From d5da49d56d7dec5f8a96c5252384d865f7efd4d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Vladan=20Vasi=C4=87?= Date: Fri, 22 Nov 2024 12:22:33 +0100 Subject: [PATCH 001/438] Revert [SPARK-50230][SQL] Added logic to support reading unknown collation name as utf8_binary MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? I propose reverting changes for new `SQLConf` entry which enables spark to read an invalid collation name as `UTF8_BINARY`. ### Why are the changes needed? Since the original changes may bring unwanted data corruption when a user writes in a table that has unknown collation and modifies its properties, the original PR must be reverted. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Not applicable. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48876 from vladanvasi-db/vladanvasi-db/allow-reading-unknown-collations-as-utf8-binary-revert. Authored-by: Vladan Vasić Signed-off-by: Max Gekk --- .../spark/sql/internal/SqlApiConf.scala | 3 - .../spark/sql/internal/SqlApiConfHelper.scala | 2 - .../org/apache/spark/sql/types/DataType.scala | 26 +-- .../apache/spark/sql/internal/SQLConf.scala | 11 -- .../spark/sql/types/DataTypeSuite.scala | 162 +----------------- 5 files changed, 5 insertions(+), 199 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index 773494f418659..d5668cc721750 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -47,7 +47,6 @@ private[sql] trait SqlApiConf { def stackTracesInDataFrameContext: Int def dataFrameQueryContextEnabled: Boolean def legacyAllowUntypedScalaUDFs: Boolean - def allowReadingUnknownCollations: Boolean } private[sql] object SqlApiConf { @@ -60,7 +59,6 @@ private[sql] object SqlApiConf { SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY } val DEFAULT_COLLATION: String = SqlApiConfHelper.DEFAULT_COLLATION - val ALLOW_READING_UNKNOWN_COLLATIONS: String = SqlApiConfHelper.ALLOW_READING_UNKNOWN_COLLATIONS def get: SqlApiConf = SqlApiConfHelper.getConfGetter.get()() @@ -89,5 +87,4 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf { override def stackTracesInDataFrameContext: Int = 1 override def dataFrameQueryContextEnabled: Boolean = true override def legacyAllowUntypedScalaUDFs: Boolean = false - override def allowReadingUnknownCollations: Boolean = false } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala index c8d6f395d4506..13ef13e5894e0 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala @@ -33,8 +33,6 @@ private[sql] object SqlApiConfHelper { val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone" val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold" val DEFAULT_COLLATION: String = "spark.sql.session.collation.default" - val ALLOW_READING_UNKNOWN_COLLATIONS: String = - "spark.sql.collation.allowReadingUnknownCollations" val confGetter: AtomicReference[() => SqlApiConf] = { new AtomicReference[() => SqlApiConf](() => DefaultSqlApiConf) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala index 4cf7d8efb96a5..036de22b4189a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -27,7 +27,7 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{SparkException, SparkIllegalArgumentException, SparkThrowable} +import org.apache.spark.{SparkIllegalArgumentException, SparkThrowable} import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.SqlApiAnalysis import org.apache.spark.sql.catalyst.parser.DataTypeParser @@ -340,17 +340,8 @@ object DataType { fields.collect { case (fieldPath, JString(collation)) => collation.split("\\.", 2) match { case Array(provider: String, collationName: String) => - try { - CollationFactory.assertValidProvider(provider) - fieldPath -> collationName - } catch { - case e: SparkException - if e.getCondition == "COLLATION_INVALID_PROVIDER" && - SqlApiConf.get.allowReadingUnknownCollations => - // If the collation provider is unknown and the config for reading such - // collations is enabled, return the UTF8_BINARY collation. - fieldPath -> "UTF8_BINARY" - } + CollationFactory.assertValidProvider(provider) + fieldPath -> collationName } }.toMap @@ -359,16 +350,7 @@ object DataType { } private def stringTypeWithCollation(collationName: String): StringType = { - try { - StringType(CollationFactory.collationNameToId(collationName)) - } catch { - case e: SparkException - if e.getCondition == "COLLATION_INVALID_NAME" && - SqlApiConf.get.allowReadingUnknownCollations => - // If the collation name is unknown and the config for reading such collations is enabled, - // return the UTF8_BINARY collation. - StringType(CollationFactory.UTF8_BINARY_COLLATION_ID) - } + StringType(CollationFactory.collationNameToId(collationName)) } protected[types] def buildFormattedString( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 123759c6c8b80..ba0a37541e490 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -778,15 +778,6 @@ object SQLConf { .booleanConf .createWithDefault(Utils.isTesting) - val ALLOW_READING_UNKNOWN_COLLATIONS = - buildConf(SqlApiConfHelper.ALLOW_READING_UNKNOWN_COLLATIONS) - .internal() - .doc("Enables spark to read unknown collation name as UTF8_BINARY. If the config is " + - "not enabled, when spark encounters an unknown collation name, it will throw an error.") - .version("4.0.0") - .booleanConf - .createWithDefault(false) - val DEFAULT_COLLATION = buildConf(SqlApiConfHelper.DEFAULT_COLLATION) .doc("Sets default collation to use for string literals, parameter markers or the string" + @@ -5582,8 +5573,6 @@ class SQLConf extends Serializable with Logging with SqlApiConf { } } - override def allowReadingUnknownCollations: Boolean = getConf(ALLOW_READING_UNKNOWN_COLLATIONS) - def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) def adaptiveExecutionLogLevel: String = getConf(ADAPTIVE_EXECUTION_LOG_LEVEL) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index d5fc4d87bb6ad..f6d8f2a66e202 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -23,13 +23,11 @@ import org.json4s.jackson.JsonMethods import org.apache.spark.{SparkException, SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.analysis.{caseInsensitiveResolution, caseSensitiveResolution} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.{CollationFactory, StringConcat} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DataTypeTestUtils.{dayTimeIntervalTypes, yearMonthIntervalTypes} -class DataTypeSuite extends SparkFunSuite with SQLHelper { +class DataTypeSuite extends SparkFunSuite { private val UNICODE_COLLATION_ID = CollationFactory.collationNameToId("UNICODE") @@ -878,90 +876,6 @@ class DataTypeSuite extends SparkFunSuite with SQLHelper { } } - test("string field with invalid collation name") { - val collationProviders = Seq("spark", "icu") - collationProviders.foreach { provider => - val json = - s""" - |{ - | "type": "struct", - | "fields": [ - | { - | "name": "c1", - | "type": "string", - | "nullable": false, - | "metadata": { - | "${DataType.COLLATIONS_METADATA_KEY}": { - | "c1": "$provider.INVALID" - | } - | } - | } - | ] - |} - |""".stripMargin - - // Check that the exception will be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME config not enabled. - checkError( - exception = intercept[SparkException] { - DataType.fromJson(json) - }, - condition = "COLLATION_INVALID_NAME", - parameters = Map( - "proposals" -> "id", - "collationName" -> "INVALID")) - - // Check that the exception will not be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME enabled, but UTF8_BINARY collation will be returned. - withSQLConf(SQLConf.ALLOW_READING_UNKNOWN_COLLATIONS.key -> "true") { - val dataType = DataType.fromJson(json) - assert(dataType === StructType( - StructField("c1", StringType(CollationFactory.UTF8_BINARY_COLLATION_ID), false) :: Nil)) - } - } - } - - test("string field with invalid collation provider") { - val json = - s""" - |{ - | "type": "struct", - | "fields": [ - | { - | "name": "c1", - | "type": "string", - | "nullable": false, - | "metadata": { - | "${DataType.COLLATIONS_METADATA_KEY}": { - | "c1": "INVALID.INVALID" - | } - | } - | } - | ] - |} - |""".stripMargin - - - // Check that the exception will be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME config not enabled. - checkError( - exception = intercept[SparkException] { - DataType.fromJson(json) - }, - condition = "COLLATION_INVALID_PROVIDER", - parameters = Map( - "supportedProviders" -> "spark, icu", - "provider" -> "INVALID")) - - // Check that the exception will not be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME enabled, but UTF8_BINARY collation will be returned. - withSQLConf(SQLConf.ALLOW_READING_UNKNOWN_COLLATIONS.key -> "true") { - val dataType = DataType.fromJson(json) - assert(dataType === StructType( - StructField("c1", StringType(CollationFactory.UTF8_BINARY_COLLATION_ID), false) :: Nil)) - } - } - test("non string field has collation metadata") { val json = s""" @@ -1109,42 +1023,6 @@ class DataTypeSuite extends SparkFunSuite with SQLHelper { assert(parsedWithCollations === ArrayType(StringType(unicodeCollationId))) } - test("parse array type with invalid collation metadata") { - val utf8BinaryCollationId = CollationFactory.UTF8_BINARY_COLLATION_ID - val arrayJson = - s""" - |{ - | "type": "array", - | "elementType": "string", - | "containsNull": true - |} - |""".stripMargin - - val collationsMap = Map("element" -> "INVALID") - - // Parse without collations map - assert(DataType.parseDataType(JsonMethods.parse(arrayJson)) === ArrayType(StringType)) - - // Check that the exception will be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME config not enabled. - checkError( - exception = intercept[SparkException] { - DataType.parseDataType(JsonMethods.parse(arrayJson), collationsMap = collationsMap) - }, - condition = "COLLATION_INVALID_NAME", - parameters = Map( - "proposals" -> "id", - "collationName" -> "INVALID")) - - // Check that the exception will not be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME enabled, but UTF8_BINARY collation will be returned. - withSQLConf(SQLConf.ALLOW_READING_UNKNOWN_COLLATIONS.key -> "true") { - val dataType = DataType.parseDataType( - JsonMethods.parse(arrayJson), collationsMap = collationsMap) - assert(dataType === ArrayType(StringType(utf8BinaryCollationId))) - } - } - test("parse map type with collation metadata") { val unicodeCollationId = CollationFactory.collationNameToId("UNICODE") val mapJson = @@ -1168,44 +1046,6 @@ class DataTypeSuite extends SparkFunSuite with SQLHelper { MapType(StringType(unicodeCollationId), StringType(unicodeCollationId))) } - test("parse map type with invalid collation metadata") { - val utf8BinaryCollationId = CollationFactory.UTF8_BINARY_COLLATION_ID - val mapJson = - s""" - |{ - | "type": "map", - | "keyType": "string", - | "valueType": "string", - | "valueContainsNull": true - |} - |""".stripMargin - - val collationsMap = Map("key" -> "INVALID", "value" -> "INVALID") - - // Parse without collations map - assert(DataType.parseDataType(JsonMethods.parse(mapJson)) === MapType(StringType, StringType)) - - // Check that the exception will be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME config not enabled. - checkError( - exception = intercept[SparkException] { - DataType.parseDataType(JsonMethods.parse(mapJson), collationsMap = collationsMap) - }, - condition = "COLLATION_INVALID_NAME", - parameters = Map( - "proposals" -> "id", - "collationName" -> "INVALID")) - - // Check that the exception will not be thrown in case of invalid collation name and - // UNKNOWN_COLLATION_NAME enabled, but UTF8_BINARY collation will be returned. - withSQLConf(SQLConf.ALLOW_READING_UNKNOWN_COLLATIONS.key -> "true") { - val dataType = DataType.parseDataType( - JsonMethods.parse(mapJson), collationsMap = collationsMap) - assert(dataType === MapType( - StringType(utf8BinaryCollationId), StringType(utf8BinaryCollationId))) - } - } - test("SPARK-48680: Add CharType and VarcharType to DataTypes JAVA API") { assert(DataTypes.createCharType(1) === CharType(1)) assert(DataTypes.createVarcharType(100) === VarcharType(100)) From 5bdaa72801d6ab35cbb975516324b536f5278f13 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2024 11:15:48 +0100 Subject: [PATCH 002/438] [SPARK-50398][CORE] Use ExitCode `0` for `--help` usage in Spark scripts ### What changes were proposed in this pull request? This PR aims to use `0` as the exit code for `--help` usage in Spark scripts consistently to provide better and consistent UX from Apache Spark 4.0.0. ### Why are the changes needed? Initially, Apache Spark was designed to exit with `0` when `--help` is given because `--help` argument is not a error case. **SparkSubmit** https://github.com/apache/spark/blob/d5da49d56d7dec5f8a96c5252384d865f7efd4d9/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala#L458-L459 **Spark Master** https://github.com/apache/spark/blob/d5da49d56d7dec5f8a96c5252384d865f7efd4d9/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala#L88-L89 **Spark Worker** https://github.com/apache/spark/blob/d5da49d56d7dec5f8a96c5252384d865f7efd4d9/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala#L107-L108 **Spark History Server** https://github.com/apache/spark/blob/d5da49d56d7dec5f8a96c5252384d865f7efd4d9/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala#L39-L40 In addition, all scripts under `bin/*` return `0` consistently like the following. This PR changes `sbin/*` scripts additionally for consistency. | Command | Before | After | | ---------- | ------- | ------- | | bin/spark-class --help | 0 | 0 | | bin/spark-shell --help | 0 | 0 | | bin/pyspark --help | 0 | 0 | | bin/spark-sql --help | 0 | 0 | | bin/beeline --help | 0 | 0 | | bin/sparkR --help | 0 | 0 | | bin/docker-image-tool.sh --help | 0 | 0 | | sbin/start-master.sh --help | 1 | 0 | | sbin/start-worker.sh --help | 1 | 0 | | sbin/start-history-server.sh --help | 1 | 0 | | sbin/start-thriftserver.sh --help | 1 | 0 | | sbin/start-connect-server.sh --help | 1 | 0 | ### Does this PR introduce _any_ user-facing change? Yes, but this is a exit status code for `--help` or `-h`. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48939 from dongjoon-hyun/SPARK-50398. Authored-by: Dongjoon Hyun Signed-off-by: Max Gekk --- sbin/start-connect-server.sh | 2 +- sbin/start-history-server.sh | 2 +- sbin/start-master.sh | 2 +- sbin/start-thriftserver.sh | 2 +- sbin/start-worker.sh | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sbin/start-connect-server.sh b/sbin/start-connect-server.sh index 668423bad1cbb..7f0c430a468a9 100755 --- a/sbin/start-connect-server.sh +++ b/sbin/start-connect-server.sh @@ -33,7 +33,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./sbin/start-connect-server.sh [--wait] [options]" "${SPARK_HOME}"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 1 + exit 0 fi . "${SPARK_HOME}/bin/load-spark-env.sh" diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 71dace47767cb..a99c8e557885b 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -40,7 +40,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then pattern+="\|Registered signal handler for" "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 - exit 1 + exit 0 fi . "${SPARK_HOME}/sbin/spark-config.sh" diff --git a/sbin/start-master.sh b/sbin/start-master.sh index 36fe4b4abeb91..25e739132f0d5 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -35,7 +35,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then pattern+="\|Registered signal handler for" "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 - exit 1 + exit 0 fi ORIGINAL_ARGS="$@" diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index b1d38713218b7..a457526979341 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -52,7 +52,7 @@ function usage { if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage - exit 1 + exit 0 fi export SUBMIT_USAGE_FUNCTION=usage diff --git a/sbin/start-worker.sh b/sbin/start-worker.sh index fd58f01bac2eb..c0147a51b3f2c 100755 --- a/sbin/start-worker.sh +++ b/sbin/start-worker.sh @@ -47,7 +47,7 @@ if [[ $# -lt 1 ]] || [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then pattern+="\|Registered signal handler for" "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 - exit 1 + [[ $# -lt 1 ]] && exit 1 || exit 0 fi . "${SPARK_HOME}/sbin/spark-config.sh" From 7ae2d81e9bc31a9549775faa6f5adf870198ac44 Mon Sep 17 00:00:00 2001 From: camilesing Date: Sat, 23 Nov 2024 11:17:52 +0100 Subject: [PATCH 003/438] [MINOR][DOCS] Fix miss semicolon on insert example sql ### What changes were proposed in this pull request? fix miss semicolon on insert example sql ### Why are the changes needed? fix miss semicolon on insert example sql. ### Does this PR introduce _any_ user-facing change? Yes. the patch fix docs miss semicolon sql. ### How was this patch tested? Manually by inspecting generated docs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48942 from camilesing/fix_docs_miss_semicolon. Authored-by: camilesing Signed-off-by: Max Gekk --- docs/sql-ref-syntax-dml-insert-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-dml-insert-table.md b/docs/sql-ref-syntax-dml-insert-table.md index 6ca062e081747..6f85d4401d3b1 100644 --- a/docs/sql-ref-syntax-dml-insert-table.md +++ b/docs/sql-ref-syntax-dml-insert-table.md @@ -379,7 +379,7 @@ SELECT * FROM persons2; +-------------+--------------------------+---------+ -- in an atomic operation, 1) delete rows with ssn = 123456789 and 2) insert rows from persons2 -INSERT INTO persons REPLACE WHERE ssn = 123456789 SELECT * FROM persons2 +INSERT INTO persons REPLACE WHERE ssn = 123456789 SELECT * FROM persons2; SELECT * FROM persons; +-------------+--------------------------+---------+ From 779a526a015d6e03dd69443a28bf5d17837bd93e Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Sat, 23 Nov 2024 11:33:16 +0100 Subject: [PATCH 004/438] [SPARK-50396][PYTHON][DOCS] Refine the docstring for datetime functions - part 3 ### What changes were proposed in this pull request? Refine the docstring for datetime functions ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? new doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48935 from zhengruifeng/py_doc_8. Authored-by: Ruifeng Zheng Signed-off-by: Max Gekk --- python/pyspark/sql/functions/builtin.py | 494 ++++++++++++++++++++---- 1 file changed, 410 insertions(+), 84 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index d2873a388617e..68b51440278cb 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -8864,6 +8864,13 @@ def curdate() -> Column: :class:`~pyspark.sql.Column` current date. + See Also + -------- + :meth:`pyspark.sql.functions.now` + :meth:`pyspark.sql.functions.current_date` + :meth:`pyspark.sql.functions.current_timestamp` + :meth:`pyspark.sql.functions.localtimestamp` + Examples -------- >>> import pyspark.sql.functions as sf @@ -8893,6 +8900,13 @@ def current_date() -> Column: :class:`~pyspark.sql.Column` current date. + See Also + -------- + :meth:`pyspark.sql.functions.now` + :meth:`pyspark.sql.functions.curdate` + :meth:`pyspark.sql.functions.current_timestamp` + :meth:`pyspark.sql.functions.localtimestamp` + Examples -------- >>> from pyspark.sql import functions as sf @@ -8920,14 +8934,26 @@ def current_timezone() -> Column: Examples -------- - >>> from pyspark.sql import functions as sf >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") + + >>> from pyspark.sql import functions as sf >>> spark.range(1).select(sf.current_timezone()).show() +-------------------+ | current_timezone()| +-------------------+ |America/Los_Angeles| +-------------------+ + + Switch the timezone to Shanghai. + + >>> spark.conf.set("spark.sql.session.timeZone", "Asia/Shanghai") + >>> spark.range(1).select(sf.current_timezone()).show() + +------------------+ + |current_timezone()| + +------------------+ + | Asia/Shanghai| + +------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function("current_timezone") @@ -8949,6 +8975,13 @@ def current_timestamp() -> Column: :class:`~pyspark.sql.Column` current date and time. + See Also + -------- + :meth:`pyspark.sql.functions.now` + :meth:`pyspark.sql.functions.curdate` + :meth:`pyspark.sql.functions.current_date` + :meth:`pyspark.sql.functions.localtimestamp` + Examples -------- >>> from pyspark.sql import functions as sf @@ -8974,6 +9007,13 @@ def now() -> Column: :class:`~pyspark.sql.Column` current timestamp at the start of query evaluation. + See Also + -------- + :meth:`pyspark.sql.functions.curdate` + :meth:`pyspark.sql.functions.current_date` + :meth:`pyspark.sql.functions.current_timestamp` + :meth:`pyspark.sql.functions.localtimestamp` + Examples -------- >>> from pyspark.sql import functions as sf @@ -9004,6 +9044,13 @@ def localtimestamp() -> Column: :class:`~pyspark.sql.Column` current local date and time. + See Also + -------- + :meth:`pyspark.sql.functions.now` + :meth:`pyspark.sql.functions.curdate` + :meth:`pyspark.sql.functions.current_date` + :meth:`pyspark.sql.functions.current_timestamp` + Examples -------- >>> from pyspark.sql import functions as sf @@ -9044,6 +9091,15 @@ def date_format(date: "ColumnOrName", format: str) -> Column: format: literal string format to use to represent datetime values. + See Also + -------- + :meth:`pyspark.sql.functions.to_date` + :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_timestamp_ltz` + :meth:`pyspark.sql.functions.to_timestamp_ntz` + :meth:`pyspark.sql.functions.to_utc_timestamp` + :meth:`pyspark.sql.functions.try_to_timestamp` + Returns ------- :class:`~pyspark.sql.Column` @@ -9130,6 +9186,18 @@ def year(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` year part of the date/timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` + :meth:`pyspark.sql.functions.extract` + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Examples -------- Example 1: Extract the year from a string column representing dates @@ -9209,6 +9277,18 @@ def quarter(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` quarter of the date/timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` + :meth:`pyspark.sql.functions.extract` + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Examples -------- Example 1: Extract the quarter from a string column representing dates @@ -9288,6 +9368,19 @@ def month(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` month part of the date/timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` + :meth:`pyspark.sql.functions.monthname` + :meth:`pyspark.sql.functions.extract` + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Examples -------- Example 1: Extract the month from a string column representing dates @@ -9368,6 +9461,12 @@ def dayofweek(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` day of the week for given date/timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.dayofyear` + :meth:`pyspark.sql.functions.dayofmonth` + Examples -------- Example 1: Extract the day of the week from a string column representing dates @@ -9442,6 +9541,12 @@ def dayofmonth(col: "ColumnOrName") -> Column: col : :class:`~pyspark.sql.Column` or column name target date/timestamp column to work on. + See Also + -------- + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.dayofyear` + :meth:`pyspark.sql.functions.dayofweek` + Returns ------- :class:`~pyspark.sql.Column` @@ -9523,6 +9628,22 @@ def day(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` day of the month for given date/timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` + :meth:`pyspark.sql.functions.dayname` + :meth:`pyspark.sql.functions.dayofyear` + :meth:`pyspark.sql.functions.dayofmonth` + :meth:`pyspark.sql.functions.dayofweek` + :meth:`pyspark.sql.functions.extract` + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Examples -------- Example 1: Extract the day of the month from a string column representing dates @@ -9602,6 +9723,12 @@ def dayofyear(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` day of the year for given date/timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.dayofyear` + :meth:`pyspark.sql.functions.dayofmonth` + Examples -------- Example 1: Extract the day of the year from a string column representing dates @@ -9681,6 +9808,18 @@ def hour(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` hour part of the timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` + :meth:`pyspark.sql.functions.extract` + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Examples -------- Example 1: Extract the hours from a string column representing timestamp @@ -9728,6 +9867,18 @@ def minute(col: "ColumnOrName") -> Column: col : :class:`~pyspark.sql.Column` or column name target date/timestamp column to work on. + See Also + -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.second` + :meth:`pyspark.sql.functions.extract` + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Returns ------- :class:`~pyspark.sql.Column` @@ -9785,6 +9936,18 @@ def second(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` `seconds` part of the timestamp as integer. + See Also + -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.extract` + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Examples -------- Example 1: Extract the seconds from a string column representing timestamp @@ -9839,6 +10002,10 @@ def weekofyear(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` `week` of the year for given date as integer. + See Also + -------- + :meth:`pyspark.sql.functions.weekday` + Examples -------- Example 1: Extract the week of the year from a string column representing dates @@ -9915,6 +10082,11 @@ def weekday(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the day of the week for date/timestamp (0 = Monday, 1 = Tuesday, ..., 6 = Sunday). + See Also + -------- + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.weekofyear` + Examples -------- Example 1: Extract the day of the week from a string column representing dates @@ -9991,6 +10163,11 @@ def monthname(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the three-letter abbreviation of month name for date/timestamp (Jan, Feb, Mar...) + See Also + -------- + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.dayname` + Examples -------- Example 1: Extract the month name from a string column representing dates @@ -10067,6 +10244,11 @@ def dayname(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the three-letter abbreviation of day name for date/timestamp (Mon, Tue, Wed...) + See Also + -------- + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.monthname` + Examples -------- Example 1: Extract the weekday name from a string column representing dates @@ -10147,6 +10329,13 @@ def extract(field: Column, source: "ColumnOrName") -> Column: See Also -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` :meth:`pyspark.sql.functions.datepart` :meth:`pyspark.sql.functions.date_part` @@ -10195,6 +10384,13 @@ def date_part(field: Column, source: "ColumnOrName") -> Column: See Also -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` :meth:`pyspark.sql.functions.datepart` :meth:`pyspark.sql.functions.extract` @@ -10243,6 +10439,13 @@ def datepart(field: Column, source: "ColumnOrName") -> Column: See Also -------- + :meth:`pyspark.sql.functions.year` + :meth:`pyspark.sql.functions.quarter` + :meth:`pyspark.sql.functions.month` + :meth:`pyspark.sql.functions.day` + :meth:`pyspark.sql.functions.hour` + :meth:`pyspark.sql.functions.minute` + :meth:`pyspark.sql.functions.second` :meth:`pyspark.sql.functions.date_part` :meth:`pyspark.sql.functions.extract` @@ -10780,7 +10983,11 @@ def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column: See Also -------- :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_timestamp_ltz` + :meth:`pyspark.sql.functions.to_timestamp_ntz` + :meth:`pyspark.sql.functions.to_utc_timestamp` :meth:`pyspark.sql.functions.try_to_timestamp` + :meth:`pyspark.sql.functions.date_format` Examples -------- @@ -11018,7 +11225,12 @@ def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column: See Also -------- :meth:`pyspark.sql.functions.to_date` + :meth:`pyspark.sql.functions.to_timestamp_ltz` + :meth:`pyspark.sql.functions.to_timestamp_ntz` + :meth:`pyspark.sql.functions.to_utc_timestamp` + :meth:`pyspark.sql.functions.to_unix_timestamp` :meth:`pyspark.sql.functions.try_to_timestamp` + :meth:`pyspark.sql.functions.date_format` Examples -------- @@ -11072,6 +11284,8 @@ def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = Non -------- :meth:`pyspark.sql.functions.to_date` :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_utc_timestamp` + :meth:`pyspark.sql.functions.date_format` Examples -------- @@ -11646,6 +11860,9 @@ def from_utc_timestamp(timestamp: "ColumnOrName", tz: Union[Column, str]) -> Col See Also -------- :meth:`pyspark.sql.functions.to_utc_timestamp` + :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_timestamp_ltz` + :meth:`pyspark.sql.functions.to_timestamp_ntz` Examples -------- @@ -11712,6 +11929,9 @@ def to_utc_timestamp(timestamp: "ColumnOrName", tz: Union[Column, str]) -> Colum See Also -------- :meth:`pyspark.sql.functions.from_utc_timestamp` + :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_timestamp_ltz` + :meth:`pyspark.sql.functions.to_timestamp_ntz` Examples -------- @@ -12034,22 +12254,22 @@ def window( Parameters ---------- - timeColumn : :class:`~pyspark.sql.Column` + timeColumn : :class:`~pyspark.sql.Column` or column name The column or the expression to use as the timestamp for windowing by time. The time column must be of TimestampType or TimestampNTZType. - windowDuration : str + windowDuration : literal string A string specifying the width of the window, e.g. `10 minutes`, `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. Note that the duration is a fixed length of time, and does not vary over time according to a calendar. For example, `1 day` always means 86,400,000 milliseconds, not a calendar day. - slideDuration : str, optional + slideDuration : literal string, optional A new window will be generated every `slideDuration`. Must be less than or equal to the `windowDuration`. Check `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. This duration is likewise absolute, and does not vary according to a calendar. - startTime : str, optional + startTime : literal string, optional The offset with respect to 1970-01-01 00:00:00 UTC with which to start window intervals. For example, in order to have hourly tumbling windows that start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide @@ -12060,24 +12280,30 @@ def window( :class:`~pyspark.sql.Column` the column for computed results. + See Also + -------- + :meth:`pyspark.sql.functions.window_time` + :meth:`pyspark.sql.functions.session_window` + Examples -------- >>> import datetime >>> from pyspark.sql import functions as sf - >>> df = spark.createDataFrame( - ... [(datetime.datetime(2016, 3, 11, 9, 0, 7), 1)], - ... ).toDF("date", "val") - >>> w = df.groupBy(sf.window("date", "5 seconds")).agg(sf.sum("val").alias("sum")) - >>> w.select( - ... w.window.start.cast("string").alias("start"), - ... w.window.end.cast("string").alias("end"), - ... "sum" - ... ).show() - +-------------------+-------------------+---+ - | start| end|sum| - +-------------------+-------------------+---+ - |2016-03-11 09:00:05|2016-03-11 09:00:10| 1| - +-------------------+-------------------+---+ + >>> df = spark.createDataFrame([(datetime.datetime(2016, 3, 11, 9, 0, 7), 1)], ['dt', 'v']) + >>> df2 = df.groupBy(sf.window('dt', '5 seconds')).agg(sf.sum('v')) + >>> df2.show(truncate=False) + +------------------------------------------+------+ + |window |sum(v)| + +------------------------------------------+------+ + |{2016-03-11 09:00:05, 2016-03-11 09:00:10}|1 | + +------------------------------------------+------+ + + >>> df2.printSchema() + root + |-- window: struct (nullable = false) + | |-- start: timestamp (nullable = true) + | |-- end: timestamp (nullable = true) + |-- sum(v): long (nullable = true) """ from pyspark.sql.classic.column import _to_java_column @@ -12123,7 +12349,7 @@ def window_time( Parameters ---------- - windowColumn : :class:`~pyspark.sql.Column` + windowColumn : :class:`~pyspark.sql.Column` or column name The window column of a window aggregate records. Returns @@ -12131,29 +12357,29 @@ def window_time( :class:`~pyspark.sql.Column` the column for computed results. - Notes - ----- - Supports Spark Connect. + See Also + -------- + :meth:`pyspark.sql.functions.window` + :meth:`pyspark.sql.functions.session_window` Examples -------- >>> import datetime - >>> df = spark.createDataFrame( - ... [(datetime.datetime(2016, 3, 11, 9, 0, 7), 1)], - ... ).toDF("date", "val") + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(datetime.datetime(2016, 3, 11, 9, 0, 7), 1)], ['dt', 'v']) Group the data into 5 second time windows and aggregate as sum. - >>> w = df.groupBy(window("date", "5 seconds")).agg(sum("val").alias("sum")) + >>> df2 = df.groupBy(sf.window('dt', '5 seconds')).agg(sf.sum('v')) Extract the window event time using the window_time function. - >>> w.select( - ... w.window.end.cast("string").alias("end"), - ... window_time(w.window).cast("string").alias("window_time"), - ... "sum" - ... ).collect() - [Row(end='2016-03-11 09:00:10', window_time='2016-03-11 09:00:09.999999', sum=1)] + >>> df2.select('*', sf.window_time('window')).show(truncate=False) + +------------------------------------------+------+--------------------------+ + |window |sum(v)|window_time(window) | + +------------------------------------------+------+--------------------------+ + |{2016-03-11 09:00:05, 2016-03-11 09:00:10}|1 |2016-03-11 09:00:09.999999| + +------------------------------------------+------+--------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -12187,10 +12413,10 @@ def session_window(timeColumn: "ColumnOrName", gapDuration: Union[Column, str]) Parameters ---------- - timeColumn : :class:`~pyspark.sql.Column` or str + timeColumn : :class:`~pyspark.sql.Column` or column name The column name or column to use as the timestamp for windowing by time. The time column must be of TimestampType or TimestampNTZType. - gapDuration : :class:`~pyspark.sql.Column` or str + gapDuration : :class:`~pyspark.sql.Column` or literal string A Python string literal or column specifying the timeout of the session. It could be static value, e.g. `10 minutes`, `1 second`, or an expression/UDF that specifies gap duration dynamically based on the input row. @@ -12200,17 +12426,29 @@ def session_window(timeColumn: "ColumnOrName", gapDuration: Union[Column, str]) :class:`~pyspark.sql.Column` the column for computed results. + See Also + -------- + :meth:`pyspark.sql.functions.window` + :meth:`pyspark.sql.functions.window_time` + Examples -------- - >>> df = spark.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val") - >>> w = df.groupBy(session_window("date", "5 seconds")).agg(sum("val").alias("sum")) - >>> w.select(w.session_window.start.cast("string").alias("start"), - ... w.session_window.end.cast("string").alias("end"), "sum").collect() - [Row(start='2016-03-11 09:00:07', end='2016-03-11 09:00:12', sum=1)] - >>> w = df.groupBy(session_window("date", lit("5 seconds"))).agg(sum("val").alias("sum")) - >>> w.select(w.session_window.start.cast("string").alias("start"), - ... w.session_window.end.cast("string").alias("end"), "sum").collect() - [Row(start='2016-03-11 09:00:07', end='2016-03-11 09:00:12', sum=1)] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('2016-03-11 09:00:07', 1)], ['dt', 'v']) + >>> df2 = df.groupBy(sf.session_window('dt', '5 seconds')).agg(sf.sum('v')) + >>> df2.show(truncate=False) + +------------------------------------------+------+ + |session_window |sum(v)| + +------------------------------------------+------+ + |{2016-03-11 09:00:07, 2016-03-11 09:00:12}|1 | + +------------------------------------------+------+ + + >>> df2.printSchema() + root + |-- session_window: struct (nullable = false) + | |-- start: timestamp (nullable = true) + | |-- end: timestamp (nullable = true) + |-- sum(v): long (nullable = true) """ from pyspark.sql.classic.column import _to_java_column @@ -12240,37 +12478,57 @@ def to_unix_timestamp( Parameters ---------- - timestamp : :class:`~pyspark.sql.Column` or str + timestamp : :class:`~pyspark.sql.Column` or column name Input column or strings. - format : :class:`~pyspark.sql.Column` or str, optional + format : :class:`~pyspark.sql.Column` or column name, optional format to use to convert UNIX timestamp values. + See Also + -------- + :meth:`pyspark.sql.functions.to_date` + :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_timestamp_ltz` + :meth:`pyspark.sql.functions.to_timestamp_ntz` + :meth:`pyspark.sql.functions.to_utc_timestamp` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - Example 1: Using default format 'yyyy-MM-dd HH:mm:ss' parses the timestamp string. + Example 1: Using default format to parse the timestamp string. >>> import pyspark.sql.functions as sf - >>> time_df = spark.createDataFrame([('2015-04-08 12:12:12',)], ['dt']) - >>> time_df.select(sf.to_unix_timestamp('dt').alias('unix_time')).show() - +----------+ - | unix_time| - +----------+ - |1428520332| - +----------+ + >>> df = spark.createDataFrame([('2015-04-08 12:12:12',)], ['ts']) + >>> df.select('*', sf.to_unix_timestamp('ts')).show() + +-------------------+------------------------------------------+ + | ts|to_unix_timestamp(ts, yyyy-MM-dd HH:mm:ss)| + +-------------------+------------------------------------------+ + |2015-04-08 12:12:12| 1428520332| + +-------------------+------------------------------------------+ - Example 2: Using user-specified format 'yyyy-MM-dd' parses the timestamp string. + Example 2: Using user-specified format 'yyyy-MM-dd' to parse the date string. >>> import pyspark.sql.functions as sf - >>> time_df = spark.createDataFrame([('2015-04-08',)], ['dt']) - >>> time_df.select( - ... sf.to_unix_timestamp('dt', sf.lit('yyyy-MM-dd')).alias('unix_time')).show() - +----------+ - | unix_time| - +----------+ - |1428476400| - +----------+ + >>> df = spark.createDataFrame([('2015-04-08',)], ['dt']) + >>> df.select('*', sf.to_unix_timestamp(df.dt, sf.lit('yyyy-MM-dd'))).show() + +----------+---------------------------------+ + | dt|to_unix_timestamp(dt, yyyy-MM-dd)| + +----------+---------------------------------+ + |2015-04-08| 1428476400| + +----------+---------------------------------+ + + Example 3: Using a format column to represent different formats. + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame( + ... [('2015-04-08', 'yyyy-MM-dd'), ('2025+01+09', 'yyyy+MM+dd')], ['dt', 'fmt']) + >>> df.select('*', sf.to_unix_timestamp('dt', 'fmt')).show() + +----------+----------+--------------------------+ + | dt| fmt|to_unix_timestamp(dt, fmt)| + +----------+----------+--------------------------+ + |2015-04-08|yyyy-MM-dd| 1428476400| + |2025+01+09|yyyy+MM+dd| 1736409600| + +----------+----------+--------------------------+ >>> spark.conf.unset("spark.sql.session.timeZone") """ @@ -12286,29 +12544,63 @@ def to_timestamp_ltz( format: Optional["ColumnOrName"] = None, ) -> Column: """ - Parses the `timestamp` with the `format` to a timestamp without time zone. + Parses the `timestamp` with the `format` to a timestamp with time zone. Returns null with invalid input. .. versionadded:: 3.5.0 Parameters ---------- - timestamp : :class:`~pyspark.sql.Column` or str + timestamp : :class:`~pyspark.sql.Column` or column name Input column or strings. - format : :class:`~pyspark.sql.Column` or str, optional + format : :class:`~pyspark.sql.Column` or column name, optional format to use to convert type `TimestampType` timestamp values. + See Also + -------- + :meth:`pyspark.sql.functions.to_date` + :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_timestamp_ntz` + :meth:`pyspark.sql.functions.to_utc_timestamp` + :meth:`pyspark.sql.functions.to_unix_timestamp` + :meth:`pyspark.sql.functions.date_format` + Examples -------- - >>> df = spark.createDataFrame([("2016-12-31",)], ["e"]) - >>> df.select(to_timestamp_ltz(df.e, lit("yyyy-MM-dd")).alias('r')).collect() - ... # doctest: +SKIP - [Row(r=datetime.datetime(2016, 12, 31, 0, 0))] + Example 1: Using default format to parse the timestamp string. - >>> df = spark.createDataFrame([("2016-12-31",)], ["e"]) - >>> df.select(to_timestamp_ltz(df.e).alias('r')).collect() - ... # doctest: +SKIP - [Row(r=datetime.datetime(2016, 12, 31, 0, 0))] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2015-04-08 12:12:12',)], ['ts']) + >>> df.select('*', sf.to_timestamp_ltz('ts')).show() + +-------------------+--------------------+ + | ts|to_timestamp_ltz(ts)| + +-------------------+--------------------+ + |2015-04-08 12:12:12| 2015-04-08 12:12:12| + +-------------------+--------------------+ + + Example 2: Using user-specified format to parse the date string. + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2016-12-31',)], ['dt']) + >>> df.select('*', sf.to_timestamp_ltz(df.dt, sf.lit('yyyy-MM-dd'))).show() + +----------+--------------------------------+ + | dt|to_timestamp_ltz(dt, yyyy-MM-dd)| + +----------+--------------------------------+ + |2016-12-31| 2016-12-31 00:00:00| + +----------+--------------------------------+ + + Example 3: Using a format column to represent different formats. + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame( + ... [('2015-04-08', 'yyyy-MM-dd'), ('2025+01+09', 'yyyy+MM+dd')], ['dt', 'fmt']) + >>> df.select('*', sf.to_timestamp_ltz('dt', 'fmt')).show() + +----------+----------+-------------------------+ + | dt| fmt|to_timestamp_ltz(dt, fmt)| + +----------+----------+-------------------------+ + |2015-04-08|yyyy-MM-dd| 2015-04-08 00:00:00| + |2025+01+09|yyyy+MM+dd| 2025-01-09 00:00:00| + +----------+----------+-------------------------+ """ if format is not None: return _invoke_function_over_columns("to_timestamp_ltz", timestamp, format) @@ -12329,22 +12621,56 @@ def to_timestamp_ntz( Parameters ---------- - timestamp : :class:`~pyspark.sql.Column` or str + timestamp : :class:`~pyspark.sql.Column` or column name Input column or strings. - format : :class:`~pyspark.sql.Column` or str, optional + format : :class:`~pyspark.sql.Column` or column name, optional format to use to convert type `TimestampNTZType` timestamp values. + See Also + -------- + :meth:`pyspark.sql.functions.to_date` + :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.to_timestamp_ltz` + :meth:`pyspark.sql.functions.to_utc_timestamp` + :meth:`pyspark.sql.functions.to_unix_timestamp` + :meth:`pyspark.sql.functions.date_format` + Examples -------- - >>> df = spark.createDataFrame([("2016-04-08",)], ["e"]) - >>> df.select(to_timestamp_ntz(df.e, lit("yyyy-MM-dd")).alias('r')).collect() - ... # doctest: +SKIP - [Row(r=datetime.datetime(2016, 4, 8, 0, 0))] + Example 1: Using default format to parse the timestamp string. - >>> df = spark.createDataFrame([("2016-04-08",)], ["e"]) - >>> df.select(to_timestamp_ntz(df.e).alias('r')).collect() - ... # doctest: +SKIP - [Row(r=datetime.datetime(2016, 4, 8, 0, 0))] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2015-04-08 12:12:12',)], ['ts']) + >>> df.select('*', sf.to_timestamp_ntz('ts')).show() + +-------------------+--------------------+ + | ts|to_timestamp_ntz(ts)| + +-------------------+--------------------+ + |2015-04-08 12:12:12| 2015-04-08 12:12:12| + +-------------------+--------------------+ + + Example 2: Using user-specified format 'yyyy-MM-dd' to parse the date string. + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2016-12-31',)], ['dt']) + >>> df.select('*', sf.to_timestamp_ntz(df.dt, sf.lit('yyyy-MM-dd'))).show() + +----------+--------------------------------+ + | dt|to_timestamp_ntz(dt, yyyy-MM-dd)| + +----------+--------------------------------+ + |2016-12-31| 2016-12-31 00:00:00| + +----------+--------------------------------+ + + Example 3: Using a format column to represent different formats. + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame( + ... [('2015-04-08', 'yyyy-MM-dd'), ('2025+01+09', 'yyyy+MM+dd')], ['dt', 'fmt']) + >>> df.select('*', sf.to_timestamp_ntz('dt', 'fmt')).show() + +----------+----------+-------------------------+ + | dt| fmt|to_timestamp_ntz(dt, fmt)| + +----------+----------+-------------------------+ + |2015-04-08|yyyy-MM-dd| 2015-04-08 00:00:00| + |2025+01+09|yyyy+MM+dd| 2025-01-09 00:00:00| + +----------+----------+-------------------------+ """ if format is not None: return _invoke_function_over_columns("to_timestamp_ntz", timestamp, format) From 656ece1adb0fb6b4aea108ed92e5939e7b2dc7e9 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sat, 23 Nov 2024 11:59:38 +0100 Subject: [PATCH 005/438] [SPARK-50081][SQL] Codegen Support for `XPath*`(by Invoke & RuntimeReplaceable) ### What changes were proposed in this pull request? The pr aims to add `Codegen` Support for `xpath*`, include: - `xpath_boolean` - `xpath_short` - `xpath_int` - `xpath_long` - `xpath_float` - `xpath_double` - `xpath_string` - `xpath` ### Why are the changes needed? - improve codegen coverage. - simplified code. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA & Existed UT (eg: `XPathFunctionsSuite`, `XPathExpressionSuite`, `CollationSQLExpressionsSuite`#`*XPath*`, `CollationExpressionWalkerSuite`) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48610 from panbingkun/xpath_codegen. Lead-authored-by: panbingkun Co-authored-by: panbingkun Signed-off-by: Max Gekk --- .../xml/XmlExpressionEvalUtils.scala | 82 ++++++++++++++++- .../sql/catalyst/expressions/xml/xpath.scala | 92 ++++++++----------- .../explain-results/function_xpath.explain | 2 +- .../function_xpath_boolean.explain | 2 +- .../function_xpath_double.explain | 2 +- .../function_xpath_float.explain | 2 +- .../function_xpath_int.explain | 2 +- .../function_xpath_long.explain | 2 +- .../function_xpath_number.explain | 2 +- .../function_xpath_short.explain | 2 +- .../function_xpath_string.explain | 2 +- .../spark/sql/XPathFunctionsSuite.scala | 36 ++++++++ 12 files changed, 162 insertions(+), 66 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala index dff88475327a2..44b98026d62d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.catalyst.expressions.xml +import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.catalyst.xml.XmlInferSchema import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, DataType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String object XmlExpressionEvalUtils { @@ -40,3 +41,82 @@ object XmlExpressionEvalUtils { UTF8String.fromString(dataType.sql) } } + +trait XPathEvaluator { + + protected val path: UTF8String + + @transient protected lazy val xpathUtil: UDFXPathUtil = new UDFXPathUtil + + final def evaluate(xml: UTF8String): Any = { + if (xml == null || xml.toString.isEmpty || path == null || path.toString.isEmpty) return null + doEvaluate(xml) + } + + def doEvaluate(xml: UTF8String): Any +} + +case class XPathBooleanEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + xpathUtil.evalBoolean(xml.toString, path.toString) + } +} + +case class XPathShortEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + val ret = xpathUtil.evalNumber(xml.toString, path.toString) + if (ret eq null) null.asInstanceOf[Short] else ret.shortValue() + } +} + +case class XPathIntEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + val ret = xpathUtil.evalNumber(xml.toString, path.toString) + if (ret eq null) null.asInstanceOf[Int] else ret.intValue() + } +} + +case class XPathLongEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + val ret = xpathUtil.evalNumber(xml.toString, path.toString) + if (ret eq null) null.asInstanceOf[Long] else ret.longValue() + } +} + +case class XPathFloatEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + val ret = xpathUtil.evalNumber(xml.toString, path.toString) + if (ret eq null) null.asInstanceOf[Float] else ret.floatValue() + } +} + +case class XPathDoubleEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + val ret = xpathUtil.evalNumber(xml.toString, path.toString) + if (ret eq null) null.asInstanceOf[Double] else ret.doubleValue() + } +} + +case class XPathStringEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + val ret = xpathUtil.evalString(xml.toString, path.toString) + UTF8String.fromString(ret) + } +} + +case class XPathListEvaluator(path: UTF8String) extends XPathEvaluator { + override def doEvaluate(xml: UTF8String): Any = { + val nodeList = xpathUtil.evalNodeList(xml.toString, path.toString) + if (nodeList ne null) { + val ret = new Array[AnyRef](nodeList.getLength) + var i = 0 + while (i < nodeList.getLength) { + ret(i) = UTF8String.fromString(nodeList.item(i).getNodeValue) + i += 1 + } + new GenericArrayData(ret) + } else { + null + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala index 9848e062a08fd..2c18ffa2abecb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala @@ -21,8 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.Cast._ -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation import org.apache.spark.sql.types._ @@ -34,10 +33,9 @@ import org.apache.spark.unsafe.types.UTF8String * This is not the world's most efficient implementation due to type conversion, but works. */ abstract class XPathExtract - extends BinaryExpression with ExpectsInputTypes with CodegenFallback { + extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes { override def left: Expression = xml override def right: Expression = path - override def nullIntolerant: Boolean = true /** XPath expressions are always nullable, e.g. if the xml string is empty. */ override def nullable: Boolean = true @@ -60,12 +58,20 @@ abstract class XPathExtract } } - @transient protected lazy val xpathUtil = new UDFXPathUtil - @transient protected lazy val pathString: String = path.eval().asInstanceOf[UTF8String].toString - /** Concrete implementations need to override the following three methods. */ def xml: Expression def path: Expression + + @transient protected lazy val pathUTF8String: UTF8String = path.eval().asInstanceOf[UTF8String] + + protected def evaluator: XPathEvaluator + + override def replacement: Expression = Invoke( + Literal.create(evaluator, ObjectType(classOf[XPathEvaluator])), + "evaluate", + dataType, + Seq(xml), + Seq(xml.dataType)) } // scalastyle:off line.size.limit @@ -81,11 +87,9 @@ abstract class XPathExtract // scalastyle:on line.size.limit case class XPathBoolean(xml: Expression, path: Expression) extends XPathExtract with Predicate { - override def prettyName: String = "xpath_boolean" + @transient override lazy val evaluator: XPathEvaluator = XPathBooleanEvaluator(pathUTF8String) - override def nullSafeEval(xml: Any, path: Any): Any = { - xpathUtil.evalBoolean(xml.asInstanceOf[UTF8String].toString, pathString) - } + override def prettyName: String = "xpath_boolean" override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): XPathBoolean = copy(xml = newLeft, path = newRight) @@ -103,14 +107,12 @@ case class XPathBoolean(xml: Expression, path: Expression) extends XPathExtract group = "xml_funcs") // scalastyle:on line.size.limit case class XPathShort(xml: Expression, path: Expression) extends XPathExtract { + + @transient override lazy val evaluator: XPathEvaluator = XPathShortEvaluator(pathUTF8String) + override def prettyName: String = "xpath_short" override def dataType: DataType = ShortType - override def nullSafeEval(xml: Any, path: Any): Any = { - val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) - if (ret eq null) null else ret.shortValue() - } - override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): XPathShort = copy(xml = newLeft, path = newRight) } @@ -127,14 +129,12 @@ case class XPathShort(xml: Expression, path: Expression) extends XPathExtract { group = "xml_funcs") // scalastyle:on line.size.limit case class XPathInt(xml: Expression, path: Expression) extends XPathExtract { + + @transient override lazy val evaluator: XPathEvaluator = XPathIntEvaluator(pathUTF8String) + override def prettyName: String = "xpath_int" override def dataType: DataType = IntegerType - override def nullSafeEval(xml: Any, path: Any): Any = { - val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) - if (ret eq null) null else ret.intValue() - } - override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): Expression = copy(xml = newLeft, path = newRight) } @@ -151,14 +151,12 @@ case class XPathInt(xml: Expression, path: Expression) extends XPathExtract { group = "xml_funcs") // scalastyle:on line.size.limit case class XPathLong(xml: Expression, path: Expression) extends XPathExtract { + + @transient override lazy val evaluator: XPathEvaluator = XPathLongEvaluator(pathUTF8String) + override def prettyName: String = "xpath_long" override def dataType: DataType = LongType - override def nullSafeEval(xml: Any, path: Any): Any = { - val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) - if (ret eq null) null else ret.longValue() - } - override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): XPathLong = copy(xml = newLeft, path = newRight) } @@ -175,14 +173,12 @@ case class XPathLong(xml: Expression, path: Expression) extends XPathExtract { group = "xml_funcs") // scalastyle:on line.size.limit case class XPathFloat(xml: Expression, path: Expression) extends XPathExtract { + + @transient override lazy val evaluator: XPathEvaluator = XPathFloatEvaluator(pathUTF8String) + override def prettyName: String = "xpath_float" override def dataType: DataType = FloatType - override def nullSafeEval(xml: Any, path: Any): Any = { - val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) - if (ret eq null) null else ret.floatValue() - } - override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): XPathFloat = copy(xml = newLeft, path = newRight) } @@ -199,15 +195,13 @@ case class XPathFloat(xml: Expression, path: Expression) extends XPathExtract { group = "xml_funcs") // scalastyle:on line.size.limit case class XPathDouble(xml: Expression, path: Expression) extends XPathExtract { + + @transient override lazy val evaluator: XPathEvaluator = XPathDoubleEvaluator(pathUTF8String) + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("xpath_double") override def dataType: DataType = DoubleType - override def nullSafeEval(xml: Any, path: Any): Any = { - val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) - if (ret eq null) null else ret.doubleValue() - } - override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): XPathDouble = copy(xml = newLeft, path = newRight) } @@ -224,14 +218,12 @@ case class XPathDouble(xml: Expression, path: Expression) extends XPathExtract { group = "xml_funcs") // scalastyle:on line.size.limit case class XPathString(xml: Expression, path: Expression) extends XPathExtract { + + @transient override lazy val evaluator: XPathEvaluator = XPathStringEvaluator(pathUTF8String) + override def prettyName: String = "xpath_string" override def dataType: DataType = SQLConf.get.defaultStringType - override def nullSafeEval(xml: Any, path: Any): Any = { - val ret = xpathUtil.evalString(xml.asInstanceOf[UTF8String].toString, pathString) - UTF8String.fromString(ret) - } - override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): Expression = copy(xml = newLeft, path = newRight) } @@ -250,24 +242,12 @@ case class XPathString(xml: Expression, path: Expression) extends XPathExtract { group = "xml_funcs") // scalastyle:on line.size.limit case class XPathList(xml: Expression, path: Expression) extends XPathExtract { + + @transient override lazy val evaluator: XPathEvaluator = XPathListEvaluator(pathUTF8String) + override def prettyName: String = "xpath" override def dataType: DataType = ArrayType(SQLConf.get.defaultStringType) - override def nullSafeEval(xml: Any, path: Any): Any = { - val nodeList = xpathUtil.evalNodeList(xml.asInstanceOf[UTF8String].toString, pathString) - if (nodeList ne null) { - val ret = new Array[AnyRef](nodeList.getLength) - var i = 0 - while (i < nodeList.getLength) { - ret(i) = UTF8String.fromString(nodeList.item(i).getNodeValue) - i += 1 - } - new GenericArrayData(ret) - } else { - null - } - } - override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): XPathList = copy(xml = newLeft, path = newRight) } diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath.explain index d9e2e55d9b12e..4752e5218bb12 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath.explain @@ -1,2 +1,2 @@ -Project [xpath(s#0, a/b/text()) AS xpath(s, a/b/text())#0] +Project [invoke(XPathListEvaluator(a/b/text()).evaluate(s#0)) AS xpath(s, a/b/text())#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_boolean.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_boolean.explain index 9b75f81802467..b537366736d25 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_boolean.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_boolean.explain @@ -1,2 +1,2 @@ -Project [xpath_boolean(s#0, a/b) AS xpath_boolean(s, a/b)#0] +Project [invoke(XPathBooleanEvaluator(a/b).evaluate(s#0)) AS xpath_boolean(s, a/b)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_double.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_double.explain index 9ce47136df242..76e0b01721841 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_double.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_double.explain @@ -1,2 +1,2 @@ -Project [xpath_double(s#0, a/b) AS xpath_double(s, a/b)#0] +Project [invoke(XPathDoubleEvaluator(a/b).evaluate(s#0)) AS xpath_double(s, a/b)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_float.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_float.explain index 02b29ec4afa9c..21aebb357928f 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_float.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_float.explain @@ -1,2 +1,2 @@ -Project [xpath_float(s#0, a/b) AS xpath_float(s, a/b)#0] +Project [invoke(XPathFloatEvaluator(a/b).evaluate(s#0)) AS xpath_float(s, a/b)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_int.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_int.explain index cdd56eaa73199..eee74472b1cff 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_int.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_int.explain @@ -1,2 +1,2 @@ -Project [xpath_int(s#0, a/b) AS xpath_int(s, a/b)#0] +Project [invoke(XPathIntEvaluator(a/b).evaluate(s#0)) AS xpath_int(s, a/b)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_long.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_long.explain index 3acefb13d0f8c..8356c2c8e18c1 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_long.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_long.explain @@ -1,2 +1,2 @@ -Project [xpath_long(s#0, a/b) AS xpath_long(s, a/b)#0L] +Project [invoke(XPathLongEvaluator(a/b).evaluate(s#0)) AS xpath_long(s, a/b)#0L] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_number.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_number.explain index 0a30685f0c6d2..bc32d4fefffb8 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_number.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_number.explain @@ -1,2 +1,2 @@ -Project [xpath_number(s#0, a/b) AS xpath_number(s, a/b)#0] +Project [invoke(XPathDoubleEvaluator(a/b).evaluate(s#0)) AS xpath_number(s, a/b)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_short.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_short.explain index ed440972bf490..e0ba76b3acd0e 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_short.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_short.explain @@ -1,2 +1,2 @@ -Project [xpath_short(s#0, a/b) AS xpath_short(s, a/b)#0] +Project [invoke(XPathShortEvaluator(a/b).evaluate(s#0)) AS xpath_short(s, a/b)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_string.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_string.explain index f4103f68c3bc3..80f2600e6cdd4 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_string.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_xpath_string.explain @@ -1,2 +1,2 @@ -Project [xpath_string(s#0, a/b) AS xpath_string(s, a/b)#0] +Project [invoke(XPathStringEvaluator(a/b).evaluate(s#0)) AS xpath_string(s, a/b)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala index f08466e8f8d9d..f2a86cbf54152 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.expressions.IsNotNull +import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession @@ -76,4 +78,38 @@ class XPathFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df.select(xpath(col("xml"), lit("a/*/text()"))), Row(Seq("b1", "b2", "b3", "c1", "c2"))) } + + test("The replacement of `xpath*` functions should be NullIntolerant") { + def check(df: DataFrame, expected: Seq[Row]): Unit = { + val filter = df.queryExecution + .sparkPlan + .find(_.isInstanceOf[FilterExec]) + .get.asInstanceOf[FilterExec] + assert(filter.condition.find(_.isInstanceOf[IsNotNull]).nonEmpty) + checkAnswer(df, expected) + } + withTable("t") { + sql("CREATE TABLE t AS SELECT * FROM VALUES ('1'), (NULL) T(xml)") + check(sql("SELECT * FROM t WHERE xpath_boolean(xml, 'a/b') = true"), + Seq(Row("1"))) + check(sql("SELECT * FROM t WHERE xpath_short(xml, 'a/b') = 1"), + Seq(Row("1"))) + check(sql("SELECT * FROM t WHERE xpath_int(xml, 'a/b') = 1"), + Seq(Row("1"))) + check(sql("SELECT * FROM t WHERE xpath_long(xml, 'a/b') = 1"), + Seq(Row("1"))) + check(sql("SELECT * FROM t WHERE xpath_float(xml, 'a/b') = 1"), + Seq(Row("1"))) + check(sql("SELECT * FROM t WHERE xpath_double(xml, 'a/b') = 1"), + Seq(Row("1"))) + check(sql("SELECT * FROM t WHERE xpath_string(xml, 'a/b') = '1'"), + Seq(Row("1"))) + } + withTable("t") { + sql("CREATE TABLE t AS SELECT * FROM VALUES " + + "('b1b2b3c1c2'), (NULL) T(xml)") + check(sql("SELECT * FROM t WHERE xpath(xml, 'a/b/text()') = array('b1', 'b2', 'b3')"), + Seq(Row("b1b2b3c1c2"))) + } + } } From d356b17f1dcab83fd7eecc93b79e37608f4bb540 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2024 18:35:50 -0800 Subject: [PATCH 006/438] [SPARK-50380][SQL][TESTS][FOLLOWUP] Enable ANSI for conditional branches with error expression test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This is a follow-up to recover non-ANSI CI. - #48918 ### Why are the changes needed? The original PR broke non-ANSI CI because the test case assumes ANSI setting. - https://github.com/apache/spark/actions/runs/11964792566 - https://github.com/apache/spark/actions/runs/11982859814 ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Manual tests. **BEFORE** ``` $ SPARK_ANSI_SQL_MODE=false build/sbt "catalyst/testOnly *.ReorderAssociativeOperatorSuite -- -z SPARK-50380" ... [info] *** 1 TEST FAILED *** [error] Failed tests: [error] org.apache.spark.sql.catalyst.optimizer.ReorderAssociativeOperatorSuite [error] (catalyst / Test / testOnly) sbt.TestsFailedException: Tests unsuccessful [error] Total time: 8 s, completed Nov 23, 2024, 11:50:45 AM ``` **AFTER** ``` $ SPARK_ANSI_SQL_MODE=false build/sbt "catalyst/testOnly *.ReorderAssociativeOperatorSuite -- -z SPARK-50380" ... [info] ReorderAssociativeOperatorSuite: [info] - SPARK-50380: conditional branches with error expression (508 milliseconds) [info] Run completed in 1 second, 413 milliseconds. [info] Total number of tests run: 1 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 1, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. [success] Total time: 11 s, completed Nov 23, 2024, 11:51:34 AM ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48943 from dongjoon-hyun/SPARK-50380. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../ReorderAssociativeOperatorSuite.scala | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala index 7733e58547fe0..69c303d4773b4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.internal.SQLConf class ReorderAssociativeOperatorSuite extends PlanTest { @@ -109,15 +110,17 @@ class ReorderAssociativeOperatorSuite extends PlanTest { } test("SPARK-50380: conditional branches with error expression") { - val originalQuery1 = testRelation.select(If($"a" === 1, 1L, Literal(1).div(0) + $"b")).analyze - val optimized1 = Optimize.execute(originalQuery1) - comparePlans(optimized1, originalQuery1) - - val originalQuery2 = testRelation.select( - If($"a" === 1, 1, ($"b" + Literal(Int.MaxValue)) + 1).as("col")).analyze - val optimized2 = Optimize.execute(originalQuery2) - val correctAnswer2 = testRelation.select( - If($"a" === 1, 1, $"b" + (Literal(Int.MaxValue) + 1)).as("col")).analyze - comparePlans(optimized2, correctAnswer2) + withSQLConf(SQLConf.ANSI_ENABLED.key -> true.toString) { + val originalQuery1 = testRelation.select(If($"a" === 1, 1L, Literal(1).div(0) + $"b")).analyze + val optimized1 = Optimize.execute(originalQuery1) + comparePlans(optimized1, originalQuery1) + + val originalQuery2 = testRelation.select( + If($"a" === 1, 1, ($"b" + Literal(Int.MaxValue)) + 1).as("col")).analyze + val optimized2 = Optimize.execute(originalQuery2) + val correctAnswer2 = testRelation.select( + If($"a" === 1, 1, $"b" + (Literal(Int.MaxValue) + 1)).as("col")).analyze + comparePlans(optimized2, correctAnswer2) + } } } From 3cdebbeb881407c94a75d0b8fff476a2c4645f74 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sat, 23 Nov 2024 21:46:41 -0800 Subject: [PATCH 007/438] [SPARK-50400][BUILD] Upgrade log4j2 to 2.24.2 ### What changes were proposed in this pull request? The pr aims to upgrade log4j2 from `2.24.1` to `2.24.2`. ### Why are the changes needed? - The full release notes: https://github.com/apache/logging-log4j2/releases/tag/rel%2F2.24.2 - This release fixes a critical bug in Log4j API initialization code, which can cause LogManager.getLogger() to return null under certain conditions. See https://github.com/apache/logging-log4j2/issues/3143 for details. Fix key removal issues in Thread Context (https://github.com/apache/logging-log4j2/issues/3048) Use hard references to Loggers in LoggerRegistry. (https://github.com/apache/logging-log4j2/issues/3143) Fix ArrayIndexOutOfBoundsException in JSON Template Layout truncated exception resolver (https://github.com/apache/logging-log4j2/pull/3216) ### 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 #48945 from panbingkun/SPARK-50400. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 10 +++++----- pom.xml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 4603ae2fc5548..902c9787adc56 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -189,11 +189,11 @@ leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.16.0//libthrift-0.16.0.jar listenablefuture/9999.0-empty-to-avoid-conflict-with-guava//listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar -log4j-1.2-api/2.24.1//log4j-1.2-api-2.24.1.jar -log4j-api/2.24.1//log4j-api-2.24.1.jar -log4j-core/2.24.1//log4j-core-2.24.1.jar -log4j-layout-template-json/2.24.1//log4j-layout-template-json-2.24.1.jar -log4j-slf4j2-impl/2.24.1//log4j-slf4j2-impl-2.24.1.jar +log4j-1.2-api/2.24.2//log4j-1.2-api-2.24.2.jar +log4j-api/2.24.2//log4j-api-2.24.2.jar +log4j-core/2.24.2//log4j-core-2.24.2.jar +log4j-layout-template-json/2.24.2//log4j-layout-template-json-2.24.2.jar +log4j-slf4j2-impl/2.24.2//log4j-slf4j2-impl-2.24.2.jar logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.8.0//lz4-java-1.8.0.jar metrics-core/4.2.28//metrics-core-4.2.28.jar diff --git a/pom.xml b/pom.xml index 9fa0b3cc8a4b7..4482f3f49badf 100644 --- a/pom.xml +++ b/pom.xml @@ -121,7 +121,7 @@ spark 9.7.1 2.0.16 - 2.24.1 + 2.24.2 3.4.1 From 8887d53117eeca7bb085cd4f9ebc1eae11e54748 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Mon, 25 Nov 2024 08:49:16 +0900 Subject: [PATCH 008/438] [SPARK-50395][SQL] Fix malformed URI syntax in Windows ### What changes were proposed in this pull request? This PR fixes an issue that the Artifact Manager is using a malformed URI string for Class Dir in windows. The issue is caused by using a platform-specific `File.separator` instead of `/`: Windows's file separator is `\`, which results in a wrong URI string: ``` java.net.URISyntaxException: Illegal character in path at index 88: spark://xxxx:57839/artifacts\bd3e1ffe-50d2-412c-8fe4-911ae160c251\classes\ ``` This failure is captured by the scheduled Windows build on `master`, such as https://github.com/apache/spark/actions/runs/11958030827. To fix this issue we just make sure that the separator is always `/` on all OSes. ### Why are the changes needed? Fix a compilation failure in Windows. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tested on my own fork with a modified Workflow that runs on PR: https://github.com/xupefei/spark/pull/1, https://github.com/xupefei/spark/actions/runs/11970330735/job/33372836765?pr=1 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48934 from xupefei/repl-class-uri-windows. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/artifact/ArtifactManager.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index d362c5bef878e..0aae5a43ca40f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -66,12 +66,11 @@ class ArtifactManager(session: SparkSession) extends Logging { // The base directory/URI where all artifacts are stored for this `sessionUUID`. protected[artifact] val (artifactPath, artifactURI): (Path, String) = (ArtifactUtils.concatenatePaths(artifactRootPath, session.sessionUUID), - s"$artifactRootURI${File.separator}${session.sessionUUID}") + s"$artifactRootURI/${session.sessionUUID}") // The base directory/URI where all class file artifacts are stored for this `sessionUUID`. protected[artifact] val (classDir, replClassURI): (Path, String) = - (ArtifactUtils.concatenatePaths(artifactPath, "classes"), - s"$artifactURI${File.separator}classes${File.separator}") + (ArtifactUtils.concatenatePaths(artifactPath, "classes"), s"$artifactURI/classes/") private lazy val alwaysApplyClassLoader = session.conf.get(SQLConf.ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER.key).toBoolean From d9757b5653b9b221dcd571354435e5be41eb8861 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 24 Nov 2024 15:53:23 -0800 Subject: [PATCH 009/438] [SPARK-50402][SQL][TESTS] Upgrade PostgreSQL docker image to 17.2 ### What changes were proposed in this pull request? This PR aims to upgrade `PostgreSQL` docker image to 17.2. ### Why are the changes needed? To test against the latest PostgreSQL images - https://hub.docker.com/layers/library/postgres/17.2-alpine ``` $ docker images postgres REPOSITORY TAG IMAGE ID CREATED SIZE postgres 17.2-alpine 03844845c1d8 2 days ago 261MB postgres 17.1-alpine 685d732c5876 9 days ago 261MB ``` ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48946 from dongjoon-hyun/SPARK-50402. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/jdbc/PostgresIntegrationSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala | 6 +++--- .../spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala | 6 +++--- .../spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala | 6 +++--- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 92a3e99586b5f..c3a705a1c7e28 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -32,9 +32,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.1-alpine): + * To run this test suite for a specific version (e.g., postgres:17.2-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.2-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.PostgresIntegrationSuite" * }}} @@ -42,7 +42,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index 7c9fc477dbb78..6bb6bbc36a2d4 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnecti import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.1-alpine): + * To run this test suite for a specific version (e.g., postgres:17.2-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.2-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly *PostgresKrbIntegrationSuite" * }}} @@ -38,7 +38,7 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val keytabFileName = "postgres.keytab" override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala index b6917df2d428a..c51ae4c9ed71a 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.tags.DockerTest /** * This suite is used to generate subqueries, and test Spark against Postgres. - * To run this test suite for a specific version (e.g., postgres:17.1-alpine): + * To run this test suite for a specific version (e.g., postgres:17.2-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.2-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.GeneratedSubquerySuite" * }}} @@ -39,7 +39,7 @@ import org.apache.spark.tags.DockerTest class GeneratedSubquerySuite extends DockerJDBCIntegrationSuite with QueryGeneratorHelper { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala index 56a83cc0a34d6..84a04cafb6453 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala @@ -30,9 +30,9 @@ import org.apache.spark.tags.DockerTest * confidence, and you won't have to manually verify the golden files generated with your test. * 2. Add this line to your .sql file: --ONLY_IF spark * - * Note: To run this test suite for a specific version (e.g., postgres:17.1-alpine): + * Note: To run this test suite for a specific version (e.g., postgres:17.2-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.2-alpine * ./build/sbt -Pdocker-integration-tests * "testOnly org.apache.spark.sql.jdbc.PostgreSQLQueryTestSuite" * }}} @@ -45,7 +45,7 @@ class PostgreSQLQueryTestSuite extends CrossDbmsQueryTestSuite { protected val customInputFilePath: String = new File(inputFilePath, "subquery").getAbsolutePath override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index 95465cc6e40c5..ce79618a011cb 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.1-alpine) + * To run this test suite for a specific version (e.g., postgres:17.2-alpine) * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.2-alpine * ./build/sbt -Pdocker-integration-tests "testOnly *v2.PostgresIntegrationSuite" * }}} */ @@ -38,7 +38,7 @@ import org.apache.spark.tags.DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest { override val catalogName: String = "postgresql" override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala index 75f7ede5bc733..649267788ad60 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala @@ -26,16 +26,16 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.1-alpine): + * To run this test suite for a specific version (e.g., postgres:17.2-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.2-alpine * ./build/sbt -Pdocker-integration-tests "testOnly *v2.PostgresNamespaceSuite" * }}} */ @DockerTest class PostgresNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespaceTest { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) From 4f8d1f575e99aeef8990c63a9614af0fc5479330 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Sun, 24 Nov 2024 17:31:22 -0800 Subject: [PATCH 010/438] [SPARK-50394][PYTHON][INFRA][FOLLOW-UP] Reduce parallelism further in Pure Python library builds ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/48932 that reduces parallelism further ### Why are the changes needed? In order to make the tests more robust: https://github.com/apache/spark/actions/workflows/build_python_connect35.yml It fails because of OOM. FWIW, https://github.com/apache/spark/actions/workflows/build_python_connect.yml is fixed by the original PR. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Will monitor the build. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48948 from HyukjinKwon/SPARK-50394-followup. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- .github/workflows/build_python_connect35.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml index 4b7a6b82b9527..ad250d95fb844 100644 --- a/.github/workflows/build_python_connect35.yml +++ b/.github/workflows/build_python_connect35.yml @@ -98,7 +98,7 @@ jobs: # Run branch-3.5 tests ./python/run-tests --parallelism=1 --python-executables=python3 --modules pyspark-connect # None of tests are dependent on each other in Pandas API on Spark so run them in parallel - ./python/run-tests --parallelism=2 --python-executables=python3 --modules pyspark-pandas-connect,pyspark-pandas-slow-connect + ./python/run-tests --parallelism=1 --python-executables=python3 --modules pyspark-pandas-connect,pyspark-pandas-slow-connect - name: Upload test results to report if: always() uses: actions/upload-artifact@v4 From bf893624161c017f80c0563b97a05fec455c9454 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 24 Nov 2024 18:26:04 -0800 Subject: [PATCH 011/438] [SPARK-50397][CORE] Remove deprecated `--ip` and `-i` arguments from `Master/Worker` ### What changes were proposed in this pull request? This PR aims to remove `--ip` and `-i` arguments and `SPARK_MASTER_IP` environment variable from `Master/Worker` from Apache Spark 4.0.0. ### Why are the changes needed? These arguments were deprecated on Apr 15, 2013 before [v0.8.0-incubating](https://github.com/apache/spark/releases/tag/v0.8.0-incubating) and have never been used in Apache Spark code base. - https://github.com/apache/spark/commit/d90d2af1036e909f81cf77c85bfe589993c4f9f3#diff-6a66898887ff9c00a900384bdc2a0c098a4e97a302fecf484b46814a5371210a ``` Utils.checkHost(value, "ip no longer supported, please use hostname " + value) ``` ### Does this PR introduce _any_ user-facing change? Yes, however, I believe these incubating-era arguments are not only misleading (because IPs are not hostnames) but also not required in Apache Spark 4.0.0 users. ### How was this patch tested? Manual review. **BEFORE** ``` $ sbin/start-master.sh --help Usage: ./sbin/start-master.sh [options] Options: -i HOST, --ip HOST Hostname to listen on (deprecated, please use --host or -h) -h HOST, --host HOST Hostname to listen on -p PORT, --port PORT Port to listen on (default: 7077) --webui-port PORT Port for web UI (default: 8080) --properties-file FILE Path to a custom Spark properties file. Default is conf/spark-defaults.conf. ``` **AFTER** ``` $ sbin/start-master.sh --help Usage: ./sbin/start-master.sh [options] Options: -h HOST, --host HOST Hostname to listen on -p PORT, --port PORT Port to listen on (default: 7077) --webui-port PORT Port for web UI (default: 8080) --properties-file FILE Path to a custom Spark properties file. Default is conf/spark-defaults.conf. ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48938 from dongjoon-hyun/SPARK-50397. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/deploy/master/MasterArguments.scala | 12 ------------ .../apache/spark/deploy/worker/WorkerArguments.scala | 6 ------ docs/spark-standalone.md | 4 ---- 3 files changed, 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 6647b11874d72..0904581d72367 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -33,12 +33,6 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) exte var webUiPort = 8080 var propertiesFile: String = null - // Check for settings in environment variables - if (System.getenv("SPARK_MASTER_IP") != null) { - logWarning("SPARK_MASTER_IP is deprecated, please use SPARK_MASTER_HOST") - host = System.getenv("SPARK_MASTER_IP") - } - if (System.getenv("SPARK_MASTER_HOST") != null) { host = System.getenv("SPARK_MASTER_HOST") } @@ -63,11 +57,6 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) exte @tailrec private def parse(args: List[String]): Unit = args match { - case ("--ip" | "-i") :: value :: tail => - Utils.checkHost(value) - host = value - parse(tail) - case ("--host" | "-h") :: value :: tail => Utils.checkHost(value) host = value @@ -103,7 +92,6 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) exte "Usage: Master [options]\n" + "\n" + "Options:\n" + - " -i HOST, --ip HOST Hostname to listen on (deprecated, please use --host or -h) \n" + " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + " --webui-port PORT Port for web UI (default: 8080)\n" + diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index f24cd59418300..87ca01fe82a97 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -70,11 +70,6 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { @tailrec private def parse(args: List[String]): Unit = args match { - case ("--ip" | "-i") :: value :: tail => - Utils.checkHost(value) - host = value - parse(tail) - case ("--host" | "-h") :: value :: tail => Utils.checkHost(value) host = value @@ -137,7 +132,6 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { " -c CORES, --cores CORES Number of cores to use\n" + " -m MEM, --memory MEM Amount of memory to use (e.g. 1000M, 2G)\n" + " -d DIR, --work-dir DIR Directory to run apps in (default: SPARK_HOME/work)\n" + - " -i HOST, --ip IP Hostname to listen on (deprecated, please use --host or -h)\n" + " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: random)\n" + " --webui-port PORT Port for web UI (default: 8081)\n" + diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 8bc7445d17c71..d828436e77340 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -59,10 +59,6 @@ Finally, the following configuration options can be passed to the master and wor -h HOST, --host HOST Hostname to listen on - - -i HOST, --ip HOST - Hostname to listen on (deprecated, use -h or --host) - -p PORT, --port PORT Port for service to listen on (default: 7077 for master, random for worker) From 658879e0aeef62ab02b086ab9f42b7ba9b6c96fd Mon Sep 17 00:00:00 2001 From: vicennial Date: Mon, 25 Nov 2024 14:27:18 +0900 Subject: [PATCH 012/438] [SPARK-50382][CONNECT] Add documentation for general information on application development with/extending Spark Connect ### What changes were proposed in this pull request? Adds a new page, `app-dev-spark-connect.md`, which is hyperlinked from the `Use Spark Connect in standalone applications` section in `spark-connect-overview`. ### Why are the changes needed? There is a lack of documentation in the area of application development (with Spark Connect) especially so on extending Spark Connect with custom logic/libraries/plugins. ### Does this PR introduce _any_ user-facing change? Yes, new page titled "Application Development with Spark Connect" Render screenshot: ![image](https://github.com/user-attachments/assets/c1d786c6-a545-483d-bb92-679d90f7e56f) ### How was this patch tested? Local rendering ### Was this patch authored or co-authored using generative AI tooling? No Closes #48922 from vicennial/plugin. Authored-by: vicennial Signed-off-by: Hyukjin Kwon --- docs/app-dev-spark-connect.md | 243 ++++++++++++++++++ docs/img/extending-spark-connect-labelled.png | Bin 0 -> 131995 bytes docs/img/extending-spark-connect.png | Bin 0 -> 118823 bytes docs/spark-connect-overview.md | 2 + 4 files changed, 245 insertions(+) create mode 100644 docs/app-dev-spark-connect.md create mode 100644 docs/img/extending-spark-connect-labelled.png create mode 100644 docs/img/extending-spark-connect.png diff --git a/docs/app-dev-spark-connect.md b/docs/app-dev-spark-connect.md new file mode 100644 index 0000000000000..218edd331aa94 --- /dev/null +++ b/docs/app-dev-spark-connect.md @@ -0,0 +1,243 @@ +--- +layout: global +title: Application Development with Spark Connect +license: | + 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. +--- +**Spark Connect Overview** + +In Apache Spark 3.4, Spark Connect introduced a decoupled client-server +architecture that allows remote connectivity to Spark clusters using the +DataFrame API and unresolved logical plans as the protocol. The separation +between client and server allows Spark and its open ecosystem to be +leveraged from everywhere. It can be embedded in modern data applications, +in IDEs, Notebooks and programming languages. + +To learn more about Spark Connect, see [Spark Connect Overview](spark-connect-overview.html). + +# Redefining Spark Applications using Spark Connect + +With its decoupled client-server architecture, Spark Connect simplifies how Spark Applications are +developed. +The notion of Spark Client Applications and Spark Server Libraries are introduced as follows: +* _Spark Client Applications_ are regular Spark applications that use Spark and its rich ecosystem for +distributed data processing. Examples include ETL pipelines, data preparation, and model training +and inference. +* _Spark Server Libraries_ build on, extend, and complement Spark's functionality, e.g. +[MLlib](ml-guide.html) (distributed ML libraries that use Spark's powerful distributed processing). Spark Connect +can be extended to expose client-side interfaces for Spark Server Libraries. + +With Spark 3.4 and Spark Connect, the development of Spark Client Applications is simplified, and +clear extension points and guidelines are provided on how to build Spark Server Libraries, making +it easy for both types of applications to evolve alongside Spark. As illustrated in Fig.1, Spark +Client applications connect to Spark using the Spark Connect API, which is essentially the +DataFrame API and fully declarative. + +

+ Extending Spark
+Connect Diagram +

+Spark Server Libraries extend Spark. They typically provide additional server-side logic integrated +with Spark, which is exposed to client applications as part of the Spark Connect API, using Spark +Connect extension points. For example, the _Spark Server Library_ consists of custom +service-side logic (as indicated by the blue box labeled _Custom Library Plugin_), which is exposed +to the client via the blue box as part of the Spark Connect API. The client uses this API, e.g., +alongside PySpark or the Spark Scala client, making it easy for Spark client applications to work +with the custom logic/library. + +## Spark Client Applications + +Spark Client Applications are the _regular Spark applications_ that Spark users develop today, e.g., +ETL pipelines, data preparation, or model training or inference. These are typically built using +Sparks declarative DataFrame and DataSet APIs. With Spark Connect, the core behaviour remains the +same, but there are a few differences: +* Lower-level, non-declarative APIs (RDDs) can no longer be directly used from Spark Client +applications. Alternatives for missing RDD functionality are provided as part of the higher-level +DataFrame API. +* Client applications no longer have direct access to the Spark driver JVM; they are fully +separated from the server. + +Client applications based on Spark Connect can be submitted in the same way as any previous job. +In addition, Spark Client Applications based on Spark Connect have several benefits compared to +classic Spark applications using earlier Spark versions (3.4 and below): +* _Upgradability_: Upgrading to new Spark Server versions is seamless, as the Spark Connect API +abstracts any changes/improvements on the server side. Client- and server APIs are cleanly +separated. +* _Simplicity_: The number of APIs exposed to the user is reduced from 3 to 2. The Spark Connect API +is fully declarative and consequently easy to learn for new users familiar with SQL. +* _Stability_: When using Spark Connect, the client applications no longer run on the Spark driver +and, therefore don’t cause and are not affected by any instability on the server. +* _Remote connectivity_: The decoupled architecture allows remote connectivity to Spark beyond SQL +and JDBC: any application can now interactively use Spark “as a service”. +* _Backwards compatibility_: The Spark Connect API is code-compatible with earlier Spark versions, +except for the usage of RDDs, for which a list of alternative APIs is provided in Spark Connect. + +## Spark Server Libraries + +Until Spark 3.4, extensions to Spark (e.g., [Spark ML](ml-guide#:~:text=What%20is%20%E2%80%9CSpark%20ML%E2%80%9D%3F,to%20emphasize%20the%20pipeline%20concept.) +or [Spark-NLP](https://github.com/JohnSnowLabs/spark-nlp)) were built and deployed like Spark +Client Applications. With Spark 3.4 and Spark Connect, explicit extension points are offered to +extend Spark via Spark Server Libraries. These extension points provide functionality that can be +exposed to a client, which differs from existing extension points in Spark such as +[SparkSession extensions](api/java/org/apache/spark/sql/SparkSessionExtensions.html) or +[Spark Plugins](api/java/org/apache/spark/api/plugin/SparkPlugin.html). + +### Getting Started: Extending Spark with Spark Server Libraries + +Spark Connect is available and supports PySpark and Scala +applications. We will walk through how to run an Apache Spark server with Spark +Connect and connect to it from a client application using the Spark Connect client +library. + +A Spark Server Library consists of the following components, illustrated in Fig. 2: + +1. The Spark Connect protocol extension (blue box _Proto_ API) +2. A Spark Connect Plugin. +3. The application logic that extends Spark. +4. The client package that exposes the Spark Server Library application logic to the Spark Client +Application, alongside PySpark or the Scala Spark Client. +

+ Extending Spark
+Connect Diagram - Labelled Steps +

+ +#### (1) Spark Connect Protocol Extension + +To extend Spark with a new Spark Server Library, developers can extend the three main operation +types in the Spark Connect protocol: _Relation_, _Expression_, and _Command_. + +{% highlight protobuf %} +message Relation { + oneof rel_type { + Read read = 1; + // ... + google.protobuf.Any extension = 998; + } +} + +message Expression { + oneof expr_type { + Literal literal = 1; + // ... + google.protobuf.Any extension = 999; + } +} + +message Command { + oneof command_type { + WriteCommand write_command = 1; + // ... + google.protobuf.Any extension = 999; + } +} +{% endhighlight %} +Their extension fields allow serializing arbitrary protobuf messages as part of the Spark Connect +protocol. These messages represent the parameters or state of the extension implementation. +To build a custom expression type, the developer first defines the custom protobuf definition +of the expression. + +{% highlight protobuf %} +message ExamplePluginExpression { + Expression child = 1; + string custom_field = 2; +} +{% endhighlight %} + +#### (2) Spark Connect Plugin implementation with (3) custom application logic + +As a next step, the developer implements the _ExpressionPlugin_ class of Spark Connect with custom +application logic based on the input parameters of the protobuf message. +{% highlight protobuf %} +class ExampleExpressionPlugin extends ExpressionPlugin { + override def transform( + relation: protobuf.Any, + planner: SparkConnectPlanner): Option[Expression] = { + // Check if the serialized value of protobuf.Any matches the type + // of our example expression. + if (!relation.is(classOf[proto.ExamplePluginExpression])) { + return None + } + val exp = relation.unpack(classOf[proto.ExamplePluginExpression]) + Some(Alias(planner.transformExpression( + exp.getChild), exp.getCustomField)(explicitMetadata = None)) + } +} +{% endhighlight %} + +Once the application logic is developed, the code must be packaged as a jar and Spark must be +configured to pick up the additional logic. The relevant Spark configuration options are: +* _spark.jars_ which define the location of the Jar file containing the application logic built for +the custom expression. +* _spark.connect.extensions.expression.classes_ specifying the full class name +of each expression extension loaded by Spark. Based on these configuration options, Spark will +load the values at startup and make them available for processing. + +#### (4) Spark Server Library Client Package + +Once the server component is deployed, any client can use it with the right protobuf messages. +In the example above, the following message payload sent to the Spark Connect endpoint would be +enough to trigger the extension mechanism. +{% highlight json %} +{ + "project": { + "input": { + "sql": { + "query": "select * from samples.nyctaxi.trips" + } + }, + "expressions": [ + { + "extension": { + "typeUrl": "type.googleapis.com/spark.connect.ExamplePluginExpression", + "value": "\n\006\022\004\n\002id\022\006testval" + } + } + ] + } +} +{% endhighlight %} +To make the example available in Python, the application developer provides a Python library that +wraps the new expression and embeds it into PySpark. The easiest way to provide a function for any +expression is to take a PySpark column instance as an argument and return a new Column instance +with the expression applied. + +{% highlight python %} +from pyspark.sql.connect.column import Expression +import pyspark.sql.connect.proto as proto + +from myxample.proto import ExamplePluginExpression + +# Internal class that satisfies the interface by the Python client +# of Spark Connect to generate the protobuf representation from +# an instance of the expression. +class ExampleExpression(Expression): + def to_plan(self, session) -> proto.Expression: + fun = proto.Expression() + plugin = ExamplePluginExpression() + plugin.child.literal.long = 10 + plugin.custom_field = "example" + fun.extension.Pack(plugin) + return fun + +# Defining the function to be used from the consumers. +def example_expression(col: Column) -> Column: + return Column(ExampleExpression()) + + +# Using the expression in the Spark Connect client code. +df = spark.read.table("samples.nyctaxi.trips") +df.select(example_expression(df["fare_amount"])).collect() +{% endhighlight %} \ No newline at end of file diff --git a/docs/img/extending-spark-connect-labelled.png b/docs/img/extending-spark-connect-labelled.png new file mode 100644 index 0000000000000000000000000000000000000000..94b8cfdc024cb30b4092adf408307b7322647d33 GIT binary patch literal 131995 zcmeFZS6Eb8+bvqQtv2S2f?y&jpoj_x5>yl^5m0iFphSt1B!d}I0WFeBQlezZl2sH0 zL^22n2#6%fIiE41``>4uyK`|a&hu=)?)IZBtJa$H4P%UV%;k1T`uw`p467*=$~vmV zSy>8Yl@o=s{L?=x@k+Gs-Us+?kG6!Y6oq2XMxorgN1;sPrCVJTiuDl+rALcG5qwUe z(BBBiyex!2Sb0tI{8`EZ`QNLYNH4rXcSAzOl0rGWpZvSbHeB>3UR-5Gl{&YoYt6Q` zN4EccrG*zSEiqI6jQ zcYwgi*FVp>M!pIZ>hSrmz}Nlz>#fXs@m2r*fZuJ+S)7*r&(H5Yo{j$B-h5KLlKy}G zjPhp(Rr7!S^j7=-{KfyXIR6t5|38i#->*n+{Q9Z;;$j2?t%s3mDM8*c+OU-J`Iqh$ zUq#y}G5r_b^45sW>!5#_Nm~p5;$PY0EX3l@C;0H*`@wJIg+J=+pA9Z0(2!qY7iIJ6 z4dUc=mYt@iriP7aaTQm|kCa<=)kE!t)_=b9gyu`!SV|<8ol&ykl_}S&o~&QK_@?ZL zf+st@SiExA4y;{#VevmVpM-J$ub;m2&~pFRPg;dbm$~%HEvEh5YyQ_C{(tlrX*-z~ zzvKIU7Z(>{yWz8X$tnlsBD^CbBP;f{YJT|eA=70cw=zPClb@eIG=H(y?5U`zTFjj` zgR&suo~Eqy)%1sBFTOY$rS;(shy0tTNAUi#&zql)5wz|J`%|CHsh(-YAshDasLq#Hp%j1R8B0^H(~QrqksPEe<@HE@;*Dsxn-X)4Hd6cxb4iyDBo( zu!e(4)cFd(>5oezze`hXhpzD&R)6Ul+_a&@k6Y=|rKgX!aYiKRmM}1_r)2w{IddjO zw`BX=bZ-Lw$+PPG7HAOasq-yBrK7}8sVURCCP`N^BP)xMDXmGsVLWzvyf5bA!-oeN zo0^))SBWKbyEF2qT>HG*q$O9qvn;qS-*tm7pZ}qbA}@BO4|i55q#7!u8rREq6#E?G zQly;vcrzPnWg^YBTu^Y^IvCGtcdA6UG8hFyC_E(XYkC2^3 zjJ24U*jQhBi{b3#kZ*>wC$n%gwusT`g@#*GmW>)Ms=d%tsU_EGOP}(xWy?4;bF;H^ za&(Wr7_HY2zW(LuUOtC}ufd|4cUNstG%$E0BrH5ur|EJ~EzMY5^XR9`J=Ia$7#O1C z)w2u{7viqVDGA$>>tpY)p*Q^VJu%a|S218>WN@&2)o#aE%a*TvI_vJL=`yFlBxvOgcL~UHD#gp&+S&$8a4JN5=T5e*sfkyg@hMzO5ob82_oTps zX~uEvd~*Gd{I2Jl|FT?))3w{Vl=o1J{rM9jqyr6)wQ^FWrKK4}9N#_Iu=_C8e)yN0 zURjU{cAuK8S4vAyPiMJa{_6DfXqAliuK2fzh?hckNn!e8>K@F(4PA05a^WjhtcY@& z8AsG;W^c}|Og^}=z;k!P1s{%XEUI|$YK?M)y8P=iD-NlpacR(LQU!1Rd5%1a_Y|3y zc>Ve{-a`4w`Y%pk3=*;n$y=CLqgB7Tyk9^-;Pdso(|1>@*iX!lH=2nDi_=nWw4R~L zPLFnv6|o0@x*mL3J@XBclNQEnRD0mak@Bu8T-5z@XV1Qo^gG<}&b(bR)v}{_8?Qn6 zDO^79wD(3Cc9|gg$J-B$c2$J>8S2l^PGMisRfq9E6{sT}?d|)yxw(-VlE~3ZSvUKB{r%K9(bS}nV{e^m)*@1H)yJeh!!oiqR%L&;3U`nF zSkGs>mM?CjLUzNSdf(ZN{7&U^aBzsR9coKtHmXn3ZAdo@?r+ZFM0`Klc`Oqli zQuK$fJPi&G&d$#E=GGLciB^yt$R3e?A{MZqWNZ5_&A2{<Ge#PH^D>ro(xHB>c+o!Nye6BUxy?4E0{FSIR^oPH5CCG+9 z;!uoz!=;k+<->>5Sb_#Dw8t)iYnH(X!x)pswBlgVyo$(G6x(HN$@Ruoi|u>x<3%>P zZBz%AgOa{nf-dupchWS|j8)X%-8kLf-;ZMb6oq$tkl|4_|^j~or`b5YL9Yc0IPwu_Iv})C=rffU8pr9aH7)oW#34bFF zSM>#*4fS3TAs1J>X3d%^l+rkKoyLInZ$(Aa<*U|LzrC7&?V_VnjG|Pb2a_E35*f>e zaMeI-PgAYS&lfwU|4qKc_mD{%zt($7sXxC=W%5-f(YaAMlZKQc!zSc5DR86$Vl|)C z=`GC95&RHNpUR6MuDY{gjePu-4CREe7svFgvup--bh>?gw%;{ECOG~juaP(>`ex90 zUn705IeE#nDf6@H1`3VV;H9`G@mgZP-(Inmy69Tcg|-7Z6ISRjFD|mxM9H21{AiopjT>(fgCSVv zE-d==>_{0(h&FYr+DIvXivj76V>DLD=ri|MU-0LPb98h}aanL?61JD$z$|phpU-5p zggfsI;6-8qzg1Vc0i9;I%C$+mkNolD{Y^0*OecHrsTaK1+-u|2858aeloSUDB(Ge% zW21K9+qXwjjq0QkUp_1Xf`ZYwXAb4JuUNW)$r^HMYP%ol;Gm3Dmi?UBJOi^ za!9+cAypDxOi5Y!IcZ@JAO3?jTd8;Q?bdz&{Ok(iD@Wh1QD3{8WI8|QVBHB5d{vQ<|?yjIkpYuCGvm?{(GKO@Uv&{)h1|iqi!14Cq?1k%RMR%QGZcrf@&iJnHd?^zXb>= zZpo$n>gWhZ>Yd)Zcdu%mvs23O`}e2F9Z0=A&NIDr(;BsjQ+4s`CTxzZQ7G08Z?E2s zR*W;WKKl|`;00EG^4t;`zg8YDN#EC$rGzx8LZcuR-@2>(^g!N1?xXF8UhXM8;WV9s z@|j{kX6S1z)Zgq<$il zi$8qWd%=MeFaq~)-J+0AN*W-I_3^y&-aUIx1qxaxu{eSk#3TRDG8WiXgq)l13iaHI z=0URa+@q~-Gzn%Q+qb#`z}KV)#;K+r*DiRXlD0>!M=!v-|sx3;R zG$Wre;^l10uw10{ImziDQI}@Jvhr)6A00$_qlE#4_o8UlAjY?D-5PfMdgU7wJks0H zj>kTB+Ru!e5GYLo839X@1O=2sF5+UbA9rxHeAMPnzw$Nrjs|h8pd%|sv7Us zu3umKU>zf=Q`ksOquN-*`lN8(On1~Miw`_G?P z+_>`G{8r$X&J?4%!-2vMia_?XyLazahEYR*WLU$R8FDJ)J)Y5+t^D~CkZ=TisQgU=$n472|NV$23=yQ|!+@-d>ZCHy@3r@TIX)Z42AQ zO9>h|*PObrB>FMJrouu(qfIs~-L-LQ4OmjZxtKV$^oCuTBN>}nFBqT{Uf%o#_ffQb z<=S!t5XqSPuC7KyZJ!BH3rXeC|F%omaq<%F_{~4c*8M*ORGH47KfjKV@1nD_^Ggw@ zOti5`)FZV%Is%u0=p<=ebf#)`AJO1*?8jtLip4#dO)TGCrks*b+bof``St>^2Vck6 z7jodyZi?do@dk~AngZ^hYitGwcbcE-B(T!=usVm499?E+CL-RHLBJwRNJyw{YjC}B zZ9}RNq1fQALIs(J4<9yc&dwlJqql1VPhcTFa;!#4^E%K-1+5i1R*vOu+dAWN|K7bf zAjt%F*9@a8T?g14GOW@$TBLJSC8xP;A)+wCuqIm33!Ce(CnC%Y91!&-eCIK}E_5Hh zUV#%Q-l!z&P5((QyCxOz4NOtfap+u1-$B{1a;$8M)6C7W{w6iV*<)D|!|DTbfKBKp zK3Ib|v^!N37wjEhBwvQzNCY-bXUra`L&IfZF1==swAE+(&-eWNaF-`Y?e&@aqaa9c zpm^}qX5sBUwQ)wNe>#I*6u~ZPvTV%eQ2i>3BQL%PL0qkV?|;ZN!H@8d_ly9dIt+Zq z&ym+}aM^11l-a4yUswuEsU&@Z?ANFPX~&y@;_6GPAL?d~D6!gRN8PCkxsTAi9&X(I+ly z6Vf&Warr6A9g+Q|q=d6UhERi=bsILMHxvLXR70CMq>}W>@Z&}M@xSVkM+#B0tm?Ur z0=4f*Vj!eq5Q2$pj^X+_qJG?(@q_{c)TK2z2&9et1N?e^w^JBaPqj%DCEh$UVgcI4 zSgY&qhKrNi4_rEhX60d%wSN8j;^#b}HA$dUDoklpc^1FFMp<=No|~CHhSk)zmD%lS?Y&}(Z zW(g~_9nj2m1TT8|El4<~Ol0OOZR6Y4=^p3;ka(k> zb(8yt#Z}(~jMek)9s+`sDtZf!VYL9}WeM zj#7Um+E|b%GzHM1Y?dg*SH%OZwA$v89s4z7IZa$I8A@Q__IFAamoUH@{67wM8-BI&r##zf7FVkWn^TC z67nr4c%h_L_y(Y0O`IBk?EveF)mzJbIF&-2=IOaLQl={XA-|xSY%^+H*oN*6NFjqJ zq@7)M?7#sD)Pdv|TMu7(_jVT(6RWV~bVJXI-Ex)u5lLTK#=`VltnEj;UZd`Onh}{n z^46r8sG)j&nh{*fETjzKEm=h)+jht{oo-L}-X&Fch_Glc-o-x4B%i zpFdJsQ^PB5SspIsUpxG^wzk}*?6Yp}X&W0G$+9bI8~7L0MET0LB-Ap`w`5or4*%pi z@3G4~Q@qT)pP2y*E~%7AYhxK5h4-HTR;iQqG7Fvx$fj_xQ;99X9z&4kft zp!xaQT_?)*67Lt(4&6)U~pEo|LTdZT9xs@tt`TI#t(8o=iHu1UM8y_8&cW2-Y zG4?-l^$wWq$j3S@EI)&wl|lBz<8J1$!NE&dLX8t_6zZxa@yYfyI#SP%jAW#2BVG_`m^<*>=&J*pRezRHbi}7vtdAqHDbhedNfLR z3VS%+;iq|R&Q7FGTT4qTFZ07x@i0%U&;P``%>&i^a!;lX`&zOv|U@K;``SC0XYx= zk31Gv2-Xq)dcB^Optt<3V~D+O1a&{{G3oI9*`51^#p$TxwB#0!KQU&f}g?tAJ z=mKStLz}P3b#kCqg3XMTi7v3%ujOvDt2`Cx=A1-5aOsUtDzUG`B%yQ$!yd#g#@u;xT7#`uGy_79%X9Z#Nz8fn+&pathJ%ZsT{JvJkcp zmg`n&l&%=3x_b5MN)SD(z6RcsFPl+9RCDZ8b?ui?!Jy^9On!j~p_VDT_d;nvUs1^Z z77i7tr!hSNY%c-aq4#&G|IoC<_^+ZZ=+$!2ao(y1vCA+M{fcfp@B{QXgibY?PM~ZS zUJnN3@5Ky>sArjg59p^HAZ0xLBS_;$p~#BR^SyUBh`fyj`?ym2l2;BgIwFKOhI<-) zQ3A{0QJ0aKl@;T%Fpo;>JKRwc3G%F-W6xW>^#ILV*x{znVRcn>1h-m@Gfu*J5c*>v z?e_?g&LBFNsBA#T1;>o5jl1;vjxV?7VbD`*S?+Q|l*t4Nr7_ic&OWAY6cKZ|zxSO1 zfCNf-3<_c~1W_cl94LEx&B|M0CG+#=h>e4i@WoHlISMNj1spQekf2-Q8;zJ_>Bh7u zah&2VxuK%abGMpr+`3i5K$nsqC(vo2nnfNbXt%FXiQFpwJtT@%a$&7A!ziPtJ~>iX z_+Mc;OkdbBR`_p^$DJFh!@hU2%qL(WV}SAOXZq4pu^0v&We)0%J%QS#BO?z>jx;6s zf8kzE8aIge!;vz4Z0MNdBcZ~<$IiKwTa(wpf(cd!lq%qSF^Lq8MmU4hOi7l8lJ2jz zIrkYxY%)M%iJ3A)Fv!b?Ku`g4O8795qClK0F<*=V!pPUV7K`}fb7(DyWRGFNM@bQp z7849_Q5R=BJG&@@iV(t<0~)))3BY2^Qn?_ksb*ST4|oviu>sAU2$rgEwQiR-r`AqF zbU{u*R(OId+}^(C0jgwjrZB_}P<_9eBw*FOf3pp z1`!D{H;BiZ)ts)D)ov_|M_!c)WO`s&m(_ zUB~T54Pq8FAGo?UUOb`J{`2Q6C;(u{=L~?c!r})S>}R7LA3uS<@jJ~9 z6;`B)4$uvwz!`u9b64N|)w*xLrr*1F79EcW<-v5hq{q<$i9j2Tjg&Udi2+*>2`YSA zzQJ5`1)!m%nct*_q|rXjAGt{a&^qy;ho+tPqB&4e7@%eAD&YeL~ z?|~Mh`tHVSbdjx{&5aP;NH$|cXJ8rM1O}zrqye=vrs!Lz@j79q?b{>J_CHPifIhlA z;a*x?Si;s+{Q)Gtac{cAqp z6LL8+fn_>?Um9X9;?CuIaTAg{<}bAQEx7OfwPN+zgC~Kb!V8Tssk`R%g?{s9T322C zrPpT^(}6a`AedD^l0WV=ZAExRvlxedu{RlJLOQE~wAh|%_RG}>k@QeeBgNqI96Fsn zPz@P%qfZrlZwAulvdr4XrF^6!!RU-+kWdDiTPRV?@VB{}G$Jm(fWpcO2?{*%u=LC< z1aLtJp#&ZDUh-VKa~*}J#EadNGz1>)0t0LS5+KSM;*PSW4w%Te)!C_^8rgvnmgaOO#6Ja8aPB7Qi|Cri zjcU1oAvr2MTG2c)@OuMHlzFr7CH5}@VtyC2)x`cR^LBCTfo65A*Q0Uu%36J52l3Cz zU@a;uE7Pk-lO3y7GyBZohhZh!belC3Fnb*GOC>PN9^(^Vzl|oPoxN>sn@vPC)au^r z9F4+t@{A{?*N9C5p_q$Ac8315?9|2o@}z8wAHbw)vYtD_Wk;t|@WjI#=H_mx`X7<0 zyw^Vc<6l%(UXH;-Zvri7%wWpQ)9=-*JtY=bn3XZ*t-KX8^`uCrl5`Wq5Bjo^YPxBl zYMSxR1ipZP0MaOkL_ua@hc&Wyw2O6gk|6^~6*bL49j1+nq*;s@44~)oq-DSqP-20J z&AP03?)JF!=1SDNG4zOoM~)n+)o4S-@rsEt@^1d~jEt`G7Ur{1C&EB_3r>_>A;c!D>WZra##j3q+N@BO033O`)A=?`%m(@0l0l4uR ze?PL0Nl+d$x;jtM`8Py=*O~dx5$s*yG#~R;Xs(foSk+p$f!R9xD)!SE?)^5*FxEmu;;tj?6UieQ1lVyVEiRMlQ}w70Iz z(6b91xpA}FuU^wqB;j$)8mVOu0^%WQysjHN$7|Mnr83!d4&A{mu^)4^I5Ob}O~+_8 zphmFFm3b|0>3gEKw($UzqH^prD;2agrC##))ZIFa0m>WcDQyPH6;sZ30J*TymKk^CPGh2)NL$ z<-2%4^{WwuV*c;6#rO+m0wPntwouFLcjRKRU-D=_C~aPk!b|?2piMtlay^%ll9Cf7 zNOQs3_qUg0klk>g5ulYU6)}Ebc}#&AJk!jIXskXWPWIGFz^CKZJ@O?_^Ply#Rw{b!Wt5cI^?P$*1HkV5sFduENyxq6?Rq=m3_c68KP$D2HW=(UOic-5 zhg}P5s{;Pu*T4Wb6@A6KW*i!KI3U*r+V8U91R2q};|+J`Fs_n%e&jT)GHF*P@LILr zlQP$DxhQJ@#tAbM1B_1fl=s9}RD~WDI(sotu*PuTm;eN1Xt!sPx*p~d%?J?TZ|2__ z()f<5xgC;`DLc^Xxm#2b0{SRMe|c>&F9xA|hl2d79P)=nM3Q~xux9SAzL8)US@=wU z>}Yp}N>WV_AL={@x~*dGm8_dm5SJ5U{V%cs?yyesUifp?0sf*;DYxMtXuqkJozgMf zG}u$3Fy^jEV;7fcR2oZKc8lNDY5wt60wOh@L8a^>>ZZ;BrqrXK={4T~HWi~|K!x7A zdGkIqGc!y}t1H{9!OEc=DxfG{6BdW{BGqO<17f)fd_lyYMn*RfAjNkc9eB*0`|#-O zprwb02Lr4G_L%>YDdi5HGaz~jFOKSZBO01AEYAbOyiHMnm~T7QMqN9QI}=a(6sWQx zWL(SvRkItP^BVa=x=4jdAW+mr0}VzN_zT?PV7PRkD)f1{;JhI)Y1K!9Ca5M}JNJaH z0lC-(Iz;>cC|#k1pPzNxO#DPBdf>MA4~XfwOt!ibZ{q8^5#8q(*$8_ez?Lv9LUS+` zxkTa?*nqeJzzwP~2Im?hx&d;r%+LXMr2*o-EMbQYfk02n7EaJYtIM$0GCJrLq zH+lg3rGY2;3-)GaS#^g%cB0apr@I%2<-^0nG(SJ3*qmC*o+!GtYf}W~P7$kLsJQ!% z2^mc8*kpoY!OXkB7%Pg~y{z2(SOdT)zhAroW0YX=G=Ga@DB@Tp&L1;!!l*F`l7%e= zgB6{FMrbEb_FHBfLZPlgYa}`7&Md4-bo-m7C>Ga7R;0XBMmy_ZP5lBdqA~B)DNVR9 zDp9vEjZ=#0fI#dq%)|#FSkYDcFieKHTgHJ&DK@$6$O+p`sNh6OWHgxp?=fyZ2D+B~ z@>iPAsI`?kV36L8y=MaS54w^1wxDe(_q|##Zb&}mtc@AQj7wkM8h2A^* z^TTrDXu1#x)yM$80WuN*(9`X*mI@ZhK@W%n${>qFB(O(Y4{lE2gFhu%y=K1X2GD~5oF%om|jLRa5{`4MeZC!iyhhyggzii#{eFTFhRpdCJLHqAmqr% z2OZ~d!8iiw4GM@M*M0$7$h1$)Zwqmuc1;0W#Gi_c$yB{w?0*Of}V_L)m; zr(%MJaP9Pd@h6ucXt35h+wNraDq2*v35Ft772YsV4yB2K7C-!goEip2AO@}9!rIzh zsS#{M5hOGPIG0Jt_ENf8i*LQLB5*n}7ZUARdc#XHVMop2gbBi2upi}um^QX?$bT`k zgBGBWZEFtk?EtuMVj5g2m*7}B)^MeK!AHPSFAiP>g2l0g)&LDt)6*PyA91B1;1b9r zJA)l-rZPHGCfbVuDe-zBWjFy9>}+kxV1!I%Af@c?9ACMPAq*mG7u4!Z^Y#aT1KjNF zk+ykUaH1ZDsD$#AbdhIcJ=$Iva_sZLZkw?lg_y?u?AqGxy}dCI9-{ENVu=Z>1?xGx z)=qpMpJv+>90S*8nL!mMXiCRh9g#56S2B`xu`>NHPnd@QBZO3ELGN{*>Yy)?-!`P% zYc>t{3qcSk#zNeM?v1$2O)%qHv*GjdG#EpvV?`g*1vDzq4m@Fg2D&a%*rkYbmneQm z6U+!{>P1UOkXQ7a1h0eBO|j8_KCtE`>PW%M(RKg)Qx1d_0alV-X+-&1`roAJ^{Lx` z@pZ0pP$Aa9uX|@hU?#*AqH%{SBf(w*0WFpo{CZ9#%7I}gXe3}&sBblrkTb+qS-RI% z)7#(QpVSmgpYv`o4!K8RzQ^MWB~%re=3(i8+UZs|1h)GcBfIRusd*&~J&oDkwwIux zqK{GEey6K%O%;{Hz+sEL!0qLtpW2VEfM%s>H4Hr0BJz=)~aUryogl`Er>{tb!Ru81}r zYXeh?#6O97!~Y$(u{D9~;geoHVGu}pL#W%ABZU5eg#y|zujAwmV8ns@4Qap8CJbA0 zb94{?B=7)b(y%JR9a3^!KQYuD*st~O-MjC`Evg^oX&8)!!=29FrXal|?V1njg7a#=H?l z*9THg=yCX%tH3!pF6h@2Qp>Tc)+;OsQ0#^h_TQL0SN*6 zW2H7h%mX7mByBl?&^|6rr-`0OUifAir?zF|#*IcF(;cUV?K=Z)LX5GoYELXv5|k6q zpu;I*XW&KF2k2`&O#CjbG>5Ss-Be@1Btgf?!8NIk;3y~9pJ2?`<~KJ3=#YOdf*Tx%Q3V|=*GdiTp7iKP^xQ#jcp|0z6(N9z z)_fYqgvs|!A)hQ(@KO4Uf2t*{SAx}s4D-gq=DucG){Fgh>&YSKO;T3I7m3FW*Ay38N3V$&J*$iQ7dgGT;q8Ic8S|A838i!{%b5>HC*DisO z_Y^FQ0!U+Z$hkG)RIh@r8#tmAhym>*;wXh|hpuH|-kbpH`0}XkBP3&Fdb54+eGd>3 zXM!*bnj=75dzy2y#s^wv?O{`tspqSA0 z{Xwt0QPcBc#E)*OjRCTY=*^w&3ra`x3yVr9r77Bl>uVA;MQe@x!1cjOWQaP09zxTxV%FCmCp-2c0MJG){p z0Njq>sIO$C1(F);&u>O>8+tPQK~DjR2y_D|CfbI%X)=gcCZKgnAB<9nPA`l{tOtLd zCWb!D37!HQ5#t3Q4>=cd-~KE!ipZ+%G^}xmfBONK2 z>H#al!B~l=hg$j!?p)b0iPr=pf?tybLAo8r9MMmt8O?zlFF$#t{@E9P+4n&#HDTQOCzlS>kuCgxbs+A)b8V+qqXmP7kP!j|v|Upim-F+(mE-8NUV}JzYGNd+Y3PtP^ApL&Pce$`3v! ze=QSH=UHWxy~@{Ou3U}aZ-fbuGDeP#z;yBJ=lee*IAim<8fk%J6P{_L8i>2Dq)_Jn zEwh_%dwP1JY{ikPf-QMeB~la^=p&#mXqTB)Qd*3}#oMS6@{OZq-)za=ixZnNd z#jysbUk`5*Z>i_5%gVOz!e^0Ca7l!LfJ&hB+;1am{9piY^WgMy+szXBql9&dsq> ziM$HEi-ZRFBqWpY*+ z)+vimm>#Qo$}F66qe)Oo%-9!ON)5u#G+VfV-bx z_#ePcVkw5wcnI(@RLo~IRw;o6%I9@rzfWWLg`8mo8H}8h#ks%>h5EJzixH-=Z3*3M zTmuq7#2%0pACv`MharF)lV+h(1_9ZOBt2fiueO#_L01M+uZM0is(mAqb<_9tn^L`G zf{fAdh@bUAfdQP7P#!MOIF#aJajr>~_SP+M)Q1JYk3d=+>WRPv+pueF@w?wM{QKQ^ z5n03rsEYX0Rl6V~<8;#C={T20EV7W1sZDa^!m~45R&J>6KleB!z!QEu;J8Rk^Um%p zo>m}7s7zaO4^iPvq9mXU9zfKv`YPT$U6(sh+HL3_V&emPKxVlv; z&9&;{Rr$3a9F34Nva)q6K%R&}T^ShZ<9vf4KrL@!$8 zIG7w%3wwBzTQ%%KO|(iHfm&e5HDH;V)V2T4I9bm1HE~j*ZF&*w^$%J0#6LeWs*+|M zMXoO1mj=68LDk}YJ-qw$>Bh0H3Ngt4$;O<|Pp02IjHM+fiJLk71a&XOJwI`Iw@vr8 z`kOd#34o)9uaFpR7v0^UGQD8u7#GK`XV!x88}X#lh#MZDkagN_F=;nyE)~C1vcM|D zxPls_mgd8$DzBEeprLY1=ZK(CK9p;xs=T~hrsH3ZCiFoUz+p47KoKXw%bs&W-D|P-?K99PLBH_1l`` z2WM}8E7}A(1Vgk)GS`NY;d4|Up`d`fd?mX@onvu4$^e}U)*u>-!kbG(>KluBSG{;0 zpBR_yhGyABW5qjDV)^4#rXqmF7#yTu?&ElV_EpTX zt#B5XR8?eB<=o7~wv8KK5uk=6GH?#BUc1&CFc+8uW`V&Ac7-C~3Jt_l`M69d&4UpANNsADhQOavsvAS9tfA3R2c zx-kPHCaYq@K#a@)Mdj^hClf*M={v&`;2|t7IC4HYD=UkzY^X|aaO9tblajp&H%Khc z(B-aUQ;AxRbKl#xZy&ET!Miz$_y-3Pq5BXDIiVYh2OmYA(w5Y)_rx-qLyQ%=ITFp; zs`GgZMW5^IyUI0;gjLh3b6n;tx&$m)sAP0Vq(hW>c8Qth=4L_X*>_?3Id)QPUN!cR z#z6R9lUXEeB$8w>2l4ZoNgh1Jm+dz%fA3$6fi#+H&A(zQ+Q*U09w6zj(4u?E*yfnN z2Rv-6w{e7_(8WNRvj73w=?9JAGgs$UXBHbzc#U(K^}97 zx>*;e#!e-03cNIKM=uU3su5g1F;NGcfS!MA9WuQJDprb4(Pn>PdnTq`^3j*}DsZYP zZoA-p=|g~k&wfc?zn8=BMUPza%gM2B17ZgO%i6HJwr^%4w0a0P0;YNV>c_vy(PeVx z0fv#o=Oa`R;}4H572M_%tkKS+u`%j7j3$gbD4*qsY%Y65a;0k0zAUP+3N!I~np|~ak#n!|YSP2z_ zjNHh%Cm($YFY!aY)tBQ|Z<7zVx(>+B-ZdnQ!&OOIr+v7TFRA;Ze$Rn9sJm`3n)6_4 z7^>4meRx|fitohgL_n-2r|d{$qp{1Fj_Ur6=K}RD0l7i(44m19+d^B#Ofkl{*{G_Z z>+K)K+O<1JoBTARn_iv9aXUdMi{u0*j=8tpZ)n8P#eMTmlX-fj3@}~s)H;D5uu$PW zA%_}Cq>kz?qvXS*wGF?Gi8HEZ{KH z7yF&Z%<>j7ti^b2_%{%;QJlmugb9EkDEsk0)G|Dr=+=(x2D8Pot@ZZy@>LuxEN5X$ zlw&KMp0x}WOL8C>G+4rN3P*O;aSCT*O)8{z)e;Ll`2s_wYjbWwyE{f+zh3rJiahrL zBsK;`8pBMg_dPRTJi?&w)#=RPMl-URkbAp**c}%*#aUIh*4t=tD$XRmKCY4(DwL3$+gM!3eeOMf85`y{q8^U|j z%?K4@IB`QOd&^rKn}I$w!$O7LT?G`-o;^TDotVNC5=e?6)BxR4)@8SpOi9gxXoSR} z*LC>z^+ip(lf#&8CsW3WlZ;FeQujxGRs6UdI%*dz@-kP)ZF{VG%zGEtFcA_# zbwdNcj9HTVdX)d4ss9zj58%9*1HFx{i<75f36BZJvgBj8+~+2>ttVlPAQrbp{R_0> zn~CxFx2Lh$v@l`>0hlvyp1>XxDVaPtfViG|s*AcL|Dq&ZgnWP*Ee508i;Vfy(nutS zip*Z>W8gcUecsGbr_7s5;NDMkC7d8ro$XCPwBa8@r$}8wM~Q-?PuvwCWU%3Fu$cpC zm4ln-C-N4^k!u0*aeH1wb9|}E%qfzBCrY`5aTY?gUOx7g5%n#i`*K8 zrsP0NeBl?dv&&D(+JK@Hm_qt2izCynUDdf)#h6qxWSJQG(_-$K>FlfwtN8P5f2u?4 z86c7Bq$bpIJow`hHUTDM11mfvjFPZoX0~7v*AM-~QF0`FbCSrr^BX$81k`uErG=rK z?bByxr;-PEzzxeBIY&GNJMFW?-5D@)dQ@i)Qwk9VB<*WUz60iUNiL(fnL?9{R!xmY z5S!F1rb%q-12eHa?=IWpeb9Y9&uMQCd5JvuHVy_mbI91(l#GMbM`09@={S{Cu@{dq z+TpQCg>SJ8fQWd3E3q83F7}>&rG662lr*eY77wPOS__0`uro=3jbKomq+rPzyb)M7 zu?BuUA{*y!r93sj*r6uR#f6-f^bLxK@O2l265usP=ffRw*XKWXG3^>PjZtjKvQ50O zjYFs2x$P4j@yem{9}^^xuZl!AFFCeUA<;Q z9kvJuj0JIE;b?&qq^notc!bW;?m(MnyVvbEaoPms-gkW72%~fR$-&pqAYQ)e#4JFa z@LZaF$XT~f@4_frLzpk-%=Bknevl;7-Vr{KH+^A@Ly1o?`>fwdHNL*dZg%&VypmoE zOqC6YdSbeT=ai`8p<<|tNA7h`UD-`yOw03smjfNWhjB&MCkI<;ELilwnK321aEYr{ z->J=6`_CIrMM&ZK4QD3))bGcWLadZ2T6F|2Lf6De?z*J7Yi^rWPuz?>8YyFDWt;Hh z0S;^%{gRkk@wbG1ZQ~S4OAM>r*RKOADddFy7R6f`}H zP_e%#S$O(TB*Fs)Mb6ix=_P+%n^~UEyD-G z2>wJ?9dMcH<6r#n&6wu^c@P!Y*Mvkez0=H5nQ50WnLt(l+!^W5!=35Ij3;iqBIDYn zD`)*rCS-Yl2@1h;v|s(dBO-W>7?@n5iV3xB+>_3(p9+S zGOZGvDwSvrd{3k{zP)S>q##`3>WRj6{^5b0{hl@unE;GV4C z+Xu(|-SeJct+C{^9Vi2Fc~`W8Zn7{uuptuL;`xS>q$`oKtfBdCJ}xNjGJ(;K>lnJc z+_q?6oNVyw=_potU<(^wEiAk4>***O4&*2-BrkmxZjjOK9*mKIO5(11^hA5nU6=`` zojfT?1F4TA@|s>UAUM8h^SUKQLH&lAw$T{2O(QnkUIzY$q zA~cl4_rO(^7956#T>svv4~>;LJ20SRp#mk3!I)?v74)O4{Lh8YZJ)=a7Uq4&u#tvn zg5s`Fw4VN}7fD#`n;0$)J{Gx{p%RG}7ksVsTcT7Wp6I?nG5uux>ER{<{#i&(%Hqzw zg!b#czT_~`w_aAixvH`NbDlMxk`S7YXq4ExL}*zAo^5$tQ|!wt7;wUz`ce4ZK9`kH za%J%-kHyRFh97}gw{&x$vRtP5J`C=pj_g`sD}_a~Cf!UEio{MP25^Bpl9>rK!dq>% zVRo!cx{@J(5Sy@tpWg34Z$^^NW%uEZ)SmQ~Kh9q|W$&*pLgdfiz71tt5w3S1Av*R9 z_usH86B!Z~-Q$8=DMK5P446eLFLv^ecyh$fpdH+7;gONflhZ~ueKnF0uL-aE{Vk9j zpxm)^9ZMj757iCh$weiR`#Gmy#oo_U55SQmX(k13CF#^+KZrsqIDNc{K$W>$bY|U#olalIPBclFFUo1oQZ<@EGgE;p!z_uW0$QROq!AM2W2H_2%a75 zNZ<>MF3R#x!>(l8RwM=RiajvMXUpGVx1Y7e2)F}IlVoj_ZtB4`eMq=*%5sKQhgt zd5c&ZS{qe-#I|sqU$x4qA;wSF3BvIroWK+8C@lXhksq4(myzcYgM&u|cg^-r2|c3y z>fEhmb%HXeIyf>?1$U>J%EDVY_6#-|Q!8bvsuu&V`~e$NDiL)EFSe4vUePU0a&ED( zT2$8MH#K$L806l-&tGDpto-iIogfCL&6FLc&`>V_9`3^ozhW<|9Z<*e%$upP+-hkT z;W6SvEu-A>TY7ToDcWY%?%p0ls}qxc4q;&lXU#MzPs8BTRP}8!t70o&b5ueouD3=G z(aE}K3O{kU?)`)fr-QvdFaI=D=&H{|5{Z!c>+aH8&asr&kF z_f~JN7xuX$9=cA@=EYqB6$UH+Ey}WY?#zq6_>%oFf!-_t$R=OhI%BSB>%e>$px4z2fzisO-V-*`4 z{Vo%VSv-_ClC@9@ zK3q{b=RMJTz(l|xNbjM%U$gYU$UW4d!NIp&Gpml6{psD;)h&1b{8{B%Vd3ZLo8SBQ zq9~Mux3N(BTf$gYbckOo5^byzesYJ z&SkA!$3<Kry*OJ+y&PW-;OTa2GJeoR=Cx4`lfUud@Ty}uDT>(F7Z z5d9^;G*{!EO73;9)%@2(=CupD(w$~v$@%?Ip#Vs2`LNUuZ9kM^T*VeD<{5IXufgJ` z1&xIqjf7-%T#yBEK`$<|_zYX>YGEf$nP;MF*duiw-MAS4zWn(Fwco&3`>&knU0d~N z^#SiXyLxnd4=!tPJL11}*Z8#=!Hn=r785t9vJ&_1-Z>K&oN||XA&6^lh|kA;Ustb_ zc^YACVf8eCX$xEP(;LYj_iN+^K0k5tg2#HgEoL{K3selK)3eL{)s$6P_4d;`zNjBr z5Mq+J>JBI!Hoq&O6)g24ak@u0{_L4#9-r=yLYA&!j z;CTU|cg{(&6(`b)b7uBU<}fnl*=*S`VcJ$ulH+gt&c@ZD z-#5M)crw$Jl-+$hr8|8_u~+xvvx{@l5!9z6B38l@jAd;73r1!g7IL8<>5g&ziEfv% z8aHkGHqrF@iQj7di$Qx#q<+l5*Bcgjy*9yLBVL+&;pd-R&F6Z5rr0lQORb*h2ahGb zu98e)Rww;KYU5|efHCiJrmbE+Ur=RHP3C+f{0@AYYY&5s# z*@U*^mRp-Unw>lx_`934igsP(@U1^5^~j!0IYcN*%T^yDMWJ4R9r;j+l3CMHQkHs3 z{e5*ktI;4b}U;bTYqqO8eF>O}Hgwhday6RC4VG zg8sU5=T|>!p0Zi?7rIIL7j3?yP7KekENi73DXYEr!h~P%cI)AzC$ne8Twgu>on=t6Ag&NjWzq7aI?W>cK5jzvomwdHT8iUiKgD(<2+12sH z&!Rgk4v7hgW;<)_w4j$9Y$qS6`%O4;$bo<8#sJ&l|BI`)j*4=9zlSlgK*c~>C6tzK zFo7YZ8!S48?g30xx`qakZs`UU1!3rv8oFcX80vS8p7S~1_x$yihb?9 zuce_MwN=rj`rd8=-!0UwP2R-N(2)3zJN(5V?gj?~77Ug{AQ5^D6i3hD(uA=fvTt`J zLz9qGx;Fjw79N|=ImuYC8V)T}$Ats6a<4zJyI;3^CZhTDRb1xj-k7^vOPNB(ZZ%Gv zdX*+xE%7|YWA%YV_iMk~Pu7uS8ioN)5xwlE+>;eKb+BhwnD6!Jx?Z`59snm?`1Mc; zV+Glc-szU2ha&z}!xfzEU99iJhbunNAYT>MR+&#;AN)W6zv+OhX8t#GtDb}r*Q}Dh7-5R264Z-qHDzzm5_8_2> z4tAsydS)+(tc-<3jK+kq=TuVYt$#d8Q}AnoVrA8@@}>f*GWs&7BkC2cE9NjSmw`FV zu9Q1jdeB=Xcmh3m&#Gb3_pcfngBFT1W;cG&8b(DqqBu$mBp`1 zdT-SAqqe^ombn(4;cO=(`d#8lD_+Pp&^gQqrI#O_AkfS9i;?l9vEr7%yDPI@wU+&Q zgGT@Urd%W&=7wVY+=qYw7#fJ$?efhdou8$ZMmn;aiSp?DHA@1IsskdjynAz(0 zx7uZfKHc2J#Kil^Ul)XgEA>%IqC-QIbk}Y(n*~_F;kf-R%p@mlU zqz)$Fj{IdT+bohh1A4vorKczerP0s(6g$*#wj{!9D7J|S&jk&_=3ze_U zNoQR;B_h8(yoJGgB>=_Gw&a-UqL5xPL&yao?Aj zo%?h?Bc&*-;$fqOD?=*M{Dm$cX;2&x6hLA8iv^z7@x&+1HdiMly+bGTsvKllODyuT z&Qo;hFoiVCd?EU39pn~>KXLpTO^fW=Q)iTY4)z^Z#~x))Li51`3JC}bzNCDT_Y~tL zBkOxRLkRe4?pZ0~kjg=7c8T0} zQP5}y^SOLG`u452Y%sV;<={!Em<5~KXTIp4Uwx6t5ySIH2D1ufy`=Y=^8Nb{ zx>rIJ*Lxd6)G;t}WJ0Ze02+{meOjLD`UE1hR7{kDP+0$_`=^ejRb_VdL^4h1wxxHKZVZqTrkMLFsp`0_{T0! zM=xHS+xSV2M;&@ZJ4m9o6I_d|;LvSsXTi(D77+c(u`#nh{oKmGeONds-w7}=9;VTZ zh!77vVv&kp6<`2AQuYQ4-lwnP-o23y?l5jlpd=X}JmyON)d`HTRyEwBy`x7bC`{uF z+@1idq&L(hN&d@nZGB4JwU;efY>??UXOcJhHQnkEZ8b{r-gY&6FOOF(}pIoTYp=(0Mc zwlwGxddt8Arkj$p9jOBNb+7RM&0@eK7iddr;Do`5sKE{AZiqOhnKnmBdZQVRpCq0y zU{+z9XJqYC{}~10=`4{xD_l|dnVk)n>cQ{&shJ{TIH)t#93RXGfeL1E;hnb#rG&l# z^v)ge6;89QVig*2aPX3$+smUsER<1G%g}ukmZ_@NZ$IlR zj=HY@Hyb!VF9bIsn3r_XdG6gn;3qr}Rf@O`CkQWcLSGf9aZA_6KnWv4p@m~^qu~W0 zu3o*$fE%+ThC|3XGoS$`z%nXVIl7WgoV7rw0_X}_wr5#u1}AJMm7%!w73Yhswr|Lf zxrNk+MFL&|Scr3lu9Gs^x_NTsE%82E2LS0!F3n|3fUN-OX(h-*Kqh-Wg!}$>t=xLF z(qr#w4kVjKjkGj%%l>rJwX4932x|{N@&3v`6opV2CZ&TzE`jse3>1=iAI?JC4g$v~ z@j-!3^wvT=mS{DHdga&Q~_MF|>V7jg=AMMoPyIKwk7Ah7*b%&vj|~aUsk5 zaK%X!@%p8%YEE?Gp$r6mqiQ<$ySigU$c#^)?pkWrD6pbAJK^M+K~`ooBnsvl)=Ptp z(J=U&Y)U5(Af0-$Y@KX^Dx#yC_m3VuHaU*Vt$_)3!@zEzCwMu{0-~?0S`tJDC&-l##5ShA>7Sz+n?gWYDTZjbUK`l#vUg z;p+?z4TdtJTww^$FVhDs0sE03rxyjC9}k?TZY%PX;tEazY{L>;dM{eQ>T&Tx--dqt z`L1LgC-8rwMD2Lp{^#P2uj7Ud2!bIX*f1jKQ{+|pK0i`RaJGSO#G3H*=^T7lcXt#c zg2QTwa=Tg4*4B{=05)gKD{QKo@PhTe24V~C#e}MQ|MXGwJ3m&U%`}p*J!&@@Ct%z* zAMbsh&v;X=Iex2(U>aAfmunY(K-qy z3`>Ap2$Tb0aO9k$meBW%6*bku?e@tTHbf3XE*dF{8B4y^6WP zcO+4(v#Gh=r{w%eNi)z@58O7p zvJ5f2c00Bw_M$7rJr>d^yEC(`#IW7d2L8d(G%@FcY5%S)aZUtRFj)W|i?VPM9=b#C zNK{NTBia+H&^-q$-X~GJ*@j|nao;KVSB$4tPIx*0ir@;#=uR`$GAK3;0gR5Nte~J0 zX#H*Rz^Qo`t7t9#-eYI{JjQk2anZN>2Vs1(Zx>*umP!4WY)T75ylo(xmdcV>NN-D` z`PEqb76vrxHFDMA?!yX=Ka39i(i#Z9q9gFRa}RyS?(!&*PXZZ|5%G`cnR3zIsp0JjN4$nihc*vGd^0_E<)MlqK%$17 zIeM;Ru*-x%u7CZ`kI;Gqhr;wrKAu}EJ4&Fv%FT&=Ko+=BH1Ie~<54M>9mT)jPqCln zb?UlpiC+@A(h^dizrPBM*vF~G)rVF9RgRJTg=&ZxV05GaWbwQ#9B5PJ@tfU=w-kLz z7mMyZJ#equnhZzNwt+oHM>vm%TEB0fFN|InloRXZCw6c##nlG~{iWtG*QioLTl{{a zWi?nBsjIh`8BRe|*#hFxRHrDuo?Zh#SoZ-FAz(_#@2smVYJa(Fc!r!`_{VXzA~}jh zz5Ivp_(NaU78DM2rhcR3vPi=&VjmqG97uY<7dZYmBLxz+M}V9gfdi8h!-0g()D6eh zxN@7VCD}rYVUTz5I)6Wo0;V(&-WZteQqJLBHpu=2ie;j&Rv=JFM-)tl z?R9+xVa$5Mn9U7yPbD8Fl8xnW#}Xg>pt^2)F#L!1&H1&jPYVcH1dlXiADD3Qsm*pF z25yQ`d#LOI^FbYyoH+HYFBX_E*63mj%}=jRAhwTw#msf))=Z9Wt*lKKSsr=-cug-C zAu$~Mgi|}KltnFvFcS!5h!OOVBbn>z=mrO%fhdqm51l2?&^!!32i?}mQ^cJ zPQKln>=7GPzPdFm>1`a|WW;>C%VNWOYx48okrYu75JU-@e;Dcn!>*2Xww=D!!~kBX zQVPx?W4acB|B7^&xsZHIHB$Le5o#Wm()YKOXZ}9)FVBFDg$D~3FvMm-5bVsW z+{>cY7c&RIWP?Wk^JyS=Zqs=@t_A&Y0|f^IYgA ztfYVs++PB0V$qxTI|`EAYU|_)@6iHwQ=&K>HK2|zGmnjp;XRB-cBK;NA3VPB9S9kr zoeuTFr3?)F+|010FBPaIjO?AIKmSd*UO?ftZvz)%f}ba*SOWGAr8{BUp=`T+HLeiL zBJOceX$CaiT;tYa(F=-kgIeSD?ir6HCNGykA3lo>tzO=BO99|T&5T+i7$ zc_%R82F*If+=MJ)4mAJAyTv$IW%D?A0@9T{Bth*fEZ1oI&66J_N@@!~eH9ZM!+_YFi^w_JDTL|yY-~$_K`_?pC&@+zx{|zSBpNRJGzDe75zv%J#=-- zS91(z)^r)$Q8qspywr!UH!S;h; zXB$7aHromF`v&KQ`AklurY~6~CVetRX1xSr!WSLDwK8PA9m`$D`+RaC|Km;9O^rH6U{ud*^xb2-s%xlZmy~|Jc*FAFm zWzY*7A1Gi|$gO+Dm>99n`EfAbP-PU}tBnDu zSd)-J0aog|$XXQ$iyhXj0vzXtbE7*_WuMRRo$yLp>XKLSoqS}zp&EDJXJe}@5#oz0 z4#Qc5oK=R&;J2B!#hyyF%tE69evE6hlhlm9iP2Dnsr?J0C-Iv|gVaY7EqPfry(xYm z+vx!+Xd}o8x_+AX7TRoCAv$K13^JgaQ_ZlaNPO@*_W9o-jPpIPh#^?akWveNB5X4O zA*oU$9SD>9FyWcn;T$a3&u_(PMbyoUEq%CSZ10~vP10TcyqQyZic< z+w6M*7_K$Wm|4L=?r0d8D3fggsrckfWbaUu+uC;+83>r`n& z!wnMyrMJMWfP6aH8mY2lH}QD0DRP{TdcC$jzyZ>uUZsZ~fn^LYI}O-49JyCqpT(w@ z|GvM-^m7Z1&ackS+8J45qU`{=zSXp?Hs&bq{8Wb7w%Ja+>!t;2sHO&5>MJ!~Lcj1IO@X^?#@7&{PX(1d56oGB$VKWcD z%8ivxR966rZF~Rp0pIh@m~bRDlI<@b66gi%gR(&)f$8nlAW+nSU{BC}Mm8JC(g4d0 zNZ9jDlNhB<2S;1wDf?LyK+U8?J zce6ao)d}LIyxNz$v;^Y!^{d@;CEc^N6&t(w6TwN$K3AEM{?v2*S&{G!-?n|Z;{fbU zy)v5z!7psB+oTMdRT@=i|5Rf=OU)RE&ENVMeA7kxP*!vG#M*pwAcVBN-x}@(CqtQF z5B)s3*m8TdpCv3nlUca@H@iw72X*da&@+II5jpS}WWmv;3-s0CL2WfuX^Oh}Ef9oc z6{y|hfW^7Fl%sFY%pSbW5%P;Af}LZH`s#JJaF3=aSx=+PefCtI{FUu*$58->iH85`i{S zQM_J}JFzxX4dDLIN6(3u@0;GYv9@jteOfy)vdpIEw6s7$Xp-BXep6V$5=mLPV`=@w#`gpbqrKc6N6FO;stP;FX>?Zh8ztIzLie5f6k5 zzUQ;b7}xZYxwJ!ayBD^fhR+gJrp<>dn$OSCvCY@IZm9#;!;h2|!K{`RSk7;rLZHY5 zuP^>?H*3!a78&p_O}_B_1=2_OMk1OJ=>=12W|L|M>kZwXL!stBLZp98X!UCQ`*id5 zNn-iS)R(W*85r<6m>l2r4*oexppR=Fe4gYR znctC_nt&h$F!aE^*M-39@+EDyhKXy}MnUUm2KRzVmb8m1J->c756wUmkTbMdowYyE ztJNd5IZJ&&u# zdZKY_hS_vWzr#$Rq0Y)G9}HDCR^=Mpw}_VWX|WaQLpmDTJ>K>>4&o_kMOB!vxaP`J zVepVX|Lha|@-XDJF#L z-b@e#n5LP)CsKdQV)&Jg3=hz{lsvfeVQG&s`OQEk;F}rJ;Q_Na=ZA0FZ!monIZvAg zA_Dx2v4=2%lL@8{5lS~dY@a*)xn(Oi^}_x=^gB?YnDn3H}%m5P5C;g*UEkXEXLuP?LRzvMb#zRRKRgfk=EHcjqZM z$U{LUQ(~43b%RcEYIC5PK`S=PC{`RgA#PzgB57^~+WvH@^H#FK1o3b*euKbiA`_Z* z+itt3x3h~(eDEsv`row!ClNt|No>rqp^V8P+^N>eDW0CnWsE$4r+LwUp_$oWPHuHn z*}e2m!+Z+~mzB@hrN6Z-BtAdVwPb48(gVguyZrvD9MVx2`A41}Ak(oMBBnW<1n z6U@C1s@=7W72=hHKL0%DcRE+%-w$_}EGYG{so(zSc}smEDb{Jfu$uS)2IlLxr3=D2 zdjmk)3xz5H5j4}tx-CUKZtS@W^kzbNDJx*n7F1xI7>WD~TgLh+idVH-?%%a3WTG z_0TT0H$zhi`~wJgC={qQVeHBe;pl=H_REa7sG`>$?jUj$dZf1Qd{C0O`R|J2&t<4x zG`vr`-Uq^;K&aUObq6UEak5;bGGQXpw|b>3U(FmK(&m}RBsOE)T=Mb4T2N`d_9j(< zv4lZ`O#rQUmV@vs3g64ZFAS4zuUPT=wn;Uxv3QOsCjn63tGKoDoiue`B7YX0UaBdwRs{>s`x% z8UsRk4Jrj@UN$!E*jGPU)6}*URWp?J-r1h?(wOKP)k9Ct5tOejRD{7XIthoB*qW&l zr@KZId;i0t)~0^1`G&x?P8hrBpeA6iQ0Q=u-I3{svQZMZD2?y z@T4l5POc_XaxGjUq2f{&!!GX2gg86zmzKF@81~*X>!^-*w9x&mwpPn6ZHNe~M1 zpgFpIlVvAIr&569lVSAsOomoo1d0niW}n zBfHZ58FM!*6OV7DhfegvaER1uwQWnjLPxJAkej=I#CrrYEDrCcf?5RD0h!oqI{W<_ zzWyNm4oklkL`n{CAb^Ul2J(wAdSed5X0$vojA3_l94W2fr&@1PmYX;4W5PMwztxbM zEWT?7pOBFD)IbJ8c;YixH6ToK84Q~u5Na7}8nCz{j7x>-#5VB^IN$bHnWyF$R4{=c z9>s6}-XAW?Y{B%h^#xm7g`9GB*EGkF7U6Q7z~W$~ocET6y2&pUS(?MIUyV=qy(VT9 zi>~J~@TX=8WDO@sW88&oZ=1h6Clkhsx+Q{2`UWp-iZ&YMH%~kY_YDMc?{WzaUzc6n1b9Z@PFfI7x0+vhdVC z1Pvm@!+WY7%>VF|A-_p_2j0fAXeqbASPL2>$k_IqO{P~^^}o-2S*HB=4T2TG#tJ12 z?(^NqPt2Xh%<}z(>ht=7o(jkCTb-j1h#e+zHQkA_w=93>`^6=nZnEIdJS4`=gJ%!; zCsK;9%5E(78i2H@K-+0Ncb?Dw*IltvPlHSk{e;S$HS>`XMPG4}N+v?W(k+r?ff`|N z-@OnyV77=YS0-0D03B#rEvv-lM)xA^1 zRUgB$IWE;mfK_aGD{UT|2B)#aTACuIuxcWH!$)OE3lnbMv^89>nQ9#;N}#_Hip}J+ zjkA~8k_>u~s=NHS8JK1| zKoYa;U(TcyuryjvXJD=mpy`H7BpvKpGpw;ML2FWw0~XukDb1>B)))D*bY#JhwkPz% zp-QVhjPprZDd{zV~)F zKX@ynroNKjt(IlsFxjNx`;e2UQqoGV=7knVut0A6=npO6bO06i;&W)E_BSSN?fzc~S7XypcEQf&jbI zd^?YR{RAdHUa{VHd@V2W^r>az`pxV@^I;{N?8WqmqHG$&OjwyyBuTXGS~G#omK7~U zg?T&Wt}VClEe)4_#~9OY@3=naeOLE8JUNV{r`YjD1Da|}_k>NWnwLe{nP7HfkNj!o zH<6L<)XZ-#q<7bfXvkw=?Asv{^WxHF>ZVV~8pGKXd9GqEo@){OR=OE7+eaq?9zWZy z^P6||rdJl)VVC{i_Y&P$=Vdo;jikrcPqu`3e}DARc6RK<&UKa(&f8-4KigiO{`_lO{>$kqPl6y)ZUmZR6B|qN^gOV>@zd-%f)Cqn{KWcINKzi@M*bsB+&4&W;wa#dk65i+cL|Ws_K0 z^`&X1bMu*X`>#>B&CNDU$x|IM8B9uNo`S=(o>;yCYcv&FbjwVBr=9k<2@R18V9U1n zQkegZ>$Q1@$ff(If>=a7tfS$vUC+|BwQ_Q*0+pjTEo~tczptG}Zr$&?hHo#gNbEoC z<~Cd>70}+HXF$;n)+pZflIZE-clBmeW=7!8{QK6tq874yYHIlC@ssN?8cjjg9Ephh ziTr^LOsF-*e!h~E(|*hAYRh4C%_07hSDk$H8fHCy$6?AI1%Yu5@x;gYiRFT=OneDxuvk_?W!ve3?8cMp{ zad=1(EAtj0!oICmZl7g2-MZ`ZVc38Ar_W5tCa63{D-gq-<(3uKc1t}LDQhKmu6ebO ziyQyCRlDszXmXWZ_pZrIwDQ+0tZGKJ=*7F5pPjqjCtUNBlnH0X6d!K>piFHyNvbD~ zU0GlL+S7vu4#6YSOJrm~FHPaL>u(LxuX(>Z(Tq{-^i0s+pyZ01A9=a&OU-|rh)RcQ zPQc_x+qYwVl{OEO909W!?d{`gCAkf6OVX9NfTakuoe}0;^4quA8;lJ*=Dj8Y_P5Ui zDa0b==g+sE#w|%NzLnN6q=m<56+AOEMLch0fPcfXSS z)vLWSn^D226h@u+9Urx?042D0Vq04LNohp#Oh%8k#EW18U+(X?)Nyk8C;6sHHfwjK zX|d{0Q}DmI0QdI9b+=9G2~N!mIE~y=_t~4()z!My_Ce~+QOm6zlX;s$Zatjt`#+6S zdOI5$KD>H+W_)3x$%mX%&}ozH<;x=$Fh_yS4Y@TMpsvBu4tk9-)n zS@VW8>qmtE$9g(YZ3qCZi|)D94xLfEls0hVeC>XbA~FU%vIeH!rrr-Dfw!fUI`k-oe|BI{Z$+f%(-=?=579zYO-h^lqCGvOaVouE$$@=?3{tx~@@C4M^|8C}?)5iglpa z>d5!t11Q58jdKQ_tDL`Wm}{)8tc>R6O`fO{lefvqhWRms9QOP7@71*?1C!2TF&XCTS=6{zmDD@);S|8RZZB~< zPf8<(kOYYwDh3XJ#lXWq#Z*&Mb9wPcgL_-HRMPPcczWBb1$k@+s005!*K~B6)J>n#oWK%?1@eL$sS?oU7|I%aAV<2cT7+`JKb3udff$lKU{8F_a~C zIsYXZjh2hvVWqQQy8HO?1$fUQnyGquEGBNl!-z=g9g9D)CVzSu!Y{YsTx(Z`N^hYA zFK=iVCY;-`OB!ZeJ9BF`syWJhE!<5$SDU6sYl$^=2HJ{roHEoi{moP42CMA6;nqo+ z4P?mc_|ppcrkGm5z|rH_$)CS*bsO_o>fUn8(0vsq`8$m67&L4BmWrppeJcuuisZ8s zmXMFVZ(?%$H6HaVVLTPK9++qK@@(uPMBTT&z}~Md+nBR4JokM_0gpzmZi-`Um%vs$ zyjnrm4lV{)Z#p{udAI?zemtXRm&b=ac`Uyj)$Yx0-J+Ka>Yk6Gx*c=@_l%cv zTmf%&3bsi0^(oant(D%TmGJn1Ebut;>B+bIY&u0(!M+0N z+YrxN9V;l*GS94(e_2)|i@$l1s&j)-I++sMf78BiJ^ZJ|W|4Bk)mZ~YKITqKKl=E) zOkRK!^n$SMo93EBg}l9;4N{4Z?{mEa0u;$y$!25!Z;*}A8=T?a-qjqabl-eKVc?W% zou0uvyN>W(I4ycgN*sr)QK*Cja7$ zg!KhR*)RpwLqkk>K=87k$;3?FUcm?wJc)k-*)CBY&yVaoL$0AgJ6I!C5?A$X0rTWe z8x&6d)Z%YaOsU1(Bhc9cKPZQtLQ)u*M}c&5tiQ;rB~j|>+Mz;(C}U5V^?0V^kn4h& z^&0~th3BKgYt9>UR{$q@_UqFTcZkjFbNe@{#22V=NqbN>$t4Hxt}l`^j#4RQ3p<$h zd}cIP&wV!Szv%4ZGP*-lVi6zeXQGHTr`uira@! z-(iO;hWSBQ^e%#H(@`T ztHFJS&7k0F?hcI~|0jyRdH1p{N-<*y!47-;5MoMuI{CqT*r zcVAf+)vVA@)zSRs7w0{IId9gL(G2mU)Dy=6>+AfLJxU;tt7huc>rb;j{_@GRud|cB zrT68hBu(KB!78P4yAC}OHB^G|r>=!C$z=NFu(H#MSR2Lt# zOSJnL++ekX5245O$iX}86~?U>r-cfpoja$N<}a3TF{>2ozRF2;!Kk~7JMw~EHlz`8d=l>6 z#@*lHN9VfygaekEqkH9Ts>RL-;^$p>@^`R9B6&+q*^X>ZJyT|X*w|Bh4=^x5O$R1Q~8#@w0T zlGw(34zFU?P51VDb&no9rdi;6Me01AQFlMdgik%SNnY3x2|3rxJ)nAP4&ZloY@Qs7 z6Y4&n>DHAsA5r|bTKdiQs*8OC1Gna|+uP2a$q%SF^aTTyKQ4*6<&EH6L|xYdq1kCG znyPqkB8U+nhpQJn7aBwWt@}Ydj8~o+f2w zHLujK{L1RTOMT(O@|0`Ub52$h+(_W3`Cll(JF2Vc z$;fz)QfBE3d)sGt<9MYS#rWI1AQtE=bmBzy7k7)eU!toKz^Q!LN2yiP{nwldmyJ>U zcp0rE@t8THPbRqfW;g7qXB}3Ek*hiK`kJ`?9XBW37?d6M7QG)WMYcxB>eEiS?Ur6L zAtm)snT#Owg^E}%lK17?wdv9E@esD&s{@rec@!dMNzCs~horWZ3==_gYmV$p-G+up z*kLwmrh1&pX~>l%O(*wA#o<81`-BIjo`?i=7NU_tllVAmD1Ms&IQHrQ*=~h$tQPa07*o&PmF5%pXV@br7eK`4!*Wp zaIc{WxC>`@vxtX>$L-e24~>E;cEVw&|FHX!giGl~7=X2rS#LVk;>tS)-GCI+58~>l zf;>u%8k-h=&Ielu@eG?nvhiZo#9l>~_?P{9I0Osu6uIl&c?Lq z!azYZ?6Jx>t;Gw5R9CNTUuXveyO5geC6(30vZCDSVQ7%KOdC}p4oeO6_VJ{uh|Zq9 z<2-ZcT>!PAHnO{%eb{6F3RtF2^ip(^g{^O_t&Pufia+S|Ei~)&r{pvLIbSLl$-}c4 zuElGvO6Mg{)gy(#3k$vr>nkoQGTb_-J&iy5dVXGLB{&2sWYui_*C;o~0=r#)ct!Jy zi$~!$#|XQSc`-qz5tPVJeExf884#3w=ben>u>)X2*q18>)seLM!cB%t!{JHDlv64n zA5N_7l@X4JCZVz*HZ~SJT4{GLfl@&9nEX8YV)eYn?Sd;;jc_<}o8kXc=Z^8ovQp_5 zOii@MvO0@a%1>F8B^J8a9-a(X()-!da&g*MYn@c6FH5`Z74VrJ7rXB`-$EV9%C=0? zE8&p6y8jw_Ih-JFzrsXHu;DieJ)aPT9#1qHC+3mmB)Hg(Py?+`Z^BJxaMdky&8V5au#*S zLiTS$RgPTc(}prwi+`7ucBqQdboli^xE8;$DD?l_-BF+I75JG_ml;M1QWY=RSZIEi z@8pI2_{`F6ezRF{SHyEE##9I(u%JyE5!c^^g9TgOHcajhhVevq7*L@w`QhX^ks@Xy^odEj)*D>gDG6~Hv+y0&d%Rd<6O|?~LXR7${)^$>PY@-@d z@*A1jvg<)anZ&u#fQDmQdq9bZs{bh$|hbL1|J4ZYfb|S>0udWWRCBP4np*UBF+r7dUZD0GnF{*wG zX+7Ry6DL~wp{$JYidoU;-7n0Vn%c88X8C44pOZ~px|S1;pFI9!zVz7cmd(t_5h6-n zGcw~{2sw89{YBTdTBCQpne%i~t;DjjvgsFFFFc+2xwFRQ8AyBqz_6HMqpAU_9W->~ zPw!kiMM5%JVqna)y=~N7s~-;1VV=d*^c8U{e5(W*8QIahiFHZ_;be@mL44-Fb&G4{ z;wA5Td1md)@0zkSQwe+mHsnfFxKN&U5h?5F-q?;EtVuH}f498Im9Caas;>UDwJ|`2 zsXp>Loru|SlKGyokO1O{Mya#V=iw^eC!#uKI(pl&abvKywF24P5-aMH@VQ}?UB`KL zk!GAT(>UUL_EJlwJW5Kw6)(iRRAN&0{#{pYxpw@r8hc$b#B&}e{EBNpbmktR_*n}Gcl zT6V{rwP8>zvFMl1wbE_g62D!2AtDbdFM9BaIo z%l8uB9s7q_jJ~gr9*bNbI*b79;{tlnFQV%7eeZDMpd7Lcw2T^eBs=n#SyronAZ$)xM75V!u+2ow<U8CuKiH1zlW5hHyv)i=lJ;dF?o=(1OH+3Mt%gnv#jlY#mTr(ZO;P4^({9j$scD#y=7*YcE>kUbBXC zDwXM)3Og>zynJ;6Y*CH=v@it&T~^C&PgS#2g&J3umilJ9k~5iKYL^)CQSw{ekIbL6 zc66K?eNC)e=KkUEU;qL5`OfO}H1w6_*nuOjPZj_h9g=e7ZavPIcFUzQpgF4B?}ShI z%DvRhZ9JYExkSf?H$DQQ+0QT1m0vVOU9qz}TWL@=wmKR1~KURcV$c?{FhAvD74N!L1iok~y(;$8Cn~y#+B?zpD2mY@HeSdj+BKYmcqJhVG zj#T`d@eM0g^9=@BN1bxxYu8rvLs@P)M zyW!H#TEH+A%oFSBAu-|(+;z{vR=DZBgkq{~SjLKQeIwuePMV@(ORQL}fv2Q=X=HRX z&$F%wO(~MY<#AppJTyzD416W&b@(k`U{d^o`XeGE+jnaN2mlskZ*KNHDqAU&|B=s? z>9MzKP5XAFz|5$J`3bgOBU?Q-){S(zFG6Xr#7^Yi%aSJhF2DebM+)7jxG5z8Kh)WT)F*eq=5nT!ooT!tAbS| zH&=!tU3IjWNdJDdhlfRfg+Pl3^ykB*6rZ;yBXp}!(e=5RKdzlS2@;ggPSXFMWa1=~ z{w4CAwj#jlzSsU0YZ-Jw|jH-M!_O^dv|AiWae&9 zmR0C&t&b;3sFbRVvX58sYvmgm&bJJ)>EuT^m>*8|G*cZXJ`vL0>?fm^(UI-eDk$5l zrKPhrU3r3f$te(UW82jqr{ly>AB+6>fIONk(`{C`*cNmj1{1TRPg!<#%H{fQqu@EO zjY{pGCMEso_`*i+<7IZEIRQ^m$=B{{i^qF%b&h>~O}xeY`1(_N>D4JreMeYknd2f0 z%#%l(W>o-6ISbQ}JCuHc!a_l?s(XQ7jsL@QSzL`CG)&urKGvDjoEH+}-ww;?e4Uzv#US6_U(cFqS zf1=M>jQRlv&@;_EcM1>&Ppl+*1GGODCnl1N2~9RHS%Q;d>G3#og~OtpmO_pG&4n?P z(kE$~7@qQxjXeq|JgEg8KGt=QAKGau(z|Vp{Vub!5VOACj`H1EevpA?PTWe4SPCflVjWQc!fR zFYG_f-?D_fLp?srJjs!;eG zz^Pg)ov#-_0PWRpC*L+GJ1N4SedGylW!#i$*8Hhe%XeH&>f!dvUd3~%SGjKX@#5}S z=#;}RHpGqB%VWCDdpbm@gHB5Sk3#S;8*Pf+1XlmOvw&avl|aRcKxK{;%sPDcn!pv zNaz%*8dq|+S2^Cid6$p-ak|qLhS2Mzh@>O`!A3!Ix08RxrJm|ryPRN^d0q%*a#ly< zK+^g#k%u_vrEcRoxxN_@e6Mp+mEAry)z$rnpK8uK=y*tDzZ;G_#6UY^q|r(HmVZo$O+tvt?nw=bUaWSy=>?_#YT8#1=Col}8B0hw5++>IpR z#@m*pj(G0v?AK>)KNsDXh+e}o z0`p?Fbw8TC%-IYXylc~{@&>q3Ar((%1qZ~}O6vi;m!Cf&tLlNnDssZzPoQCyn};87 zh-oyKZQEk4dT{Cf``flt4N@x|lY*YY7NN5p^3mJFfj3;e!=3;&qA*7zpl4%lmPY0? zTVG*A;b4*$&%)y3#{chGu|QO_oo__s#7Z=V6%a3PY;rGR{;6n8BBNf|Vm1`UY?^~# zSH8vAOfDR0p~gyP8zG`S)uGSwAui!qX>H=_GU@P(k}ppP?GQ;)UBou~C>0L{7)8UQ zyxQFF$i;{Xx9I=TOXumU#X?OIt?#mn&o^zQ?R524&?DTPi~rr71S(Ghq?Mk~&!RQ& zGIt}1doPcqphiDnpG}(GN((iyZ44IU)M%j{iL6IeA;T`*XQ+shuXt`{MPXbq@`)%; ztgOD7_15X-wIknUuHuFY1KUQf&lTez<75%t)G?pYHt3F)`|rE55cB?fdqdMe^;rGY z86sBiuT-MUSXJwa%<8WOF4|opT^_^FjLGyWf4Zygoo+P;4em?(mTbVx~qbcd3Hv?3iMCEXngNP~2Tl+vBj z-AH$L4qY=aziWKP$w6vrm$O)D%>V>hd)pN@n{F10`p%3#wP5!)If?h2nXdy1@b4>1p zFAj`n>1>Y;a-YNQGgr(G7NzBoVA>KCis@~Z)2}u=wL>~0fR9Vy-4#`W5U`vsGdNF$ zli45!nH{Ybr4Q=w{nDu(77%HQvb&!*ZrWT_NDR5OTbFrr-6-Z&sAHjF33{}6czCY< z)FS~GtE9^RmGczo2wyC9b{J?0dHE9Khoc-Wof}5Ch=$d7AkSHe-Z5%}oanS-9twuB z&k`@HwZI9j3RN7bKuQ=+YlhA+#2!Y7$EtFVeV$3_x2#p?vO-N94#+Wug9#<~g5JM= zvpr-oJ;ndWYR{Y{WkFU^590DD#>H6WSwICCpTL@^N-wFXIAzi_k>YlE7Oi`Rv70hm zX8BAvoCMO$dh$W3P+kn^No+8oi`fR5z2NRfd3l`w#(r1eGNo%cN0wD(#HlS+NUP2z ziqhp|$HQ%;VgCW9?eWJcT!;+4@D__MHull~xoGHjnyU=q)s{u3^Lj78{}!cx6qoxd zLV3KzNv45@g`;*o5_7}v=?XJ=@br;^36h0p+}J%J0<_c?V%Gg0=ob^poSn}(A%Gt7 zZEwD`C;IT-RHdO%nnLoo+AA2>$`Aq3XAKMwZT;sB0yfTR^vjP+5KR_!)E@bhrqU5B z*|qk!oS?Kpj<`({L0pJ5SSi$9wAH)htbYq7QmIh0X%NFU4RF7a|L5DWKi^w;|A(`o zO`5H^UXZPxY#H?9kb5XQVO{+cYbxaL}#q5P%?Cx2hO2R2A`L9k#o+VAVw{Cj0*5NZgv($}z zYMH|s2ED|)y!mf41Fl(#E;gZ^iXe4r1IL)l@zQ+_PBbS?0jZ$a!Z&A<L!K*DH>85sFpPa&klwwG=l@`^Q$(p~*Nj#VZ?7Em?Z?=-mm$RN& zxvTgiqd$`g`+twnE|6tP1V#$xfSD#>Ho&uL`P&DHox@(n+oT#nJ@L~*ClVjL1YDv#&sh@}HHo@CBeg00_GU;MQxLk3L45JR4Y)12q~>+HObMtElmt5QxyWgY`02U-GsK_X51&6{3& z`li`VmmtK+i`E&t0w+s<{GdqYuknilC*9GZYBa-BdBSqVinmYlxc0t=rrZHRfYH2&$}lIDkrCc0 z>B-};!Gw={UITC_^0PrK`5up09ImOY^0|Ey(EM<7-#a_|$(^Q499c8sRN>mLmi@k3 z$AhPyjm4Ct1{M}l2d##`l&eXnPw-@qcPUQG?N0b1dwcu2ij^7YLg zbZaBe|E=1HKLWFQJ^>dg;1>B2^JIh(Fl*(Z7P?{Nm{(pC#Gp2hvv}O(;w?dg5t`(0 zY5n}NpPh#h;;xID$nu}Tm}KGlP)85puFdfiX^~k}xVwSe^|iAIPshf|vHINP@v&)^ z8QP=c&9yIb@akJu@Qr2)y?EXn*Z1?n?&{RD_npP-)i82!`zEYSO4Mt9LAfQFYIx$k z<3I!?dvZWE9)Edd9rEMH-dsI*&l_D(+u<0Z1(TUq66rZ6xZjY!A;QH){)i4Yp{z#V zDD~Vk&Esf~zOhjlD0x;A`OLCh9B_I~I9IOLumC#!Q}@=_lm@ZHdW?s}`>>8I!c%i? zeDF*Cb|5VkBIwWCERa%$%D6%6|0!li+`qiMECFn0 zU@q;M9lZamt4Hy6sF9%0;x@tKYQ!-m3teWMo*q8F*6SNJm^m5KZ63TZ;l`}x^q2udz zW`;XokKRivgB)`;HTBiqo5&BWt->J}7akpD)4CDGCRa{Go5S;Fyd@T>n-l z1BdcKfO0AI*=uLX$h|@L&r#;lFX|WW?Y|vNpk6?IhN}3}<}pM|i$kT(-B6y4uh~m; zg5x@gd?lt^lO4)~HARkRnQTnr+1AOCbWjY#&kN?z z@yrKK>W46NM@t?5%_yFx^4Vzt#Y$BV*HdztqmiSgZx1dG4mU%f)3q+73&ji34MV!W z==ST*bHpwh%{9tH_(os&DEGtNXlhTMg^Ua#I?*?#z6kO18-E)9CS;R(anuIuM|kjq zgWhA8e=ql41BNyXJwt1KC$LeWLpCncu*vD~7{4i zuHv5!(4VZsw{en1=F_ zS4WGnXL}2cfi0O$ITy{#u8R{U7HXC8foYgsi$kov!n)*ERaWr0l%&ZKyXKeo!*BTq z#bE~)#8-P!!o%lIH@y;#2v^+S)m9nye!V%99WcYwlOH1WPH`|ND+5?R$KdW%y%8U% z8)COPTpofy2Cp_0D&%)Gm~#@*HzaK;yRafYpLW_?sKP4V9CBV?6}w#uoHAA6!XsCS1{`9RVP{e&C<%n zlBpHC5rI8Lc?Nv(M$j-h$h!g^VgVSBZespy%{`G@E0V;VQoGR^XBrPGq9dgbsPLQa zl;Y>}B1EqnUrQWboUG5~t}>s`BpAEEBoAe4sbwOk!zPL z$FS?*la1YGZkK_h)oe3pO@nD4?ICY;mzS2LVV2||grHJyvc=L3C4l(8UpAe~yTi51 zLy_brpXw^SJT~mZql8S`NQfW>WuAThw80LHeWY`*2KONhQKJ}#r8>cGpx*gN? zZ6LzZidoI#5I&nxUH$RZluGN>Dajm%+aIDXKdOm9!cqmgd>6fp;a%iSK<2AWD5#J6 z!D{h-GbnAQAa zpT1aBtt!-&$c)`L*k`2!qyh?q&e6@)h=}uX%c+nY3jvM-1u6Iq{L-a$bIf#%B<^kQ z>V->6VPzrjbmrc=5Qc$z?3*lK zE5Ifvl?Jwii!!lwT@fNMaCw9VWP08`ljX76V-wq};Q?%I7IG6qs?2 z>Btbscr8~fN&Or)*$=a!(-~Tfu4h%wxxa2!A^jTny2fSu?kx~&pJ6}SNtC(n%9d%s zP*ZE1jZj<9u5W(BfW`;8@O9VwuiPB$H{&LU(Rty@NMF=Vqw|Xd;%b*Ec!UEaY}|;H8eCC z=cPMzFNugLNu-O@>bM5#85DW=Bu>cK(o?&pGrIm}H;z2h!U_IIr%1Q`sVeq659u&os`GzIQNh^wRhR$7I-F^(G6t zen0scwz_)r$n8Ow8f!DWS-^=xHw5djY4x6nW0{Qs+x2KWq{eA=BTe4EG_sPa=@ zVp=yt9PK_P+1sD;DQn?p?^vuNY%8|8!a`+pY!DL?AXO$X(DZ2$1p&9|lqb-O{xlU9 zUWk}zXwao8qool4yqwI&I1U`-fCgjAzWjLUYOBa}dS)i9fBKku>k?F;7EDJb=?!w6 ztGPHhy6DATfYk7TsCSxEefYC$K8L-hLuoF^IQlem^JI-xJQeO&<8bo`gq?;{#)}=l z5e!_r9n(3s1FqnUcq#{S<#Xz1`Qu>?9_UoEZG3#j=_Y|A$s!(KWjE#Z(VujrpFn$2 zZawGfJ@0u(8ZzoxT4HUo0rjjwR}L3}m{_gnIyQ)2gDVidekH@FMjIRN@>CfIZ)KwC z)4-sP-MF^i8hj>YRL$QtoBIb7PA?^7qg%kHEUTlj*okz%7ks$BM(EDAN5v_(3SM^w zF@H4Bh=(;4T0~yU4}to%Ozq@&QQw& zi^2kn6)q_&P!Hnsy1ol|CqOTQjz8H?@4;wrcDa1#wXQb}nxU1J2r$z{-|m?|%%Ee;uz2H=9hgl%8% zJ&xzLAPOgwxYg?dI(jfUSu&hV&hOC?YbD)BV?$DI%B9*(v9e8OJd32bA#xD8xAgKK zM4;Q0Ryt)Ur&OUaY|>-@bPY{RZnX9TdfV@;UMT$o>zSY;5DW}E?XB$kJ?*EITMpX> z5MqI@yX3_eCK3^iKo)T@nVs?&pYfU5^n6Jc^c}>}QUgfti&H%2hH6t%8ce6VXwQ2| z&d)K9_=_!kg>T&5?G>)CCF1*LUyx-Qv=z0mv>pj7ojdJ1jBK`=U1e=$rIr|m?qq^>VJXq zYn1Lhh(JOnnk`guzSLB%eh4e=xXFsv#J~V&$XBOb7%0hExyY8FRvKYYga?RKM#~Vp zcdf)E&y`HR54i>;9 zR|Zyruw)^#8E*GAx@%0h8uApR{%^0S-3!WSTC!-)2q5G2- z^?>_mYr&zxDgrFAPq^4{&|pjr@C|HvPM#mMhQsEfS^cI@ZI&VX7tD2VQGln|ZvU*l z$Oa7^J)k_aDXrLax|QJ^fe$2UNBmap!pthK48H`Rjr{rZ-I)|8b?jbA&_<&#ZMe^! zRVRhkK3Q5nLtoz4Uwu?Ij5?x}gIJr=-mI#UzMYlVsh9R?^r(exDPg9a+7;1n{Mu6u$}EK8m$=$Q=WYoF|lGd{v&62JNOq%=(bwgISF zO57&pfndB}mX+0f-`u8eLw>#Y{V&x_qvSVfnNk>bS0gkeq@H$GzqdgjU?YQ(B(LfT^;gXU{MrJ-;k0v*2GM0q~!d4IC;=-q< z>g{}ip+|4w;-B7r``+1i-Ytm7i&zA3+45BpGcqn4Ewq z0{`CP^I&x={*dohd}W>f-8UAWMuWB$Q&6c4rp2^OI8R8>RIHWP{LBT7VcI-Ccl|b$ zZ){WLcEo9J|8=NuQ?Sl$ykee476aIjR|IvMPmG5y*H%#=pxI9$0_S8NG!PULBmkb$ z*=mc>GOJcZr%JxOwrA?m*=)K(p$20rrc&(r^-DtrG`e+rFvtd3D&N-Y#MF5Z% zKX*Q_lK7WWIUcwM(!7#efyLU!uCH_Zah#G&hikDB;`}kk&l_}6zaW2_ZANiHb+s7~ zy8T*m4pR$}e*9z{hAqx-v)-zHu!sgr>Ds;GsN>F0NMCg_*J|@#PV1OHSP$N`sdCx5 zYOmN!9;TV7y7lw~OT3fM^#Y{x&&@f1seGG)x#Meb7)#Lm<(N7PhAq;Kbzv^R=LVM& z>3cml(FswUoe^Xd6fN;xY{pjYYpx%sXQmAdEyVP^MZy=Z8(y=5&#tuE)FouhlHc~SQr@@;WDd+*cHnPJ^=1JV6B?~ zmMb7hQ2}(qhBmxy-+zs3&`+dV=HO|H$}by*<2cvtgqx~%qOsl}s6rxw^Wky<-B;xU7p4U4tj6agI% z4EUB9nXvm)L*3zz9aM#B0&d?N3lmq@*YA1P*7ErU<8}-TJOP+>Fk!hHBnGv>w++5g z{o#>kV?~;*g9I34Qwp#N=8SjGAaok10}l&1h}+=YT$5dJ zKK@Y0ZhWw@0;+!C?pKK*sp&L;sRL>DE6{SqA56}AS3AY%FxTiAlf_Pej!UN%eezSm z`fP*`;e;wvr%>hNXAHeOD#lVoXUWNR=Y+luJ~%j?PiW}qzAI)%1lny_ zU-lDjc*2+y*p1#soaEdGGf*7g;XYD|mWDyzdE=(Pp316Q zCk+Jj1hDBibTWRXK~@+V7XH7-#0VFt=}#1DCUBZ!gA{=|=EMJTgqOjqKG;W=@kyGI z#nLXv&*;2)7q|25?CGS)blto zFffvZ>=J?9KWPTQ-v2<`baH&HU|}g&J4uKLuKc82IpF05n#HAZwlvUgX>Y#u;B!4L z$aS~32hu9SKh>Dv20-dQ<+bY~MZ8cz_&!saUsF?;I8$XhnVRMWM|ntGNTNA%0aol> z4H@^MJ>Z0&d2ahkJ8GcH($QwEl@;D3eI^@9ol~$jziqUM>wQKkd^<}Ld+J~_&_9v+ z;U!1-qSU!tmDR^7{}jknsVTFEe@IZ}P0qM1A?LAOfklQwy>N|_KaSM-9sNvqyY)af zjo|men;nU#^Pmdb(xOiVCL0tVhe6Hz8C3H-ZUm}pJl84X22(Pm1LLLC8(p_M&O;HX ziZyEVd!!kvs6efU{aBZkRrgS;;LW13f)KBxiB*}FRBi7#GU#Sa6lot&Mo zoL6$hCush$=)PF%{~!o14)S^ZbKh_}^;(KEO|PsAntC2EjajW+p98oiDJ@vR(lFz( z{2_nFfW(KW&?=M|j!GtSTVa`b0Yh=RWZ_Id!WiuwOS1A^A>NhG4iF_LME(L~hmFZ; zyNr4fq^2|T18~{8_aHjGdce`5-eAiop6;ZTxf6+o#e*OfwQX%Tr_v&Kk%L=VvmgFRN%wqC;tUb=PE9(R+5-T2SJ6yVXj=?z8#-+qvY^;2wadoweT9>A#5?Z zmSbf=zCte&ZVprw>Sfyy&_!sSa9~6^biYmZ)b$!zG@Cd)iD$#@Bfp4yto&Q-rviuh z@h6u0Galn`orVP7s|QSllyRcfrKRh@wj%|MT?AvrtibwoR}aW-!Tssu7aaHI3@^fD zyUF_>A^1uIz(uqbsF%|zW!wd$Ho~J1-^&cw*cdn-%=Mh_&w_EYfz|89^*;KTuRa^EVv_;sX}fS%jK+1 zVP`i#>?Sea_W5Iyg>Zb63{83ooN6m)sel=xhI#>{;=o_Z-RZcTogy2{AlGPr_#^kJ z>O_RpbKyEC=2u-($!gAik6`p5vKklaN9)zAiFAoAvO*S`pbQf|*Yb3sgRaTVc zxTZ*ouDN_MIGHRKVZZ8&a27^O>Qf!%98Bfy?V6vlw!UzLup19ZLSIp>KJ#JYjHVEp zEwvriy?g)i&Qs90-Qeomp0A=9O7rvJ%Xn4J9bKr#)pM~s7@Due@5Jy5QKS`cZC7Dk zs%(G2)oc%St<3Ej9N2w$h$(yU&G)X|@z$$rCyN%)E<&Nx&AvY{>k!Dz0_=mrH?C#d zsh%94pRRs8U0xgZ5%k*WX@faUS32C8*BoC5eR`W%{jlO>m*ZiOZbA+g>RuIBolRHi zSJJ;B#m4U6UiBxOD<|#S7(!{4jK(Cok|;|S5eA+91bCmIaMB@16|mCm10hR>E2#Cu zu8alKx`Y<(Kih9Ei(QA4E3}0R2Spz`*fo$-=rd;Zr@LbR zAVq-FLM@y*iD%oxg)%fU;vaHGfc@%t{s#b4ZGV@@mHhu{EWTV6Fb+RhnyGDj1Kp}b zFR~o7IMe?c{uu4iU7*c;(O?xAAf`nHc#)QEQ%Y1IHCt-Es@YuKoyepGG-sQtz6cIl zpZi|e#RS?MhON`ngsiNrf`URT<{~dsZ%`{tXH$D{8t>@=6DuyO61z;&Z>IxmK!_EU zyb3yP8)O5H@pB*sfLNdHQG5kvyo##86EzS=IXiF4)DOiJOY;Fqwqz!jH!yaoWV{yY zUuC=A2RbmIEd6z{kG(DGJTeJvL66>>0T&H_sAdV+|Bxyx4Ke7KHUslnuS?$B_*)_H z()Fj|e7uI;0ZqlMuZq1yzLt~?j^1i~i?W+rH#~X9^t$n@#8j0&7lgAR2A950 z1{>!fK0pi^-YxHv8`1QE0VltQa2)0qPA+V$?Sm>T+VOD?U5`TK5GZ|=r3{Vuuj_!aki+Bi z@;+%TdR9!#GY}ygj{Qy8kE@@DcnET%D%>67D{5czruZfuymiV>^|g!0B9MZow9 zG(&TSkc2r~8iwuNL@wF^eoPv(u*>j(3a4J_%KItZDm8lowI3)8*0g(+vkfc&D^9ed zH)gupn5WumzSV@Y^ZJ!}f^Cn82&Z>L!yQ0UIbs+ku(4dJS?fuP1FitThkCd7lpdYoFk3}$>+!nYOoko=Vn|^M3aAYlb|s3K%o4Woe*OsuNV{4Bt2?B(8TqGP2K3sO39Ls`xmyxfsEvdG4Gt z3%cTg92?3&i++2%Z|v?-FGuBulAO%(2XrD`y#!HZW%HfS2vgPwo)OPOAA!)@O=8*$`^i0~1$5r+0U`If8}!3|cn`>8;_=OgG~0`bClFMhua{mGnZKSLvIeqb>l|Gw*-JDk<7(Y`tO~={I4Tp%RH!dx79ccI6 z1#HUR{lq@{*Agq9m=(={5`tRS*f0d72bS}7Y#{zAGxE4t{1jf?weTa^3-_s^+b^X% zBMKwcpE_ItG>&%qy1dd!X3U1$5hJ>Afl0q>il3UHg$a9Tc$iIx+`)_t zXdn>ArBi%4a(g&3xZeBWxm#E=TGR0BEajM;iuIKqp`5!1fj7i2Ie30r_6e)0Ny^pL zyC1a7_^CKs?<#*CdS5*5KUeSm_4`r@t6qvwS&FEaZlXO&h)=m$^!bIA_Q(BJdLZm7 zxUc|+*ankj!J-$2+-P?+pKQi;NqM#LXv^cvtE-O&I2S_+pB;$GavbjzX$kfXqz{QO4ih0th7RQ7u` zoUIPn!Joxrox>94(5lIWKLO4k(|kqXt`k&pI!!=<>{}>-#7`D&xxnl8ybnzj#MKAs znsH$OcJ#oTuiW?uer0C_zb4*5bI0p|y-CIXkJ3UsF(Wz;BLpQ8C4yZH)D74SI$t^; zE;+~8K7)R9JD>y&xC|!(%Fq(0l5peSJmgD&1<}c;YxTN7t;)9tr)gD(ll-885s5EJ zDsB`V<>W5X?JB`aOFVaR7F?bhrh-(MPU=(>_(`K3)IL_pe8Y2YsbMnXXn#S(k~}5$ zBl3OEUg`r`Su?nMeGTviOtO&Ba8eU^qbHQn{Msusn@Hw_@+Vh5_7Zgy`n=ma?W5J? z98&Ms`<+J0lZn~l@XT`%c>3f54YP4A`5qk~TEhF@JU-&~NgzDnv!$s`3n@zImWKZ< zQj}2w>CYp4#u)R-k3izq@wrlEg&QZL<+K%l z^ry=2j}ibn{2KVz=wn5^3dXe~8K^mBBiZTl+m3}Gee+z?0b6d>P z@=X&fEmKNNmw5o#zWvf#AP%HJt1-H{8RD75fiK%P7FCa*8})A`#zk01_=($4T;FNbal~69Qgcpb7WA>>j%`Q z3~CSH<}_h2oiB4p!sTmv>Y6v5MeMo2%G+q=W13Wa@mQX0ir|C|2>TU-=85sCx3p*& z#@bq2gU%3eX;E+1r>ZR1Gk%#`zpXiC+zdejWgpVd%mgZFzS1SXt1YcTYzYB6wct>T z>)eAVf>_lF`@JJ9i{IxTS9c5xG~oD~@?;$Qm}pN7yMBo6@#3lmEp|*N0H=pi;akE- z>pfz($$UM)3!KqQ?oSjt=W+Sn%n53HbnSqV%-?YvMSFC97FyNNqK6cIo^H>E1DwH6 zwc}|nl^3c-)1tr?JJrN$9+V7_WlPY_p~c7e`ld0bSf{3;y6F)@JPYta3%uoZ%XjT4 zW_I*mqyTIAPj~2@EWP1#b8+YgQTr&{R1|JVT9ak5Q#a+4hZv8Rwhk&urYnhnymPn; zARN~iH6rSm-JE*?t*X+_I!FKa7hG0de-BMO8iv-^?EEMpD>h)lYp=}|c!+)&T$KWZQ=H}k z-g_=V*0zXm(y*1=L*0%N5*4#&eMi7r-VGa<6^5;efhrTEV&{)G^tk@oBgwEwcj4Q^&VP2OZkr@w%M^pM1sqUGDqjeX%J^xsN zv(RMdWz#wId*;3lbMisl71Ua*XLcQ5x6{)#r`_X_JlZ;V4T+{I@3# zC{F)5)2RS56kuuwZmsB0;t4jrKYE~7)bnLBQy4H=eww*c?$c{$vkkK7>*AJFl(tb4 zZp?-RyA`!^BJ_AQeqk9JJvUxhITEfCJ(9tt(+LEw72BH?HsJAS-W-1nU>BtzKrH&2 z-J|l0b7Mn)okdplS9nST2Ly;7r=~i*@*cQ#p>uBY#$Ol0jt|eDr3%&uHWl-1;HmQR z8d4kIV&}L$SlYzDYUrKeKF2;e)AVRR$q0MT&vvfmpPJ&2FX|X<;d^z_@G8ku z(9IST@}W8_?kV1DYI$(!Iaz8OL&KPnt%XztG89GNW(G?k$H_TY8QhYDr`vJpBvPNtviN1G1u{!~O+`JJ=Ou$|u(sX+ZTnDvyrg8tCfw|w5dr`LTgxoJ{sMNf}jf%r1 zCxE+507}-T-Bsh|F7_7;31u5u64eoGgIw_5z9>?&+qKpsZ8+?L8WgE9&63s#r*EDYpxfGpI35CYBk<@#ke~}4w8cM@R1V5AfX|w%n)w0{em(-5 zGdclh{Jtg+(BFlC2nk6UA3g~_r8O{eVmEU1`ng$ARtly!ym+(I#%%m*&K57EJS9pN zuUFM7Hp@<3O(QB_kKXoKh)3gS+>qegJ{fxRz2uJ;XiuL@GTfOT#TEDh(C{-`Gd1-sH%ulbrb~-K zEr9c#E_19k z-jdc{gSf6V(`s^j;ti&w()97^HlDIG|E}p2dNkD4<^NMbNCmEgOp`&2&jvn)oRZT2 z?My#A#cZrV4p1p^MVg*cx&n(a2eZI$qmKT!GO~{-TAhVm+GUG>sRVo)itz`Dz@Ja?mKY%jrg()|CUwG-X?&!$h0w(q zNizX&YfF+ohL`}u{q9tKrNC1)47kE*#ja}?!$>3TMKJ(9E7vKvso^v(M=QTGn@RvL z2%JM)4Q*b<@GF@4wdWYQZ2p#rZxY+S)Tk}MTScuW6S992M|Osg#iYS=JEI{=U}5cm zN|fU<6P|2aBO!H_?u5T}hV&=v3ZotIUb9vgC9iC+@|?WgV16eL{q7r}8l_5VJ(NJq z;X?@UGc>YkVx69CD&bw99Nenk-x-MnB0ub>Pa)nunK+iywu%*>2p{3nv(UIiCDe8# z{ixwPd6qdUl?MDTK_wLxpAO*n>@62#0bvw!xQe+L1)%hGj#0HgI*9)@2iDukQsNcCQRK>~r2G z9?Bp10f=+u?wBEf2ZMhn1#nq#xC~dg1UuP!Qiy*75e+ZD?U`y}`4&Q38D zvatX6upBbEJIw^?goo#M#TC4#n>po1>rcV%5SggLfLvZR1YNKJZ;KAOS+Ub4bPn9_ zB?hpnyQCofyfnS;cOLL3jsmQvRp@3HAUN{ctPohtIy5tDa+8U;w&#Qr*SOOc^!^6N z4TteS=giED;Z4_B;1k-KE_@8hltf>JHf@Zg>r<80{yqUQzvW_G4*)eFnZpCj#GpWM z34>)f&xaQ{{5}Ub2*2j$5+yN*-SIAv7j_IW`UdRv+z=ON^ZNZ%Sm>un$QONcj2HX2 zPNtmd<8ueWq%9wH?Et0~u>oI)`&7O_Qq_Kz;xZ+2*`xgJ1WBthnA3@4q(4`DNQb_5 zFBwIk8O0iP4=pKk81oRNUWZi#r3iJ>yRVr`ZZvhrLmO+Lbmym6>+kd#Y9Fztmn|G9_l?uHzoKqoA%3Z^*Xo61jXA!g5INTW|EpbC6OB*p! zCX$)Cq2Pd!6Yz-iEl2Kjv?w7r3_NDdSNTSvM`vdpv$IL5^K~T-n`Ft!E2*uLSyq1+ zNCdSEyO!bz+R{O5}~_*)1K~+$O>v zAoiyQ+a6#7C^$7JZjTkI#Icau(*pEsX{S<-Ip1jIT7L}V!qLU;Ar%p)d3&S;!Ww`9 zfhI=QM4GNBfn6!j-V}@vkiWVAL`h~Cu>YK$1%H6%$ME@hjj7#SnPJ-mcViQo)X>60 zD#LhQso5rmxrK%4SdD4qHO_1Cp~cB0@-Byf=&8`alm;B_`*O`(X7b=ZfBo<%l^!&0 znt<<`Oj|&olc?5Z^*;FpwC`(2-=~GGi{!sS0d>4`FPaHv#huSjV{7EzSo#iJO|73@ zk)vpS>1g&!%<kj1yVc+Z0vdZqYSP-TV_cJk}VW2m|)>VMXdEhKqYB)qPkLN%J;K|M~5GmcMLJt6k zAfmuSLtT^O%Ab7I{9-yzS4zVP)T77ZZ9<4RSd=ykwXzw2ghX!6qdOu>$w0*59m&3A zI81QjX5Q^Q!)-_>u+eE53GnnfV1k4a@pORY*yHhtBIEetB#>h{AN_t%T%las`JK|r zIIQj{=sD7{{*ap7Yv*@Kbld31 z?M_Jo5xMfq4=n74{Q-rcPaG#Ec=>qstjuJkP3SyPP(Uo+GCPTn@P-@+ySAVGcB*yU z!~hDC<&J!l%{jX}t_`;riu9FtFQrB+MqL+aDSUxNHk-s-xS6i`rf{C91S|fR=?q1K z!80i9#hl-j-r=W-!L+ZFCtbj8A=p_JJHd?v^W;6@-zKec9@?KRCsx43gv>Z@0VdV! zEX^MjuZAso;uE5#nR%{)-lxBf>HK8vFe>m4Y_zY-R8lq@r}W$aE7Tk=;;yh=wr=vL zBx2zQTFPJKf_Oo+a#-{8C7jmCLZvA+b*0izZ$8JeDj<__rF4j659jsLb}&APuv=_H zS9m!RE6g#qnOp@JwFstY|hxO%)r-|f~$B!BJNI1Ta3z2s)j21L!&96SFASVRjww{GqIM8FpYpU&#rh+gFD^NKI z2oYa_)3(NTogH-K!eJNdO&$+=Qic2%meEN6^RN}s$}G)_dJmW2=eKVOIYxdcQE7}7 z$@yW^lvKS<5b@UL_>}HasKHtM{;~Hat8N*Pc*R;xw^sQ)KzoD{HdT*WLR6C#oruab z!pY3LYz#=OSmZLfbIsQ2tcT~-9;jc|n-QFHzl)p-Ue^!I%|%jX=c{DdJ2Z%4QL1WI zn;UJ0SwKz?U|2vz0xb6rv2c7^LQp#xw@JVK=}}R@yl-!j>qPK6;BJlACh%r%4T6m? zvdNmIG_L`QcgySR9e=_fl&vf}us1>7r2ppAlu9zwD7p|I7qcQAkR&Do&wpHwoT*|Z z1KtMowo6))mUf2QM0EE46X&CyXZclD#?u=EB*J#z9*(}oL)k!Cq|5F!NvwLe_#1%W zwu6=PcwZ)#)J`nk>K{J3B#tjIouL8?6Qs0T0J9?Ke&|g)TXnq4%JGLY)O>nwn@2Al zEF}tY{>-lWK+blfIz$qPJ;cj>|S_K0q?O2MGF1^T3RVDs>HY zHrP;8N$HzZ`gh<&^92J!AdF_3sCVE!APy2lY!jI(H}1r3&TG{Q8EH4QFA%l{|u-{mwxT{f}Vce6{!Xf3I)w^Ln0fvy$@i zOrUlRx!QXS6nwc(pTLGX_+&lM8-3+gYkv*iG`Req`+$^a(j@UTY@{YvWC)bLRzcO$ zANZW_WA1eoc{~6B%3%S{qp-g#egVXijlMMQaw0c{Ow5GC<+N>X=j{_S8qlXW)LUA*`0LR?^b}IHsBN#l5(SWP_*t&U~w(GK}^u1Cpay>P`mp*rF5as z?oMFD2MNG;-~TXIrN0A_54-i@)6wfa^(O1f@9#LAb-Q&j!F5TdEE9t&4Pu#$ zLbxM@l7LN4F;x&i<;=mPvhJ3SV8V-GsraM>KdUOpi#rI0Z1x?}Ot8%E%g{f_9#dR; z@sgYlSHJqcIV8_CQB`}i;w%yG$L+6w3M$}5`jxIWri0iI0!Z=SypGvWP}qq_VWP-g ze3YsrQ;CX!a<8p=oJlv;Cm4UUPDiMrO%`h+HWs`K5%6g`h3qy0erupeMJv0Ug(-o{ zEF-!%E1C#%4kZWW0Lzj1%QI1+r_?Y>e4&d8LV*8Ntd-dS_AX~A*>8Fm%%{df~fgvGP@0D~dB zcZU>^`Riq6a zo8sxkDd$z@nEpORKCN;j@@OL3dOgap!kakShypTOYR+5FMny& zmz=Qgb$hiy<4!e)DS5urk!;o2(mEFnwZ6;U#Nx#a&c`t2yaMIVeAakd@@u}u4-TJv zR~fDN5qaq0G)FQ(e_$kM!($d?Seetk>ouLie2O9W&(|PMWMpB{7M&}SG!afB+TmVT$W3358atVi%&bF|{<@2U4MT9_hID#i5-@Z_^iJ>Va74_7)K1Md;N zIC8B;gwbbTqbzJ4rorV4GrdCPysQP}hW<#O zjV_amCzG4mES!F&DL05J^yO<}j@%^YfG#xtzjII?M!;XZP5qjZLJAp56*k^#_X0BriN8b`cwKWRw?RFHQd*MXhu9im2SUr?nG2 zH8{@!?%;C_t7^zq)C+w}BfQ3*&=NjR(x^W`#9fagL+|$gv&oUMFw=i3J>W2NTFpec z)04zX4gm=t9Jd6PA+4xZ#9WZ7>Q zpEBNs=Q?SDr~)VZ-HYro=Ov7nPprdBW~J52s=wx#GIPoa{Q5uJ%=vv1}O%h`rUu^wT`2il&prp$iOagTM+?JEg!7|5z&wvWk^Eb$ekO#lvj!Nq! zF#vUy9L%3#FvcsBY`vj_We6TC8%Trvk(Ra>0%V<$Zr8#B7v9geAr=Y=%YwVMpkDkw*z2z zcVkxy?Xw$iVI5gO3_^Dmnaxz>KB>r2pzI@jpL*&*x`;tMz-?2VYkB@(sVk!fb9%u1 z*^f=Ep{9F%8>JAUG} zJfjGY`nlhOWvl>NV>e3sx0wVZ6Eyd4@-Gc#C0Kg>+O-6;305dx$zEP&mb{S*>%iE> zGQ3#LivE&H9|7(?3$Bsh%a*QkAmH`7aHli__K`aW_EznwkR94<)JmBV8{~V?JddJP zy5*Qy-Q2&q3CCMsHwj{-Pl|Lt=z;cTgqJNX`pQ*(QA80+aSnC9`eQu*t)YR=Q!uhBa-uI4hvx8Qs-8)^1>oK#%S1 zGpro1yC(Lvc^*;|6bfjn!F>n=wd8UhjO8s%2hwABN7wNGR+266FDZ<8fTX|oUisok z1}Ni$D}f4tsqHuVZtl$pdzhTLpQFS$$dRx}ri6oW_cz|UD1f$O^33Y`%%3Rv5S-Ay z(X7^^U?YKw#(M_+-0c53gqD+qxEWdnBz@4lzt!*T1Q<%X*Z~B1s8K*M4ZQ?0z@*LC zB1N~yT>oU<(Z-jU&B0)~g4M*_AH#5oT26=Eg}(c;`(c6%6`BGHsZ}CrCa{e@^)FzgmB++x}+6V&(AtJ1Do zTkl3K)Tw^-bi@4FK~nz|0+1*UU)%@e#Bv_tO(T{>}kou6-8K zir8jndekq-{l{ufl6)V!W^W3cckDQ$w86av%{+6C0|KpYz3-ZC4H5{bU4Y~d z3TpgWym;v0@<S|ZI~aQbavq>3Xo};*^RaPS;UVbB>ScS*a>FJ z(eqWCS$V%QG$@!&x8=SUjxcjcJP>r4s7=|plKUC$XnoW9$GR}xw>j&z9nCC7411)$nx zihnpx(&YI@Z|Clt)mE$pez(&~(2flG<#rcXSor{kVZM5X|Kta{2X?aPj<49Goee>o z_obf750lhwg|d^ikGozIL@#txTSv_=H=@=7eufV3U-ao``7inec30;7o4X8fOJI9T zI36~hXbSKdo}WGfO?E+Ja)c7Q&q5uq*o$4XXi~418^OG!%N2Q1qF-f04rb?N{unNn zKzn!R&K@qOh8%NY{GHAQLFYi*!bl+Vvi&eMRKW0~sBk7SEcg~Qnke>*@4~}j@M2|+ zyn~HqQ~2>_-}&Lbf0y11i~{?5Xv+v<%53mL$}7nxkSo9zjoI8QOWjSWbFwTO2l4Dt ziN0W6b3}qpjA4L|dpm8Q?9Cilj#?p_9fFq3K;Q;KI=zZqe;LYu&(D1@Q;PL&btMFv z41!3X71+&|%R&*FH70B($XrHdyRv45LOF8{-0TE5_@!$$0)Bn}n#`F6wpAW5MKndE z>IH6uCe)I|zfmD7M_S3jXechU$D{bR$a2hGvH!6})ZfUj4~%GF6>wfQ(J`!d3Ij*z z8Vj$5%%%uxBE$UEk>hgv!FK0tYwt5nInI1XbFiQaKsXXocUxgS9)?@&380&D1zU|2 zO}~Cc9{LQ+Y8C8b0+b_oNh$m`3bXplBS(sT2cU{*0?a)GZcXTt0qDbh*jFKXL+N<3 z(j(!{$$?Bwu~<9{aaWS&2L9alt~S&Pg*PflD+7{Wn&YH+h^_&Gnk1fa^E1%#^tqVfc3SX!_MX-BQ z^0-(%@JA|ue{BCZ`~$a#_frInpFBZr1lODRpU(z@GvRx^Lg5Rf-?bCJ{^cm>)T>1} z-yP4owRaHhJT;AXKa@{|4eBSoJ$=-fpgQJkJ9>BUg|l;KGXHGh+sQ3-YEfo&ifc&R z(N7L+ntpAiWLYNqAXfji8iB61l~rb0pz$!6%7li_x}{g{=>|fw50|~TvngjD(ZUZfJBENXx4Wly~#UvEU0*Ve3U+e*X#CF zX;8aznsiOEPkWS>uFf)rNoMVy4j)}P6}9B-?*I&|Q6Q7S6A#2+Y9RjlL#n`~T=JCL zX6UBlPRUjlxDNtnq(DPM6KeQ9U|Di&gpZpfA>4d@E}(+A_gO`0S+??cxqC>0g>4-v zK##K&|I_IK@cHzQ)BA5|g&y+?Z#>_!}SD6hGZN_#?|V|F>781Q^t-vPe3$;3=54MNc`M1fV@tGx=M2B7jTs;(<5)yvb1w#c0kek!{_lYEtZ z&wU4VK}6-&NDMs3BH%qY$b*7|wDTv8v-#!fX&J7JgRI0|^DgmLxp;n0rd6&QG?mHScDJ>JdUSNO~iJ*JB(rvqyCE0TQOf6Ak5M9Kl!FET*-x14l*W(gg0TVJ67 zQU9e_h7u2zs1kwhrS1DRFK(Q?0h(ta>t+rF1*xp$U4f-$S~83ag2Ds+Nzdri%Aa6@bH z#{!CPy_YmCB)v(P>4MJOp$l{%=CK@efLdXY+A>E8p2fLF+>WSnx^J#!KaZlkpsoL( zKM5R^UPhDD7jP>94z@0wS9=8M2ymD;K*(>|?$}5%{q>`{dpnK6j_bM5Y~c$5Vj2XP z+AC3Wqnj#xi42$NG988Ivq7?R%VcF)&fbEM_zQe%?D1y|hTVV6Lrgfha1rX}cmWxK zxl>N-$9~}3E3UphYDE;o*UDvmks`u%ZFy}xMq@8ZCbUeH*$mGU-4pY1bnaJ#X%EHz zp%rlW%-@?7jJK*o-9ibzMPh6;v-ilVSPX1P9g=_l;M{@iTV*3{a9AMOZfTxlZ24-j znbvKl_*PIu+sxxEW7*UB#K0Rf>Y{g@h_M0?lHbKG1S1Fg<^*63NwY$p0XVKlK@3*4 z8ow~fJAWQ&h%0nIb140E*U0jfQ1OiXg^JoSt8CQ0Ctr70p8|Gq7Ka`I8`Z8%DU~<4 z5g+Fw9TusX`hyq;{tvqDXW_xD-On&%&3UDoSu|GJ~fRx_2NmB_mC zDZc;=Jp-n@O^0{E_w?)AMpzwVmKDmZMf|RJEh4eVMukGJmcBJluqK||tNJd`I$q_( zR$)b#5)m&AY>AoUNKbaYY3VM>&c`HAop`6r%B1I|yF3?PuxgRwc*D-im7mtk8pZG8 zNf{oKAKCb9*YDeQT9~&rE*E|68xE2|u&zI3k(!G)2fJgpz1s1SJVvO7SteEA&AIvO z$0YC6ANb+MPmHB3>~7tkoP3eifK}mdM&uOr)7@EW_X>`MFlQVNE%rUK`GOY4x=I)XaQmL^Y?^5`|M6ZyQzTTC#&2M(9Ax1pYpI#<$7O znr`pE`_aRhMs&SD&7-3ER%`xJh)f&3p+XzoRxg)4;XFtvkoB9^ItwD zmHr%6z= z2SZjR${+~oNm47UR(mRJM+hgGHz=Cqgckm{rR#h3?nvI$l1Ssr9d5pt?ZZl-1x@H> zDj}wP71MBp^?{+jWnx?Xm*USwE&KG6I=QgBOgTDCIk-go^C7=%dHC^mk+(rmuHVls z0#Bs<#S9qc99*=z8QZyg9vk^BA493=6&G|9;F?!J||(QOsFLDe(yMAAcTs9R(R$gtuW%KQ;}fzTSkx^RyOIHg$o80hNT*)&U*6cN0r) z)OCJTQG0te_okck!&jGD_ya}{PBzAz_gTaD4rZhJtAUM zdfb=&VCL-(x8h2ci^5ki7IT2H#!AUK5Q71%a+n{wgSqe>a3dQzpar{>o-i|p;a0s0 zNZfk)#fU~(0K%<)nA@gq@&o2peaXkWMr7Zetw>trqdlNQ^U|i)duAdx!-`*gg-v^s zeZ$U9JRzkT`gP-eH|1vE96wb4g!*xb7aCC-y&AB|Pva9e=EuKbvnaM&nH81XRTkXp z_?YAX1oD5cD&8RN#~4o`ekz-=*ES#Ss~f+5#XeKO`Kb<6nOm~IY(cKzp(wwvkIVXF zJpskz8?a1;7#i0)cM>y0__X`xxv3OE+M@G37a{R%K)bPW9PWuERDe8`B0L8!_^PM= zpJ$dFQ^{4h4#;Y7*`%qvI~q*DRs+5200O!4a+&e9O2=%YWnq9k&g~@ZmSf((qnGb; zt+)DQksw4VEvnM;4Isy2)L?4beh7_rjew;P3lzQp_NpcmB-{UrFOAvg$cPHKNUxO4 zhtSxYB}2cIB$vIEU946}xXEvn<#HQj+i$d+Kh;5Ssc z6$*g27A?Q#D#q@5wS&q56CLO7hpwHmNcUTJqaL&<8oTyZC9X--&B?C8)N=d?;Ku5! zo{*n*BlgW}AIq}t@Dn;wg7^pa9d>q@H{)~n7Wj;+pX&TF4)^lNli*V{BCRS1@z zz23mPZ|;SXeSP*^f7n29;af6S_Zu=P*I&uB;oc7Ia+1Fv?PB-6Wvz;56wvf!orgn+ z#6?1XT>a5Ju5YO9vyiT|_SzWQjQ2V8E=+DsYmCa9lizWmTTINxv$1ogv$KJm4((FR z)X&g*4A+@TS;g?J5fqS(624gSLKn~oUAW2Xi4TMXS5J&*q54|}ikj+b8YU)mcX#)t z#hAO|O3KG_sJI(=Sgcr8&9`Ve2vt>kwS+vFX=rH(@QDNO$V3@EE;euoiD$e$TCOO- zrkg|k;ssU*1)*vphJQM8I?rrHXMW5VdUohGYKbOst3}luJiP1o-c*;s6bVFog*}~^xJ`LDzGT5kWp)q{d0bXa zkMng3wuvg*cwHANdKXzqOB!^47Tpt02$6)b_}{X?@}EX;L}|pOQ#hPN?9m`@Wyxe(f86xISqZNSu8j2?8Wc}>{vM1F-G(z$(D`pH8(|P z(v;c}_)yZTnZnC6nYM8V|7_dwdx!as2ki>SPw*6u2?^$ne6Tn0)Opk)-=q*Ujg3)S z7ul+zpYxprWMm>sy_pvLpu4@=`peK*dUXFcx6ru|Dv`9HTT{w;Sy-cS2 zF9i+${QZvW-l)22em*FPckW|e9ylifxcmi; zSLBqQQT5+K_n$B(pnrgmOfH760okE&Ao-pv=Se{Ckd5wyn^36|PjyhR|CTj^`P;1# zWFd)(H=tWAESPL;Yz`r?1`v+7Zi z%E#+0!BWdeUn0AVPQ#WX>7;F zWL=zKwovo~Lz>K+N-~;gb9vFPIv9)eK36620&yCC{n1-ksJ0e@o-QXY^18p7peCRP z%X)tcoe@C@wTIkyTRHCs^g`UT)Fqb*uBGySw3Od4jF3Ltl)-EMaxZoGh8Wf?uaWN;>0 zsrTv6gtdEu%Z{fh85s11;_f_w|E+vX{~okPEQk>rIGvAe z4^`gRz?ZVmV+!weXjk~NsH*Pn2U8Wc;MkO|WPr{1;lz!&UoR3m5;lG`$F7ZoO_K?a zh-gk<>Ib}S17=ZSB5JVm6so%Vb4lsI*|DXR*TS_J$2?sjGix>+xw`z=PqiVRz4yUf zCz><@0(X18^Kc3yIDd4^tnCP@{n$An;{-qS?Ru}my0HYYl&7N)yUR5gvKW3DikKbp zG@=ojuUdni1D2?+D)HM>kx_D1mv4W*7UpqQUHg$K90*e0r65H(Z48s#8Rtpp*cmEr zkCci2iObslroo=JXrh5V{)CTBI>uyv>(}TVg++CgBvRaF!G)U8u&BVqU{dZ+&#A*{ z>MQHpQhDMyZH)8}U8@-vhuixp3@D32!%uDFe|!n$aCNI}8eWNbXS#3a5pfiC$9{H- zzO+pY3!A+f=^6M|Rxi8Q4%~7X$b74L`bn6})u6}z|tK)j! zOOunX(`UAq?>W`wcfgw4nN72Z?Bl%>rX0e6Aece;~+sc zW`JBEEa>OSk`uA7J|q{BIU-HmiuNBdcZ(=0V#CJ9&fZ?=PMgPNRyKv3wuIikp{BmQ zi1Z8s(3nd_7JXGF(*Fc>>kfIfmQrVrB=O{X>?>+8j2jV&u}Df%3w`L@a>(mNS>h6= zVC|LSbc45(JzU(c#+J$7G@VBmT;nt)hrwW5-_L}VPHXr^J;q>#;Uea1;tjnQUxYn7 z7?8zENRf?EKr;FX)N`@qgzg&;Sxc&}(uULU%2WT*mr>B)KvKZj(J@XiStt^&^LqIT zM`~L2Zp<)s7y`ib1|{V86vnb%5xO64(;q9QzIZ)yXrqF&L)LiMDwCd35NT!Z+%&a7 zj!aJv=v+qN))+>W2i);r?;gVq z%4fD)OFte4A|G|0!j@c+#qycZo0Omp=mm26R=*_`^bsN^km^+>mU0_LV*K;XplQvf zk6?^_ZgK6SpTNV1-j!*7vd8JM<)lEtMFie7x6r-F=Rg-#q=#WjOIy|4sXK8>T zL&Gg}6H3p?ipG$S`aE*`{LZJE1Y9JDW$-CRMCN?#TCvT>R>^Gx0?@HJuylK)Veo#H z7wO6Zj`;mJ@z)J&MNL&BBiE7zD^VA4i~JfYLLS+^?8d%Ra~}3eUt09R1MQ6O1AgV3 z=ld8D&caP=SZH<*2Wp__KLml3QmuC<+P(e>ncAv8Bj(ML8m+FYV9pm8Ph+Eh}`|sYo_+iB$h7VH#tJNYc4y~-z5~^DXbc=eA zPe6$Ysi>+4gjJE0IqgEwjYI(s&WhtT>nCy|;5U%PENIc2C?H;FM2vIh1fkIgYL!&g z`yBY!8=A?0Rh#u`8YII#U?rY%|4G*QX zwoQIVdO@#2mEzP` zcMS`T&CbTnfAf_>s*7j_R}?0Qd3nT{*gKAW2P?zH4k2+@W*Y598YgP-5q8PrE{D)e zbY&_kZWp`8sH2_UY>hiW!W}Z1OnQgjzpnAFgY^vd4K*(wov0S7=?7UMc?##4>IZmg z6B<5}yw*jbhXgIoa>!H9U_n9eB)V+&z~1L9O&BEwQ0UP zeaG@dLNpS2O8o1cE+f6?UZeu2+x#dph9QI>n5Hp7^6|-&t`JQ;{2MnRfKF>H6C{fZ zbSiLw-9bh>o}F81$LP|`9~J6#2FpT#!~;%0KECaiatqzv{}vyOch#?X6@|k#I8+P< zP*LIV51?07GBq)wp*T4{rn&bjV)$ziT%WdII;V__DjVWup+!o#x*|)ewp4qaY@YrQ zv`*v2LwON7kIVV+hxaz}S+0U#yRvQa8xpqgFknWVV9phJzm0r zM75yBc z(Bg%I0A`~%v?HbJN$c|NkN=jmG&}9zfSH<_&MYkCsmN%Dz-_iruJOlXzQr=KD5UnL z`5!V+0rxWLv)7@O{PseK(*>*?LD_wr7-Rh3A*VVf4KK*vaKYk?2lVi3SQD7vZmdG9mE@}ozCgbjxa zZ#rQ!z{Q{fgpc6c#tsmyHA;yVaVz_OqB59)fPg?xU!S;^)-~Tl`qVeO@n~`Nl9FyJ zA7Z8Kf6gL&ODcuXoGB2Y!^4xTsh?j*91c0o-|!L8zkffE%O#;#=R=i6#@6D&%UhrX z_Hf@vAwWO6H@uUVQCwaZPpb0~9F4dhEwbDwTTfaQ7M_?g7BBAjVy<%yiSeCM38SsU z0CyP#<)Xajv|rkKTs%CukQ(klgq(rs`vG!?RLtimbASI8u(1)Bn=B_GLE-qSOSH9+gkz!D4IM8Uf+k#d(LZ_Y2byk7UK?%@DdXXp z$K_F+5EA0BUs6l1c2>-oEK4Vm{x*T}mM~szLX$*Su-36YLWURIqwl9U`hMHn;7kwWHBmI) zg_16zuf`vzkonEX+WI!VmzSsKPh7@swOB!jh9*`s4|G^s?_w^qIGHN#Qtj>4s(sEX zS78}&M9xcdi%E8!_A)|nMMax$ zZri4vN7DL2ZZ`3SvVY~sYo~+wag41obbmXvMjXf)=%lyksX=|K)c-K`^G}YCkNfQH z?Cb>GGk4*kVW<&^tVog)On!95GiiV=2a#zVg3GcD;Y$?a_(l-Dc@J8*KBIjQ}qh)N}JkJ5r5SMLo((nuS$;RGy4hv73ibpOIz=w47QWQQpF-H^=s&RV4 zo;W_1DYG(Up;2>h;E}pn5#RBX zh1b7S@ZiRu`mNgL!B)$U&JeiSRkYE^zE(pck(v`$MGs9kcb;4uD@T8UrDf8UOMbo2 zMTrsJUtl(_>LEl|UVd-~!je)IV*Zcq(q3E0ww09)otXKl4UfrV5ET`1_>^w?od=v4 zE#&?fsdD2*JVQc&VpLW6?2qm$>D6P?dU=iT+D6^Oxsm=P2(#yrW_Rpi?b)f_*^IhF)MobWzhz1eQ=krR)L^ z_b21=Of-T0G(_Qm)1~T!;8(c0?**dI$jzsh3n(f*j!K%2)JSZ4%?H$o%h}?nP2c{g z_PjYL8G%EFxb@^eHjpkM zEv{Z~Qu{u9!K#_N*t;Airz*QtV`v^OHyfjHE}A;~gv-g)Xm_DBF2HjLA-Ucz5M_1VGeWKU86{F zPq3>yCNOr5uU6z3Zz)1>cw>6xb6lnCF=10PZMGFnR;Jy~mg$7&>9rl+XUGsS$35u} zG*T+HvIN9;)R2#Gu+!h?pitbob8T^1Tt*&?^TP*p3@iy*t!sz@0R)7k!M2l=t_Mp* zlQmT{DM{r1vD&yT96ksGPbaRI-*Inj%C&waOa^3{_|8LNbsK<5i3XV8=8NkNPgkrB z4DhICdcKGKkkDlxtsgjV@Ub;{@jbk8f6PI`9s?Sr|Dn+o`Wm{1$yBEs}AI|CJD1-&Dd z>Q!FX`scRD)dw#$NM3}8$Nu(NG;%yP6gEz)L%0+*pCqiUZyE@B_<4YaJxxdEt_QOS zG)>}tK5=Q+esW2C#eA+(?cfC~(xZA3Wc||{8HR=KZ-iw;<2wWe8Y+1pYT+7vVhYdr zk7&d6&ysf|!@^?zzKzVxjQ&x+9wiA&f=uL45CO@1M3If+&C^o^%JmHps#4VZoP}=o zh)M>A&;a?I$q)-g#|tVgy*t4391g+nPkBM7gNZ++nTs1LnEEHNN%mHiJ<+Oht6_dDh5U3!{KZ1cx@_|k1Cs$fY~64Z!L zXlCe~AQ>sjCZgyPz^!OT2)-avZ2S6;Yg1L)a_r+35YPcw^){Si2xS^3LPl_5r((uz zTupdb0o^8++RC2b`~w^b<|SEJE^^2Ls)>%BK)?s2ommiRN(leO1qW^;GsGjMT4wf- z>|_EmHE>COH3O!QfM6a2XZ6? zxkuv7_s+~B0{(tCQx5zH4u6J)GHLFd{|fG8EYDrk0RY#DLI~@-08R}3P;@cXEMNul zNAro`7U1^-qm`7FmTSLp;|2(M%;C(Gq?`20vnUlc4UK#(JS=J1bgI;;8sCzhTdR7U zxYfr;DsLZ{S2h1-$!R+FL6?JUQ+<8?Dv(XSlmelwo%HJ<_LJY$^DSn4f@6bitIWGP z#fQX0J}!ZeTqW^0pr+{<@(&2eF%=dThW|F~l2%|6j|(06)-%tkTgUz310 zmvkc7(k9vbPDK^|k$0-GL#l(zC#dNEEt`OCBU$*;rtSK8Pa1;$gZfBIBPZjG->A&c zK-2(1iDT70Lp-8j%O@KuLk`&2XL=(azOJ2pA@0C)mxg#{fq(k*r#T1${PWXA*=bWn6(1;>c#8Amz&QoreC0#j?;XZ(hb*rs7nUC z4Z6DnQgPZuV+2~l44+H1cp9)VC3e<=hu#<1x4d&5J-1!4D1QhAH4T|<&d!(tHX@Lv z{NmHj!BKGQoM7XR`5-3|FIu9?D!#e?6Z1+pZ*~SY{;1O@qz*pGf5cJ}hT?57`R2iI zE57SP7X1tnvanqF+?1RAsKGlGskXsO8~{O1r)5&&QWg{_6dmhx*bjMolVOuuFK_Z_ z+f-=~X=OW~3@J7TShZ(lnFoH3<4retPl}I>keVi9bTmMDID3oOIz>z zwRk?Vo0w+y+I+b=vUlxHNNF^Ps=AYM1DZPpxx_gnkUlToIb{= z^|2>w?HGcMy-%DZub&cWE_Iax{n9W%FiZ`Yw&Cvc2#U`3nbzh`43jYM=fW+8XV$k7 z;h~Z1wlpz2*;F2gI_|Mu3)hS;HXi+1^!n2b%Omx{`EyyI%w(YL&F6Sl&AU0aJUl#Q zR>N#(6jvfKZ|vvi*Y}~mzWxuYQ%xlRSdm1Xr{H$DJ^JPuE)WX4vu|udb&j}DPwkG~ zQZ-;$w)T%~(EWX9xXPO+GAjoRB;?WbQNK6t)GTZ*opE`d?dr%KyZf93Mzr%HLDF4E zWI3#Zabxf>JI3+6wPjL$v+Ia$Vp2o~Y(7!T$>GrTcgd6FA2@pf%qK`3$9u?k@oP!Z zwX!ZKyinjgxr)d=xUG+$LYQ`!{3EvVdU8-h1zNqgJ1xD}hJip}Hbcv4ke3rt_3Em;0p}wSRWvvNY1tBKgP9Fm+#> zoD>+fVi~fI0!a_~c|_9I66-WB#It%4r6+(u0uU@ z5>O>&kDXGEf-0TMapyHhQJzazE2F2WX^-tU5sQguM*dcCEJ*K>N`0G76#nzEv3C_P zi2-rh&i)|<6&iTjr!9AQ{_a_4Dqq=iw$nOc;^hrPVN57rFE3ua=>QE!(PrS;7>YUG zyEY)YZ-8U{{dCMrss=_!M?ZSfr+EczQ*$#5alIr=;hdk2qCl2(04_cZmc7%_!2(FW z*jI|cP6oE;RczY%gcO6*?>fSrtgA$WLLZ__^^S_^?PY2>IU!fwwHCkyHBMwoW}dMy zGiO;86uM4(iD#Vm*D!7!guQ44N4msmMC z@HI3v9PqBN6jS@o!GVm^u1^CQ&%Jv}GPW?vOZ1-l;Vd?-X}gP=gn}ZB+}veu_JiW# z^6?EolJ@bNnUUld7SJ*XsAYVQ{+SnJ|M}6>#>sFr*amo%CF;Yzs&}8XIhQf@aU?ok zYkb_e-Tcpco+qc2CVI(EN>b_+9V~^ITZp2Y7p7d_Y+pUb9&?w*SQhs`MQG_JKIhcs zW2P4KE}-NkAn4u5FDz{8>EVz#Ry$roVZS;RIYH1vcu7lNp8{m1L4N1Ti|&&5_Qq&& zjN#g*O(8&w*KcHLy}1E6ZUj=P&Wy(>y4J5nu7S80aAInL^kSj5R3tIW9$*ifjr-iM zk(IPe#C*3unMHVH+e4|o@=3jt?KU_1#}J-2vf*I}k}!27W_?KUs+Cpl!*>_!plDl4 z=CX&XValEtva<{fd_BCql~B5$zKB-ByVDa+CRX$BXn#$sWa$=QSE3>#BdJd>4l-eT zaGyGQdO+n*LSgu~*ja-_%SkN@Hu*=MP# zsePOWJMm=0WnkNnBk_uT(x{njq{C^ch3SAZlyh-2Cp@47cnI^^>CwtRNi}!OA^(C_;Pdq5c!rj)g%?m zfCVPx>fZjq*m)aos-e2+$GnvPG2y%w0+hUAsIU!&@e1JeQ!LLl=4!t=EPzAM%SY9D z6SQ$zs#M7k|HB|l?vhtuOpV`b9OmjpUh<3NA|NFcNxH{Eqfl6^Q+%haY!82NBZ%un zgTcZDaNVxn_I@BuSvX)5_wx)M9QsUPLXCh2u)kR5TuLrX6!2{^CyKDQKXe&HW}Z29 zw8k0V4FR$A{d%`Rq8d`x7p%AG@PSi%G(vtDf@*UN@K$B5cc^?jFC*UuaQ8|Z`cXh! zK64oMouCl?{qXha^>sE1T`jHbEJiu@OW?apZdQ1&WfbWxKwv!gzPX6b?IUGmSs{0` zQ`sH0Xb6E64S<9AHy0y9H&qH!z&8ML$5#<6WTvt0I$)n7Zr}*Q8CZ5o1Hb-^=a6sI z8KXq^tqWVze65fC%iL?>x@7yNC#9-P+8MaJ*paHO7c1|h@Bw&F6~=s*>Lr;j=A=e;Y{pxRk7v+ARSig|UFOJ$k@c>ryAXI=PNQd`}=IgSh7a!Wvfu&{88%WkD&9ZB^?GsxW9&rLj= z;H%Jm`u1HW=7~sdreZfIb9mP97b!b$I}9;|`y=Pw7B7;|*q1Ao19-ci&E)nQ@KNY6 z(#Ik+_Ke(Y&a;(}eM#kfn<^q9x>$@@3f&)qN^E+Op5D+Ise3}GH+A1{WtZ$6_%D>n?*f~<;9tnB6^HAWQZPgMhSf*n4X(Ke;@-abXDLR?bBrf z+o^gfCnxSl>gxSJ79z;D#;?6jO6n9Ax-@&4H$ejIBOW|pW71?*Eh0VFaVGSgHK;1+ zbv&HPAJ1f*yx)LRTuhZwlo8o<&rBsln#;ms3G9PfwT!h>8EhBD7uESp<&%M+P|ZqU zaoXgqyEND!NtdrvjI;fQY2CiJv52bUktMnpZ5%S^;_^cCz~I8n!X62f{d4;zb5c*) zoRO!i8`b30M0!RJ)>}dnc&0DE(DD|Da8YiDx3`lL;lR;Q_cG2{md=?oT#^tyg)_`w zQU%#`h~py%hM*upWlq?{vj7sVohV33)SSCIZwS>_HOK$}{Nno^WA$2^@K*(F2Qo@v zR}{8pTa6z-MivnfSsnR+goK8x;F3mu&l-XTiu85YXRefK-ominGbW0^U(?{yc@^cF zTf8V$C(o^+!b&PFDPc}pBfJa_*{OU5`aNtogRU&JHJR7<@3oZ+%DsrA7G1x&@mAtxv2wtl_8v4%x@yD^4V27-`Kv0r+)UU>!n zSJBaU;^ODy8c|$c^R4hWIY(D7*_Q~f%;?>FWFBwxuluL9^+r?2mK>~l#5)y}eu>q& zO-KDF8n;+zu#bZonAFN@CuTq_Y`hK92^F?f^*$F-42p^ApaHwZSQ#@sC7-#U$<#DB zND_ia)fRw;2SGn-Tx3!Np8ur0ab-QD_SSz0E{%Nof*u?UikV<}TbetPlCWo8>rX3x zy$f<4Y|OUBg-gup6bBLcX9B@M#And-l7Fz1#=GwQi2qV`UG(mFvrA-a>@YOCH8V?ZW3q>B9qp z@g-7H>Ms zD!0BKMJbF)Mn(n#>)gt^s3_dRM+N@pXD3@MPE zNbdGhap|en?gmkFbjEh~!-T2uDadwFQ-=yXv0KD*Q% z(6|%iArPUmva-XE+V>hS*%t75W>z3l0ljCMzbEmMEHf&+_I<-;ZUV#vb1wiD$G@V2 z_epnZhJH{ z-cc|FrFm$Ve)4)7yJX<4P^Q?lDFayp1HTWO7*QZc^AC4`QgI{*R$hKAOzs`n6{k<=C^sml_E+`LEqO+NgOPrf(Rc^qVCvxD*IS_F0OdriSO z(#fW@Giopf(H6Re7#+DYk-W`!irijkj%!sHjkS$<*)KgPd+)B68+#@I3uOfqO>vhqr>+3>G{@gX}=boMiXgf&%){Auw62 z(dX-L%1y4}!Hh6WPAl6*m`BW0Fx6B1YN@8Bsi zY+$1phC400MaF*%G_$dZ>`D>LJ9Gy}1(ds~t1HSjg-EqLD+iSDqJ!;$Awv|8a|#N-;*`Pr=-a!y@fNyLnt=&GNl? z_20DQ{52&dr9OM0P4F^P>R$H0ud`kQcFkg{UKp^P{R8JFM>Bc=l>;e%y9LVR7Hkhx3%voj<2f>&1`r5mY=n-%usL?b3ihH%Lo8%V8_aBI?uAN z{`TFkaBZ9eHT<0vxby*kk97zU38^<%ksMTcOJsP1f4&y>xOaZM!XD%f2DgIwBc}VS zqfDAb$rCeI0opvD?PRU0)>su_w!kNTo&Ng)?{Ti*m6SUyeET*FAPIQM2LS8IUEe-0^DEJ z%`muY^g1D(s`rvOSO+nF9D^dLDS17R3ueIe^!AE@`Vvt89~2sj{`ZVbTUuM0Z?8?f z*X6B}$Rz&fbKiqY8REoJ#Oww)FHU#xKvD*R0AVsx?y1A9 zL%uIxzC>QY#=${=GBPq8@UF5qOfqn^+|=BR0D<>N0O;})H%u+`udtK@q6`TF{=^`2 z!Dn?V=^8IDxE(?7b0znS2ViI#GX+=%`SS8IkPYT+8W#viC+r@$urN|?Lk{MsMgvD8 zAti;~FqF{azz6tE3Bp{n{%gf6v-F6td(nO-4 zd{4}&Tmum=^grC>=p-2J2%lmC`QAcb9Iq$__JQTn9>b6n)9zFFT%7Tmn3xbRXJuu@ zDIAAqhMIN0G6wO};rUt3qvGuBz~jA@xip{i8FoT6F92tq@ z*Z01(1%)4AnCcAel2@USYm|%{`@5p)l#l8()Pbfh2_ePj8&Bl7R@kF@mEb zSY2^3yYC@yUyaieB5vMj3jo*QeMZGW(C=eW7kC)p>0F?rx zn}dPNrUpOYfEN$`pXh0Qt;oeJH3I``#j_uBb}0!72~E$To{#3r)|0i`>zCdY0BsN5 za%*5Jk)d=^KSYb+q7?Y*Y3v6oK;9r|XlQ5=98Y-h@W4Rem9>j_I#~Gi z4{$Fhqo87ng1dgK59w4FL1^^XYTQQ~nObtX0CWze;0A5Q#j2}2M!dE5n0DrhDqPn`|^PLIGs!3kp>V)mm+0jB;L*4$ueGqE0AHc)UtA{C0zsdncIaMg8qd6ES z4wwmAkj=vZUoboSM&8wF0|OKYVDM_XAfq4yCI|8@Ok~;JbppkY=pE`T5#j0`S)kCc zXP^%z=!96}b*3cbc8D!{YR>bdl=d4I$xFd^#3$$9u@brsTT`rDz+4qUPv;Z=k=I&F zwMD78?;>=vJm*XEhb!5Z2Z~Y5NanexHhv9`^Bry&7fgIoHXkU`L;ZU z5<>L9m99I*_WGWWTD1vvhvn(n83F{?s#vv-$#ESZ6HsGcA0ZgA(NH%8#<2P|FXmeu9v^-lZpVw> z5c!PT!Z^DsP#)a;1%ZS8G24O4cG(=7@=W}WRe|+zhI*9jakckhpnAmc?8Mg{%my9_VV?&P{~vsEyXD_&tuRhri$(h9f?> zbF`yHEVcTho{Hf~+qHX7N+zOXnK~t<+ltTAb8_M{SBmwj{LQ;(4dt%iv%Ut4_qm8| zi9SaV1#oH|PH$X2stoSCiv&gbH4O3XQ50}F#04q;pKbF=uZ5nX)(X}24czBrLiVWG zXb+Rf<8Usdb>H1U6~q5YNjeYN+uLhZSkn&OnF?LZUwl+?uzMI4#Fbz))fJ#s?oHjH z@$Iz2n@vzZp0Hj-&}W$@t{#@RxV(%AfjI|iPG%Rs@yJcXtIL)Cl7<=nJUJ-OThB{3&cpmiQTNKMQ%UJ>R@EYY-x&r;Db+2T zTpmrW&NC^ph{&fFXLejVJ8RlV`%U1KGHolTuWD%+jcMjbeM*NdH~ zKjn;7B;USAQ_IxuOvcX69-4LkRi{*pxq0;vJvmho3}C5~ZTkwh&t2HZ;d7xwT3WiV zdg*p+suT4Mu?~Bud_Su&j z`+qdHRGg1@R?_&*Eoc*s;7cOmgWnx*Kf`aQ+b@@0h-h;#$Mi`h*TH_= znNR`yxY8S+SL_)j8a4d%I-WI7vPDsQDfG2IqNcEumTT1;3^p!yZUgMu)bykCBipO& zTfSLxXZ5}}%r#hlQ&v`{6CdM_C~zQi*ya#M<@kCzt5?v9OoLnJHRvS_lRgcfD=W953oAi4#B13t!Vh@7K6dTiSnHze( z4BU^}_Wnv{ zj-GgJcv6;6Gu?WW?gwgcbe zA+M>a9h{*>no+g+q5J3GbcUIiE|^@te3_2dg*A>lG)ZZ4c$#T6#Im}Hmy$u4?kY=U z)z8NaR`Wl}le&u-M@Ft(SLr6+NG5#-Ai=BmndI*2$sW;@VX_CWAd^TJFJ25iXY<=~ zWhW(Nc@Uc_kNub?>EY9-KyeNLnBy*7mGK{04QZ*K-Z zSt(A^>`J(}Y*g*|JBBx+_Q8$`LEjly7kHeUS7Xw#$3G`D!+wY5b*v`=Q3w^g>$@-j)DNvWWzI>eBNg(t4Jb0WNFHQ3e&>Lv0x+ zqod($+$><;dT4yh>IWRaC?It_XU@C~wpLbGuL$PSe#WgUmggn=zxm)o7~u}SYi%7^ z&1X{U9!S;bk+(^_f^&AcLfgZ7AlR>=%cSE1RaNnckf#bvV(*kDIy0z~ytU#!)G53A zJ&cR-fARkkx06_x{P8@bi(#ypK(Y?By_Z`>Bc+9 z#>Zcb{i*b0KwIRIIh&K1$mY*@ZnLnkaGsaU|NFU&&}lVT`5ip-18^|<70S{W%L6lW z^Xja$>8zoL*93BIzgM!qx5kMH`wZk1J4*ka^v)<2wzG_!aPgD(Q_gvwEZNn?urSDc zO8JeRsp%7jvv-J^BH>yF9-Hdd{miQLmeihO8K9kWC(3nWy?Fec!fT-f^$!Px>_*(P zZnyNKFZ+Kz5&H%WOkkx$FpHf*H%)BwGr$qQj_XJ1Qw;jH({wb%J&|mKa&;m~MPTN4 zv#^Kro4H+c%K1)U(?8ii))I7Gj#hJ78F}4VcDx#j+?TvYU-nYcpC%&_z{{C4Su{Dt z5B|peZ*z<&6wOJ+#N=nz7$-XzCm$YtqYxanH2o9V9?qC|@$Z!EH)hZ2I4-a2=^4FM z+D%R_pY2x_VtIePvAbCPw#G%57pp4gSP$(`h(Fa^UwCqAe&)RC-5pTdD!tX)q?Vw< zJQv<1U9@_Ue%T2MW2?0pt^#Wp4@H))eN>EYU#4V+Mn)xjB=#WlTpApCsg!tYYeGWY zxZFampW}|%J!B;Od9=cs7R4XKXv4dIDs=?KACR+H>YAF}E-g^n#DA-M@6F$F^g`V% zfJzF(7xqnUfcVz@8cl@x!)vY!iyxH)xHS)z{rtnNrfHR;WXTn8=kMOewP!pKA3KZa+1{ySsjkU;ipyy{h;_Ly1Z$L&0u{VI+PuFHl6~#nk^v zKB5cIIdR=zIOWnG47?S9+-;u(CeN#>sYP2Xj0=+B&Ca~lvKe+` z&$L9V{C?Ms-&-^|HT^!&rW{?~_?lK8A?9#^r1bVJ{W9ffy!fvO= z_wT9y#ORW3hqh2VEcyOr3#!P?rF0b&H@}v#3t1wU&?!K@aZr9I#wtDN|brcb5^#| z=KogxNc7lAH1;6No>i|c&q<*bL4rC>hnSn7fLrD=6{9AWI?5w7H}tS!upy`EPCb>t z-FzE%bqfQ-0ji*4*T0gJySMbt*K@Va*DKfKFC6!d;KCjld4Sju3u*i-YBF9 zziKJJw{FsDW#lPpz}}i^8_l!_Y!vIKJNUn@JhdXHa$_0Bs4&|cajhlkO(z+* zrrSGu8%!ckvt5feGGVH!tUS^2XR%o&vt;+M=0+__v)QK8@T*{U_wz$Thp_xSPSbm0ff z;&RW_)W}oI=L(5Gc+@?vb+7()50z&QAtbx>D1T>5l+U(`FkqxW2oh`UxkH7Yd(#J<3*%? zFlfzbjW@StM`wXG$9LZHib{j=^j+{jYP->4BE#F*@F2B6o*p((r@nH4PSAjKIfR28>g@8m zzyim=%mOKG{-4fH$hK5gmK&rG@N4eRx-y+Kct)k32eX^& zM*}7j%(SIml`tGQ=&9W@p|`U9`u_ceG|83tfH#KgO&%!K9Re3j@3hcdWfqp@R)x@9 zBYT`CSV7!w%{2O}K28V?tLcfVDk<@Ej`N%#2!e+f5&`YRVIJlGuQdlOqz`)K$U!c zQ&=kxohORu5rK5+u8Y~Nd1hzV-(>Hw+R!yPd*lceu20FMj88A*wHzKe=>L76Okszl z)oOp2I?3nS@%D?D_3(ZCRkW-R3PGz=yT_I-#rwVPu?&=k$(jDlmwjI)ZlJtZ;mk%OOwa!MMP|Cq49nW4#%6GH3Z26!^+x1zSYo$%GTbIeEy zHP;3A&x;D55R8w-fX?KD@60|_widaZJWzX`eeK+qhmSXyFBX?L>!~-L{tD=U?bL&h z$5?BFxXrH#)?2C0PZ7|w$bFtQrsD%Z$O%MD`Auw0Ow5$)e=S?5DTo!on#Et001SJ2 zUR>7ji5c?|@2h^>^GqyM$Td*XVBZr(&yUwLx9ws-Th@A*diqyMBWFNzfK&?PTI3%Y zn)pgim)Eff4>aU)e9abh>?Gf}Z-YyjW5?!TORYd7RO!|1)uDNJ*{iRfidQN&87;E) z>nJk0#8_;-<9i3c+*BUCG)Hw^-pz`4ZUi`w3=PAEH)^1vS?qX*zcG)K`}ip#_JbQW zx=~6k2@ke#UKiz-ep=_>)!sL`--c%7ROBZ&T1DNsb8**v;!nPsT2n7R?96l1`<4B!_d_zC;&kFHRxRp8jpZFAD za`DI6rmDyRuh_-q^GCORChEhksIE(a77O#J4y`NL2bB}GDrkDHUX!Lr5Mgf;&0iox zIcawAo=^67o_lRIS0ara2UpMdGR%jRGFP6sd7?GlwXn3IG$`%PjY2;}g%~{AA#{`L zP4N4$4vX>eA2G&9J@U&O17g{jJGN!jCpxas=!)OHutD}Mt|!-8xMG~`552HG=LO{} zGsVyiP%?{X<`dMG?dn zxp*TtB|_J5CSllAGm&0{>&8wN%bqJf@e7}s8;YXYq7eP>IOnrAbrm=;^y&8%Tb#9A zGQaL@y-7I0yrk2ZkJnj$B!o6uwK%x+O)(PHT^l<^JMnx6q0kK5n{kux99Ncg1lM^r zw2sh`P!*;bZ!D9^O{-K2-Rm!q%)1tTf`?ab?Zn(}e^=HZ?ufjgxcd(4^`h!6JKY|3 zKXzbm%5%1rLg#f;12;<=2OK@C8h&=4YPM)^&GAg1 zah<-16F2R!(G`0EliCqEQ%!w26 z7hm5>yrcTzjvtwwOHty6Lwp4Hv14SQQa6*Jlc0noLf)!_I1{aDhG5d&;^R+(Ck^V) z#+SCv2TNgYZr(80(-12+_`mXuU@+!oL-p@st%{psaQzhPQ7GT@{mjv} zadnF2_w{f&S|N72lX2Ak_Cbxc{@UxV`G)K)-H-i5zWwoqr)#0kGLS_&iBqMgR*=Qq zyQ8;jbU~$SAkKw>R^|-Vk%#>cw0}RpW9%dY@MNwG2y>8wZ&7 z<9qhdj5N#yZrZzHlBpN9d1`%vtGUo=T}YiT%g)yBZSuRwSnuInPexuQUz#JMJznVRZ-d3HEd(CP^IM6cqFb?txb^KlM5Y0o0=a^)%uSla(1 zmE?Sxgb(0-NfHSf8G!DG>%V#it?&$o@7kYs%8l*sdTD~JYyX#`fyH~@n4PSxRC7<9 zK26o}6Jva(;-?p?^AcKG1)LYINR=*l1sY#wJ?31T>-zoG_LrMDPHV>)c7A<;CSjW` zeZ;TR1t`kPX5{5gkFS$0t}GorbxI-Xqh4PoxBl~ukha0~qC{=xtEzH*@(b!JDsI{S z0&XssXV-;JRWpFDC6R)IgO$|Pv6)?zjlBjIQG({CUg_@Zp5ymofn8pe&>g- z)j_*k)HAB*S$C+Q&GlCcaq@LZjk*h$c9CE1p}c@? z>zy18=zXhK{?s741@*<-&u?wJwm<1d#VAI`8fUw|6v{MxBO??bGVuQ85p)^n&!5K- z05?2{Iy*NP6(7GHk6_TSDjjO`(!fa{@drv4tguWMm z?jSWMrwTlC8W_TVGQ|JU-59fj>+oefr^4RyUVSGXjeIW5SuJ^G`8NsBRYFh9sgbjZ zJ|Ad58dkV9-ydAn65{qglAYW^e*8Q+^-X{IM$ya$!7ckF+(fOU+@w^S#n9u&`@bpi zXbhR^Zj8SV5R6ZN#)Fef8H4yKQ~Ep&e2 z7bk<|jlZMjnurWmgTeg8jMM9WK9c8-0cYq0@JfL%f{1?_&k4K5D@I~fc%&^LLQzgL zkXC!IMh=wZ5;i+3DJkNx)qZxGjld^fUfUyHzgGTuJx^`H3P|cd%#5F(KYi#aaC+n9 z-`~RxVKe|lxF`SKq<;vvZYlV!iPZlP($mscu28No4(u7e7YeGG7-1J;S8Is2TK^z_ zvKN9xn%%g4()HYJ2>%UvT(@(|YC370869HtduD4~)G%O!)yp%6bC`i?zuztn0mH`k zT=>Z~(I-Qfe#`45C<*YNF3*0>;Gx{+xy{<^HAh7y zd+qG5JprE^^1ksUZR~|d&gw7;Pp;#_N3|Py*2l37kFgAofF1D+)on^p&)K$o&+1w5&TS8m+XTPU z|GO>m#%WERql2=K$)*!nYxQ>S36RP5O9^1vX3rGOV|eatLvv$E@D>K8%nPf!A1mW3 zO}mPYqn$V``k_mEUVl;WRN6CYE8VWn0f-_)MIM|!rIR1jdmYHOa<=)kAZ}e|TzGd7 z)3dX)dU|>fTU#~Bwr)L)4_;DIvN$_T4fvPb#KZ(bi7g}%QPDJqxm!usGPYr}%kZ_7 zQQ-4en3N~V&*Dkv} zsaiX*G<;bbPD_7di>U*gsVOLle1JnO+9F|yQ<>6QHWu@|>w z$L`%1B_v3YyglAYd#V?6a#UETPh62en*HT6iP%F4DtuKe@o0Z;~)z$c&nMO%h(2@}LpMESH=QCEF(>1vPoYGjcho$Ee!3C*Y ztCQf?KJI*mzbPvE{gLnPd&nuyUys;fJlHv%b&>v^{djw>^NIj8058oJfDA~14y{i! zyhLPbEiJR2OH5BTw$7u$OSK-nJl2^<4I$Ig_#9M`rfuoRAxwWgrt$*^C+{mE^x6GC z@<~;qdilqe1NxN0G5odL$EY00{L+~y`h|_edHRy-O7`=wiz8h}0*i)a$Xx7KqLqF& z%Pn(J#lN7XnLp2UJ!bET8oKb?R7E77J0qt2cj~8dT8P$JI?3nrxZ$d~cuv5b-23yV zUuD%U((lMGc4AswTE-VI67zN*9*$kTed@!|(9ogJ;v}dq;x?^s`Vt~;@oi3JLJe3B zXQ3|c7hepEg@whHSN!CJE(#X|5n~hh2q+^k=WR2@O@z)7HM+i;Sy-&$v44#(fwTK7ukcQOQKL^TgNDNHM24rR>pGiubqPn=#~uX5S3^{yxDty@e3}r${LhpnzJA z@m%&5*<~$Sbtksoj1F2tVt`HUaxQk7zZ_+5rgN-pK#}{eH*pB1xwJRUqryVb6IVh>Iyx00dvMYzjvhU#sj2yeN#Gwn zuIwoU#06x+iI=yV5=zE9caWY<-~Or+?JMuMzq%GJb6UK6k@K9I%p!#w$?(N$;>b~~ z^W=?U#ks;raVyWrlwwJpMC;>$1?w$FIkEEXFWig7=tBfUq^jb{J(4Kt%td5F{`1NbHw`2?I z&a2Dks-S&9dX=BMAW{7qsq%=ii!3ts17;v<^bHGha%cBfODA!?jefN(+j(i8NzN`@ zRC|FVNWEE2xj=3DSW1Amw+bs@)Wckz*Sq>AX`fY|3>G!LWfm>R?dXw@TEk?~8B+l3 zQ?G6D$p8K!Csug+-dw3&!Gk4@sE=0PE{4q{HN10peT!o7Z~Up4ycAIM{lEJ7DCC0v zT!=axW$i^MVuB2y6sO}i-FxKd(Wuy1kG?W5LnzFM-+`lNdZd|y655z3zfYfpQKzQb z4y!;As{X%PqS%hA0G4`W6n8JLEhLE6tQM#A3B6~ANi`Sook4JW{Uhc;iuE+o#((OW zf(cyblT_44AVp>)w?R83YT36B_-t2T5&ocbHAkPx!)m>nH8M(aY5c=IGjBT(0(l16 z29u9OX{csF$2|V_DS+N3@}sj5N02VKd!*5ZPTkd($s->52FHD(B?rfHWSsz?@9(x2 zav7o#%gc?;k3=B$wWkZPk~$e}p!i0VrrpHRKaj7UJ1cihQkT?ac`pL-76IPk2^)*)xS|(8k z8VJv}kvMhoX^4ri|Eu5o4qcHq;KkCas_hc12O>Y&kK>(*?iT_;t)RRj;|z zD8=uoe@T!sA8N;BrJg*Ks-El&wmW{eX)R?2UwA3#m$x2HY}cI!Gv8YLSMOD2XjdqJ zYw7Ttf_A?sNrz6I+O6F({<*;Qrty&T`=3d^?Q^&K=jT4Dbh**{=-)^ny^686zaw*Z zhvx#z%R^Sne-@+6mW67~FD_BmzjxlrD2zEut1TY~KBu)h8wlIl+Nm^QkSk+cz-zAK za=IeUPkDC_SPf1;$ny$0vh2A`v2rj{bi#ck>q9k@Mz%3nW6V#!GVBQ9TLJ`a`73St zb{q)$e$bVAEJOTm`2E$HA++vp0G@9zv_-GvN|@rogiP=N#Z5rL1LZUpE&6G6_Zo^9m4vfRJEU-c4TZiWS#k(FH&Tf+N)n)y!;tq*)L ziP$p`=Xd$^BU(lpH}az=IF9Yz^j30k#gyWo3x220y%EBF5EsgjroZto{L|lX zN`j~=wqO`yc>?ISWH`h)is|C|;e}L{B%?5@yDN)5CfTlkHmQBQL2eWVTEOMXjFvrZFcfJ3k!?wWH%FuhN>9YgCN>(Q0#7yQ4`I|i4!Nn7HQC7 zoc2eZLwal8mj3k3WP26|s(mt4?;Z|>2ZfG@gaYz2JSc&Mdys6j9ymDkL*0_=x*F2k!pZx_bb4Pb;rS4pn7=lD_Vvt{$zM!2;7#?LL8mQD1793fIRpH8r4qWfZFuJx} zXWOcZjF41|u3EBh61P$HcY9y^T05Jh@rj*XJ^5`aKa{--Kcej!W*Ise{IvPq!SAxN zRiub8GavB3TAN}uJhq>)%azfkXnn@zN!Z)#vFQma?@lv!Ibs$EeVkx0rkX=cqDjEf6ALUxt`;KHH-GV|~>d0}n>YLvdEX6l7IqM!4q z709X9NJ{zF*=n@Kv?r<-wGRY)@ZR)3`de|uQg$RKr8`RJsAHyi2PIcWmP8wcv7Xi; zG4kKwa_a8r-d*b4Kh|YMI={&~gDpJkPN3e`v9YA{jShE(mU@zYL00L@Ahb{Wr@7SY zZCT2xcQ>(ed;q&l%thW9ry7579>V5*jEoUub^$Eno<2T1Vc?-;2=6Htxl2gFSLV68 z(jdW4GpvHL9N3=t#wzvv{JfH~GBGrR2O=nFZ47`r6Bb0uT>~La3$v?}#_!N5MjA2JKX;Jtt*N&Q!2=}sH%zzY-6sa*SwxfB%@fz@iU!=5x48Z!yY8xP=W)WyPI zZ(}Y6fdIjT7IDsI8``i7Fv9WJji~Y8iOA;K{FHTcadGjjl-KV2^JB$M)njcLyw<;D zh~VUn;G_Ya&zMO1*2j*!z2d9xdDsvX$dX}2P9ts<#;zVcj<-cW+}V}XkwtOvpqY01 zJJp5x`wPFfL_}hExJbbY-JVIYU;{9fl0e|1Eg+OnwOx~>;ZWvpTfrip`*t(>Qb!TR zQ+^h17S6E0dxC40w%@vS>qm3rc_SklNjE8~ag)~85c<{Mf}Bf7rw=uhk!#OBKIrg06QQonk&YRXC~bztfM-9h^5gdzvcG3{eHtX_p=;VNtZ$WyJPmL6vT5+EUKReBxyPs2YFD@?qVH8351v%hF z$O|B$fsfGxIPvF_l1(r%T-sP)tN*AYta%MoY$Pao_$?meN`4XLuihl)yzD;G`~l=1 zuxm-=pZYu-6GBY|`|`zjh1Y@CYaih66KXFJ5fP|+_pK2IXZctB$})?v0Ut!Pc!}-$ z4Ylo4K;LmSub=qrsqL?uE(^T!@I)+xN}(bqn@@=gdEIcet$TLO3%H=*-J54C6_~b) zKJF>*Gd4Ba;q~0Lx90mm!nK@>?KN9E4WlyLq$U-or8_-y!4s1r-o4Gk@cro#k!}bQ!Vcrey5aU`KQFTDwqb< z%i_YKebUCgGv*9Gp9k+#^hZwHMmg$}k)x>h%e~Lc!G-+IRf-TEm5sr?_wiO5D=cNp zeY1XsrY(8$-hnw7B{=jmXQx7sh^M~c_LD2oo~4@2E=(V8d3UUjwEPKbPvJv*d3Uw? z@M@v;DRYJ<1?vM391LP={tOIkA@Q6&t7xs*4O!eKKjRvc&-(2Fm$#yhB)ugBCqxx){S+@*7{Yi)VHXz{rwMUrk@E`Spy>PZ z{3t;>YNXuoL+##=IK8+yeU$i9X`chK+SIEM#d+Y8mwo*zJJF{s@EZXDI@*$o3h5}q zvqRV)3JMB{mpI^OB#r(|E`uc53ZC2orzI+8X6DdyHYb4i(X^upI0%qhakfI}NMIFi z=?~DjU^8UxP>xoOwY53=NLMpk*QtGFtS)$V4iZ@JeFI*YLAjIrO;LFV;VYnX?|r+M zZ2MNJtBSOs8hQ9nUwGP3p`2|%|LdD&(0ceuNl>`Ek8XT^80BmQ(UXYtBIAvo!o3Hn zwuw56hSIV>qqkxZCmqM&_1PQda6!oxeXnME^$IzS1Ozdj}<@+k0;cNsf)WLZ9c`RwB?7Yd$;>x(6;mTh2ZL&TLWFM z9byCNmTLOlsYX+9D^ZB}m~c;ze>+RIxLxwvHF~U1D*G)Dbox;a=d^ z&4Av4hU@vV3)`rvnb{^1XqXoTm7rfgyOQ_oz62so>h0@$1U!h0+W?%s@mZp^=Fuaa zx}N(GVD%wRPD+eZc<8UNdf@0kXC%sweq@FnyLMGT{lJStBsV`_Qc-aaD$f^$*Enb= ze!~DQDTN)5O%;wLpzdepj(mK4zAWO*aEX-Zx~|2hhM23V(GWaE0iG8AHm(v)sDdq) zW~Ja+Ni%CaF*LHizVKu#1}R9Jpz-LaW)^;q;>3}S?jeIGR+HUBMrw~L9I?B_BP65fB#)@N!JNP026fH8oQ zgfKJDAyzGmPv8t85q#fHlXfcb{U{%Ye>s6PT^R1CXV$$uG&~xO*59=Ky;WR~oSgPf zWA-2gJ7GJ!Ikmbciv9?H2hUhOy!f5Ls9JODd6~a*q~g=uM!Ww?{-!?|j!@=(vt4z5 z+R*Isy?YBrDd-#|u3Wj|7CYtY^5LwANCY9{uAV16Y0&-34({jB(5vv?PyP}8_Ca%V z8eKhARDI~f^~@vIT~lD3v}jG;1h=wE9$F7UmsP>0X4Ra7VcFRN`1>AiTX!13=&!N{ zJ`vl4mx#^JKPZEjhj1Ku(;z(6Z4J9nU61e&Vw43Mxj-C#>mRoR#!xS}> z6e|~E@H6S!-i_k5H}DG(KN}i{FH8pq5B-}h%<7c@XDZow2_>axrBvMFa(8a#hlJ2R zTr7+WbfgAQ&MN%G$45@}6EQ#nZnp-P1!S#ZyRlYYj~R3$s;{Tk+He#+K>KnujdvGn zFMEjB?@k9Q{gcHT!H%I99;n>C%Y$I$h061?>jncI z9f9>5{ohj-VZxW>hX%@U%L4G2R z^H-ueGtC+)ujR>M(_pBiuoEwKlDIJZ7vNxPM@JdbAja2Wt^-^uWJD=LRYaI%z+~)Z z5vy_Mj@|ih{AuiYpf|*$XoOc7s7eWDa8PsN@z2t&f`Wo165tZ<3Zbn*Kieh z%%qV2E*EiLrh#UP)c;~D22Q9y8APlV#2TW;<~siT@8CDDz~ge!SP@gyk)Gx2e2^T$ zqp%fSWF z7t;S=(HVS5rL{r-K;`s3rWqtNO*Q^1{ee#}U%r%VYi#8E z?hNH>AB+eDPXfZ4=ow+@grTbxp#>%F1uTeijLdWX9=s_;oLz8|r2gVK4)ymjcjb+Y zjsY-nhr^g4KtU;_W@md6C;Nrj#Kc6d?J(6uXWnBNe2)qW(gEEfz7>Ww+-mvghi`SW z)kHpzm|l>EPM4Tjgd2JZfIX?)a$k;l+i75#YWnt;pkC7~It0)O$xZxVqy6XEX5EEm zIPFfC5dZC;lnawRmw;&!%uW!i@@7!t1hjHWw+ODK=hzh}bvU~gR+qMa$xJ|0()f5i ztXKYjUhG;f#37Z!AQg;lCH04t7DBT#0)PHQkz6n*IpzW+Mg%%Df|LH3I?@F769MlK zxp8r-520wxvV#_dJl3e}$v&bLDbn4?0!up%e>qw;Ran#KbnQ_N}0gu5d(oRCuyc_MsMc%>62q%3ywHB7(GIRMa0}7*?G1^%cw7iCD^iqm7jh zomiP#RH=;E9YxA2EqITdBJND$Ouv2+jZi$64oK4@5uM|A*Xaok!pEwHL-GD&)tg` z(9D?m_KJ_@!$o_4dDbk@V$0^uZy|gd;ayjNA$n|VjE1TRr&|z8X*c@J;px%b#_*Vp zHXwz>ae-B%3=j389FS9A!D)Yo1{?ldJMWJHc>W75qZJ}J!6~8^Q*8GA8&k9mYV7)W zMHWm);QX5X{Q2{*-5dy>h;bE&98{egY_RBSX58FGh^=sG%4%y*sn%e&E0X%{*)4_qr~JFYo?0nPB)wl+^YD*9~U0^%Vf6 zhyj7fG=$X+Ey+c6kBw_PAF@E}Kujyh^TNdGpxA=KLL|R;wsZBV`cnT!E98B~&)FYX zgjD+Q^=l^L`AKk(*9>i}txKb&eNbb6RI)TM*o-Y9YER@e@^(=;;jmYTJ!h42@AlF3 zOoFjb&C03({6!_2f&m9mu_S!>P*zoSvLvt>4>;k_J%ZyMuIx9zMQ>pfFzV*Dv`&yH;&nKOS5>gecp>2ME8F#);jy@}A4X>!`_+r7xF=;-J^|0fQ`J1Lc5*Of39fYR|N zGRF`;kQ;q1henbF{vzL)SR0Oi9hLlIds~}6aIFOGoKqOk6|;B+t2eI;Sl1+Y~3WY=BY@3a+}26d`H>Hy~;H2jQt<6wq#^yS7QBDnlI8w zbKWSRLl7hM6(RCRF|0lV^OS+K_5$Kj=_2?}$Z4BCT-`&!LO7Kw1x8jE8=IRePy&D* z{!aF%rlw}W?7OzS< zH+0wXQ*tph3}P%d3|PU0h`~(|A1c56aFysMf8pxM96+mFf-9|@Xs2~F* za@-qn*WD=SVbv5TUtCRKdl+Y3bv{p3DVB<7$w}(Waf&(C|s7MT@u`lZJ0c2B^wI0_EKM?OHKP`--4uLGdo2CS+xBWNm zPgH=5LRlvb<}&73SY#v%tf-Im%RFOm6^K(60dEJ$bvsbDtm(uH*L&jV+{Zd{6r(U! zaXnP{-W|`*qbE`MXOvEg;*Gnz=>9TMtF>hV03#YWc)mRm?B(XL0*G18uYw<}PuuaJ zPO5n=e72r;Ix90XaCO-io##}#6z0C&MY%;&$ob>LRc>Ubx5#fQoG7Bov1G~srfU{= z$ZW&2IiY869{|Z0J(VtNjrK3qb;G50HIDON`7wA7>9q=Bn_$r)=NFN8#$yP*=xK>9 zC1`g@cuCXZpKr!JeE3jpZQwMU^~ULdC+WL(tp@RY-vt#yU{|Bxj6Y&eZ~0SdL7*9L zt}Nf_%vGM)`()-A)KwnzH^ng`Iz(8cny7_|$xP_@J$Vf)WyJTa6BrFW-%UN-r=g}q zYMjh4^CJRBA;%J{_@IqMN9aL`22G8Yj_wKQl_>ZRiY-6$@F@vF0A>j~`2UV^#z44rZgmFfY|^r!$)8o!Aoj=AM8H!~e!}4xSV5p22J^dG$|7f5fRI z%mX-T)HQG4tHe@l+ru1TW+e_GC=%-TXGM^_sEYhO8Lz6`GQF+$&DWgz?T7y4ci$pq zHg?G^54mk*AB#38N<}=qHLi`PJEBF70oBsix1KCNe~qv%5B-SRg2MhOfEU&=bqW%0 zf9t1v-18G%J~(X#Xp4ssyF9thtbTx*ICf-1#WkKpdPtJJa9(X0HS+&x#j@4z{rd78 zBRR149$@r9K@lYpq9aFhTK28x%ZG?G#0S^P#+Z1x-og3sb&QqTq+;Jy*@o0kcr=%PB)jpZwwEzt`E{&Trs*TB*1u%Mu!Nl|0$`8mw#g2_`N zs(r0a-5_qR^re5S{O)2O8VAinhciTD)cfZwH2Z`R3Y~EY{-7Q$CuJBp#OYo9 zSu1wuh7drB^Vj%-2GNgV5c8>xUVJZZXl|A!8B*07sws z&UUyV-4~KjR*u}fov8W#ofz3ln&uuZa9lX0zM{L+ z?b+`XXVMOGa@kbIjezm<4W&&@YLIef>jm+#0V9XnfXGScto#9b^`UTKyM-x|#0Tt0 zkx+qgCQ(467Vff!5U&q!jy1JOUySr8LP>h*&Xm3R2CW{+4is6$=ygmNj=)04u>(ZP*XpKHs>?W z5?Z?_etz<^`7GKzF_&(xU-{&?a0k4*ntq%;F8RMfj|G^*ArdkRI8BK)JS;5sLoo)< z$v`l$gM#9+x+5%-w#U~Y!rx!*cIXh$c(aMgmY9@I4tb?Y71)hY%o37o6u1U}RGzw2 z78l8EArY`VeEGa4wFD;n*ilq9PXFK1B@xOpl((ULCc7~n?1aK8(`fizVeBSixU?BH7ucado@8=w|+0W$iImbnEu`0?*<%U0DUfj za0GLiyrGw+R;D3vd!jT2xTE5X>}7?tL>OLDbc?s`-7sj!$e4Qc;GFc9_CR!!loTd+ z&J<^F*j;q8Zdf@pH7ZArfhV9Tue~yf7I$7&i>isSoQQ$k6Z=fuC*lhcxpOLu;lP1y z&>x6Am>~n_5i;WJ7xPS_98U=gszK7bf{rl z3mwm*q1c6A1S$+H>^v@5hAnrGugV7vuA$MUv z@d@L`@Y%Jw97G`T1D6gyLZr8?qV2D7+J650R;o!t<81@f6F_|fqm(fSkm z{U$LtRD5;nF|f%qm<9XJatn#%gBs`-K(vEmPPwQJOov@p7t?_EyhR;+(eVl`Z3tb< z-Ea_3q*W6 zzJ6uX7$|gdK{~9@p^Lu}!Q~&rVOZ&VP}n}HXnBOgk6BcU_;zTfnlK(US-1EGT&zlH zzRqBL64$l#@)06jEc@33<7DBDDOasg+w33W74?4f0Ax!2^5Po|XZ&nA%BGxoP zIYeyM((*DK9jh-lgq@ddMZ~*WIj-lXf+~E6r{6V2!y$OM)((F1pr}J;ee8+Q5R;(o zTenu+DhMf_r*iNJrrj_fkb)S6de%iRi71Vra%?Yj%mEY|gS)6D*7`!+)dd1tIdl!G zI09UM7FSko1F?Ir{LT~ggyk&PA@W1BTSl(~ zA(*oP+D~CRDK7kNqf6C)|N+8$Q{IG7%6C2 z-(tX)De6w42Qx9vbw|*J*W({NV_Z%1A(@!Z8&9#H-4tsrdo$08?x@wqCDTn~^@6G4s>7CRDj0gCv+{eeYfi$OgOBMXOy0`M(TVlSohQ0UHZE_FB~O=a34C zB)};ss0p;G2}dqduVgdCTfDWE_I7qCtdf(K0V2|jTNzLB2TLNTPpYP&ypLh0U|NPSCKs6A-I~?QX>ejV|A*5WJYtg+Na_V)gE8-d6UkF4r zziB(1dfpu)Y05WYz^Qt&mO6q64?Z{PtqI>5Vs^N!zC1)Exi2vYN(%{ zWK8S6>00b>C{eZp_d=($^7+x0eVfPbbwBv?vdYuL!wg8DhBLm0PrV)W=u9lcX4Tf^thUZLU%LJ+QoLAw1+b@;z&x6hr1rY2vQMvIz*)PV5X zM8#TITBaa9mbVp3t79T+QoDPRj44h;3(kr%oFv(fb_hTU7CP* z(~V?5%;U<+49gfad0w^d_Jp(nYFBpBWSj zO%M=iDj+35s3I*m7LYE1fYd06bdZjr{cAIN&$<8oo-^~zkw}v7+xsnRt#|EpAcGzx zpH)Q(rRR9z{_0%`^{8LH21fK&gbsZx+6ghpQO6sKs<9Wh1CK0Ypg4U{94HX-#dh0m zKq6tlNn)Y@H9>nI98$If6(+iGn;&0JPE8I~tWpc((7p4i_U@DggTE}qc&cOuj$d69 zl`JkDEx)frpJW8OG8BzuV|n+TemTt6uS*aO&$Ml|8|nHiqc9cTJLhXlOTwcy9#pld zrxxNHQZlcM>>lZC)SENsPd{GKTcT6KZHB6esvkNSm-XR?_YC@DWkf1vXg#DoH$KNA zm2t>Ll0g$}+g7{|2GvyAM?2&~j+lost+C;s=(v-Myy%8 zHfm<7-&M_-wsZIAY<;#~wq08`b7uW?mIf!!_(mOFzpoP?)q~3QUWv0> zWiHSMLSalgGGp~@*N=dxH*dB%SLr7^@ubXgbXxj@^OM_FmewD7ltI@nnv32F6*pJ6 z%C>)`(H_QPsw7@B?In&LK{;X9i9MYSlsKv#F;NcR8Eb`o!BgPXKhZ$g#F#Jecme|mcQsrYRx&j`({ z7*dFVnn+^=c}T20D9O&sN(tFMkqz(aXd*1MF9pYIgq*78tf)D_2c9=xBjrW(7SMrs zo|5&oQqVlRPd&%ma>Y~##6z}ZEb`7CU+&=3f6R`AFcJ2h2aEO)KWG#-S3XX?86}Ys z)m?^GeN#h&HmpfJGxOd(viadp^AfY`czr`Rf8BGI*@~(HW=M5{?V3pWbFT#TatDjQ zOvSO3!^B*)4E6_l{}X#JeyO2Q@z{X>qA9wMlkMp6v32`&KXw9zZmu2&1F zg&b-<=hhUk4wYCliEO2p=i+vs-!T*R;^D)G%`Ow>D9DQ#qCncw0yrurkt~8|RE-6q zZ{u_SJOHTS-b_|;VS8QwA3j@-zr^p0!#%Kc2t$rHLQrsE57+Uo!5pdc_*aj5VEiat zS0|<*gT2IK*A!s(c6P>r-aOkmiHxB=|0!;n66(On+|*(ehSm5cB_6|3h7;>+V-r{q|)^Nx&X0cGH6dxWroT%^UCBbSjUQ93;3jRO_VjmqA1tQBy?UCF6oB~=^N5<4 z2~i7%-#;)6g@X?xFlr>`Rztg+sG0UXB2^O>Z(?6l@fKC9N+69OH|}-#Q_<%-D;EyNsda;4q)_8V13;- z-vSC>4HFAC5XMud^;~{__|)dRaM{LZ;N?!E91D@P>Z;y{Lk|>5_1^3})>Iqb-QFI9 z1!IbjqBf=26h7~sr-z3Jibhh|P7M_D_=T7G&3yTrGaUt}!AIYJI#xegl#-j~rIjx{ zXyHBD^SMO$?E(kBciQpeF7Kz59|Z8f$elmPt)u(GW}tDGs-Tt*Q=X|fqGUU)nf9b# z^QnN0TEKqw9j9D_A4CU87#@kVSa;+yav7{o)KQj=G+5;49YNFveFL$B3~5+N5*0Qm z{)od>z&4=(jYfXo>t7Uw`lAcw&w!RA&MHXPFI<5;BY! zf+Qr74Rr_pdoR3%DJW<^0m8Edt}=hY0rc?#C|<`E>@Glx^X$i7|Ycn z!{)|0v{1y+7VnjV68&26d9Kx~U!X#YLqy!K_`=D z7lcV#N~;U%WTdl-e*`K;k1~S&DYD;gkM%ornC7(3)N( zC*M?!S5rJKy}~a~A|lN`0MnpUyUUj^E|YN>^=kjzncp$tB4yHDVD1++W~y5GW!2oe z$hC@tPDPJtNE=>yhp@m?+~WcNxV8h|q@Y4z>nbD(Wh)TLrB zldq8i_L6uy!c<=%p-9$>tQcG?g)o3M(pgn#a{CfaLO?N%Ow+(<@95YIZyso{WX78c zAD-Loe|N=}cv125pP?I)VmJrg5@%CK7VR71WfSb9cPzFVBZF4&!;GrhaSs- z&?W2j>)Zh`bcGCX_B;6JLmp{o&Z6u1Jr=VyiI#nI3PlgC(UpPrl{{>~87-X(`>q%i z`JKhyhpc3X-x*lQE0ejU{4@23qAE}pH+&-DN6U=VBIxQR! zN};Eo#`Zr`Pj;)_NH(dBSJyc&X?%@2mL2@&o@<;^41<>MQ~l>dN81jD?)fJoYR0Uq zVN>pYM|8NoA_k}{blK_)h5DGEjUxlZ&R36pIWcGpq6##tJop@%j`Ytgpbedf#Bvdm zQ>O14v9c)+uPiQ{`uOBvBzP`Cv~zbnJV?vRMi^W3efuAH#3xqQ0uCZO38BKV*AiP} zaA4rJ`4Uxr{{NWA*C`S;68wvU6JN*tB+=EJBNNo3FlQZt(!_!zA? zhKgPPMikhV7Hs$dU2?TjR$~s8flgNi<%>F<2TREKOZT^LQT_CZ3kVbbf1hp%4jRSB z%pWy94p@}&rnb3R7yWb%a#mYgo5676w#B5Bb{YS=u!L0y6%bjZJzJLZxzmprH7)7> zlzF|WD_)a!mnrq>_^Td2)*7iu`SYVd7_RkSwy@zUT3^9aIpQQ0iGilQfJA5#H2}+i zGWkUTn<@fY$Iy?M`4PI3lyY{QOPo`SuMU*1%=E05B%6LV0x~_0#*G_Dasf3-I#ke! zCP4HD$Hr-F0dtXsfvWGj#i%h!DS4XPOyBH?JaMardd(ApjufTf{;yJ}13H#|JF`$S%$kRabi^SXZF^}F9e zMA$-+E`MAsEk=I_K4BLQ4?wvciC}JurV4`s>Ev8IB#Pj3yGeLP9jr_~9|WD)?R(0* zafqcdUdcpZOj0OHc?ROCKw6`6lk%xji5OYw?|iVUnNg^hWp!4mappivcQ+MmL07GwE0al`=^Y6_Vy@*pT4&f* z8R;Q>y%b8p8*x|}VpiYo!R%!lru`A?_!I@6&*OO0O~gn*GF4BUaJ!_VGlH;7#i7>( zawFLiLZEIuC3W0hTM%3VmtJ2)r(Y||a@q-n=;*739Pf`q(X>uLO9d2&NcLKa5?(rj z({uKG%z#)EJCFOU<98#F2a&e9{{H@OzuekBl0VUxV=3&7(~r<3tZE-Qdlru;06X?{ z($ov!aWeh6Ll<&&Bk@TV+ez(Hin@sMx*EZ1Ue4~7>+s*daE(2kb@VRu>I@JbPdXlz zk|MV**~DKp$02NHG&DgoGn?tyAliS?&Gp1zH5*oMzS{Wi(X|yK+LFCNo5g2Tv`;xS zNggipT2;X_z=x=Ab@S#YlpA_EYlkpSU;`o?OhUfp=tbZ-=uCF(>vK79%yBdUq5cI~ zCWttJZ$u&fCXECEdRFa})(Gq}2*agRjz7k;Ihp-(e#k=1uO$j?kKUg4`vyRV8kop@YS2v@35nNZCm==~ z93E6j5jO$RWe1ELK40o_K@a_+9 zRmH~pw}(ktEBZKX-G|BuXQ0UiNhUscyf*=}k#TTtx3YE}a#^EjIr(F8qE3n~_@dgM zRqZ1m+aHyFKY*uJbO_O_v@2l} zx7F~`_z?V1%E?WrX|3C*D7%jOxOU-k)57te*b}CMG-UxjQRAv;#T=n}4ID)}Mz5jO z$DTKxsq``xwHX_g2tIwfEudaLIQ5!R-IyFzEswic?*I^X&LSpLm2extmJb7Uavu1w zVU_pd#fu3g$4f!+I|AC{TBT3#LvvUu{8>Fk#X22p@nM7kSqrzg zGn_+!5<~+6ibZ51(R&&!QY4Qk*u+%$O~MmL1tpqR>b3jmEd~p$0pF267JESftGf~s z77-jj5os}>EWs9cWDtw|sh23gw04IQZ^~m<-!1}Y@e<@*uzfgV`)v8B9ba`+(rJFWv?X7<$ zvqoA$$v=>%v)dxy&#<_oUwyFQv5+Zhf3Dze?Zor5 z>l6K*SbBf*w5W05esxy`)|9wxaN6|;<}9BJp--L@6fCdItv`h0RhS>2=XboXRHiur zF!QfC&GS=yxbo58G9sqMm)Zn;Wv1@BB;s(de9K9ud~H_+*U%1U*4QK3ONA$^Iw$&7 z6Jz}hcIuV4w`9qywZ>3F#0sAGw=Gp4e)ncoJ^gkeAB%Ivikh-+QQIN*`9#fE7o?VN zjmD0=I3ZO3@t=bZGxzhXyOVZLv2&t(VY^|cm;UHjbpn^ap()&7mn`!RWmlt5{~+Zix1hwcrj2b#55bo#)Ebfj6B-vwW@4{MbitwsP_a+oKDq$sPl;^@IP z)}*5YuLOU;+36z*jYiC*fs+}F3RPU=`XiP%v^X7W1~_bM%2L+9>*K@teBJuVba>ju z<BTi(4I@_q3hFb=6+$H+jvq1X;xW0HL5cdapE@oz zQtPhh&-^k4hv)B3=}CqwM7j%4N>R;QD|Rk-+G#yneav?Jb(+g2k)n>m^@aqj9^rFc zYy4$mzl*};VTaEfMW{pfZ?JOLv<>Z8gVwm*oL=2=MW4JeKjKtK%?(BWr##(Qi_(&k zAp|-i_y(GOW#oHOh5?Q$7+hYB!gd!0RJ3cGqzHIK2Ff@lGCRmNT~JjQq5cs8({ePf zqOwvEpmPnXh76QIdl%9x5Zt;^yVat$^Kt>*eWnP0jJx(}{n>9_q5kc)^`22y&>Yx8@6%uRk#F)j)j)+K_fiBm42Oa@h-$Gquv6XQtmx&0d7wzTp2%B=8S;n30@6S>ew z)5N~7AR~kw<)STcNlHO8)xdjTWRG|FRa@uIIz~u?UngzFgxh%vgRInNRY^;-I zl&4UKT~nQ4Cx3p*)K4yeZ&kGEXt(4=zV%5DKJ|m)aq9o3oIPwDuv^FE&ODC}tOPNu zA5SJD*xKGWT(=63zU_P8%Zlvuhr?o7Lv1QGJu?C&&=5yC-s;n=A6+q8{P-s`O=U~i z<`K2KHUra>dwhc9Q>3OwiwzZD+iy203qOl*jWBd(k9TM_v!ph``?%%sLbrq^w;PoO+d5-GXrQSEs0n-_2y2Z*HoFttjq<6{b;DQs--DuGV9(bWg z?MC0Os8us)3upsn^O_2U)f{fGzkd1AQ>X@whf%}?3Q|giS`OKJZc%&D_^?rX5TU^Q z`SUx$-Lb#)CzFuwP)}G+^83^cS7C9-~JH$HX z-!K`ZM5_03ISzi_cYdQtU6;#L1exo*a(m>W7(kEVI)Y}-v*>onJw!oS_Y*E8{Ds1;ubx}XcA_>X< z#9blZOUk|PYbXknl*d5qx^K?)A8 zia^*dG;u9jw$IH=V}{{Er)kw6OC38xYk1-N#rG&J&LnaF(>{r|fIn%x0sPvRn)%w} zucUD%vK1nZM&@6i+S1+Kjzs4w7%@`(M%~Y@A}Q%>roSR zBA#lGkkU(DR7fRcg@7IxU|HOKBSz=|x0p$C&z2s6DLk6P&)P487&_}v=KE#k`ALIR zRlTR_aTzz8#4{$-Kk&`f>(pHN)3V{^pzer=lKu1scTn`5wLFD|<6LLfzuYws_juTO zh9^<-+9Lqe0(FP=dg(@MIz0rIY!Hg#TKY1+T~tqt>H4+j?0L4WNrK6sTwC-pdBa$I zWUQM`&7-Wi!uF?`Ru|kk4}}8z>{c_o*A@NaFQ6z!gWrm>U1#V7O$C6^h3JgZYRWWU zdV_#g6q+K^XR^PCShVo}fGj5~UN2_rb3YOmE3B4h=q#epr z-WBc?_p!J;qC6U^sJ;FuT8A#FbRSh#xvO30KT{reB}<+>IcD%`tK_%^&B$l%RxB9J zI;(5Wty$<3a4J)^p-zMLNJY4q`dELnkD>%>N^Vi-bPx=K1^Po9N@EruD%7|$?U(+8 zwo6*PnD@iGcc)Q5OCw*MNPWCVVXJD=B~6e41T_Wgtml|ZqtPJaOh7jVfy)_?2Q@&p zH-Dg#Mef#1Bv*2e$1QRSL8En|nvo8=Gw)P~>D(hp&1UO-1NSOlimQrxFl-MFQkAC6 zjJIeiejV01X242PTyX*mab!e34QgmQHEgEm!N7>N^Yne;YjQkot@+%aPkUeR4t!O& z`6U9=`_4o6m!g-^5_8Q`)$w#a)?w{oQq*;;NFWE?qnGrZtHNJ+B|}FdoQ_R&&Pfx;Hx1UzZ=ED!Y00<`*BD zORlOYmpL$1NAqI)^R%!CM9j2U@%?YY=Pp^X!n-|YAr_G5?*KmG$aN{_Cjnb`R1 zj7(Br?W>(3>1IQxsJ`O+H*eme^1|jImARn-`aQz~RiuiYt#a4*K9zCYEVD-@ zqwfwyKv9h#O#FizPQXa4e=^z+0hfaXE2l^?6zo$Sjg;|B;Vh&$fKx?O43uiK? zTF`(IsGM78{c1Oeb51xpd>fK=yw#hQetx%qyz9jgul9g_dUD~q;e*cK_$Y&D99?hq zt7jRNZuC4?<{)y-a6?|j(kJJ3GtE`B{at=GF>V>LzHZdX2Q;{9RKV*&wwH-^nZ2aL zkJqKozb}&i%H3I7l&02;LdLN-p|5tI4m}FD5~qs8gfr%`?mmosj21}c40Ev_SWXE= zn3|Rx=Q-~5HeTn&xr1o?3yl)SOx0?9=E`W)$0&p<+S}WwkMRkrb(NKf+ZPF;j!Ld8K+c(ir_d>j2C|NQai}CS( z+x=8$_XxIxSAU_QGQP+L3#n}Yd34|hab)(s_%$}|blKLA8*(W7Up&T!NI1Y*bpQ}t zvUI63+MNtcE@WkAbL()oEG{{|$LgSF{m-nRAp7QI2Y;OX`@+fPT66KrwBTnJ?ZsD^ zD}Hb$m!9$V9_EUq2lYQ>yp-RGq6UOwWA>WJl$H`wX4W6GZuex`?=^C_fSnW z8_;03eP5rZ@Mo0c2ftDm7li;})#2;H1;cDcTAs5rk5o#w%hi^#_u1hp#<`w?l-i_A zwfA@Ku^lUy`1zAv#9~P?^vC)>Z}57wCL+l2ox|B7(;1F z0hr<}d)j^R8c1&VDM~1QsxdiF#H=vT8~pL(6GmQ##|QKYVgXs+C%SV5eH8dBO+jj^ zx}yQ&=h-U&M+_4nA4`yz0`aB_NGPbP=10n(7T-2}{{5>2*2^znF3TS`49^VUv#2(7 z$$ov5s+2L7e&d6KJPUG&t32%NoDUu12tdcgj%f+}>>hhB^iR@Ey!Dy}zf2*IzmifG zz1DAS3?6zab?XsMCNj647%V6#(c>&HPYK!H%Vt>BDokO`FNJ|*z+T0cfPNzS2H>X# zDIr01RB2m(dYE$`HhKmuMGX=$Rh0bb`iz(~Jw50b4w3~od!0ZzQH|L=8w?zvV(Kgr z%C81t($-UI@`$qhxy6prs~UV~Edu+kMI>XdifT8hz$pCK(YCGcy(0 zGA&2Ias!1o1*%J1?xaT^#?)YmP*-DC8vW<@O_1@5oHV6F4h)?#4bk@_EC}R~8vyVF z5+bPXzN(P12S{Dyy6i}RB}iL=IG;zJhtQ%5Z7jYp7_HNrcMhMA`eSl#x_-$8D!t1# zUfMAf(aaC9u|dHPlqfdCpAJHMbhIF&uA$J0uOq0!`5?!U7B;N14{qg?%*}02ZJMEF zM67h$%8u1(T_NV&Qp(4=HFi_|)n?6rw^gTCte{OR6n+>AvYEJ!^>$3`t)9!Y-NBig z?!0vknyz`lW%s+gm#-p>B_!{@P*tFWUYlsG4Qv)OP)5LI2S-K%Vqy%_%Y7E(goP>q$vw z<*O(>bG^Jw=0P|R5$Bo&W%oCt=Gli$ei|=D{?^GF{O;7cY#?HRPj$jR>oud#KWvA* zv9o!u{PDeekFJj_>grE6aN;!q#TvjH4OXC@ZwHdZM*xmspCLC;%e_mVGiMIr^Dw@< zwmSv6C&Qp*J<&ygCQrnq%xVkUU;FGp!H{_u&h(B0^A7@E28&n(L1p_X))|wj`15ML zc!&k|l<$Zu+IQ%rxnyn6)4!yBV%ceR>)2y&OPim#Up|;eUwiP(d9&?}C640@=JA$V zy*aeIqUL^NOadFHS;wAymrt>&*ei+jrkyc&?$r(13+cn{L80N%iB{H*qlPd_6Oh2f z3~Y21CC3o72gh7rS)_S*$mB9N0#yW7ZUQXSj~9>?*!)xYCW^$}b<*N2P!Xd3!_0;$ zG9RLM6{d#+K%YR5A<*3QYt{Px5EmC8M(0e$ANxgtxHQJ^tJED>d4B^o-&$~O0^NyY ze-Gatm#uOMSrs|Z*PLa@q(kp`VY5W^Uw4+iH_Cl-H|f?5nMOeZ~G}#TmEKp=d=0Kwg^Tw`ou}fp0P~_Ds0x)azy3tpjA{159&Qq>m&6^)%66 zk(~!gat~|Kzl)UPzA{pQDx3<)G=|?{X!D^MDG}yc@XG6x=&Jgk6*nYC98H*+Rs%S! zG*_%VDaEDjEdg^T0dxO2*U;X>ltX}f(8kc(pv~$3_Sod`N}){smtrg{oq{) zKKmw^7Pt+;^sFQmuDsyI_HR0=7t5$VlcZ^ZMr~Jc`U#IJQ}~ zmtRUY3EHoIYM*knXQfEl^Y8Oqu3C1tah|%AeREfVep8lhZP&X^Xhezw_Mo-Dw|9Jc zq95vAF`~1>Zo`;l5TQQFSOgwheTu#ZX+oh-@lLcvTKO({+(*n}k*qp!`_MuTZ>n&lSEnMX z=Q3Y`4WU$A=XD*fl(Cd_fwF-RaVRAeAa7TKAwg>|&SJZ_A7wCb*b=iN-H^X?i`m2y zwiI|A6}mxoN2JeVDc4e$S7b@KeT&7={ov?m4SvfAEDEN6yyGN%VWq+LK-34p631r( z`giKdH&hlWHmtCC9HZWI${i|yk@%W(G2Rich7+|@wo5c- zV&ZgBpk6<3b83w2*MZ{Jo-GMMiC&$OZ{f#UpEktW7-ni{nAh>9wUB<0d(W#S>jd_0 z*kB0F9mVfP)9b{%jmPmSxm4RF?jz7<{A8Y ztb2MvfW4IwEeYiviWAkAT9P^2s( z=Tm(D)YAaH+dU-{+&qo9G_G7xbbVDZ1%Qak$$!QJ_g{6QuQw;#bs5aHs2>;6 z9wxA*XAY>Le5vF%EK9Ym&u@RGsu$_AY4`ZlHoL~gMxxb2iKPm~2?KkU$Q}R%qBYa; z^YQT^o^?oFRD|?_t>bcjJ{4(GnG)jSSC28Mc^i49^4D{4+&_tFD57Nn51@x3Of?Lt z@KOW?D_%X!vLASXh)ILGnamL=VmvPcs~>UOSNe6{YX z>-wc35>hv|8Xb0SPRwKWwCj6#n;OmVL{3inY!OR3skc0(i+VBZlKa$OSG3Vq6M+I0 zm-{iOT>NOK2ZknA=XyiwHOeZ3PQ9RHoOA1BW}_pP^qR?O?D{1{y8?BGUpnd}a}2bR znN6s0cTtG&4u)`$FKZ&zoJUOggdAjWWK-Im**wcDjbS2``o7V5(By)O+qQn^TO(9~ zAODnCIQ(VWHUDE~W~Oq6n5zE47a8dORcm@Q6WCV|HDumupI#Ms&PDOWiD!3Rk1Iub zc%^jRvawRBu%LU6Or6^-an|2uN`i5q97_JiPS-f?iSSj|=evp|%eWhOt$En^(<*MH zJwjtdVA47zJEQw&K^^V*bY1kz46TmF&LOoa9eP(3@@@WL=>ARSHVE64)*qkDhX^ z&>SxaUUln66H~NlVcbm6-&Zp@X*nK_?rm=gD-!JAu5qV!@pjhn+swhL^=i&T?Sd*n zN1F6<3@x#%)=y=eG2i4Wn7?PHH=-AEI187KkY{6VQDnr>FA4&tF6f9XC8wbi%RDnN zq!&7u!}rOtqeo+bm_9vf`GVACfF`N7*gLXke z(x+x4+#4`wq#}Fso`c=EmTUQw7*6gQH8j|5;F28C`GUp+;&-R7DQm;m$?4M*d)1pm z57Ir_YlEUJckVVC@xxiTx%4G#_iq!TJ$AS`^BnbF93iG516x&YQ6HaTQj@qfra~lX zWb0?;!WCOGwRYH7)eE4i5@g2fDYxwpQ2d?{jN)NOgoh7Uu6^bjd6YuVhP(6VjVLR$M))uzd3M9E=14^HG1378yUk3VO4#- zjv2PCN2tlQJMVa?e&O1}b2uT-7(|tf&IBA<@LQ=Otyi=Rq7ywr4Qr-aL}e1LW1Z z!DO#0;2!P1RqQXsb~M~+Y5(KPpFq5KA~DK;F^NhwD3Nm*UH%!uoZ&K2*89$J0($dO zfZ(b@spdu3;ox|Y2xsp78p+Q{H4buVlR@j~~>ck>mx?l8z}aDt&y>Zfb4Z^>ysn8@sFa6U0%Cu3gh+OE5W*_F0lQS69t{)H{kk50R$%*$>s z53P;uFFe!Ps5N3@^yA(fMX?%Llw?oEZ~c|+S}&iYr@?*vzyaBk^$DL+=kvWdd|_(0 z?MUmB=c$>lQ&H2LA=On#-8*~cI`s|To?^GY-mBC7Fnr6}jT>VD%NKNnH`tj)6Y@*L zY7&Dty-fo&RXrbiL$-0K_^^RB?!y zjb6!R-XFJy&oG7(RFB*BB$wJ(gox7UGt-VD7R`-apNTMzXf&b1H;5}5_XD$}8mDv? zGk-pJIeC&GlgD}Lr&4${7|`h@B7jl+AdKXq@g!hx!Hrl&7T#7k(DrXznAnhc4wO-q zki33tZ6%dwM}^dkijh%lWi*lNx?LgiPh@cubjyBGm^st@)fRF93tKI-Vr!a~Sj{!z zIET%B?Gu-PpYJT)>fN`x0(DB2XTB@x)gzrQ6sznLY02bixu)5ExKqjW`(MWU)ly|2 zS6EtEF@l`4jY`uit*opb*e+lE&mn^)=f}bkF-O*QR#-DGlq2tZC3kl~uwmNh)?pVN)~F;5#0 zrMC$N3M%t1r9*3j3`22oaf#Q;z#zKTI_dy)m81hfW{~$Is~V8P`w3?ZtossaDm3w0 zWXOXl$7NApK92-8zrP{s(bl!7Pp4o*cz!TCu;h^Qn)^dB6(!&l7=k)UQNjjA8{2DB zGBc|Rd!HKj4d?Saw_gbJ89jREK_0bDdxI(4olEP8>zyQ>tm)!bVZ*Yh-4RYvJ`Ezu zU5DdZdL)Ifmp{u_E3ZUCHJWZbD|Wl5n$Jf*Kr8^q%2L%LGA@p0rk$M-+Zw*_F>Y(1 zM1JZuuX@jpYGY+SnWUb;XrDs`KPniYrJ5$}R9n}35!zEL#VFtg%V|V190ZU=W%agG`3l;GK0)4bzZual-j4=R*wNP1wyc?HWE(2hn%ckRWYS7rPsEx+>z zC(ymqu03qFdW|YS8jM~S$ONr?D}QgEW6$|E&^oYNeoDIR09yJjn4t`hbPS?{O~#0d z-JReFiJ8q=R`17uj&;<8Dz~;~^;j7C)B61l$F5k=4tFx%+b4Xx4!a>T-y0el zO6pC*S-}B^q$eW!gAq(Ta1~4r(|C(&$Liu20&7tvf>z9!=Z(~852)#pp&87MbwOHL zi@FZ8i4sIYi4~>>aWcku#zd#LWZfPj6Z+9a?2Ae^8|SF=7D1*4BIiLZ8yLM2sL2}W z1~M>KhDxnse@EGqgMF^w$u8v3R)*RYlX3;Zacx8y3engV8_j#W{RLc8>#hmw_&+}D zH?4J08<4VSb`)>>_YrM}eEaiMdSvY=1`^NXl|Ew4W$Z$QrFK%zHH$bSQcX_WEtjsHw{7~#xt9<8 zUdbz=EVKn&8!0chmRLnxFAGl;SBkxSV@p%-mRo-}ILhAmxhy9W4d|?7`Q{h52OE6G zt-k=BAj~jP1wdV%?jQuUe!OywH|Fi);Ru>P@J`9HtLP+UVB(Pd-9{1H_P)~;1`sY4 z==R)mq&O_V(Oj`vjJUl7!wOG()D~#vIb;}^9>?GbmR0!pJ|?a@Z(Cot zw;zv@u3fv~N$10(kB;jJqQ5Lj?lvs;qXeg(u~3Uk1W%)%Tl$A?E9gck6!D*!W;uD0hnL`rKmp=4iP(hw3iB8#k6X7x5Q`2zY@Q`^XLX<{Wvr8 z)?e;)Yz1Y|ZHF;Mm#C3)icAhusok6AV!C&D(n8t=n~01)I%D*|6zuwE{kvG{Qfwl$H6hJ7q3zg811rknqCSx)|e4hiP*4i<3_aK z8%i9}Nc)F`gy6I)TVP@H7CY;G>+CcZ`FqTkOf5;dPR~o^g#70Vh6B+vcR(NH7lk+nyZAr>&BKL_>&v5|TM00Y`=(rvV_PVom@~2E)+nbOH|}F@!8`xU6yMV2U4j;Xd3Iljhv4XbG?y=sMP>#New4 z9mNa+nJvL~M$4i^Tx&4Wa^?060*+r4IX@W()u4@M5Qh}BIVC_ISWZLdbD7@}cR(zV zQUC5kH$huNR*QkHmJzh%XU-yV*uuP2jX~brZ5{)}D>072M+MaD3%|r?m}K3kv%cAF z)^*4U#W6Y@UtWAeWeh($w^;%!%`gc$x0|>sfUuCuW!hn@P$fiK2oBuV*%`d8_79FP zwzxIlFq~w=Mc&ER49D1?Aj4E>+ICS;`|O5ng};9p$D5^_H@_IWO$P0NKgw@9V)hBy z#@yy7`D?gVnj(Ouxtn|S;Uhn%9>SF0DEI z(Q5@b0@6{;DpLLu!^k13=m~J+m}A9!!2MA-DE%w(-2i2=Hj-H%ryE zcs&T%Z_+U`l;>aDAwypIW*eyE8UUy*=99!A0uUGekPntvHLKaca`VMpr*o~^%gvFS z=SkJgrta0m&m|sJxuTdIL<)7~n$ysPn>M==PiStWw-Pij1_Kxc7n^y=O zN@!S24r3hP-aN9O5OXT`D@EPIDDKK=eWD$NUfTpb2dybW62f&%w@I_O$?JF>&`)Q$ z!;*GN6okfgLmO-!-rZQ*!olYbPU}Giehj!v>W= zb6$drA6n!{kI`(Dd5!G&Yku z%r+jw%f_DC3kVk02%4duFkI2ZAuA0Gj36ar)$XB?fk>Q;z{iYJ-qYMS8w0V2%yFV% z+^%6AV3x4#l0lG;EO{$WoO0|>VtNO@4sLViR>M!4cytj{D||xSjE(qqPUf?nD3rE? zgcl_V3sRG8lx;YtB=+BpsC%koDaQ$ZP;BJ12?EHIY5-#*9RNVRf=r2)0&^THICVLp zu-I?cLsdsiv=hA^%oAfF^VK7x!M^RksHkn0@*zhOVdY6f!6)yhmY{L6BV-X-RV(Fj zAdAW8ctM8Zv^%-$t@@#!On2Fl8tJxT6Q96Bh*}-hy-1q-xge6$10+ei*scfVnFTp+1D1jF_ z&EYsAfJf>Vg*o`V(bu5cWgSNU5kS_8EFsI(?FYbl8H3t0QqHZqEzk7 zq_z=cHoURW6$RwqmJ_I~iDN)y3D&nJZj`6w%%nA`B%#P9dzd_BG7(423h-fIe|eBH zJ4%Q4j!Q?F{Nm4^P@#2$PVwXIZOT#qRT@GC_wfxRRHaj=|#zWWhe+eV;f z0PaNwH_6l$W`?3*jwa|!+7kFH3?y6 zn^G})AXL;M94?QuY+m;U4&QIY7XktPK=v|nXJwVpMVl;k7KLY(%`y8qB*Y29B0J%a zKmKSj=z-VJ2M15=3hh5w;1y!=|H(8A=qr90ki)%98;MjI4%lRLv#7982EnifI+J05 z7ae$Jq?#l{F@fo`oi{R}9tAn7tNNR0y4#A+48T#a0Rf zbNSC;mvh#!2L)e2HXYE&evBX^ermzZ3o1Y z;T$6wm`6k-c7{F6AlU$T!DC+H5_sy%gcyJbXt{n%NXo*F2Vv{XUMJFF$n+gwXaHdIQokPl&$rHo1xoI&$E>Lw1i z&p0^v9&AXukuFicLVPFsuY%>xOAMUM&O#6K51(y0K_cQ>%%zhen0O}~#_9l2b%`K^ z@G5DTmo*JMZx;pPA3vPa;#u$!VUxELF@JDq$RE7^TXAs?j;rK^gnHbSTu*a6;o)AA zN8HNxsV9ZDjbNs=4W|x4dJ|b9H#+Nkb{CBx8Lz-S~SI9n#^dRHYGE+sQ3={H57UtZuYI>aN9t@umNl1v8U({kxBvjMqgYC|d7 zIvZv>;35KnlMtF{gbJb+sZTTXfz1d1@tlt{bvWcB8ST?PoateoNJE`Xwk5AI>KOER zKlpq|2*;sgu@Dm(!QKp%d>}8vwcJy5ZKnOdDiIFdJ#Ygl_TBz>wDY%*s{kB~j(d?9 zhbEvB2_+tFf+@5!7MG-gOUIok*CUp8#Phe)n!?L3=w^P_wBGAXZ3>NuWHt=q4&q zDvTEQLN)_?Cl%0%s#qUnj=H(~SG4kV!dXmY{?DH`?3i^yhn1hi>;qgG@<#M|o=HI; zG}{S?ai2S8`xZQIUQ67=wA|gpB3nz2{}%0`CJ78W&BBt!m#-n4_V;y09-#tV5HAG@ zl?jolr{F%#orqYxrerMw9m_z;W1mm7r>dDZbf!>QOS6vr`s?o>0G+@L`8DIT_h~7g ztB_j~u=P#RjT3N$xEP?lh(&<}v*e4Ro~}4Sid1{e&CR5%;w8h=2f6tW(TSm*zkw3r z4RL7A`#S4ixx#nyTS-ub@H?EGHAGBDD|hYEhC!j*wM2X@Qm@0m_62&BE)3_Bv4O_~ zfrk(WIbV@f@bJMxF73+D$4b>IIEpt#H471)pyn23tYV^%lg`*GW+RJy?d?Ayh!}@K zf`9{@v)vilvyVkg)k?Ynws7fONR`H>m92p;h=$^rxTbRBAUQG@3a1%3B;oA6(qyrk zk`j4jD7j$aXbk%!57Xeb+Kqh!a}BHyAKhb1YtfbBV={>aD=PVd*EPfvGN=Fb6M@;3 zj56sqG>ai^BF||l+?}0 zGqBVQh6^aU|Gd)9u41|~|5krEh%rwT+q0LO7wIS3MgR?ZA|T8b0bEdS-s6R7%!kz+ zzqaOyb9o!tm|$jMYYVtEASDP704F2RgWqGM#K@9gd)0RyR&Dv<(@(WWi9sjtToeaP z;z#!LB^5mYv7wy3wk@{D>6MOb?!tcv)O4bHoXs}D9k`>FI4}o2p1-oQ&x$(X! zD=VA*SOm0i^MYFQpkyuldH~xoz_AePieJ;s_%y2ru6A_y_gi3bttVJe@+zW{!lCgm zGIG-Za57$=r=oW)A%%hVzwf}|`+g;R_X_f*KQAH!m5Ex+j@ga?KY#PrCsE7Ga1#8V z>tGAcL%&U&RLOh%+pW+3&}}0ep95?0oTY>O%}CGS>xly=RU!KTUf1d=_$ynSSbd@+ z|T>+nc!H2-P5n0**7T5qa?+^YIwP=P9UFNs0u6X3)!mhlahF@>=V>eW& zfwhow{{H%xccQ*e2kb{tf-@{R8^-`H zWJUkGu~v$pWdp7l$V+OY$`xo?XRmEZc}J^)el6C|_b?r+t-FCwt(4dR&t7hyL?03O z;mv}xhCIK6&pn}@&XA+Y1n4{R0>mi+fpE$R%tGQfdMSc-L1a)sB_zVO`}J#h=fGJd zuoe>b|9matH;DWB-oz8FWITMKrCCdUy(w=C#sC9rf%qQE4!!oUE$ly;@cl=Waso7H zpG^PH$|2AAInDCF6X*Ly;0tG~sb8Ojqw+fdaCBVq$H@i>dOW+ve*f^^D_Hkx>mC;q z>q^XQ7S4{k_V9P3fR~&|*U58W4vGOmfG-eOqyO)xh!Jq%Y(k6>A?g43RXKXK1Ba@q zMV3(hcT#V7_}l_54j$8N5dHo7mF56OQ**E1HqcOrH zti{>Wk1?Y4e}4)+a$qfgZzxV}gJJ>~_E*TRw1`U8+5fl93JI>H8>yE2-&!4idM_!N zqiDwS_bUh?K+JRMQDFczx%c0z=}aX$%XQq``zgqajN|>e`Zko6{~b5>qQ?qqgGFqc zfuT&}lZ`)OX5V)9U)$t#S2#bKjX}Tv&_43bn-g%K{xoYAP`T@0)Z;%t#hA=qL=C@h zeSJx&DL3k@o}PvNf4#y@l$7Xgw)mvqIy#JODC4gG{aBifQ2qwiLh>e>y*Za4+`q3k zrUc-Qxe%6`4olW6Hk5zif#1Kjmi$_1WOh-|&sPI@GygxYxN6+~-s``9;|cQA5>ByZ zySsnks^4$6O^HAU2r|1UfZXC?M1?Q9YlZZNmFnNi!f68CncO-HIC$k1n&c?^HgVy< z|GlsFVON(zzPkgN9n9noj=pH;xS4+nz~uaW4cbYo>u z9K2>?*c!Z&x3-34+^kTJEIA>3i!FVxa@VhSIAVc%gxE;&79qg`z7)ykJ@Xgt?aG)N zl7~&dua_bK+rQU)KP0U-Tf#i__1&^HL!c=Pbwl6->=nxADh z+N6&xmYyH5Y%SFMtPa1G^Z)x-UxT|SXeRj_z;RY~cBm-N%)(&5e?=mWY9QKBUZSJJ zcKA(p{r>s0#cOtJ=!%+sz6k1qJe5`dwby-1w@5nSl#$-lbo|Q%&tk!UMe%LHo1kxK zgQHNR;t_G^9^9)~wLkYr!Y)(+z37Al)C6b7tWia9zSz^!)6p<;mvHC3Qm z142-!1*Hf~jv%6C~& z-sFDYci-=QpXYhs7h-!-zfiLVT}OwTSLe8|n{2m7Bt_+|A|ZehQo{j2{JUF4UJjh2 z&XCXztef;=Wlt_eEjf!a0NqDi?KsmXmr_LgQ?frhUrN>Q z^-+l{z)DEbc9w*&IKQi}mo|5UU7=BY`cCOX$Gjh{v5;c|6`>#K6el8en>O!ICPk(g z3UztMoY8w-2STG(JQnN@3yos;aT~}IY7L-X4;{%}^UB)`Zdk)OT%!y{!;bQvo@GkdRAyCAv?dE?GeF1aWzMdIe=GWn#;5}T3ZC*(+t*#&r|v}0LQIP1 z;PJ?BKxAeVkH8xF!lhvMrSoZsG=E9pw-6EQ958g#@DyHoDK=XOX;U>=>g70C^7?Sb zfp<&4p9l3PD(Zf=5G{_T4&_gXyraCg#{hI(gYsDf6cY`6Y6T$MNmLs z8^RgzP%i@e@x2H2$1LPU1Ry`jjyqr5cur0ch>96kSaT?&*01}0t4t##t9_IT2ECt@PuoHe@6u4imro)okwADGXs52+mL>grm{yx!!#)G~BES%jdr6(FHV z1q>N!Y!Qe=GsP}E>S|*f)#0e-S^)?pA@MN;4whI~=*Rs^2e7~@Soxzh2_eEA~*@yCBH*`eCF8T}*L888iM4(>egd=a) znY$|O29o9h&#tZc84{{cbs9_0AHn(cf}HU%0SFMai72(C2G;J6N#}FEN$xE}9b;B} z^c%=riM)i7c8?jy=MB3LiB(9rsXhs9EG1F^kvEE6wsE7mY)8CjD!`1#Y5?$6z)nm8 zY!RBDp~$eTN_IJLykAihVt`AS2B9TBH`q(G=iDhcL3n8@1<6c#aYQ;{<=x_obu@kQ zeOg%onZ1V(BpzHu{^GP-(?WN_-##=C|R(WHYJ{k?q4<)ihQb41~?>E zpQ`+XW9J#W>-8HK5_=4I=Y@@M72Cl1QXN%&YV`}ql>3zcn&(0=5}N5tUaw!{SD-kW>}}fuQJo(s1M-AaBp5^o zJahW4tvRT+*-cU;*l6F41Uo>uR}8~K{XyKryRIf$ z0)QOldsH;kVm9F{O?-1|eCF<<=tdrdf~wm9#vcT#7Civ$@2@Zt2N17@=D#H*KMf|e*-dzb*iWrvmu&FLJ)5B4t&Hc+-vX*?3bwY2ilMK#xma}|=H&4= zzBG@;FFM~X%~5cJLJ6rp&hHIghzJLyb#3Jh!XxR^VEnG)(@oq@x9ZUsg_=}gMJ@v3 zth|9PSY+=>62KU3fkgJtqjMd0Wm^t41xUR8Dz z?edc;4=#f&sEIr6fc^4#*T=M3aN5pCUzc6Lz)G4!rB*|w{1ZCK7XxipnLH;<;IY_w zlLs2v#nf+zrp8C8Hcr(qsK6WW%^kg-0I7%-1ppR-%;ZmQ$*(!wei&&b3J5aOfvOk) zfC~j$IqYrol`ByYxEb3%z|k}r2=veY*7GvXaH{?)1JZ$N#kAZ8#X*pzByAC?h|Ov> z)BmjwCqNdcM#+W^pe>+27ZGQVe{%QlM`9zR7<@A*Q<1XYEv&C!G#Y{s6cjFmctS+JDR^z#+3nM$n zVxX44QceO0bp;+GG;g8PmtT~l@}(|l8K|VrQ=3kaz79UmA1+-&^DLymlEV-A4OM!d zhpJvAUeQfpveSmX+muc{bx_VIua6WFx*I`g;e}E^aEYcvwWwYXRSIaRdbE}G4&D!o zz=l_Ksb6T(sid>}>T`@Rjv!O(uX{vF1X#d|dV&f-mm)m6|c_$1db9mC9k#+qO_5&mrgsZfpw-DYGwFq#ldIG7V`xES8)zGpD zqWjT9ibPUyXFmh1)kZaOZwD5ULnaBwiGBZyZuQL#k0 zBRX^n3mE7d!KmKZLLsVUWV>Q&OMu=?0ty5HdU)XTK#J(Szx)!8n?K_1-S;X$=))g| z*$^@L5q_>fR@^dz5%>(v3xJ(}W5;m4;AgT$zmH#dheg9_9py>4ZZ#b0LK`&Go(aib zIr@itK)l0|d8N>^ai=8my;e=TXYSc@YIq<9>dP!waED?cdm^wPrlY* zC2izEaTdNtxf!t~!dr*iE7=Q7qb=4+xfx_JseS3RHjjE0hol0mV=x8>x1DL%+EJo5a6Bce5^BrKBM%@OxpuS0D_X9NPBGt8;~_z zP7m4SMkO*99(Ru!&0Ztl9B-N2B2@iA&Njke!MX*Q`ul8n;#>)+H?~xF5vIIaD*}D5`!DyZP@K C76Y6B literal 0 HcmV?d00001 diff --git a/docs/img/extending-spark-connect.png b/docs/img/extending-spark-connect.png new file mode 100644 index 0000000000000000000000000000000000000000..381d99bdda86556303771994717132934be989eb GIT binary patch literal 118823 zcmeFacU08b)-6gK+uSBpz|cw%5Ksvs64W+OfTH9epn~Kek~3->5D+aC0|-inB8LKr zCO}j`LJ zr7SEgD=4x@lvr4nTxDTdVDrI0dHLfI};W1vGg>vMOier6uql2vqqp7f;=RW7X z1;79O@j%7l`%N$9`xg8Wp>c3sIG^gF9anCK{=WTY#XZ4XFO`M&bR!%N3tj%3^Y1jL z>9f{Ne}CZgKfI{0&q$zJ>O!B}u(Yt|d-*Ny4KEG$k|+Dm*Z1=_X+sPC^Lz5EnByCl z|CjG{vz)^-|LZGh{@*YE^;;H}7c2i*H2=SR=fJ`Ld6EBFlmCg6|2aD>2R@s3{mMGu zb1q%H7`@ZvaySP)c-d^ecwt(@^)pZ*l1UGib>6X@Bw!ZL3zMAdGln?lQc5vZwZeru zR`0yM!`EW_Z{*uaA4;m%&i#4#TaO=o4l%Poyxw`FH(pRDtJaxq?k_BJpZaN3@<_O- zx-yCP)!CVEYy9cZ`ziL+vn%mYGp~qmCT&wsI71J)IzUIUo$8Wa}_cSbB zvgB@HU|@<7ud|F$v_~O{0 zNl#ts@wT#%np~F=PD)+M1?4ZVuBpariLU2WdQ+`DTM6zI&OY4PsM}j*61>*(n0|YA z|JKc$vT5;tetx20ixQmlEP=gv!sRgSt{ouFe@`SZx}<7T~| zo-c-Yjn(SOFSNht!>MeEv8L5&tp|-kyjQjd)L^g zbNBAuENkZV+gqe!@uKbcu+;F+pU)2UHfkdv%0E8)%OZE3MhE^)e(><&!#-xuFMU%A z6thrye(3_%MxNWBvd{9<$2_<3wDSc^8*`kLskWbA{KlZED|Xc+s$+M|bJs~8pBiPZ zx_0eaab2C}iN_*$y}iBTqkb*AC+BRrcyW6I3pLgnXyr4Y7O&OOV3n_Wn`8g3;Y|nZ z;{pVAVxRqWqZ8Xt4d0_*7;Ri0YTBHa8~@;M&QIA6iklDEi(ENdT3X7s|5Etaa1Z~! zeHjaS4mkA2;V&Ckgj3k|Sq7aye}3J%b(|g^9zFHx@{fd#=tI=+LElG4_VG3KH01^+ zZJP<(7t1{7&f1Q(+zPwtf^pd2*3u_|#@HT?&6~gV^(7+A_&la23WxBiw9*SW&H1-& zJ6Myb_keT%{{1FRIZk7rude#A=w-lxX8gqW^h-`>4+=~7CT$K`noMf3}o?-v)Boe2rv znzBbuox0=x^VOr&0||;(D92?96hfr^X9^D#}c*v!YREo{5D;h;{ZP*LuBp zaq#G|V{*#}Z3j6(g7WcbKT>R{5 zeI~7)^h~?ENB9{w zDJl+@%C-)w4Y+Jn8dQFJ^8rqZySsa=N(?o;s6jPBH&@GLxVt>vtmbQHXY8lvHeKHy z=&4~bzJB>~w?$Mp-(81&pJf%E$8MnY#I+kY-d9#ClQ2g7GM*lqPpjcrz4{rKe85f# z39a7V-p;BR|Ky{sR~N5Jvuu=jBIBXEYOAygcAzT9$p$~e&j_h(%yvk}A09n^T%M|x zX8eo;VP_nNg=J=HB%kLER!%-&kY-ZJO{u_1@Mnx5ZtIiIU0u0ZB0O-P72o#lhc#0T zRdG605_P5S3+YK~GJ6^_6p#CDxj&exq~o=Qi&JV!LCn1NDLw{?dK8dTI1MTha_h-i zC&4?mDSvV}aEqklHXa_HbgSm3(Yoih9aJPDn)e1F@Rdcs-o@ zd;4t+zrBB;s2ZzsI_#LQ&(ezlXR_^2u#1`B;VdXDd}pwC(@21TcBGWc@EfzfZ{H&5 z77dZ6O~3Y5R(lpSC#@~;-tYiNKQuIylj6-Ppp>YaOO-l#bsp88n5s`i!k_$!IicotDIG|{6>q%Ou4iLoOPsv9 zdWS8uaK(<@0c$cW87Pa?Owa{<~38QM%3@ zT(p|K&V$ARx;eGA!Bs(sbCZeDel;s?iF2@)3Nbg#xHh%oRz8w3No=DiGZ0vTb|MBg;YO3Lhc+FJq z@@)?}k>5>_X-%u5Pv=+&mpDm0)ZhE4il#2KaKQr3->*=VP;!;mFIcv8sS-^s-e=px zjEoGmgfqNWR(v34 zr{1LX!y`4nT*DGJM)c{wJSO^&<0UB)uA@mf91$j!k(yFdIBs8&-hZ3Fh?8xB%a7X# zsj#u3uC7(%jBaA? zaJ{fwZ=o-HI^Gd$r$W+ltY&JYtE;Q+P^W6or{~I*ktc-bY2hH9>TAjs1ui>M)cD*c zs>!)aAgumTSXdaFn0YiHUG&+PSAGl+t6jX9v~sI-9FD&!5CC<{VlxDPq$*8KbF;?J z{-Ukv7rz{4O8#|y`MU=~A!l@S?7qBOh&01h(8t5cdBmcD&H!NGlu9^@W0D^udv|-; zpp6qNNlQ4gY+rUAKYkpUL>Ki@xxc00TX%OHDOWffkELAb$OJfNG(VXsjf8}R{@7%$ z3n+BXqu-x##$b7!hZyRrh~MW~S)D)I}S#zEnb<2&)MJw3Cw;#O~ZP}RRbvxUxODz{= zvGCJg!w=E>nq8GiB&a++_#3it7t;RB8}8D08Xdi};LRK3_K(l_PCfN}EN1bzef0F{ z)A&r@cu2!D{Casg&V%jGaCD2`zn6dc@+B!zhzk_36gt zuWGr)`1s_ka)$z}`zjS>RG%Hb){^d*Eh_LU!3D2cNtvV_nfc8Wje~=ORj!khg7d?g zq3F_NUY|z6fw`rgTwGSBhzzMxbbIqRR3`4SO$g8A6JOVI$=&;iZM-LJRJx$2uQ^}n z7eaZTDF%dbB!d!%m_hE}PWkcu`^V6u-lS%vTIlQRi@J_pJn=+=5!;Un6^9_+)V6Zv z%Ds;i_>?1kP}L)@uV8=2V5m~pZP_B|Uwm)d#F{_;c)DaQw=C6^5gUOj&*ON~(J@mi z!_p$Ip`hR>>V_H;ywc<*5p2o<`|dk~;&H;pAD^oDJ$m@|t*mOIu1=Ik3 zp8|6DK+m`Dd5lVNe!<-{XU+@)Lqs3(TEiMr9;3{oA1K?=ul9Gd+sG3Yw8|@w zJ{{&WaCgUX)3w$Zb#80k>N zePWa+N(MbzhJ{qrV4>LEUkCmXmevGEq(62Pb^l0T{&e204MGV7pMH2GY%+C5U0q#3 zC#%Z*4+ZBld(4rw-#s~yVcmoBlOo=PoQ~Yb>AP{ytMlY9JTt|i_%SdboKO-X)7*)c zkXzL9lZJMNr7E(6@3x17;x_}Re6p)ouO`T3^#YrV7e9_PdxUTw+O4gTdGQ&(bLmnA z;$qjLmz=!FAzIHbMXbJZ>((tfHHILCO~N+10B=Zgp3L+_s`Zx5o8z4ZS~aEe{Qdn| zcWIp6)Bg4I=R0Rt9>2{|4%j7ddq1G&j^A6qe0dfj|6nHZzrG;91bDW=?kJV6Tv&9@cHE^%osP*AU_{_1xM5pSvDSNs9DruPZq{*H_&j&4*65eFx6b zeY8~8QpnoMQ72U+0dKuFh-;@9coopQs^f*5o}o#V z9&6u0s~+pk?u!L3Ec^*z5$CS>D{naNnd^K`IypMxkR6h+?|vAm@F=vUz?*#pJ)f9G zJ;}?l$W#ECchBqVD-Y$op}Y9dus)X@9QKutbl2(ApP-ORPkjFiofQR#@(zBh%6!15 z`c8Z$YORXOJ=IC);qDkbRaEC@$<=GuhGjH4%jZu|x#aZ6FADRN`nH}{|Lwyg<@@~VVx@_C z`IdAYq!AoXLwj8c+QqBZIW8lX-ZK6-R&KsEay!MS%#tyB`}S?pMOsgVzhnQk0^N(} znujw8t*TrXhn~jlnr^x~Qg0OsR6O>_1Z&CbeBIkyKIff+SMfbwe;{z9prRG@@N1cu zk=4R)-u#_5#zkLv;nJJJ!cHJQ`J|E%e}Co23BtL^ynIdByEIe68E!gAPQbaF+N#}K z^YZf4WOi7pN|^EcMYL{04KQp7l6I5OP0yY9IiM}&4y@wK{P^*dPM&L>d3NTx)UU29 zsRYcF1nyHL?FKq%Dr3X1=O~kz1y&Qs#U&(A;tdJdr&~2Apme`wFc_I0SW7evT3g*m zpKXzLeHfQ)^Y!gHyM;WIcY*s7t<8DAQE=2!Vx>3jHAr%v_?qU+-&=N2DX#ADV0FB9VRBfQeoB)v z3P#JdMFMVm^XAX@)i38};&Z`2B$el&dY-G(i4bWWWYVh9 zex|6$lq*W>dxO0u^=T#&ZewWyyEM<+X>dMo;Fqr3dy0zIF(ko#a;z91q`LA1J@edd zGd=?;f!@c=+ZU|J(P1Vf?M^pk@D=E5j@3CzJp6?aStJk5N&(U6J;+DE@s37gB})am zh$5g4DTp$x*lTZN{rU(H3Twk=8TT^?`U(UYQ%6W6=_zSAE|dmoJ_4cS&<7NS!1^mI z|8?g2XJaFMiRWHlr%*v{#3B24cGsnnM`|q#NyPc)Z)|&@W&PJl>x{i=YM-R7i&FOS zI_(S4SSk}aocb$lJtw!%GKWvZod=`ATkSxPKx(|8pkQ%l$e({LLP^p?{Stnqj%JI_ zw>;pJoFWQQVRFQ~V>mRXjalh?HW(QhYVyojp6UaKBaWco&ql+m| z^|%}Jz1}R5^o`W3SO2=w72-DhsJAgYYM`ymtmLsM9^WoDWJAVwZ36>?TSbrd=wIhn zc({#XS`qGrvqF7(;?o^vxp}hI^RG_d-~I6F)vJqEay)D;yUG5+n>}Tg3gobm`o-DZ z5o=j=M``VH0^Y1$$A>LwW1Rbz+kbnvo>^O2S!o`KKxHdJ@9Wfi^(^R$Du4i?<3)da zG`65p?`v{=Ej|tCRQdIhHNT_gxaOk)PFo&|W{Sa{25w}mLTe3;hp%3}q7M0FhCF}% zPJeiN@5p{h$%#jQr?=$go7E)LITz1XrrZBirZ+3PBxC*v(6Xj+aJ4q&x~RzrgS_yV zp3JbGLNettUdzEjdIa{i;+Rk{2lu(n6cZ4MJXu4`1ARIA=8ey-nx)a|2s6rj#(VAi znx3>+x2SpYqwB2``&Bb@(6pokTerMf!mKST6JnTo=zc5qKaTXayA*9vU~Ab<)!2L1 zu7KzULs)Ve7WwIVezkU-X}Zhl+0V>9hcS2%N@qC|wKD_@m>l94Ux=5Y7h+5jf>t#f zC#$GoZxT-Al6nw_LQPI|^SZd;B7g>^oM9sZpG<2Ko>&L7GG2C8$D46%1;V?rYU|12 zjV>d-j2Pu8p8*NG&W9Fl9UU>K0c>pz8CDr%n+SHoYx@q|TF)Qp+*8I6gn;T;Dn8v7 z;z4ChqpIBU^*xbx;#0^YkEx%>x&4bf`w{vHo}MgrJ@u*~?qe~G$rj5VBF#bw_!k=p zro@Md$2CUd03r?u?Xj@YZR@Fb{D~Tk4#0%BZr?tc=38uVYr_?Q0(BtMwP`NSw8Lie zb(52mAA`K<)cq=46{Wb9>G=Bk3JIql2C;=St@PNX>-ji4`m-lNRZ`QB8Xr0GH-Qn% z91tF{_~U|Mz_C5I@hL5bQn^nC8FM@k($ht&rWm+kEQpj6Ye|y|PBdE@j6M_$@-u8A z#(xOi`&=U#LN8yoMM|7Yc_^+N4p|1%JcfL&)5 z1q^da)-;YkBkcGp{Vs7-1=9PejmUt#!uH=W2Cn~kEZe?kZ|MQs_B-1~xhRBDLk{yD zz$S5P3h;7sA0y>6<4jYU@5e0e|Ge@p|*YIq$%O>%L9XnmNp};8I4{K zSp|?O+En@F^$k%}GtTBpFtme(>{dH*TE<8E6w^%AQJEMi7v!0)-#@=xx_r5U?F9aS zhNwE`DI_1TtN)c-tD}nxd(ryK{rVaFXExLKk7Afaw;Eqdn zb|*dxNlKmphf}dZ&vkX;yqdRnclhhPH0iEQ&T<(^Kuu*8lD!0mDXY$V^K4RCt3Fl#wDG8L?hY=DoQadG4(;FVgCNvCBE{L^MHe(281AZ3oE#s?7<&#bKoz`{pk7{FM`aX|zX+u{OL75# z%K_RPb3s0wMBEX$Td?N__KYNOKzV97*eeq(I-hP%CIJCxQ@XIp5;)bp;6ggABX)rg z5ztC|TfOz#QT#fa5rXJ|5fC-Rq?yiK9hKGNKL$miI5iv+cKOZD*+KF_75oCdcduYGM<=`1DwmIUor2CsP;F)L;*MxmXAMs#tR5)XBzpN6H zmk4MvY*S6=od7^CZpGlZ^98p^b}CFIK?<}I*i#9X7DV3(NWrNE{25C?e3m@i^A}c^ z@wM3B_v|MuufWvdlvB?R69y4*@Ek*!kg+Hs(MDy#5*|}IoV=WzZ=xkfM1+Kd$O(ri zlwmpna>BUfjTfM5=4wITu8upKaDu(F9r|L?en?oi9Jy7bk!t9>l0*C{B0mOH8C`b` z)dWF}lj(5lsxfkIXH8-VQM~bd#rUe0p9hVeZSJU^0zOT59x|OA@5_HI@xF$_o`5-Vmk zR0}*=4px(kS2N$S$!N$ek(8q;$QBzfJCg!^?X~~jzL^TWO?vU-8`r236eUnr z=q8N2cq8OM@erthoI})2aN12xP1q&nTZQak%6_S7w7b~WZB3Kt&vnpsp~FL7FuHa4 zxT5#Xn;)?o>(;EUes&cdRrLdZnF zEmCVNn60g(yg)?kL=j#>|LCp-T+w22fB68CfEoQ#JKzdR@LSu!;{Y=02)`o@KuILk zqQ0t~I&~`4VlxLvn){?PBTiGeu^OwSWclIY9yS^G3IEEF6Wo;z^k8M@u)V)LDbuUT zlOr@P;o}IF0c&YLwp!NS+8POkC+ht~&t(KVgsmu8OQZ&joSd8@D4)xkush60YOL0F zc7**m&sAVyF*ig@$=dgGeB2CDLw~Fvpc7F_##+3E22pt)_hf(*fRq&y`%uUe8DD$( z6}((WYMHb$3P{W3)rX+lO^I6sLSbEe@(FH#V%0!as&jV2W2jNaJr@0H2gufkr$R`C zO`)M%3iS}_Q23=-6|r=n2S-+p#Oc)F81LZAbd(0vDCxFaym)b#>8Hy3cgf)=8Wk4s z9hJ+wZVJh$8Zh1>msmJy|FY}7Sk{MUXU%Z!#(@cS6E=AJbd#{5MqKDPSp9HET@don zN-N+dS+{ZH!>{yv#e_{n73@ILBwM|G$Bs&Ka~m*2PXQnzJJ>2J5H}xmg`l|*!QQwT zS%VPXtbIHq2mXMXgiz=Jw=3cJ{V9_3gn~>6QqICfi}*-qDz$^lvN-yXi&OBFvbqG681lUXVCnsty!xXWP{R%-hgnwyoH+F**I|j+i?&psS6u*o} zJV!1mMawts<#bs=we6`Fg&2GgutWA@88Q?}xX|iD;`VHnb~PT(5)H}cwCk=t4M9>Y zTjwkG!@t{V@1sG$Mq|+1V_k{)gP&hsbsUkBl=LZ!1c!df+L~gUgCI$V@!+K^sh*7v z%|vxVi(2#dlC^0T4PwmfQ(*br#(OfzcA#TOKx91H>_H^CuK0{y($hpFhhV&>Z*sqc z1RsDqVcYrC6U^z&kmiO+{srnSbZA|I00U^118S@0#1}INrj2L6Gz-h{^?zB#bRE4Gy4!cxJ_EGk-hb@nm4zj` z`!d!Kd;V&e=n(~khUKot6v?Qc1nYIE`fVu%40)>E7O`g|aE!J;nZ zD4w?*J<>A-F1J>4H}D-RKlRCen`3yWkKez%UeC_X-q!?#lm@ySC5D2Jxi@=WMu;@4+qH2cDb)w?brk(ciwgkkEJ80@xIcPcuE4n_T!)0M7`rDsFC;sWXFY z7z3gwaNF<0M*!`(q!QMsC7nH#YDTt_)JQ^2WjerP;C(0w&cPU{GW`i)0%?c>uj^le zSz!f&aY}{oIp{?E1$R8VKa#HNz==BC#^)?9`r(0NF*vHJGi+EN!@QNeNt(bm>VfGHToB%^d-F^h%vM{ zr;Lq_Z?%blR6!Xw!1p7fBQxdU9uKN1* zYxu*54;PFOWQ_u6Hv0WbCzO~1qbe9eG|Nz^!wCU>=gu92vVD8^DxqsoBaR|~KZ7A1 zi1h_75Rdz%LxAMX&^9coM^#4iZvmg3+Z7kQU$VG6-kbRZSeb^LU%d260cp2U)VK~T zFy=iq`gcf(-mTX8EHAuE%jeviLXpV$vEa{Ru^iO(tXMM7!S62@Z`FtsT!-QTkQ0rK zaQ=2*T?FKt%EGvc3Ix;;j6IKY_dy8}dDKRfJBZ`qbwTe@xx1xhCI|h2^Un_lqgVr* zL?D909Z#atr~y$4H059Vb_-o<7-7YU`4YfQX=o+nGi1@kbqikx-}LdR0z2s+*<27s zqf2dWy7gYE-3$-MIxZ>IR28GVc7>m&KGLjKzPsDw-1%NeQ=jv_XOoreT?li=I8~st z?&E#*yE;hiA-pZ2roP(WL|Rb_k#JPfysm!6`wgk57$_ z0(2sGgE(E^SC1S?6bH?2WtbXB@?6Vvk|>kdHe-4gbYNSk@@XR4+Lx}1fgr26vDMwU ze+nHGTx?fW%FNK=AgL>bT9ERKTup;K5PRl%3ipY}#C66!t_i?V4gRpC$;RGZ4Ifg( z@{>4{b8r=Kw3Iyb=dGa2_ z&rr#}W@AJuF~!!}0|^<@vLgP526c;8^Bc?eP&fd{;>>*2`ern{Yo9xJZfun4=fB($ zXw?i364H?@-5?l8JRKw{?1jy*(iuwHjzzkU!FekLIaI32 zk&oU*E636Lif0{4Vghz%rt0YZ@wc^)E-PUgi!#8%6g^n<+@gcz@c zM1A)p&=a@WviENcKym;OD&zdx4z`zbHjq=f)ot*u^v|zrC;L36D~X1I9z|;?4jKJ% zZZF^hI`bHWV1EKmz8=wT;Nx36h(Er4ebrXWJ-ZFPQd3iL9-~ORR8n#R#MCLgK}$MO zF@SXk(cN;kZ{4^t8WA2%MB;R0PwhDSVO!NCB;$6hwqe6?N9A!~OYafk-RIx@Zr9h; zXudva*#<#lNbLkpr8HgU{8kpChoDprmUOZOKEmlmL@E?-fgI)X@2M2~2x!~b+Mp(v zJ}DqAEk+yI>9O`OKNuby3BLi%p$1w_X>bCdxND$HM)B0M55N(Gq$WKFy%|f^a_uGQ zhw&lW!F>>I9{qec`7P2Fp%Uuk^wEo{!-$!N1zIqICNCh)P8<}VRdXJBJL1m_b+GQJ z-~yD=UcQ`w2^kR3RK_Rj4&)mGx}?Xug$a~0ylSoJjZ}!$C$3CYkh#nvKXo7x0eU`E zFZhox4C^9Oq0WRC-S5hq9M*;b%WO$6aH2fnX{M(}WN0;-Fhdc%Ae1|(M76;8EyR{{ zOEvNBwcNNXSzv3(%BzG%G0(<>o1yqDkyZ!wToNp$joMU=W3pxt3O^;~)miLtH89nO zJY^P^DcZjwsC^VdvOd+w56os1QIi|{u!qEe3}qo6yi-(ros{!nlE#-9fM3=OZ+MrC zu9yMAki$ALIXQ&v*p7a_25^RH!5B%~d*R)~T+5if54e!Dt5NioITrYbFI>jjPSASa zDp(=`(-rzh1K{Q(X`nvOEnC-ZNY%T+=&{ZB&xb&GAK7utCut)m@_AsvJDbG3vDkuS zZ4f&sh=I9Ayk`tg;v^DQfp+CARVCNi?&s*}!`u*^vuf2U>|S8*2s#9f$(5WPKzrO6CwrkB{ja%zY;>dCB((bS#uJDcoNx@p zAb#Y?5vrRa{CR||!d4NZLrSnV;I;P%Y8x^7q`nLR_o+4MgA?4C>!SYb*)zg)dg-i3Q%2|xIOBj?MQFqf*LHT!BAEfj7@Pp3dCO$dUV<3#6a1SUW?{D z9YC5(_V(2{CTS42QuRqhHo5%1e7Oqxi8?wuOs~B8^5=@dH1We?3m_tY=>(wNEp-au zgXFP)p)dQA;$jK{l6ZaALrR0ogxD03@-;#y6ZG;EaMBPqI>oWN7h>jBu7z>IFj2;N zQ1;80FNXZq#OQ{+K-?k2-F5!P>G5fwySszpZkh6?qa`MkTIW)AUcx|=!Uc@~PU!*xb8Ozgy#xFuH$8tc)LO~}l;OiA*8;c*=*FX1x5 zmJqw7b5DBBJxI*nJwm!U(Y(r$oL!p$1{C_;Z2k;`1%ZeSaG=(JQW^em^=t)mcWosj zDR?0qALLt)WOxz~{RF8<*tGf(jQI2CPp*1+w)(RG5_8M~_|8WZBA3Qnnwcysd{dR6 z({TQ6XL0o%oO)50=9(IHAZ{-QfnB?Hl{6o^)by8J6NK35h5wrI`yN6J*CbycK2+e0 z+eO!g!ktp+3oUOjFDz^*KeQp;>;XC4aRLiMNInu~->seFm<-!RrJFQ#dqVa+!+e4< zXyG4&gCY%rkc1E!ehwu{4Q-E59Z>%)hnxJdw$pA}C_rmnt#Wg6f{P`=2=wS;A^*!n zjjkgM(@KbuwG*4f%r&TP17Itf-N#P=-0X9iMl!*GOtP~e8g;qBDI|lC865(sNePuJ(qPl=lr-@tIf-iB7R@BO7n^N(Bm-c}4hGl_vnWta#fqaGAXLa)d5 z8kMNsF8&6V1?fg+U<4RYD$E`Orn6tVjsf(5Y@jyP$tOWUz{;y8QFWH|64oA!5;W{& z(>SQ}WbA{rFN6@qcF4=HO>Jxg3P=~9#z+A#x|gH3)=@6@kro!6=wmSM@qn{nQsl2()VH-wo2iI?FkO)r>9h0sb4hSmM#FzX*A+xOQB?jj zBKl%knbN2s;0z>Nx~UJi0F9gt`@-+9v$>qYL$D3v{$Z|qoJO)^8(|b05-xsu^%&OM z0#9WH1%;(epHshJ5y4XP0wYlgBfArzc}YD%Qb3}-?kCTGp^*&TX^jW|pj!qTW6l7n z9~MS?ACXo!&E>Mm2Y5muqzJo|RgBQ_Vut*YPR`Cm`Y;?fZ_Hv;%P(a4EHejp$sS>Y zuAD|1O3jk12IEi__WY#A5)9y5g@{tLAA9+y7i)`WaZ(O!zad#}jZ75*zPcUwSa!sL#M-Pq zh&+ks23<7tyPC!UyYWgwP=#K1Pr{^37<{w`VlMeeQil9HG%taQj>=dZ{A-OnRbLO=oyc27N@rUpx66PhQNN zo3UYeApx{n161NWfYvD*LB_kKkIbkf>mN@UGkAB8vj5W8Q*eO$r>jyq$?P0J`z2dj z738;)sd;}u9*W>oiQXn7Bjc^TTM)GzJke={lq%{qT33D$H1G;k10~pE6}|PfmrioF z5{Ccs{KZwMrSOQ8szwkVr0Tk1B>pM@w0y^!oZQ@!MYTkFK(X$En@{+#Dd>kWbVj0v zA2#B=)^^(@9Ru-F6|2fmnx`ebCm!uRn(I6ym-gck7pRXKfVYySY)H|>fCQ={21YAB zrSO}dK7Ar1AAoTuQ3F&#Vb|s9>gt--CY`&|MZ`kGvS16bv+#Uxx2R1zL>h7kB==xQ z0!%iAuR_9QIIg@G4Kgt@swL`LSa%htwf6QV0P2|`A5!QRXc|MO8&SLL;JZHx-wpB+ zKp+Zb0Q@90{F@Kl0@2Cgg^9<&2wPtX9Cql|Z+Us0hN-{u8`QV6WwYuVOU|}=^XAPc z(>bu~vt}eG7a}(e0wRV-TLGl2U;+a$yR2KNFqjpSJMboN%AX*kbZgD8V#ec+7V+c~ zhqpC*A=TrqQHVu!+b*F1qOu53z!kpC1615MLLX9J%Vu5Z51G zXmi(|5yJZ8oJOI+D>Tmso7;tOIB{tLy>T=`o5;P;6CQUunE7ITwbw67HIy$VbCCE{ zyR}sX;N~D~8VVGZqYVQ-ROpU``w}F&h)cn&H=9Msv$nQ&K?5uKun_1^Fdp)f>_o^o z^v23nvI?NWO+h8Pr3J0<@P67&!}Q4w{Lh^}~I$ZDB{n~=Nq z@LsN1^+Z8M#WImV2PJ?7bFt5?>R>Rc(71CJ!B(6@To9e)=+GccbPU6mk)%FXI2?cL zCbU26A&}TAh%#j1{?X{^B4{&8G=Kik*w2qc5)&D17ch|h3~~9Km{X=CaB)N9nfWF z@b=Cjgo6?mGCtP*!tMP0eD~2WH;#n(SCd%|5PPP`9jpzYBQz{6E!BKSh*J;t6ViOw zSqS?pfe;bUOgVCG*@lxDB%|ks&gwZYmXR68LhDa27-+C73?{(z0tjp?mjQpvAUgt= zoEiAGSfz+ttTf2lXJ!o1Fr#Ifp&!7>=6?u8SEn&KOt7{l#m`2TO`@y zhK2*hbihCcwh3atesT%8WY|_u(GHKG^Qr<<7++uC=kyWA5pPBkB!!19)~V1$QDM5! z$r105e`7utogj6F>K8J+4CcTDEoO0SuPP`?BpX>U018P*jh$sXU8tY>PUB;w+xKVz z)@n8ufjXFRGj-R60eM7@9{{A@a(dIR;nC0gQQgsy6DUmTqBXmJLHysTChQ1~UZ(gek3b0^EpqrZ^cw>tOlutud%_V@Nne zk$3`obrZb-!!D;lS5{?M5o4)5F=)bsvDODIO2UE~6qU1d3osbe^j1U?M48<$hcc%< zKbt`5TKfC%yT?~?1O){_1qvmm?snmeU{Qc{2Vv7RL2nDbE*ID|0xQW`+k(WSL)2)(cda%Vn3bBmb`@j5~oDigH@pFQ2SLkV)7HY7{m5x0`N zJ8obmD-jTrMi5v218Y(9+K=YnauEHIVh|{ zn)>YFfVpNcv@IdFY(8QL{RuKfCEMOYJ70^1<@?rkXzr2b0VVlS%`p@zS1ykE=Tip{ z9=y^8^KJjUgJh5>0WIPM=)r_>7-EBNVmCeI2B3bSEV$A9)S>{4P!P@vtpODcAGRQ* zE)~WsG`G9pUw53An$TN=I)#*j9gqdiIBVv_@7DT8XIJdow~xRNAX(m(SGXR) z%mx`b5R)<;*4JV%O8Ty-=*BjUr3S^+qr@kH;0-px5Y9y1_}JK#5$Q55df{|V0JvMb zU_Q&X&2!?YV`==E=N00XFg=DBX=jOJl$-X1=#oe+sdW>?AHt!RdGV#;z|01M6Y83) z;F3r>2MibFk`4qcCWTZkf{fErFWBvd_{|jGx%2J+=g+lV&p|$Ax8e`uQGFFY%6MI& zjnQ!HAcR|B$*g^NG;t2xe&^5zA8q{!theJx%k`JQli?@^bq2RFP9bvJ&CShXFSx*& zuelyKn1m+-{0JB2*LVhKR}d9S=vz4{@ThWYABF86YyLqz5eTM7clKt7?_1|C%lW*2 zy+c)Egt zn$S<6>lFp%L8*ZCf%vYzp@QkOj)2iM{ruu`p%Yqf0(98(_8@N$4xr-FAkbsdtv?}f z*>|!O9oJ^r`4={X6(e}Ux(SCqwFkgGjPRI1VfX>U*@O7ZHINYk6NI*2A*GSnvA`%1 zN(AR(PeNNspgK89@p>3kIhaL;PH^yyx=hI0_5TwRD@jO1@BPx&76lZoVcr0q2QxUu z?GLECYf*1ikZ;$OAP-ca!{4Ghcb5?Rf36<7^AVNDhlv(|t6r>k63j>kDon@T;|L2k zFe=0n3G{V7aKl3K)=2pWFZ_BL&$bUheN2m&wa8&d8nE`7poX&CC$k`P5E^f&yG~fM zY!EY}sJi~<52Re|aYA)4I8Ss6j?F(auFt;Yz?FYi&JdLlHs$-AxEA9GMzZA6a=`@= z_b0}2PggMGxBIg%;>T1aFsXU2?XMSLiFqN@2&9QG$I>^A+fz_6)rP#ZrOXldS>R^l zAN(@7vut~^dbSsKy!{DG-i(dB2{4{EhBuZ+SN?#wiH%Wg=(@Ac2T$8My6p3Q07SN# zk^4(+rST~DT8y$^7fS3Y#R8agMa+I6S))>V@a(Ur^gan}2hSdhqoCQCapw6YA~$e$ zfUYD6>Yk{XnzqpF{J=qpKW911??J!Ay@?F!19s(s87xe^Y{U;sjGG8b%dQKvk0i@u zzV_Eoe-F~guzWIOfXG3`taYZ$eu9eC_h0_{W$;?-jGMdQWhZ=5_a0(KLuV;RGIb>K z;e}#qLACtsxxTaVpZu{Hu6Mb&)=IIoDJda&#z+>ax^{iHWaWU4fl@ za7JT*?I|+@Nc0mh3i!6c$uRXOp&a1=$Te&AA_KKXW5nZvuh026XT#jnmo(VUZBHC3 zY(F41#{wyO(M{^pVu;gE(9G7>)+b{<_pcxGOe>meEq;CD|J)5Yb4xb+zW;mI=={%% z{Qq=6(SNgr{m*qvyLm8Shv{`#RsW8Ux-$JcFckzxV!g>2Qkgv!>*f+=u)MJ0hmTNF zETmRU6oEWc1@9A(ot+&Y1wcl=IN2MH%>2@`tpaJvTx#LOV#2xtBhrJ>e~_;`+vj@S zvfME{KAO$=be%I>lJe=(<(=a%AN=R27dDB5^4Az$zV`UD?yA=}{N}Rd3&EuS zYwoc6Zx$o}?H|WTTk%&svb@kI1Su*jamB*#0($&iLxVQ#1oU9G;LGUxh+maBP$2-2 zW*+u^KJdTz)0tOrJT!-NZtmc&K1975AJrf7_48S3*wy0dN?8Z^hsn)0#F2*!hMGRO zSf3((X|RU5aA1>(>-eaN!*$8}H=5i=bSPAe%EQ@UQ2FyrmMsNMH3RQ%hk-euAs%C9 zR~HQR z1~7f30ZrT!DHk=;5x^x8N=!u1Ncqrb$3OUfI1@-tP&a2MW$DtTDRq)4`@}usHqz)| zIRC6Q@f4GZAmG`1xXy@-q9^A3xK+{$CJSG>9CICqr>#90VQ@yDe&?l7ffT?sMgX4k zgWZFydl&sG%onF&NMabnJSYvuT^n^=1uu@9#5Dc{vyA7g0Ay;>5jvBZglXm&r~woz zrY7vj6an0I+?7l|C)`s2iunl{4}u(ATP^u^g0u3R4ZO$aKL=FvB8e zU$6i)D}#y_X=uYZ+ZbGmLjZexFtvj>RDv*^Zcw}iqeyS7(@OX&H4;NUSMVBP1%XQd7T+q^^H3bB|zbZO?DU)X&3#~#HpB~PY0z_-X!(IBRQ zG{Ce`1#X6*l7#^&t;??#cM4n~r_{>(fc%O8e;628V~s_cgv+cD^omCU{4kohpog@U z$9y)*$~?aydEA!}?kn;0JC-4P;b&FA+b9KjL?mV;H`{R5(7#%nLi-o{M*uigNUDK1_rw zXSN444^_M_(-o8o`P9iC)9%bnGzjo1%Bqp6NHVBh_!Cr$c`oP7OMT~r*XOjW&W80+Bw`<5JJAhrLQ+FKV?1xHBN3cKz{_&qwJ6b`W_Nah(lFK z?PRVJ0=sIY!Y16U6je9wI>kiHUdwCzQ8R{0kLp8pf!JEG;zw6;W1LI{qCh}*;VYfZWeEz(+sHkXW z098rkBt+c%AWVo^Ps_p-l-s+Y{i%fe#fg=3=2%W>QM6Q%;Dm_Tk*M>7m@wU`7%bTd zHY6v0g}*+=ATj4~1Fp|7^usi8PM{|O;aDNpsF9IfQ0(P+oOm{vQ}O1<(FChu5Sq+e zQ~YT9E4E6XK`TYB7E(p3!@Vu15YaVQN9D==WWWa=n8Iv$lbe+!)joZOhd5hFgxCha ziF1LQhb84LW)ouFF^v?34*w}kjqUf)&zz!uCDAUP?4L;wxYueFM3%7 zG4maqQ!ob;R*}d9F}PC?Cr9~~wADdJ8|tF_D_cLhAS8BzP??}h}Tw!w(Ye+gc=UpZ$uVbC61=(#m@)zEg`Ha zVO~K=QWw4hA10APAz&A8*?}J*&I=eF$i-?o7pfJ3tRZ0XsOPsY}#itZP*x|`RpW2QiKn(|Z z5Q8fW$lY3GECI9;yDAlYizs|^u#9h7e`go$x@v@Gzwn-w`55ZT=|77>D5H1A(D*&l z=i{~#N=1&-z&hN*13z7Hr|5!(3p-%cXJ7{BLz585mDOT$USJ)F_^FJW;y8)p@$v3+ zhhv*u5D)LH|FI7Hl{We$KVo#b3#sEjn)xR=lZvc97q zNNy^`7esaxs*C1i983l@jFF3ECdL5!|AcU%-K9ht2rV}G&0A^r6A3gm(=auX!yiY<~FNICbj zvt4q;Fd6oM#4;X@QUPPKh6OZS*-6xpHC%F>#x7me0{8?CwcMA; z)eabFCaea7jg(5PfIJl^LjXfH_jjH?OdLCi@pbUOw2TZuyMYzGNl8J$XU`O3^j@c& zKHMnf5GG@>ibUro*I?ikiMJYsFBV8DhHcsehws*g1uj5GL&ZEHZ@++e2EpW4V1>>E=oBKVW9; zL(7&dTjsqXi@YzCmNjX_gGB763tmD%AtO5vCk1wQ#G z)5t6xR$DBA@TQ&5VG-AX0{s!>TpU0`hwr|onL#`-tPp<5qw_lRAn29BY5MNg#M=`` z#RXV-PBb||@l3<-!b~3_hkyhp(gra(!IGM0`u;MK4B1LRmT= z6e4bY7@j5JhGD2ZkZ_0u1HIy`^pZ@?7KoX@&0n;!4PfOoN*%*&lz3n;xgp}hLmcZk zd_GC1AyuM|?gbgiRTBIsg%oHcz0bTh+aZzoDjw)MQ;51>Ah0{0+*Gt< zIT?c{dUsu$$Mkr}%mtT{`_JvBjCDkXR6%>AH5}N#pCq#xyhmLwUAV%BlwPo<3Lm+9 z#iFQ#0JHq05Zw+0Ryh$YStkzMnGK@Ud`JstB?{-RzEjS0CW(YEojH-%kOm*`e0ItU;FpO})p^sCl@bY@t!EZ7+c z6s*y8DT#H$HI}c%Z+m%_V>9DA+Yo{zIb5{IvQ1yOdZe6X+X#9=L?JA!t~@M zjO}F91YKLB89ZGsIeD#omgKC73#vrlL9IJG#D?|sDq_9BbRBylpC-ZhX~yLuF4wsLb81kP z3D6=pvx?FE?uFwzMk1_`V8^1{wSL${DTIDW_)lQvNyv$5w*y8%oJ(dy4*a!^<0f(5ov$|CcZ!Hv( z)PIJX98a!8ibHLwg1{w;=F;)qGTXgj3=8sVMpG2CSP=M4cio01i%ebGxr31Nm(Y1X~ z;8iURklFks9Nzc0MMXSK8r8BLZC~2kHwwuHXl&fon~T8; zjfs_dIgaJxrC7U^*0mXHg3`_W#s_Aa))xbB{?p6+p@&90q(} z=}e2bpU4%9a3FHh=Hc`k-Kt--c(Hb)kcfy1xwZf+0VUSl{{2g=L_~WoW*+u4`i9Z( z5k;FISPHo{^HS@F7^owi$RgybMBJMZrXs_$AGv?-5)xQk7;vHkvcUAfn38^?5u7Q1B)jnmh#9I%z%8v8$k`nXa^a5(!!;SK~L6BrCUNO$QdBFfgD;$ zfAeO=Ax-w1a=t_hg?j|E-}eSH=hfl}Gt=wPo#e<_cwKvmDEDPzj;|fWLyEH?0CTaf z%caBkkE-A+oM%qCVF|W@jhx(WOKf7uwZ)lJU=mD`rqq0acJIcq$bWiw9KZ6i{3ed8 zhY5s-#N}pr^(=MWrcEY)(*T~IksAhZbu2l}BvTOXrUFpbLQ0j~F$uFFxhLDlj0{r( zM17!pj14#;6JcW8YA7W0T{~u8 z>ESW6O-YO9WG>GQM@SLD^fOWQ2DGuRk`=|w8$mZ=ucJPT&DP4rH}>(rGblrYEh1>0xjtC z)SYn#{c%q_4d~bmJu)?%z^#%dGDO7SB z0~)KH006jnwgKw-8ro8Bsi`KZFOZyC`p8B06&HGDHc68IKU~(2FE_S!!eR7)({1W! z8}fu|O40YBAqF-b_W+%O(`J>AkB`D1na)~p8-g>7TlqWgQ2!`?n?22|3n!I<84{lX za%Th%B4v@a`KA=|6p*t7Wn^H7JYd%uPUaEG94&wn8E3U?|M;hB*%+dMa7%Jj3EXl| z%B~CX6O)Ulae4G*&JMs+!fhZL*CjqdBFGs)Fw}`$H1fqV60dKT>(3OlfaC4@x-@;=?2ean;CkgYX5!u@IChwlJ9oBg<`#-DKY#v1=goCGhN!zTwNyx6Pja3kuiA}{R=?=bBxEiJ7!!eHNlDz(Ml?59ZiK7CZ+IUVtsmR#qX>ezpt+{30woOnZl2OJVqFNecL1vc}OIIfb9t+U4!ABEPJ=ubc4YNMsi zpHpiv_l&EZWT|3SO>r1>jdYy3j97ag3>fmbD^IapoLCoe0S7MZ*9^e5Xy)0(r&ru<({o*mP_`rfyEeC1d9~!A$TXJ3zS+X@Vc6e2Rb=2--@eD zp7ff{1nRc<|B#RrLRQ3{1i?ofi8*|F)wQ@++kLEqhxw_wxw$is{Kr7BYHKalf)-Si z{`p3HU4I@;bl&McaEuMU{13 zqqMfU1#Jl`$pA_a5CstsFrt8DiVUS7smM8lnU<&}L9*nWb1ZEHB&v7v-e(W%{Av-knW!1`i!p}EleqztZSp$fs zu&MMgzvf`GA%7qggy3v3fHQ5P)MaOJHRCNO-FE(lXis1`jJ&oW)hvvz{Fw|`gFcLo z*prwpdjZZ3r#>%82O%tr;UVinq*rD`a&Xi^zV=-wx?qPQs~$VGzd>x@IamxwQZ!)J z5T=QS$j;{Kljn5wb=$P9Q2ZDICkHM2<05aOmq3{a(ESG{a6Co|8Fv~d32(?U065=j zhRg@d_7edM9tSX^<@q1`eE}Z0zYbZpEYc!DaRCzTqtpP_WjMn@gaq{Hi{KHzc^qWG zlLG_-)W@f?ec&rV{+$BV%GnMeif_lPuP-kwpg9XxkWxZaXF&!IdUezvr2hh#5%T3C z@a9~CEuIKjH*y@?F%;Q^hCeczMINQiLjZuA6=M5G&E74CsR1XpJE)-Hs7pdk2FT+Sf%v5WOjOkC0&fBI=HU4XOyj>z zd+vhAC-O%^1l&b?ST#+sOS3_%fK;ZSyMgE}0~{`dj|0Ly0r3PugQ&xR^cCZ35I*mh z={CqA%`qsn>^j&KljGqKRuy^+`hVfARc-Y3fPE(n`FI*Y$Vsr?t3O`$L1-apVO) zK>JBX{!wtag9&3a6Du%^VDE6Vy=T4z*N1#yFqk4pvKNs%KO8>YaAq-0z+r%#L_UTL zL4P2boF8;ivvM-@a=Lm>Sjx9tIckTh0l5%O0;LO*VsstqkmPxDKIL4Ow9 z!1EL4GNzE}RJquwLpst199CYPYD&Pm!Os{Lmmq%IM3I>hP!bGx%^A^AXa$WXaE3sGaG?DY0LLWikHDX^Kcib& zz?={5J9xf0AmJjVD0Fsq+BEPf@YVd1k_%g+|JnG_epL5Et7wjg2-WJVba8~JsE)_OjULitf-HJ zZZ`na%~1YD!9HLC{s?3UX!inT3kF4vJUh@0JuEFFQ`P84A9yr?x(+XcJ;Em0FFIrl z0u@$(Ndr6F*EXuEWQ0fpYrG;c7YO6&i5hUu%kf@@IERMiJOD1M^9Us7{%POe15gCd zTn6^I)1@I1sH)>3_g6GJ*m>jf#d?6vx%l#co z_vIw-lrTHj1^@H^*I38@ytc{CnU@tRHI)c#Cz%_z z+czf*{{v>=at-Cc=v5|f6 zgAc={?$H{Gp2?@3`@O;ntF(e$!Qj0%E&lK_b_fdQDc0RzJQuomYTDJ?>ooX1qooP> z%iY30I;7xO436&ijp5>a_#Z>!_dmUN*eiy9|HJ>sz5Am1>9JVv4HbUSkD=Fdnv=gR z?Z^b*e-(n5RnJ8{2Y`}d35)PAW74ec$z zNwvTJy4mCK`*?g*a+EhH!6~OgAlvYjXtAv1l4@y$(;M*#+54Z!x3caWvPu3>^-Rt! z?_m|Z&Hw8KB8P+7I7?pQLRj}o2V2UT&fVhmdd^hb_UOYk?0SE0*zV~=_FV7v9wjl< z%(5Z=5AD&Z#U>4b)fIuh68i`6E7fZGzKQLINz=k4BcTVo_kyfpZ{!x4pnS0rFBO+s zU3<$H%X3Uj!`q!jv*Rp@!k$A7GqEeTpEur}eckUUdt&0zsR#FE6{aIQ*AudwsV@9(wl&_GRUsAFJ{%bi=f3By!^cnl&TS55$svv4fTk+XX{!j0kD zHD~B->Jl#O{;K*x_piHudv0h1FgT3alYt5WX7$LPTd{*CFmwqGvg5#u7IKr(u5HjU zipBy}GjP*=_t(tM?71#{DK8P2JXjveA2@5S^KYWzq8cHXp79btq%ZPgyFV;rCG(rw z`(scl^dFhtz<4Jg%ava|f&z~b2n#*v27Nd9qEPARR_-o_+kZs<&i}rY3$6SN7nhJM z)DLgU{&i7YLacfAR7DW|PUQnf7Vq8%+&(EEwbeyDzhb0eU_-h4?a4Y$ z?3M78+&6lGL^3YTva@64>1f`Q9+;**VNQREm+8xn1kzV zFbfqxRY0y;zagl`tD zUZGo3FOiSlW25xC&7~p1M3g$680gi=L(YqIq>`}e;66x=OhLi%VHV2z04AMh&&C17R)inp`kn`! zbnR)x*3W>H5e<_Ib>Xrp(txnTQz8B7>d$InyPfZB*0;B4KL3-To|O^iVw(F_WbQ?m z?32vw>}++PP;RhyLbD*C_{HdszDcSrFE5`!Sq>!W$bmO*^s$&2xA&=2r>JOX;vhOt zLjOb&w#K~yB+CM>WkLNyISn=*yuFYQ&-}D@UQZ4%$%Hn3m7ys|P~!N{I)U{!YQ={+ zFRLKI;!7u}g++@Hyr!7^iHVh$7Xy*qYE8owYB9j|1DI5U2=a0F_378XknF zBQGQ4;YZP+Hc>&yjgX8F-t>w``Cua-2ZtoGW!Pc)A>sq#)RmA%gQZ4-c?|F$(qQ|c z0XqPW#eqTqIdVa^dGy#Z%hN3cSJ$Z1MvxJsS_1MN>ng`kwZGa4B0WVr=hyR^hriF9 z+vm-u8#$fl=jjD(^p#ROq(g5;?-%SanCNBXjna}7T3(}k=T?gO@cx2w8$W^);>E!GjTE7D1!$}}CKvQ4)g^mdch%r$B z=mSc@dinB0*Nr*1830>6`k_@Wgi2(lDqKz?e45f?!Or3Md@8tY`u7=^pZ~g3zhur1 zhJj{FjIgye8%I@4edxF}(M2&jo|6_Z5xBD5SeCXG{bHpw`F-P6dY{geIiD2;3$2i@ zCfR@bz;v#F`empg)KQ3l*#t#LCcHvQD*?AygjOk5=CFT2NC=&FObYDEQI*3ssQ~1k zPI?;?7l+i}B!@%bNksU3;tJP|;JS!UhLHIO&{@tTK0!)y_<_*+-P77kc<)#=GjF+6 znwo!I7S|r(!(jLHWpD2SXGlTD?4y#ES~6mHvAU(p*6sbX&Rx^1EYYy&OC^@gZEZV1 z0G$Zj1T5a__iS6(xdp?@P*(t;eghkbdO~7)9)-z@V=fKQ&dr1+gUsof#h~t&@>&3T z6Zp^h&#tt?DnigTJ{r<*c3?Wfs<&U>hI5auVmAt^ANik)H4Lcr4aiG_^X%t?V_%u7 zdOENx!V7PVOlDLGR{n4oANfHe7Kl!kiKalAe3sSaxeSXs1L&@0993~aBgcT;h>xzVeOwnR$k$K`z-J3!0nDKqm}G$Q1D1q$ zANAIQyerZ#gzsgUW5b^|53{_6LwnR}0BRys zW$H8DvTIB*A2Z=nX{_f^tTCTA9JE+~qp#1RhC_5+OYOlHl-ij+M0_;pNRc~p$8_k< zr#_E-_j@cXnYyy7cqCI*OAi-zsJM93H`Fhp)|1EHj`V5sG|My4M$NYoz;e36{mHv` z@1(ru7HS5f+!VLDw`GK+6=+sjIK^t9`b~nMZO|WIw=U0HCJ0xOla7DFC;E zp1bisUykzk$mRU>&w~*bE5DkXUJ3es{K%2iW>dv|W+Fkr#V&8(O9{EdZ)^TOecG;T zQ(j&^^sIJ-$ zGr=+fCT!O2uw5;ZI(9D-e0igitcXeKQP-1VXd7rLwZhdhejHE|mn+7#o9fs&QObb` zjWjtsF=0R_sP>ml0Ww1cBE%#_VAk}z=}A9(2h%`Lt6{bY6b}VOMSmt}1xi6qaYOLh zbnlm!P!y1f-M_OdGho@r@;iGlGtcW?3b~?~zJ%$F`{ z{*ngNGL}~D?f4=$N(xWtCz$z~Ok1VW4rmDKWdlZDVsDBe%c?AWICakWmIxR^*dUVx$p(ycF zmYlr&QsX@e){7VK;S~TfLu_gI0~?qx&v2V?2O?Z50K{nYmAI5tJD|YUkU2}><>cg` z(SBj)fQ5FpdEIV%7*?j%@2kE0qoQ;rQEO=}h*Z|oU10tW|F@Q}?}?$t%Cee=UTOo$ z{kER61>!li-kx`_K;IR+*}%a7qxIO|KL(hx1KLPTVmE>%kzo$vBKAvI+=bLU2SzW7 z=_&17&pSO0>-2`A{1l+!FG{ZOYz(u_h~H`Q=lAi7aRs6TQZVERl=Sw>2~|hLCB=O6 zdrq@zDnOz``4Cr0XKCr_HEx?u4gx}gl({JyMT(F$f!r|=#UYqE zD$(4MTT#ddQ}G&ug6IwE1fQ1a0uGlyOP zP^nIyl)cJTbp-VCeBK}rNC1@mY`G<*x&grGH|W~Af47_N?ZxSni{Da~7HdQ6>bzgv zoravS_}NKhioOVcJxv70-FF{RP+Nyw3zmCy2BM}fAGrQ_d z1_Xc>2uVj>eg*Z1olp6Ji`siihpVfqWNmExySQckiHVH`YYI(!>emRr2UZJ4O3IrF z%v7oo-A=swE^AsUXu$UhD)2#p*KoZs5|jqX3fONHIq(TL)~7&;!( zr{JI+MluhTvcdVZ)ZjNz){nA>#{HC@P{l{`#+p(2w-g;i6Hp$JmB8JOR zKW+g5q8*MTF2~|FTpJ?9LAydd=P?v!NT`gupQv{Ybq8uUppS#1&;Ue$V8EJC_6dpM zAw$2CzJi7jK$H&@gPl9N?m(yc6;3;Y$Nc0gVhyOJPpSh0cQ^ zEJX4#pvtg zsruefHfywLr30{7qnkB>n1rIDZ&4Qt&B7>H&gn98fJCZ|;JUrfS(@780pQ#v@R

cr}A&GUCkHOIw0qH^uwbw!QY5Zce=o2(=CLl~H>FUOsdvjdaxwzYRKL1y()~ypQ zArr$SwB0Xx>#dmaw+xfjqb_mPXU=4v-5RCKKX-O%OQs37~O(a27;nlckoUek3gztdRoCc@j8!jnKrV zWMm9C6an3GpLr!c`Zd(4cXQO37E(7TEv5w25o$w3d}R_t0}dJ^$}qvZ+xG|dzxbAa z%FVV0mow4S;i_o%$RMu5&d#BxuT0(#$HOBiY=J5#grO$Z>g1?mfUc&V1`S%tgoFer zV1d%J@nDPu1rcyJnZ*(niPLNr2#1H%5qUA7pM+mQ$tfxnL35n|h&8EttxGTb%nDeu zPU;Ay!LCHUi97cirf4853|RY+NB?-=jO$Dc?n*m;Kz}SYMj+kNxwMjWi29?e%se(!EB9CD%tX5gUMt*&AI+Y_te$3de0o7qOdMJ@mFdZq>OB8%gyY? z8i22h^Y=mh+svl1_OFOGga>f5Lrv+lo6?#q`E<}<2ZcZulr&CNrS+w=NrAKKDk>^# zkVAUf+%Zi!LUw?e^}J}7tAmoq)~Xb%*j&IgHdjls@7DO!9HLEda(hzV;DsN_kOi+< z^gfDm?unn9yfGfJuV?_zKQQG+@sO7Is)+A2PO!-<5?;v zbqO~pKcjyf#kguT(J_lFwH|k#ZSr!S`QR4Nz`UNOaqHekVdo0{4qR*MZI25YO(Ram z`))tAarWET&2cI3ww0+RI@ozQd9kX*MvD8V>f2b6;lcdmGdzAVIhdRVEKnUACbI8& z$7{A5$Il<<(W`coqIxOTJoiuE+FISA#laFHmG$)VE}!R9@B2(_jcQy1uk-5>@qsiU zY5_I;Kg91p6dfOH2y6NDw9aa}v0h@C@S~xmf&pa8+26bIn{E=DV3yxCW=Jk+qSf90(NjYdt zMJJ~^!=MQWk-~7(W_Rh+59FJ!J%SU{H|)6jd!!nQw=1m+-dIaXW`hTR*9hpN1koQMo7WkuU=CWHi6VdV#MtC) zE zK6)j*2%}9<;e_HqR|H>EO*G8X_7Oy|+<~{)JG=QUTyr{1nCt^r2vJA4zigPvHMZwu zGWoD$C#LuqAUfnSRcU4DcWTd0lutvy6lqozK;7xQ{q>ZtNIP#p zsGoh)4R`OiDTNbNd9ev2$M?5RzJ1{^8G0tMV?#7cJK~>o(BCx9Unb~vm6&jwRiJr$y zvJ`h6pX{{EcXHZa>wQe!wz;_(`3WPOt{be?01S)u^!!T*Ge2Y2v8#%44CBdV>$&3ic8C_`V*ZN6TrDMN-Ew;ZuRB^@_@zXlUrqMA%j?Dv4XXoaCALTwn4q-Y&sn zeu=^#%3HaQr~E=rCw;t3J^?JKwunC3rTR~5JX=LMN6A@@ZJd4-KFv_3>qacumRK(| zvJ5)3!Sobl^UT7dz898_xx0-E|MGW>ThXb*#F-!RW6_Xp`fw5-O+Q5wxM@cwGmg?7 zlNp8zt|P>Og8HZHk06`$_?^R&(X903Y5UXli5iqi_&JZ-ANv(eEcqb$^}TLQZ7YFy=EL zusO}ozc%T$?)M?=g4krQ<7*rrZg$bxdwL)?P^6TjiZiJ#HiCkX^8CqI?u6t&f+@Tn zu?E}Ayiou8z$N?n1x9?)Vy03~_Qkat##9(P#y?hG@~8)qX-ZmuJ~nV70Uv!m&RbAD z{8#Yl$EmaXPOwjLy;s-j0JA!6b+6z7}#B6AE?!EpY5WM#h4Te^ztoHBEbf@0q#`nH0VJ~qw7;ue05 zt~^tbXOzM>40UVIJomV#bz6vI_O}S2RS-!=l zWO#Ju=VHi1ikU{w*@5>~%MH~;iV_1S-HfKoiw(7=*|t4wq22;)wzd=i_9f=_nJB?8 zaSF^%#9ve;ttb#X^Y6Q@7-Qt_dPZj@#jYKC6K~-qn%I|L(rr!4MtJ@tZei>C5G}Dg z`#xTw(8+pSCp>`>kRnbMDI{Q;gA**1<^<+lS~%MP0*x6-v?dbBM92 zW%#d!X5~a?xonJ!m508=Gri#}x!um9{d0^_|kKfsA{CyqiUQi?b_@ zPkl-*6P{~H`CtWB9vYLjUESWkR%}dJ@u}Tlrg@^=c5(GRp#mMY` zgKZS2IA38>Opb{)uf=3#IPL__=L zZ*p&WqKD#DGsZIC>B3wQ>feTwzioB|liKy@xSF+mP5u7)rJN&VO0@s2_$!dnz#2(x z$$!Pux>cw(rKeYD`pL;r&-F@s3C8{IM&sKi*%P zUtVg+!Rm?z`ET-8=_|#na=Dg2Ih1Qy7il26KKj*1J@fPxW>!rgkhoLVhs94P*B(o) zydTan+Fd-!CAm5Opv|~hHm`rXkWC>ept4A!Wu9N}8^sK-skkWb@tV1dmh{2CCz7IH9~Xdy zJu0>N2ymKWhyLY!d04|0BbJnmjy7*DLGTU0{FQ4pH0P2)A*44JK@wN z2|d4BaNva9NDGS_AUJDm&NwLYQ8;uaTqsX595ozr zFTh9Rb=VTN|Do{>3W=A-(x=uXZCQ*vjt*jXOrxOh0Sr*)w~l5_>KLcT;0_*8TX3lO5K*vu01Pt|C*aj>aiR6dOjwU z^Pq>9a75a`*_af=t}M4~@d5}IBH4kSJp+bVQitnALQ4>afUXmhC2O5boX zHo@jKeiAvya{r#FSBpC#ZnBahoaZP*gSEiqKai+3b(ddXguq}uV2Mpqe04)J{TqRm7pCBg$5FPACXzLnmNvpn-u0bLia z2~4JI7UzX-g>7L{MfgS((=(knF9^9x({6%GXF`H&Hf>*YD_IG*L7CP7O> zJpho8=)#_NFnD&zSb3{bM@Il)o*@R}2Se@-(zRx}#m#je!%+$-HqbW?n3)=H?M$xy z3oY8sjV{XozbqD&n^S7eo1^;?*v>WC`A5mzeXf*M$4Q-m*_9({Qnu;mpk@Mgr6B0Rz2(=yqNtAz0KO-lH)Ui=PdUb16BuO1SbvU%g`jbrMKin>dNtbzoCbzxGyhRokuQPyk2C!;iypYdm1)bNL`Fd4HhFsgdkMi^2D z%=SiNUf6V0^y=x$x&#OREz-x*(}EW|?xrt<^4lwz-s+c!96dkw%pX6n8>%cjk?;Aj zV~06erMa;5{Y4J~&lmtUTR8j)7H8@}< zJf?w$MQCX{*9GOwv!^uqQU}%-#|o&8hcQ+cLp2hJC3*)Zzoafz@ME{n^);?&^5WVg zBSw;->Xe_$h|4x?xwd(I(~!Qm{ ziI3l$wXj@}nVvY+ojorNXu@@7=5J@UU4pMFS_fOTLiEEp)rtXX&WmfgM}&MnJ@M!K z{G1Z}qp@1iZA({@GDID^8KtZ{w2GSShp8F^kKGOfRvEQAQgX2NvMH|CD{F$9&LN{6rk52jK zr){lc-rS^WK{!j;MR|$nlnm_^b!= zQvvmI5XAYN8{oh?Q;A~wZJF;>PX8)(K+-_xJ~^4+nW$6$2vP$_X^N|gPkigQE+|wo zl(o*ky~C!I;NROPMCo{Ga(cxxfpMbs-M#K2tIzB^@>ebo1(5+Q?F`|A(Di^(c=N z@ajPaUC<5l;)7mJBaxd#n<5v|{DYX7OGzqjkPkxt$xH=BKu8m=cJCK0j_r~7+n~NFix19p0hzh`UI8;TPZmJ$v>u2J#rEueJo7_-(Z?WYr!AqMhTXoHE z-p_i}$5fE=B_^pw-KSJo7gcqNH6U2BA|FI;gp-z@E4n?zQ`61rR6`c5A>wyTav^3xw`?

2cgs`d6WZMjrD zBlDj2g!*0S;<=>6R@`7?nnG7KTU%acw+(G%gJ#KkrjlT>`f>=JnA{uO(&9H-LZb4! zO!AIb?rw>yp`(+nQ=Cv_V?%GbJZkRxw+dV8pyK1RUGCX&?1wUJO3_qsm$ ztji@yKAOcEMS00G4yu{=!lBbe>?u`cEHw+>3Y_zT(z8`hdCR2*tl@q*Sj;SIv`z9f zJ_EKMJ> zq3aG#k2fJw_%DtA31JEdL5%;bvQ{-`h^P>;J(YpGM@^=BP-J zP4laeRF1LXqp)WA^P{icTT74p%&coU=H=>fGxjso9}s3@(a-i=m*0BueZNa)_}@bf zr^ld3&8{&bOeIIZR*w22jlXrPgm*MF5QVIomKr6B()ki-6P1kG?|&sR36OyX+?3SD zdCbzcV1giJpNTqm$I}D%Z;DGxGy(9A9D|U176{&SAMSJibB{x%uCCl>=H*}z32&cc zGxwL5b9~awjGXvIlcR0lXGW|{^RAtBx%ZEz{f~nwSOE#)(mxO+LDDpbJQW_>{OX;d zeBD>=Qa&g)y}Tz*OSDva#C90sk%>{-WwCpVE)xBvTT)&`-bFD#@+95tfZEOT*cDEC zMadKsI;$!dsA1&RXYSG0Ls~5r>M-D3l>w`|(Ee8$`4}8@)=SuD=YLL&B!%UPm{+;D z{PA_tw!@NqOmXYucAMLb^O_}wqG4bek1^!KcsZmfKxxBk(sK`@mjG|7b--70b_-)H zfd#vAoSZeKvO>}vhe@-(gKU8R$>r(v5GVh<{j%3TySx43}lQ}z9C@WDwl_xlf zHT1X1Rgx2VQh7~U?im-67bJG2zRB^3V`abIr*AuH9RO*xZI$`VOxRho28oVarYq-t zG&9F#@u-n}yyW4{kRI7D56a%CJhqAk^f?-!V`S|E1Zz2H60|{U8(9-TL$erYC@Ijs z2%@KAQP}I5ems9zDT<(8(xGauOY&B8+%-^74e}#$XF#yEY4#{<+u~H>7?a|U< z{#G_i@vEQYk6ux1PMBK?>N^MA5`}?-9EG^8S?p7yuH{VHg{i#pL`IP{Q=MD@ zTMw+<)lm1S&%2qtYRyewI1q6L_?N4l^7wh|eWetsg&I{N4iGPm_N=bnUQB}MJz{6N zGM>_MQn|EZ<@9|1#y)qA_7XK4hR+iNUI9$TYU@7DEqHpxEI};f0mPn+cKy8Z-tEj8 zNA5T&x@K5~oLvnjdF;%r!X1VXY1%d$TrYa${7U&F%bK`vs1rJOd3PxIg>p3)Xh(=s zX|2QG4Ef%s{a86A7e}E3unAIKU zycZOjSs^DK^G;W`P34*u-kienPUcwYGeY77<)9_bXijb~Hi1MFJpNA)JL) zybqU6uY1>EQiDWCf?US5VszXrqB}+>U+Xc(t=v_1|J`C z4}!bq?%`3SKL92q=nw&oiZ66L3aut0p+l7f0(uKe_7Lo$+jm%YR4J_R{jT?LvQAF|ZuQ`gT7NJf{X>vvH?yY~@R!X*< z)EuKqRzIegL?nm`3+n;$;6_wgNHQu9I+`Tcp`WHEun@V~!4(XIFa0hIWD(asLb?9L zOzaIG5fG@7Sc=#gneCHYl-+l)!fc@WYO|3Bl2M4Y%OAKq^X3+Y{jP93zt?RNW)1ke zR64F0K9$e3Iza3zQUb0ZMoTLaUfJM{in{X1L>|~b3Z)xw%F9_)!AXo}9Y zwfW*H$7D`Zou2M>mmZrxcGw^837lDYPFLm&>a_jBBz6I4Mc@Gs=^|m;Aj+oVmRn>W zTER&j!*rR2m;dWi_IB3?41KnNvm@16xsXzLZxBvfz{Btj4CJ)6Z8)|DKX}uqp*ymr zZQz2OK+$}E{*iiP;=~lA!nrfG_`pCDCu!&*4Mvc{E{c>Vj(#6t^S)UFb~VuW14^&$ zO;?Z(qT#79?z9O+cYdE%2dE=}=!WKbB8wIr5Hu0d6*|Z!%TZ*pds_~zoDOFR(Qk7h!4>+r%gN6q$I3}Kq$ zo31|};KZAGoevKCBG2FC3}5v6mt(1ng4y!dUF{^edV8dND9Ae&XDirmtbOCH32@Q; z>0dN9lJa`!=iGq5R|4yijw>SbmAITLA;um-blCUX6AN1pGlt(RxRvbekh5_jT>q@-?regMjZcgKM4J6 zYhpJB`5*u#0oiP0Qa1+tzk}EW8Q#Lzh=GGZ9q0+%g4fLIWI2B`>4A$g?i7DH{?=)EkK$Nn`BSFV;n1t)9Jh0fF+3B60Q)T}v=Q zS@D?R14?G7#>t6>84emwwX8ezJfZxupH61)t|daM0bNQ~Q^sv|B}dh(3$*n8d2QJ% zZJCy?s7S?Uo0I5teCjQfpR!%QC<&k)Uw55gpO$L-1!A#C9Di3ADSRk|!^(UUCSEmN z^tlyzh|x&~DgXNI$LPUC8t5B-{~}f*GRkuR8xmfN;YS4pvKL-W?RRb7WmmydllhFW z3XNd@?>&4n0DmbY-Ju15;0u)HP)=eNyqysm4~3rr0Ce3MPct%fLYQ=-<8aH-E7^Ke z;zDy3rUwe^f`R68Pw=`BF>{pPmWsg?yIm8s{{YwJ%W7vPSmzflT>PS2pH=$x7dsu@ zTG(`&?t@+!Vyl0TH%}ZWu3^rz>@Be`NGN|&C%Duy+z|c|Dxi0Bdgv3{OgiiHO97wJ zPLhP$zRl3)m4S*!=7Q2bLBeYLnXUl&+ix!b!@vN9nMg8;maUSe{R40d_{fCl#lRqL z2jcJ&^tuy3?yFmQ;3!?pHrl5%bO0MH=QxpTykhNwp z&PAd$KedPGFrQ+uW~BIf#&wIf&9tdJqM_e7Y3ZxwE1iZa)^xftn#a%7MZdm=a?imb zv8j~0{BFJzKpUhf*Z$o12Q;Sh?Ax&2r7jw52M8|-(B^jlL7Lx(59SAK|4i4Uom6W8x{7n?w&Qsf?{DLVX z;pGucvYxl3)r0YnXrgcEFjFQ%bwd5${9MBY3uDzKy|1@C>Rrgks5`%fgT{b=Q0k!o zWisgiC3|{tz6CV2Nt59rr^gzXmyTU&bV5k#O&iOV#{CVh>|CZ}O4lq8a0mba2x1n; z`ZmE7IxdDo^ZeUO+8&RF`i8Y%%a<74;Byn<15kBXP)!vlS6caF?I#Q)0mh~w3{@!{ z@)q>uu*%BJM3b}8sA8l8@8W*3Z;v^Iz8{@$f=pRNiECvyd(iBc)Z-QAkf7*TS~ipn zZEt5TV_CKXj7u=!*lx~8UPCJW>Y@7 zbZn9SXZ18duGmeA*&Z!AwJX$qDGEim7wEf-x^AXIMFbUZRmZ9kjdERr%DQl-sxdyg zbNmz#5ki>^8vOkL^tmW@^zczxqA0P>+o4XSew`s?z;6w-nlDfV^g}Ol)Q|cYY$6FnlIrPsXs1()=6YFFYTD&t<)5bq);x->>eU{ z51l`onhF$B2ZCmMF*IoF>kf|H>MgN-Utrvl1(dE$TRi4 zm3bLE6T zDNdpJ?-#|T37N$7Ka~Ej$BqDnTYBs{_Z)8+Ed4vcWta*jeOJWu1p7)CniA8hW-8C+ zdP#{8O(GoIlC(;-=82}jV29336oOX)j1pkT5X3e-v)ne-$&#QN=@O`YV8{-P)kTH^<`V6N-=Dk24@8= z^-$Ug+e(`b-Gty=T2XPIvExPP)UjJp*Hx{yOpkqsleF3-PteJjceZugB9>zjp5u6< zCc!GVMH^HrF`X?Rq*l`U;zP}i@_G=_@}`i@W@Z;jGv_bltpU8}G;ldm0j6>V35;&W zM}tkFRr{o^@k&nRv+B4!e~*k5I-O-6ozW4IrTK8rioz zsBFdk!oIkzDvsB6*^ZT82@;wRiu2GfE_S9;xYeqkruxgHh1~>E$}jtD=4ogH@CI_G zjlm>(TI#VW`WsW74Ntwph3!?LJ1XV1IWffTE~3XAw=6XCy`#jOh{8noH((%AOmP&0;OTbLa!dDxg(thu?nzoP2c=M5|E_Cg~ z=;m_QgUVU}j3u|XyqtuvNwf#gRyyXUHd8Qqmp+RShaA9pdD2X~=5cVHB!>Xj=|BXu zSn0z^&}naap;4da4VZ9}+RR$o_d%=HlU}aLZjn*sWBixcuipc%uP^oXT&Q^})B-ri z7Gsjb!>g75stn-j8yJV}g)VYDm=(GH2o*>5ZH6vUiEeo;zl@gDdiCkz;+UrvGA{al z@V98AuJm!)?~pfH5YBsBV~*QhQ5Z^WI`@f`m&rTOu^a1(x$$^m~Xl&Ku1# z^qmEq4crC^RcoswPrC`)#-%eO0A>Jqs*`PLk-4;$^@=`MSO^QyUkT|xSCkn=zB&-< zJHU0p8U_uV9rK2P)4?W%Vqn90U68@7JeA<(y_4TO!$nk=pXRd$-wHCVS!`37HD z9!z&dQ@Kj}SuJyI63)7YZ++S*?Y1hx08t4}D|>r~bv9^QMtaY5)O$%}JM}*UgkGT5 zdi|bhqV8sUK(U@*x1_Z;;X}~&It{*_`r^iFle}ulFWa117dJMHDGMOJ@EhYlW+YXi zzq*rdLAN3W3emW*l!kQf?sTVINQ;=n)*RQAx=VE46ao~nq-Nvi`ouhHHt^%a9Njsq zP3{RHx0JcUfwg#X{{*Y1_0`n_fHCSKbQ>f_{IY4nX0K3Bwo5#RQZBI=m@5eGaB~cs zs&KiO0|||>DhU+sze-aa*1UCbwhKgD>zc0Lkt9wyKgvVBTM>osNkxZ%`Li_ce&8ZR zX@;-_6W6wqzQX=v7i6DuA7hn|C_I}tVcv#l{J>_?(bJni?j99XmTfuP`w{wH?7Sb* zk&A9{v~3t**|tK6qMQ3Y3|CJvs(-ZYOz7wXbU{0Ysayh|n@UpCe#_e;xt~EMw1S=r zF8Ye1$KONsELyX=DcCKycFrIZCk@qP#WPR(n;+06W*!D%N=uY=0s_( zNrjhM#K{)pff_}Kb54Li_DzoKYlkM9GDO&cvD7k4i0{q>;z@FtYx#mgvLfNO``3DE zYv*{>@PcBbJIgv|*+|2_i5pNzhe0na=0f(}7bSU?2mTCb{H{lU)VD7JdX`{o1_jQ> zc!GpC(Ry9~!9!8~wwD8`V(k(Jy@l%s$|acZvd07WL-*lz#C{61TTm)l`*Pe}h@N#~ zdL?;e!KHoPUSy0v6^K@_`Z}K){*cH}HFj8S3txnSt~(vtlP(SI7`Prx*bv->JMYZn zc%j{0=c2)~59~ZsfuA}7lkgPo-hBy#dT=yg0l~uLi$e@o;RsPlZCK*?eS~~?{`mwf zUqe!H_|lxjJ>EoVD(AK8P}L&43g*0L@I(1>kc7MyS`G_ya4f9GBoY^I!s7;jaB}Gg z`vT_h@~jXeqX-6C&mV!d#JtWo+bJzwa$RWPzyi=%mL`=Xtfuy-)pUq_|EUUkDUe?P z%#Ip86r~#_qMjrL#iY7~^;T#h`FI6eG-1AN^*LPQLLDBXUdq0$#^Cw6V^));$5s2I zJ4!WURgd?KPL+JQCE2sm9+kG8484_rX*DRjO6EuY#-y}N2k9F1#=3>syLtl#Yn$kr zkmPDj$;5mZ3Y;)U(@Tw)nuruNSt z()1g@D{)O}qU*4le%7~MOT(s_+52GqAQeU1D=ybGVO!6F;g!edL`;h3MA2kSWOg5& zwJpT}^Xox*q-VqPBXf_A_uRwL`CoBZ@rs5t(kc}`F=`LY`G}_h#XfKq=$O`6^J<;pV9&O$8*mq}xeYj-AEm3Ru*;pJhdCpTVx|&o zqLvd|;!CcyZ_s4(=r>`OE)T?S3U=UTmwDZV%^y{X%~e;`GKhu%1J$=wC%6Be^6Nr&FBK<5`) zz7fB4L)+ppeA+eZyocd_hoI=6TmJN za(FiM=~^PnwfqoZ;s&Tyxl*r}?eNtUDR8u;!)4^|COd7u!X)2614Z)#4nc~mr5!g6 zI%-&wH&^&WflY{NYh25cB0A+EO0~QzbM*BOV&XCtp<-821a)eH$ww*C7iPd$)KLVr8r`p?jZ*=Fr|_-NGes*kSI zefaWYzf)vhZi?DDv7;PH4`v_tYuPZVAH+&eyzNfKWFeDla7KXyR->W-sT8Dztc|w_ zl&QVog0lprz*f&jyGfqURl$~=is~nj(8BA@Kjk<-0EwZ*b=VrR7qFcC1mPh~l6||i zJ!f0LrCKD%U?d!>*F@F0gdp4LxKa9b{+v1(t_c*Iu3nZu_sfg}WmTUzI3Zf3GbUmB z?r_I@U=@Oh_6$)gpi}ej?>@-=7o3R-LA>krG95f*C4slP5y^7&_+F3hXa+3P1lX~y z2^|_B6>A#xb2X?NoaoLqi2(FIrKCr3Do#2~F$Z{_#o8k1-oh&572MGteZrN^^TRKF zuv!i<>9#Jan|@^03=Au}+;_>amAgL3e0%-+wLW<3pex7$5x-co2UjTzC^FDMiD|np z%pfYg_;SmsKMJMix$oo z1WYOgA+;Cssm0jZ9If_|5Ni#u{%Izk82=1l_cF z0tz41M5utfa|{B>XZp_-^>K=2DyN*ihARedh%WB)6P*znYs)EzMka_+zR`ci#4@o6 z%%^8wyUsa21tm$7sib^h`lN1Uh4p7pup&7=42F-&&gMZ1Yyj+$wKZt>>%0%CKi5jP z1oS{e>=YC44MBi?Sty&Cyz+X8B7yY49aASj>4lIgL;`}O^)YN*Jl(M`s#cR}z>KT& zK8kK&NAJdfaK|A3z?g&a4JEnBOJjl_EvEgyxqC42w9RuEf-_2Q|bch0F9RMj@=bA2Q z?NA{7j;DdTH>fp_mu~Y~j|pei2L0i-!xV`P%Ra_uU!37SRBEQUM=;0DL5nG}>y>py>?m41uIsQWgB&lw#XIMvKf&Xm8b^@gWsCRuFZ zs^^w+4G0!(pLLYbA#hyP#xE`)t?T5@#%Jb*To7`O>w*VEPW*i~_Rd8tx8OlDSx~DX zJ9uvfp+uOSA&v)OT?|-Qfv&i3+U_}%``3AeWV>|L9K)x(pQyI%$a1k}8aa|2m{oZ8 z8mKf1ak7rQvyDoU-7Ee9d1 zwfBFLaCMwTq=iLhTDKE}W`8oz?mL^kf%&(*;LZy=L^;rc!_0;za0ZV9qv7vb8K3AW zI#~CDgL?1BJU#_r2qoRX5`H!$4*b%qbGfo}eb4Wv0lnX0$cv8$N&vBD&;u{Q*q-q) z#~E(Hv#IiX=WG5==#_$IMI&1p6xp!1Am^B${ume-;h9aD#kgApetuK{XfhA@jG(g` zzX-bUBsj(rWg=d}LZjcqSHV(eZnIEg+1^3-~D_uf6rL`%YglGO6nPC5z1xqK;sWrwY2ht_(B!|3u6Ge6Ol1U0s?sx~vCv zFucBu6t46;wt|+%0gR7Pb;X^>WCWfz8 zh<|FMzVjZ>FF-8Q1<%v>O)fC4fQpgP6fWS<|6BkP@`-_>>Y_lgX;K+4Fq=^86FEj8 z-9eG8VAjy?x1qF$Cbl18yNjwDJQPxF2S73MD4#q(aL>a@33ToLzzQ?r2w6(er0df3 zQXNR1_nU&lMog_^3!P#hej2Z@>_=5jm-+4%axVs5AQGXXeQ`ky8PV`ScMWC?wNLA_ z{Hy$XGWq9!@)zb-S{HIJgJV#(N!4S?;G0rl#2O9n3X%v`p^a`PUdV)}Kka;NvJNjm zGtQJ_fvSLpq@uJYV^0os@@Y;tkNzp^IR{Y!!S!FC>WFoSfD7~-30Wd*$ zHVchlA$QJ0t=*zIg$Z)RAumZ%ZrhKDoFzzw%MT0;kn-4P+>(tnu%{(ZINqiNijXcs z&?dS_HX#UJtQVC411>2<(|~d|GnPH=p1R%S;I>3L4bY@vlw<+} zqD?m^r%V8c=*bYdI$*l5{x4ap6$ApKh|!?rVmzFj&d;{KV?~|NYTu359_0HgtU>sDrV@xP7suc z86%^c6|pSvO!$mr#wjN*So;~`ndMOj>d2Ub#W)9G21qhpN%w>QqPLquh_rzFN->9-Tl+v9kGkRDiEs0J#meCqqg=udIuSUPsY4&UL2D&P zg#+q3G!+^*Atj8Uu)y34BP)i#l9uvOgQOtAXn{?ZEBL|%=Ck(}NArY#J(g>N^4g@j zv~;N`iZtbXg)Dub%NM#0m`VX59IL6hQ62&D2kW~D0Y`-r;GgL|D!>#+YB#)dfn3)G zd0+xyVOp6(eSfn_ekxGNk^KI^)k|RS<00rdb~j+MJkmda;6NI5IzyE0K~BdN67x<+ z2V1QHUN>I;-kOgfB7;SEM+(rKYXIq{G{8~n@92;W77bA9#q|TXy(tyo(to(M==4+L zY|+C3Q`u)*om#~iAbg>Ti0Ld{UfaR+dW{KTVDT9S-TqnH*8Oq1 zjSAO--LZoTE4TjSBqX&#w(GWasE<6BDnQ%eswT*3;%A>I^Dk{~yLri7*>4-l#! zq$;Y?<>jY$q+`3+RVR>`0x*OxT#Weg=iNMM1g#f?oAdaIK zRDsh}i%TiNF}zH=rS1Us`X^UAlzS!i;<~Fr(GfxpMkp5mn}@=g1Ud~yE=R9a57&G< zt4)(M5c;l4hUyq+e_UfU@;^AW@cqXx!8k@}ZUY*((uh*XI=BeoW0laK2;7AF3MF*bI6~ zw5Acr6iGff@U9{RTK_fR!*>BreGBR)9IM@uRXJci14%&+RPDMg#wwCuyJ}KOIq+}B zz|nHg`FEW^a*sRX(1y)07J49b0dE7*Ce2TX!#=Lh^bo$mbw{pi!0hV5 zN#BT2{iWc*;FwTaK*qdvqki1S-poeoB9j+Ev7V@9Uloe5O4|UzeX-=YYXgj4DU+_O zFcUV@+fv*W?om(Lk7E?%tM8v*DxIbP*<=u_0Ab|W?vyAPom0o#+3RKfqvJ|wSZm$Z z8Bkn_R*rLW^-@WVdRz^xbSX428w>U8CbafgO-E$eHJ5yQ56gESs2c5x6-R<0`2C_b z1<60qgua$=0H*;2$zrFD$w34cpVv|$WO6`pf%nCrd-UOH%s7<&`tV#b{4r4FrV!(* zgudiA`MNTyH$x;AcQ=?`m#+QSXw``ZD-&t-FrPZX8eHLBezDW8B=7t$HDCBRGgx08$WCMRvfZxTxGrk*YUKDFNNXJHG^ zw4iohJ;bWb-v5Nqbz~VS9X)e$oaP>@*3|udjKQ_F0zohBA(=erq)AChfvvdScnr;W ziPoudJedn)fF9^s2NL8?!?Ww(ey*h$_XS$WpkMLthn8)+f6rrvv=V9Kh&ONYlIK;X zy~01k4XauMWQn_3+IbT|YK@AD>IsCp$YD806$8T z%w0-_KLUZ6NtCYQKWJOMa2AmKFGf6jd#Au3&@F)McqsbPWQ&{Gzb~S19vaewn1Xc> zx@a-;T*TozcI9y1nPXfHri7q=eyEs2+1|d7s`1{=1m-@d*?cPdsXvMWX8LZ<|hVQ)*$=Pq~eNCH196ZFva=npN&On^@3iA30byQsUnDW zA%iB4=)?$K?*jM;&ggfE`I=%?M+`aZ7A-0`A&KBHkRuMF3x!eex>(CK%4w#oa z9LMMJHFsg*y7eYv-{VXRP__`FzDccxBasMFjeQgHTu3?Z-K}EvOs5}(QoGwpaR@KT z-r_+V2gP#8(=ntZ9AdDnU7n94Ini)}Ls4=XpQf4#RKI~UBTUD$RXw-$Z-rAEgIGS& zOg3#{!IYfaRuYoE0Q;h1*D@(ylWyDS587V9=PTb327$>@d(9vK#Q7nw%9 zrC-S3~t7a;mG@hFdGMUbNcofByga}T0~P%iorqZZNcwVuoE&%;TeMRmIYLG<@~wEz7Hs>UEmIz$nK^Zhmf8EO4L{s0r={YFmE z>nR$rx{v)=GxE29D%BOh;QUHH${ji|>N9vE=&U?E=I*Y1%=_NT&> zA=>OsRg&PbUwBIiy0Y@%6;>lB(tic7Su#l{7w!`3*@JoVm?sq5afHSvz6xjA-49)S zYTq*a>$~^+t4dx26h+zylcEF|0|Z6?eJ$iqN?t>lNkdI7i4Km5$pWb)(DU%ZKHjna zZq~!s(#A#!nn3w&0#=PY%CWcg?|oE_7gU$@aT*yi{y6 zV9MUV%Vf6H6Nn_NRyTk3-_*Yy^;g&y43lKOIxRx*M^uoc9vAy8fXq3JCw58S7&9y% z0wx6seB-@cu^mz+?u^JK=S>=w)RCTr?N^sBI2QoN0El`lJpcTF8aqU|Q=I7=WJ+2C zJcVBz@6M-*iml>+t5tk=(J;wFPqfdhe@$p=YPtX8oh#{PKNE!VnPEU=vnA_NiB8;Gn8l&b>SK{yFmC@F!B#x3Qr z5P$1qAy>Nu%uEUB2qP^er2z6!fLSF&%ot9NOW$U4+)Cy>P?3}as}A!|z7VFA8seZ; zgp#U!{XbY1EWK0Ukwe~}aS*6-EN}{*I2Mtyybg&^lfkM$@RP)$9t;shsikWAF~|YM z>k{}2ke2c@$o&EVAR?d0V4<6`gh(jpUnsYQn&_WDetbj<%YZ(Sp{n_uPZtafXj>=% zg9m)pW{`da@+u7_YfI`cM1l~=-N35}focJ$NxK0aj*ForRD6P-2sIZI+{a;>dS(^W zxDb3}`DbXfvS81eAd-WL&*OAXonF5aRO^vt z0ZXR0MAqS4)iN8H^*jAL-HK?|`4oqOXKh>3ul?;yhybHJbl}T|!{)2n6qusKfuPY4 zN+_9^3Ji?8fzNPtau*~Iphoh2A8lmTdchCehOlfc2`K(Z2d{FP+a1jRVfcE?;qHbK zRYNRO8|1n^iUd9+bs1CVb5O%Y3g`Y-PS%n^UZS@>mM;ZiMc=+X#A%VMp;Bxmaqcpk z+)tYR(x%v3e-}*6PaX%&FMySTX!E}Zh2m~2G9a3b4-ThZeX^R;PPzzb;Z=MbL`M)Q zM+L+I%8om-WSiF;ww@x|7IoVopW$>c9));ciTU;VNzH01P9{EbvApq6A+TOw;!J3_ z{{e|?lpNjnrq~kBXMpyAlfeQdWRWlT2a*q_S_Rs3;IspX1v$wFP;WHTlBy4@y#pAC z>kzcej;08xl}CAVx5P%gOGbDTKpW(X8kkCk>e=9kcUeJ=_;azdx(IgC^hL6rWisHL z@5Z5xXGkdISD>0fRniy}bRHCkBdE48p!wK&(Uw$$_HrZd}` zH-`!&n8A`y#5<Bl@t#LeW#kma5ge|Uz_V#H-`uvv(mt%{C-)|K}SI; zkaGbbTor`sAghxE+=M7NAN+m~)mz{eg0>8@eD1RCEcCr4cOJe2a9Be7r--q;JUlXh zeIUmN_(Wt=)T;3rh|(;@oSH0@tRrczx(!4PxL2SWnu)EKC^_&SKCj!r>7AQ>!{Oc^ z&O=HAMRv8XHgAiW-$OvDr z*>_J!CWasmArMXf01@cK&)Y1Kd^@gEil9}s{^h9vNcvu$Y*tCjd^~z2g4F7zv_DW- z)PhouG!S($%rBiyElx@A;=;5?E0-%c^Bs(u^6_P)Dxxp3f>a3W>Hvf64&*g0&m|oM z85V1{1XyX~th6xl{9a}rz`4mpI&}AC>w?3Ys}H8Fg!LjQ$3e0YB4+)8F zy1ohj(nZ`Aj|&Xk?{O{q9#oyE2T?$Kq3hqO_fNi1Ni(HFn4n+Z8UKY%3BGpf5Ks zp61d8))WGYN-0V!zE?O~gw#7ECP|`U$#%w_Zz=KC{osV}st8rf9hm*_zy(uBG?@Sd zuDuY>arX*MkB5T&uC7u61Q2B1e}hkk16kDmuz--j2=YJHff0;LBVgSnB!8ay%gsId zfySw+EF$u~Co?yfzce@IT4pHBFM-~dbIdWv7nEX>Qj$p)Z5Gw5CDF=R)@GRCNZP~< zU3FTEzdQ{xA4=7{njcjYbYw=3_Bl$4o=7P|&haiwqEn0woYF`YGmKL+XILcKfXo71 zd^AGRn6q2cNkWdI(ijlv@P{!!iY?p_i%l1QCPNp!mypNYZD-tW1RNK|ziS_5gfw!t zy2nI9f2{H8@iY14Ue!-Ve1eX}`$D#CG+cpLyzz~8CV3~JTXZXb^M$^9} zI0IxQ3cRl)yMiI|hGX@akh@B=d9l|4X9yB-?K8gmtx(}M-m(h9?d948)DuxNxe>T< za3rut;AmS7!GHVwj_wj@mY@^gcHlrh*0GHoDwBVQ1#^~%J#Vllw}C8&X!1{SHCBu1 zqiv^p6#k{Y;d$1ePj)vaKXgYJu=+l6j8B$`19NhQFq<7A`W$SJ2l)hWgY0%xRJlp- zQ>X^tx(n;6jtxMT&o4=&&?&{shGSzfK{urk{V+u83`T6DcYANWYZ}*j58s7M$?(Xb)kgA_vqAh~{?Ij$C1#p2qK)-|Qq_ zmwoNu*Suw?yf7;%g+@}J_!g-qc2NkIuSqQ{0n8!7VJ#){bllkn4P2#*43U7h@|ZI0 zFyOZz9f}wOjT1bTyK;Q`vpFVe`;PzoBCT`aTLgJ~dowdLLkcGW+L7^O5MjMPmfG?V zz6Q2cz?D}VWDbDzNV>Lk53;V5{Z&UQm#2ey0+-iwyuoAp~2K<#@ z8mFSQ$dMP#d)Q1dMXQuCg5;b0#h~m@zl9yqiLrq>BXbb z?qm>NTqkEZdYZ@Y=op^twgW4X*NF?ket*9NpMO;RSv?|;N5-jVPgcVCJEDBv;%7Vg z&7|4cs{cXID4Xh+CeFuxlnA4b5<%?OQe5B7Z-c=+CJCn+d9tsrZdMv+mC16anH3LF zef>tis&)B@$o>JXBhd?}9aFgy+TObHAj)om1|)~jiO^15>`AF9=4Em_bkMOmt)8Q+EMCg#6Tfg1xS)i zI>GvOU7uL=3FM=ansXewBxiVKwT2Uuh=xiE5$phEa6_&1`=1;_NoK;s^-W@f&AO`*4LXm2N!JF4t0B38CW#I`T9|dl4 z_<_Q|@Qstx!4P(`CDXCq<9!dObL5$T%|pQ#-W^vl8<;%O-Ibv^;7%iHX{*QaKR|vM zk>GNAh70^}0UzQmYHc5Q64ClEduh_Zvg7;rX7A-IK@L{|VCOA)E-N_HO({Z%_kBBQQ zJNjL%X%H?+>7$Q{1LrdR;d~Sbo?L@02r#eO=Zl*;=`WDlcfLpJ%7Hfgvt_UtfSHvqa6!_+|^0Czc)r530>X&Cbzer{nwRrU>x}P-O?aS zKnTVApy5KDqb`L>$Uq0|8Yq_f`(c6-4-eF2_#?(2zt!gnNFl$ClTK>>mkR*M-qW`B z_I6G=IXQM7JS~E~nZKVs{+GV3uC9)=?aLQNT0(Tupu3LJzsmk~!;zF5l{u`Hoc6`~>)gR)58Cl9gwL3VV3 z$>ZO8*ccc*Sl^=8O)|=R5F8PyEU8pK5nvw9YBMB%G_T>o`rii5#>jaPnI#e%cNN^7$I&rye!G5%`eP?1~2q5%D-9b z^*5RD)?3c5vL~skcz`L32Ddon&P`tv1muDv!X0`xspdWsgOCaNL~h@o9yiDDqQ1w& zxI855htLsDg5Tis?{|=&^d^v(Xs`lF{FVR=;=kRV{=EM^zD@gb#&NO#bKn1(HRNqe zA!yOc?XsQI31Gh#&i|Y}xQMh4Wd10>f))sL zn2&HzmH>y%s8Q&;;Qt&5F93|OAmez_^?8Al`qZ&o{pYUXAFX?)rWsJeU|etq%ai4Q z-)wJ1=PhAHtk z|PG7Eis804Y*awPd|9@!ht zv;$|n8H1iXR8*8wXcw1Cm3ZYdDw#TBgz3s}d!2r6ls$-e08+;9C1UCpg$IGnm z&IKpSMx+Y3#inEI;gOET2uCuCM#3xmIx`>q3)Ps~g)qP;A!r=>sMy^3f z=i$U+SzXM)h(!p+3wGr;bYxr+AtlH7jWm;fQ%%7J+^+y^3kca9AjBL zyCa#E<(r_56%>RfAAED>O&HnQ_?1{r0_YY{1m)dppHpvDiT%Aph_QCT6c<{K3nz>u z$q?t?jC%`so!IEtFG_O`b=O=sI*XgynyMBYBy9QhiPT+XDH_3`H>}n5?a6l) zXpLP9T(I9WY|S;=(by=rdYkNWG|%?+z1sLd)}nLP*MomP3rM{cLL;2|&+oM>>R|MN zrHCZWVN^ul5xFvJi0qy19~Y4zYjbvVOw;ejGwDlvlh&J&T=N0txc7{P>&}BSzupWM zJI5B8eoMx5r&CmXgs%IxV{@ordM~3YazDbw#kIvvv?NC$xwSV4pVy}LuI=ck&YZ21 zM4@r_J(V;yRlWNkfE_zAIYDT(lA}teo#1|`HItL3W+a}hpB{s%5N}dop;TZ4~9?ZOaUNtE>aW3qtd$+fR+uZ%| zeDhu`f5#OrSKeQ)`ByK{o6I+uk*;;vfKUsV**fvjEa$5faTC|;t;zdyo?vW{_h`bJ z+1(8~v%A^7Og8mwx)5JE&Kh&%na%XcX1D%>x`H3EhIlj)m{_MxR&t&jRjY;-J8bcf zbS|4GXlE;wxE(yr8_507N+381qCAiThvEY^n3wCwyc9lc6>p8=8yg>{v8etSHtMnz z@1-{lY0|!&cOq=gUs+YsG&;uz_qmRY$8~*M-0qcp$Y`PEJ6KtRgbs5X90+s1vHYDZ z+591ouYW5&>50L%>(#iv(H=n4fsi-1KhyXG`ZMJ0w+^v(ztpW(Y%Xe5iBk+O+bmJ2 zZi!A*Op=M*dxZOZtvJ55t!h8>$$wUpmTq6hg9H#3l0-w(a$8V^0!oz^AXC(NhvAzk zBIQrp=}B!RKnEw?bccEONrPwzwaTaQ{gh?bC!6}8Fk80`L-7{)j0@<9d`EX}-*XRt zqaU*=oc*S;?79e+t5_e}J(Rn^JAlMmY|Sh9 zZ0B0Gz@nvjV`wxt94G^UIK*oQY64sZVgWjIB1WVX*x1zAF!??{yiW?u*wabb%q;8I>}Rb!rGxLr%>;F#+sp@wjq15W zaA}@y-mDW7CF9J1Q-40$>GoO@rRp#Xw-18B^f{_ znBD%B@7q}dtv8_$BVwKEoAjvB6OSf=qiI%v;d#=~CSUtkg(|Q=2H9;m9ai7Rv*=Bh zwc;(}gnUxY6z`>p;V8Iwr#DNjd3?Cff2hSx3neBBNi?8q=@ZxjV@@#Ebu$8K76+mU z&F$?B{Z`Ol=9sHs1&J@DzcBQ?Vv zkq6O^4`}yWVFqn)p8~)|H{?Z*Vq6m0V2XTnsY&UjOK4l>)kJG+Yl8D6A1)?Ie=(x@ z7TMb32OWwoXC{6|D9xCsu&7Ja-&VL*;I3DR^*-l2O*VKgvZva_hQm}*cZg$|anO5H zBK)8vjEbLKhw1Snh+l6U3A$rqV&1(*{XNo1QdX)rqMHf&DJk90y@puj0}YgtD<~5U z7A=}SpIsTRgBmE^udRz+0=J<1=Lj_CTc|#L)oJE><>Bu9>ef~mfVS462-go87w9fD zQ#53*1YH$d+@L)kP9e0Gf+ntof>Umr1CM(@2w}j9bvz>9%YxD9i5#&X2z?C9^CJS! zILTFf?6=UtzN@<%s_P#NaJ$UBzN?mQTW{m+$V23B@c6N|bxXOv0He<{Q*(2thr``z z_e)<;&Psk(kUcn5YmMFs1Gm7IWO{x&eb@g}?_}swS!ug0WCzObEUclwKpUcB|;+;9k)ShR1g7^E& zSl-Xv+b3^2w-4+sxRRI{MHhCorGJMKWOd#QQ`ZQ+x{_Z~W za;#U0`_^@opr#D2jarK$cU{E2TXoU%4pz3+xJ#oI-pjg`m&o-ahA#f32@}az#j`Z- zluWIW4IIMHF(1JCV!(_N>{gZSC{;U+`Zb2pR#Gyte}`kZ!s$xf^|<(4d(721r|7hq zj!bgtFb1kEE_rf3&zpMzXBI!4iVfPz9!sz|okH0-H#4WVv>K6|$ycFeXr_ z&CShkB~}Sq&-IGUM?USWO`kvk5wZ@_oT7o#&0+ZT^z_tfS!B>A%nKSUGn4qeeogHU z(oq7RKYvba9)^y_`ivw%DnwrIB1=k2uCcNn^4-(d*GHOgj>yE{^Q+YhrjnwJq<&Dw zN#VqMaTjly&*uzFW3uCt>Y}Egt}gyjAcxQ*7Ml{e&G-UUh5MS2akcXSh2z?EeJI;d zuKlc&sJJ*47dQr67u^+AEEwNIZe%UJGIIQukyHDIoA z-N3WZt*m>l?X6I#N=U24cACA2-8y!@yRp5*ezvS*^VcTfi&|cDfg0)*tL^bM5!PEZ z0;m&*CsrzsNHz^+;ve5-wdyfDVh_%6xEGmd(_90^M ztFSL({PGZmf#Is0u0pjKbZc^|cj(!kL?}!4R%|d1mh;X6VRAsko7?!Y$C!bHTIgCHPyb~GGymjM7)B0Rb zDh4?@c}R=h(V^?9Q>Q*U%RmpEa;lKV*!x7?Zjz%b%k7q0n1i4EqkB>KSX-peFI;-{Xr@F6C$@bf_{yzw zv60m3QWOqZV`B4IRIX5;Y+s78xE+ z>1=wi`)jN?QSnX;uj32o@tEq|3zK~#wKRaQQiuD{pXdJ>99On zJ96M490t{9>CRHYUynygj&Df6d2n%MZ3E-H?k)2l9=YM{?l|0AbEtFY&J|{w?d+>O zTU*WS2sM_vpRRK*IP*Rh7oGJDO6pUvj6L_fo?@THCh*rTd`4$$YxkX1>(GOB3%8%q zq=xfzef^u=jg5`_Nk)d>2)~?e9uDp3?iaDtV~pW*3*g~cdLuoHc^XUKc=^INcPD%t zLb{JVRk3_-&!6ZUC6PeUqL(kBIG#waVSwY>9Nn%1>Fe(w&d0Sk4PMmTwD3rcNNs8p zsXi#4L>bAxza4D4PbIH&rRSq2`3~-s2woi-HqP0Gs4xqwgH_p18$Nn^yymGa_|zHh zjfD?Aq##e|`+Ve;&&gUGqB|a|Ax$?@9OdBH_&%RvQDV0!7oYb|m*2I%hC;+@W`Un1 ziuAT~0I6Ntb;j=wPdZ`++$Kgd_k@S8d7m)h_~?~bH6lrQ@ke34O7pb$j6CGyFj4B7 znm5(5w5KA_ybjVlx>H}M`vf4NfZ=*8ItJhKVs6~H+|1*jiY~fIdzr!K1o2wnL+kYvQQf$cDY%UId*6N`x!i(T7iVedPlP(ILlnr*RT_!8+|tw3PdmT zIDGWo_gbx^O}X>J^Tn_4OCkhx3ZjPngY$K_*B8J0p^nB3<^5#+atx2KzJ1e3U*KII z5($(D-)k3ExE5Y2Mt@I}5fhK@ija~LOJ_KMbqixw=a1;s>2O9PSsYKK^<|tbf8q0Z z#%XJ}a>Z(==_jLO+cQ{ow>quh5Qjc#9DTIf3^>aB+ikeMMs#%*?vJUQ17NQkPt05^ zh;`x@arUMcE*Bi%S1+`S;O0LkYK@DBhlMK5_ubyvc?gX?Q4=h02&wO1tDh_^ER0J{ zeN$4x`O=#vCK~;BufV_5={~Pt*MI_pR}mE5mxYNw@Gu%oxz<#Y)|25!7!@Wcqe-?k zW3_jpjB8Uo3X9z8*|Gvyf>cA-pLauR`!KGcTk#ST5|`Z=1qN(d*NI}lga)UHv}rC-_V$=5ss#h1;(y8 zEn}i`+JdfJ*~+Sy+Dafu8AF8{Nhcn!Yienc8OguDefL}hgS!ry^#Q)DCKu&pY^=}jI!^ctZz}0|LBKv8_w;X_fejdroQx2lfc*;W(;UY zjj9Q}Dxk>6&dN&gv@hcXD)^2P7Ss>kvuo*2k@pG&Nz2To#f!%d$?qSABq04r#1D!Z zEPj4|U5r;0c(FVcitd-W+#xG}hV3cAQ`hl@v2#-BgwVG#^V4K$MK#}J>a3=BM{ISK z0uM*@B8C)~Cd=37f@iiTPs=BAXThC)R7fE;UHkc5H2#eo9}jhb9317gbJr0Dez%@URGh!e=2-G}Ly}^#yin}jBQG)6;;d1*>2&c#9V+S1?*6;m+uzrw zgE1I5%m!b^3b>y{i3d{9_3z)7`+Stq)!Qouy*7K0(=YzsqFS=HW1eX9yc$;fkG$=_ zBcrfeVmD8;G7$n#8z$7O;_$TqF%WL#-lH_MgE-Io`STMf zJAr;$M4J5Kv{AyKe7<*Y;wa!ig@po?U(vrz(pPn3;B#u9PA%>&(`E%T?lev#O|0}OoXUt$> z1!rr?rIy}R&AetJ;Jkh|mnj#gRUyMbL9&1nC$`+y(OF^W=&>NTHdDNxkgo1ka_-Nk z_2{)q&O?YNZF|qafD&4KUx)T0Y)^>_3YF-qcPAV10aYiht^Fmy046hjDHE23p_3O5 z;YpNMskIo@+QI(LrS@2X(mB&Ph{;YdZSHQh4aa?I!y_n8iSBaNLK~Xv^U(G}-7hSLSwob5T z>u}NAwpvxn(E5#k$X!-%xQui#eT?~2KOdh@-*s-Tti(KyZ{)8ycwL-5mb7Qrv~-c5 zICD+?_JdFriiTU_9lgDTgYVuctQ;;SbdV(*B}+&1q?B@QGXbuDC5xz$Giu%=EEh=0fN#4H#n!bxDm>5 zP$&TSh1RB;3~`WUGX_E!*w^?yc&MqVTh~DK;50P(hClT#ES%t={d?rRvxc?~n5hs3 zZQT&_`Cv8KfNZYoXR&Ucyq_dal`&XErbbcr5H^<-*1HObbX>VPAC3y(c587ngNKyV z0KpOj2;x1{(|by0{j!~WPfe2Bi{F0>SNB^GU?0%qi@5h{ao#>{@1lX@^TX%yMM)u! zkyFiv*AKbRoWWBtu3&ypixQNO&`h-wjd%b`PPwrBjZI8$P*Y<=(bOc9JI-B-TPI&L zUj+Y@gx?K&7k}1MJ8J9Ed>J`5-o7@T!mXf({Zp0C4p!;d7d2^`v=XtwdrRs!e)uGS zgJBG35t3kD({H6GS5^mTcwq-v=v0XrOH#w}Ax3q0NI*^=J@qs_B|B`23w@FJc73?e zC99o^@{SbnJmyvie?oE$B_%4PES2Bz@MnDCrx-ahE`Nm}?>gpU!?FxdkCIbK%B|KB zNl&WYPjUT1tQYykjBT6f*FX9bB=Ex{BOk4Nuiak#DHGpFPDYkvF~ z(exYRz4x37v$0`tfKY~VSm@B|RFJ@f*MO+Bo`ppqNbki7K0j$)tw(bHya(v2T%)8s zH1Y-gk2vsR=SE5YtQ&=*T$A3rFTK4nP#GB+9i=vp?HJzVkzjdVpsEEY7|Y*UxVTC? zXM9ZdTdoyxqWh_`;VS#n`lsbSH)S>l(bC;%-*4(-Ie!}M<@M$2wMjU4oQsy~etF*h zRX5$JTY{do@LM}IY*|fJz{bkT=-Md&7X4C{%i$~Ss7(~=QM`W6(={Exs0Ng07?{QwErx01ol9TG4uU$5}HaDsadwWhCOE_t7=2vWCZU?VnjzV95EXF2xy z+kf{@YG`$}02HI%D3I_Yd6#U!EGqSokwgIpJYGg{O*lE(8(v-y8s8WXS_vPD@Kw(@ zAQzV-OF*;7%fZDx7xr|xYH=^ubV>T<-SfPmm-oRHLro?JiuaF3qfnLa z%k-D~T!pWVL4?X^9DM1l|JmNk1<=8`#-eU(%gXQUs$gC8`NqwJo=UNJQy~e7(@^Ro z26;6?ilKNhTswwXa9}k_m<_N8i!B0y*7qGcy$gpjIL^$>e9)_XVLVhs*A&K(y_EU) z=rp8(_V9x0>MnNaaKy&_vAIHUETBb7eGGFN4SAdjmS}LWhboM&Kydham|Yd8H7vP2 z%aB=t1#p;=p;&!GA6DU&p8H>iBO@;cr$Y-=O*pRq>!sL)rjj{NN~-&n z2AUI)FzR_A;fC^*@2hKQPmx}tmBT3vR7Y3ervry#m5aPeIg%#|Oozzmz2$IRx6GZW z$<7i{5XG&N-uP6nuY`lOb~xp!;@j`D>k~@-;+%8eHF=#VR^33qc%^o{aMq_T=s=Y; z)e}ig1Cy!KEU22T>`_kFeOe6%+k0V!l9ZA0_U~!szD8W+so0`U_K(Iyff*B*+%p+p^s);`7xXy4u5|m48qWh#=UWdgH3j~ZjMjiq`oF;Xyfd@ zb%8$sCjY)(PE&tuQKc`&rGtZg2&QlBRznTWGiT?NJx6e~^&Y;nb8CRrZzu|xe?XtO zaxxANZn#Zn3#*QT!Dy<1fl;qQO$ zeRMPut$#wIb#zft(MR$N6esjA-K2Vasrl(`H>-p88Wu;){j(M}^(tnR>}bvrw5|#NDy{PRTu`S4Y=gvZ~&l zqtQuwme=Sm{kAYewUSginuD8LBh|{?GgNhsDN|!mW2Un@>Z1ir<7{5c#AdzdVmSxj zwUD%Ar~jQ1118>6T3Motyo|L=FLt~<;{-9>~h8nt2Ttea|C8u&v%DwNTD3_!7GCgj)*`5TxKKh|+ z!L1c`i+BKho!DchQg(`I>ZK8mr@HWB^CZSetH~zWo+s)wvTOY7&w@XS8Yh_$9!(J5 zi%w3y1INieFe*_&gbzU$Mp5r|#}c%n@f%dL||~j~_n{rj8q2PKBvn zj{>|OysHx^DCDk#v*=|b)MdT|SaE#5KO5vGPlKw#2^0xf$xoj?<$xbBT;X^LfE4{R zC;uE{@zqGDwi3(n7acL+$5UQ>gY0}VWap=id?3owsh}^wU*vkh@9FfZr-fB^t)aet zyr+{TzUnU9T^im%Wm4}09{fi?5+7Oz{yMts0rCBt-MgxEu?xQj2M6OnP@=F`@L&38 z7JizyS`1|hBDH4&@_y-(C-u(2>aJj?O*2VdNgZe=UjlAM{FeKcI4kds@e zu+{e?KH`JH6A$+fa2lUNebg#tzN;p8INb{ogNt_n~z23{q8 zQJayOnG3C$fE}X`eK5Y|8RB}@{#~PoXaLDTzDz*491CA+WUuIIP5N}Um@A-cci8&dWSzesDIGF9&4-!^8W;qBY|nM| zH3;nQ=n$m0vFgi-S}G>cnQnB@`BrJD8=0jk-Z@n2O`S8oaHd~$ph!9HsDnR#Q>p#2 zI=e%XbIag2D{Yn*qaV#uCDcbR61vOrI%>K>_PrH zK%&nZn#jlka-mA6<69`h@p{)cgx04hTJz{<`t4X=N3-u8dAS_{ZvtvNMsKbhww^AX zF8SevX>*I=JQ7lFAF|m5-)%LFQ6e~d8Wp8xp|q9vT5%nM0If3HGo+-X>eX(1)<3?) zb!@{{#~-CHs&7}WM!G71HlYj+1=g6#>N)Z(t1CW3-PD$jd$1B*@;F+ zN9RUy{@8Yf(5;=d$F!P7_lzN@g$;V7Ud|HK95+mj!KoQ!{v4Y^;XYMJXbF@KF7^zTN@^?+g;G;m)u?x6TdlL zDRF_|eB$a_i#GqIsnDtTD%td+c$wxH%WBJW%n`KpzlyU*gtk;&bL?laI&*iib^#tg z0pMNlGeRZyhYji^KN!ysj#WD)*sLvmD|EDoU$A$_Ig9=I#jGzAoCTs{2mJK9hZQ88 z)*fZWRK%_ruEEsW!aZAK0T|)z_Cyt<&Z*9hqEcmUd{r5Fg@tUuv`p$D!^8U}V3DO$ z89BGbfh=1&h~DEeGyUA}tsfvhTgocK`_Vtyq?S6OfmglQiuU44;4D^MhYzW7% z^YZe~`gnO^!o(#mQ7N#;CoN%~Pa+0?bb-5!5W?~C#dHT@cS%cXWbO3I^u-y!xk0Uv z#tCU7&o`+|>*v-l5MK8GP;<@2W$S`G1%Dl^k4vp)08p8oX8*)ooTT`)*-RgE*@7Q6 zfzd;%YWhOF^-BD`9Ysb~cHw90oI1RYCyMjK&XbY6H+Er|rZDa(a7&xEGVa7COjoUe zbNXeK{8fPjiFtl%l{SjPxn99TVf&8$eu-z#PF1-czJi9{Nb>LXn}7=qKbm8SWBJ`w z$8IZmrS+w~xR~TKS>GLo@>t2STo|%K0lt~O&Kp`vmuqnUc>(j+ceG7n8mUeKLpPt@=Uq^{iA**Sp%7cw}e z?i**7sEM*N0nqEgsHQEM!Au%*!JK}4kbf&ntMnuX2gh}iJ^AXBRTpqEZb zjEv0B!vHW0zh66Kd`<-5_+J=Ajy_P&M2>fC#B}E8vd)P zvW}L3l}E$QAhf)RaWQ1pd2WYRp+;J0*tAu{f?>Jp#bmk$g0*Y2sLZ88cEc$ru6zO*IB zWS-e{xSYt4CRo9V!p2vNgzE`OfHrSyMfK|_RSbd1O%Vi)E)OvCdY8L8@Q68{$SkrJLKfq_gY7e6p0~Sy zYT08hS}sfo`Mqa7+i=TAR{{X+b@E}{lx*^L*5`WFwlkx6yGds(C{r z!K{6a1Z6yLtlwQ_H}kWHTw~C$G}$j3nCU1F;HaAp70HhO0>#*GWp*^E8Xz7naU{W$ zOIksu;WKDOTh8~<6$j9&s-D_FSXsc#M9IorMv}Y+OC=ggcHA5X7I%b2{WyOMj zoc-V=kbJv5R_%ToMM6TN2k0YCrM!G1j!SVsp1+g-{0E&$?q`sPmAlG^D<3HFY;&9I zg{Lr@SLa5jD3}0+viE#*Ts>lSw>DHrzr|%H_Cn5+ ziOizn&8GVp+1XU*w2Zh~bBtSg*ROP4Ic-ng(Cgd3GB7U_KWS@crw}L2)svxyL0(qu z^*}Cv`)S-Aq1LkA3nV0`P>k&C(rh_*AmJOFtxVK;xWf_xoAr z7-+5_VOx4Cju91~-!kA1D!$%a!@#m~-(OEuS@5K5aCYw)h>DU|Fjlv-k57tc;}A%` zp$~z~a>gHtj+Q+?S!|{k{kMiEa5M6lZj{KxhpYJhVkN&SeCGq_*_tDC%jv|n?FaJ^ zZdBcvevsC!?~@s4)`lFaSj1hoX`@O3Ycvohr`~d#{Y9}zMEzUr!mpRc>z<$U2%rhRSuLGLJ>`sF5R?tp1cEP|`uOxW+1S(sr z7>Z=hWxQ~*HqwiKTsz10Sonxk2w^Ly2WcHF1^4dTXML^ZP@1EJS-!-5xm(L9PznU+ zOs=L|PXF>%D=S{Tb;9dSb+PX3vMZkowcl$f`upSk<@u8n>0^f+PMtb7P!TD0BNP1Q zlfl09rt!2IJQ;8^(NSvcw3HWR)8(_zS69BQWZMug5^A%`0#jJcC;p_Qhq_->Jo+WB z+}@_w<>KcXc`qJg6u+{g|E{W;NrUv_aiaJ2WmExfdzf4{G!l)rno>Sbu=;Dr@97Q`k_a?5*>*8w=L3K31F6X)sw zJNUzKrr7i~q-ZI|GIB{-ss5LUMcPPy6l`f z#+>U(Hy&R2+;-@Ufb{pLc}A0p3&l|UymCC5IcUe)xc1hqTb*590RRSf;Mf2LuHro= zK`DRriUD#Pg6VjAs{At>%J84Hu?c|7rV{Aq*Tce?SehWA6hBU{ADNKAEh8&yn4%RT z8ua{s)zuw*9t*${ivOKEYc@NzmP_rpxB24FrN0jT(I2@>w$MDHvt*swiTRVAJ@&WC z?%lhyj+XFyczE=GTSgkumL;h2zeO0v-un3DYTO^S@d+B!%g9bn2iOn^jS5+%cT-XV zpae`oc9ye?O9f<|Bj4*dpaO!TixWHt(7zQplMU|eVM8tzAAv*jj#W38xcz&`F8xP= z=jcxsY#6yG0-Nk97=g($-wF&He`3lT8d$$RKffFIhkY#JB&Q8qh6y8PY9Q>?9YOT{ zN6|em{V_~nU9%<%c!MxLn{rj!d}EeX*%~z6{YUxjTmhN>`{=<$N5nD@AgbBRHM%=@V z@6A9_TKeAPr|kIe+__`%+(VYAfshbbR0Aw?8Rf-`7X}$dVvV+N&+KfZ>U=Db{r8(u z_x8NIt;&nw;RbXmiud>5emH2ywh{mq`F`N?U;h2^+NG~S_=Vph`HM*$eW8NelLK*2 zQV&J11@v490l$um>w! z{_MBygdM@=)GE7liJp;XW7blKw)9}_$asUtxx#mDiiXnA*jRxlVqS(gx5D>4T^Ld+T=F zj50XqXD4}%Ce!$XJq9gI)|H`1QvXqkp~rAX#*xCw8T!GJQkv*z&kRaDOhLrpo@)v# z_#12sT1o`BYy^^LyO8^(a(+-y-EZLv)*vlq|1FgOJpPqEZ{ED=?Cd1+E=oD{xg&o* zZ*e~xiZ}2&w(j`U!n`WwE^dhaf7bg`MMS&b+U{j!A1%0*9n6s#mTD}t8`LsQb90`W z`!zVsFU8~m?!^KZE%f-4WN_vbn7dbra>>KjQ&NOcHfdE=RVn3&p9`)9Gu7GOAAw(_ z1+tvruZZF-eujTH&JGwlfK^6{_36{ugFpWFR0No~Wy+xm4feoo8}g2yWUvJ!4*s*( z>TAB0`K_jq5+Eof#G&@+!Gln2SQ^Tij?&wGBZ!o%tf-)&IQKV8{`dFXBYJ)FX?{#t zbkKr^)cizQK2q$q10ZN~3CsUB2~lKS*YGQCcx0B%k2}~`hHF!?JjC?%zcOLriHM9( z$8*V`Mb)aS4Sr~nv8?lar-c^bC~-Is|87%)0&a)#wMl_{i6=LR7c^ckPG#_awRk>c zS=jVqZ3X4t;-U(xKeMKw|J}vO>iXmPtnNc7hjCkgDy7wsvjTC!$;F8W{;L`p(9!b% zT4!s=8S@fBeJMFPIrv6tVHMWoXWFh86>X)UByMW6{rmH+v>=Co0F$ijNX}Kj(iH#e z*Lmqytau4i=GV{l`GI8`H?Aj)=+e#UEDM|J@k{TsHGRt?dabT|VbBzc%YXk+f8#Ow zkfsp^1S{&(f`V#9P}@M&PChu|-w(DxrUDHGKe!X)5Slj*p=+p%T|M^#D~>DP7|x4| ziYk7l9nmd_~6Y@8m_PHi7C3Nv#bVqerFRB!0s$B*W{Ei(ER$EJY1?9D+fmq4&7r&J1o37 ztthg7y|IvSuJw*54j<@YrAEyyEPi(N&)Z~;wRkW*?fBUmT+s2cM5YKg;8>%mw?02F zhaws5AO1aV;}L#fzn30AR>JOTF1aW|J$||^e>pI4uF{-?78a!WlQ2oYTAa8`bjv?S zo}DdwI30)h-f}>^3fNLADk_?X4<8;J!82)ndRl;T7k4f7f7_p6p3ze3a&jTBIJ-}U zg(dRdvfA&{7HuvO?{3p-n(0X=9$eO9B9t}JTgz;@UT+2VbGmuX8hkPBd5EEE&`LND ze%Wf0X|}IVP1U%;B198_`NI^1nK7(fSrI_Hg4F3%Lfbl_X2%pmPq{cXjM@ZO z@YK{)IpV|&i%ycASAP*nm!Y_PM>u;DAv0d=@S?Xqap2f?eh{aap?K%eQ1r$l#pQ3` z?)$YBKTm2R1n!-9m#>mF%8H-qXwzkwMBv!6#hI#QReEu{v%5PG-zXEP6=RRI|33vI zF_8zIA|Pu9iVD(p$nF4yjZTo2qAf;`CYKTC1o@81>OSm8-JHG@Ds1!koj&--fV;e0 znf{U3`ki!-PPgXL2yAxRFZTVbg*laiy!t48&lkv7DwY$IL?3E{#ZK`@ zBWDANs}`jjR~8s#A5tcg7t-V4Z3Oo~xRD3*-oG3}5K&(~(;{EiK_0hx?7g{J*3b{- zkE|h+bORbZ$IzGO^8EbjGiS~?pFhvDJZAZd6{%=V!lI&fA;$h=iCTR* zz?z0!o1G{d;@(+!H};Lx<#2eiNJ~qD{Dc01PQ!Pu9Ba|#=NmqlfiBnJN2z#$u(53e za^GB{CQ@GmCZwUv&+nGH&!Il)Lr)~o09;qQrYQU0*7EiB4Ilh|85+_*pLNYe_D)e* zD7e;y0lB1s_wV1w${aISaE=Uc)D6FKg>!l1iQ^AfSsLqa;rNKfs{>S`SZ`p0>B6(IRog@htuGiG&0Zaq(SW6LrNDjXU{iaM>>%>)NS!f&)cE)-D7dpf+`PH2V@4 zfHlenxK;c3@q4VnMBt%)<2AO0aJ*iK2SjAfU6sp0bJnTtMo;Y0UA2--B^b&$^MiHM zjFwflHSpE%MGeZOv!=s}dGq~b91WW1ld`YB4YWG=J@@?8I@WCU4Usc`Zg2MJ(ZCuuZ z1ABtDkC{SBJbKcQT5M=mF0T0RwPk@>zRNA=`8?BJ6$Y=}puD-r1r_k1xO1ex;MQ{# z&HVGP|9==?!uBLy)w-BHtT_6?IemYh3~&=OoruVCpTz|sbcS5*;6qvq922zyto``G z>fz{7K!=wyg!QH3^;fk0>FLbDKSuQIkSl5V)$BX7R+Hja%-h3}Zf4qC7hDS5gvjV*j-GsP%hRf^pE=x)V}a zcexEZ{mhovP72bt7$riQg!#G$EY8)sqLGVqf36zxDL#(~U^9!TvPsQ^jBpx=`W>Y>&*C7XlGM z(_micZ1cOlfC*A4jA&sd|2Aq^esMN)e1#|M?b8Q;9G8F%zZ%UGIsP}#wzvtNkjUv}*h7f$SZ*bqA`_0Y`keP{ztFc*?+s6s#gftwr7G@y zv$whAiOhOpyZUlgct3w~e{rFY!gsmjiT#D;{+d}9x%87m+x=tm*tU&3MQ^@sGFr=! zHVDO#nT$nb=;U^rQzrs9mZ+vc=?<&~ZvcSAhjhdBdz00LFk)z-PLtz?Q`&5=XUwR1 z+ubFTKxhx_8KgHt^Rb%3yWKMvEt>4y!DY&+uloWVv9|RR*J7*HpJM;U(20PISg+Gg zPTHq|Bxq=8pv6q{8icFq;=&vs4hZ>8o3tVPj)CmXtz)NHHfS zbGGGyI@;RS*!vbwo%nFt1^aSvaL62b!hk->97BH-3)bBCb@4SR=btYO&yE;3cNC|t z=gzsEGx)=fkLTUd%HR~^V2|6E3Ua5*#w9^zM(xUZIzQ4s?U*vOaC-#5c%XRK&dnoGc(P-R{N ztMVLas;*uR@tN+L#AcyvTzjO2G}Y#kkiDtE%EQ9L*#e6fn1|89=pbsNI5C}_*;4;- zXM1w3qH|>5P#csD7%dv_?g%=rFUn!3voa|C=nK!3STAr(;AtekSWv1~1gCw+yhHRr zXfY^vaT}CHt~CvX(4=^8RX|N${3kY>Y%4Me5cH3}qf1#0qgioLzP^+Kl@++I!*2~j=s6}1rs;91sH$F$+NTR8A}k#dI2^msH|wfD$j-_#czj|X zQB6XbY_nOT-s9M@W8@LM!4Lyl$D-gA7j%bm+kMgR!iKDDc;g48rZ!M#=01ID23<}Q z7g<%EF0a=cf&;PKex46*rW*_IqM-ZXS~TQMyn8!tAn{DtvI^gWi$f>qgXWhfzB|Ib zJa9hWrdk|j+K!!k4(@)``}q(^&613w$q+|Kr7ezcd0d|P=8oFUv)83N;it=e@?`IL zA@+c_j*i&$NRDRay=C=A0*S*@_iE~uq5>TIE6SCrW9|!cUZe%zym@mUd>T0USF1j6 z&SYY>^S}wnUGVuss0BRJm}k2?>rjE-FDDz!y;I2YP{1gINjS~Z2@?57R%KDp;@Cp^ zaDAJ*6oHHyhF-zo7ySJB^9J9M?&Y|uGZX?hj8Qh2!ozE(W@fsv0C807o7|Df#(QW$ zI`J)}o%VfbYN~`$(8MEZcz9Ub2T~!an`O9;j6gpi;B2s0h9^gn`h@{{IUo@|PS2?a zC@`cW-*Wa_6=o;}G&{_KDh6F;G1vebX0~HuTZ8K${GlZJ-S$`khn;qSJ(ATB#UF%V zQp*Z!;N;a;g?A34$KZ?}9T};>u5uacGvwT8$&21!6P3nvJVUrX7cfKcx?J>Nv|)0S ztspAkFh-lrf0;*a|JdVKO-<~;f;6F!Z!TF*Eej;4gR@%vm|P$1#)U3FxcgO`O()qO z%dS1!Av#M-3#gfGC1@RYIR*XE={sZNlj9R1o6H!)xrelc>-$W+YI%p$0om$V!HTc{ z_wDt2pc^d?9ijq4aM-A?RX1x|&+6aQwtOL>OsMdFp9=k>j1{yk5=uZUXebwFCs?p) zoczCITnsnUfpEXQ!r(&oI*)r-Bg^ZA&h@DnwfmJJR)Mx8U^9rC<%eG?}cujfK`HcA^NGNOq_yamIB3m4qw z;`gLIdGbUapN1;CuODYEaQ-?p}@7{1vm&0(K1!{I9dK^wE69%-^ zWDi085rM@Tp}4^Za#+;sK!Dj!PEOyqSpgiw^LGKVpdD?B;T5e{C={fS)T7-GK?l?W z)V9!T^nPC}FrQRR5tuukHC-42FZ(dSnqpAm?H+&R?2WzjU@rt!oHd&@u+^N5`4`Za zq?=}Pk*31=qwk!l!`f|Ez;uR}42U#;2y zJ^(d<%;n)T(1M3SNO>2|0KycZksvD-(-r|+R5s%1+j)sO02&oc3bR{avGjWj)RRCE zUkY7zHlMY&_6L7+7i$74dM>s5AaA&34_V{mmmsKTnqw{CbtJf=2Q@C8Rcw2J{I3D7 zs+uY-{hL<%`wj9=EmwJZHVS$y2A|^M;_IcuYu)8!Ulx0fb8pUBIPyVVW|QSF>)a&w z%v6`rOH_Ydkm`GHYaIOgSyW`JgT4Ja>8fx2@UhqT`XgRUj=ZaA*3&$ zIC0-ScKr1#aFv70aOaMcA*^zeLO=#|(kvf2ry#q)ya(r21u8`F-PP`}H-3rGB7T-Du76Z+!JgMblUkhDmQH-jg2#r7V8tZDL zuW@}!c4p>fv>2(7AWB%|t{WR08~*W{$Fjs*xtcu&2+KYI2KmQzo!XyBwlziL@sjra z#QQt4b5WuwD}YLxADS|8b94Jma_o#-g-;4aXj-`5*n;Ht?SkMNxW?xhm)+d%rHC?@ zIw;GU!IEkoX4K^kX{-nTx7ED|j`$;OfZ~7Gf8Cc-7aqut4B!*U78~HEo=GhpF8l+@ zwe6E%&5R)tM5>)1Y5F;#r>e`?)AKzCyjS)uY0AGD+cPd4)E?s@Z5k+v>+&|sFsohx zuRz7$>)$z`(dt?oCQ>GK{WD*3qWKCFHqk#yJKK4v@{M&}k=LmBnQt$`QD~5^x2mtb zAX86ilr$7gnwK?Tg>geELv$!xXc|uGI zfDw{}8Zyo0e&YhoKH#6V07)8~R=IV`*a0uNN40Rhy`Y*pDf3ufqG@;-f z@5OGe29l!VqwM&p3MiA&#r*>V1Gdn#rW>ZxqpuCXa6{45)TH2%V)1Xj_u-_!#Jrl% z45#bQHx>9dIQhwT%EDb=*=>s9ij3q$sQmM!Q2V&??iH_&mnSHf4Hc{S)kjuXpSqWl zLQ^~OwyU)iiq}|KNCJrR#{Pruh(M-Ai(z&S3Xj4J)Z6H5fuGY2?=LPJ<2 z)NQ_%FJIDVGHj%+3b6A__{|KC07DRk;fB^fJOWKs;{F-aZ9m@^87B>yJ0CJIU`3Zt zI*{r7h&SwoCypPd3)fyosdU|O6u(jHX)eG~DtbE72i~>02R>0!2lg9a4dnpQ6hPb< zK79e#Jznc-Lc0QQA|@=5mCF0XhB*~*b!<`n2VGHiz~5W|%{Z3Z%|n`*K_B$>g_ly} zWH;XYeSU_C%h=usoeCoAO&@}yLnKSc?X-y!;1bLNn`OQ}8jypXcwgL&GE@-)Phr^m zFh9q)1AhtVdxZOMD%$CxvX{a1L+qzhQzNMcn~ea%VzsFFExc$N;Fq_iY(DcPy}1{V zEX(l72wOXtR^Y%_zr6ni%fK--WqqHY}2AKZCA{Nsap%I(iJjEF1|F_YTI%Iv|W#JxX6Tc0{82E=3D z6az}`W|GUAGXtC5TJ~_!4?c1;0*KUz&5+GJZaK>( zdfg1p(Ml{s1^0fFC;hEF66#039$s%00j&an;IK<6~8odf_@y&aY8F{@^UK(z z;FPJMd}~XN71y1eunD7H7*_w#Z^W6SfD4WdlL3Lh!{pZGH<8|sU#a~E62^VUPVK%B>#vFDmxB;gkoMXfA&$j{udM9&{fx*?j>!hN5D4|V9AVH$t>jleMSMB#XQTI=CXe(=i`=))m!G4xK119~%;Qgl4sBvz9&}s7hi&oA z&A36icMuG{9R0Tb;^I#hzcLCh{||qtarNS5S29Uob}QQmwVP-6G^f>z((cgrtNVTp ztbb6%2fNJ9(m5~GK|f>Q6L&Pc!Vj!f7nYFc1-vGHzb5x>;tH|XaUci#jpgtGJ>#QO zy+`dWctt8jteuY0!AH+2dz5aPsV!|<(8VR6?$4EjQtLL0bIaW+t%64aEb}D{kNC0H z*7mz@TQBqp^gWp6$B8lg^T*+fi~TH%$uw8@Yy!L(L6pmzt=70(iBOq^adV&N?1U^z zb()Ek4;D#m2l};;EtSTw;Bt)(b*v>!30g_x?N8ASA^sqt_!-feQ6RF}uWYbSBt~Kr zf%Y379Z`Ma*s`knGE&$l)=v!`7e-Q01gB&met|VgJ2?u-IEqs$JSwK{IMIE9og&SK zj8&&=`9(JKyVP8(eVdp+Kcp&ad zu$YLzrtB5rpN4q&rA#v!9^X0v3lN1eLs>+J2Z5@MNuePM2 zz_rnudrT>6?RJ{vp;FzbXNX!6DlXXhp609du@kWylN0X|b#>3Bz3lB#e{S0Tn$gjF ze}2AxpJ1yO0|G)sz|_^*iXLKb&N?h4bm7Tv2t?t2Oc|R-H}NjH)ZcdYh%avAR)Kfbo3Y?O6kXnWjuTW&M%!ilM45woh$p~HnK z_C(d~Y^c}byTJTG%d*Rh9&4)7T+M_3(yY{dB8s$+ZN=lig6a3gKOgwno;JYS>LYQl zX-ebX;qz`)-36mG6yH)`i^U=FTi9Hs-*h!PMj^B$(#AyB^Su0dtG3$VE?AG;$NCp? zrO>*k?fs^@=vUJ9ASWj}a5>X`&nz~S{?DYq1@beh}8DG86Lh87VSVZur`n(LMD-8MDrX?AwjRe#t|Bnz^Mq7 z4opp5&&)Diy2j2YOHMP6x#e0nP6sg&zZiSnhPw`GBiHF@jz`Q)zqDdNKu_}GiWKB7 z(Z6*ps7X=V9gEPO?#DOY-r2M^d*FOI;#WA;{hXb1Sz+cbRUhds^XsKK)O_Z0r1UdU z0sMC`o7j%Wy4#yQHciS25kbDc&YyCB>$IQK6F&rQ6|dU0xS!T#VJw8x$|AN#4{O4m zc}SK2$l-~~0iLsQ+j`-yYDm#FztXpbKJoqY2P7g(V}1?}INwc~ZH&@9*=);h82x>@ zdt~fWS?$7}0`>FO{wiNR4FaFOxLD$&ko8^y4vV!Xk64nlq3)c;0{s?+!WM?HGa8iS zl2VC_>Zo1Rfl2AUf$v=(@GZF}oaW`)a4=4djVKW_-LCkXJ(pCKB>$6T&9l^%oJfRT zn5Y@IocTgbSPF28(gzwoeSo1KQ$RQ1KmvN={j-0a8u;-e1VyF`&6wUDF?EUE(3@oB ztqgJq42r+t)A9N>6G3wzRcQny{p8d~MzGuaCje;9HeK`?A2YF1qoaKJ`n3@?uyGQ% zM?JIw;Gm^S!o3n?^%F-=lAu!RQ-Wqa9A6ro%C`H)TbY?vQWDXoN~$Gli|B4rw<)rq zz7a(_7nyI?X!u?MGA@3zQ0vakFPIAuPC~??1egSyR*8X**zDb^p?4F#=sC-VdcYQq-9_5NO#AbEB{$mdVlp?mlcp=Vw`^r`8X}^^|i$7_-3;u+@?*01V;sUXo zXAH1joc+6RPyw$^A%D@r$*N304N1&u2z#1{e17lx4(YVK+escT>(*mZI{&Gf>+7M3 zD&qsyer?E?K-%Ii<^ffykN&w&j?3!5w$MS^grHE8TL=vFjL{`Ne^#}ty5Y{L$)*=k zA8OwntIU;@q(36g*}0ZNYRuo^UuEaA0gNsE$$zX7v>x<_g6FU`fxyU+0UFx?7vs9i z#YpUwGQQ7@W5aoN;xu~V9b?x`q;w#ma2cU?=e%FG$M{lwaK)wwx*!PlY%s_W@ab>w z`)ELzwt&QlmKToda3-FOqO(Id>3(?@#Gl6$1h*s(KdnA^^8|5N1)`~_8yfWGB^g*Y;z2MvG`qN~_s`8CD6wz4(?f`quGx)89AQBtA zhZKPzsCkcRLbq^DFnQQf4GkAUUv%oXh$e6O|H#k%HxF;NLE#r`pR zg+IT0W<|p*))e89vUR?*e_0NwC8t21FLki?bK;P^O)F>0nimOAxep-=iGJUJaL#R6 z7!$FXT#(NosOqd0S1lnT!k$WhrMlGkxsBERV58n>xZL5gVO_($aqQ!169I$)>7j6z2#5tgjwhn}2H*q=dvcHetHGCAda`@U=6 zDF6GtV8?5Zw}$sT*(`eNSYSs7YuMBCJQ@(W_`5qQ(XH1z%C1^&JI}}TK(0n(R9x)B zqekOho$wcF%~pI3h1*caJ~6{!o1t}&9CZF^zuxhCk=5_BZJV%)!Q5_N=+3)l4j()W z7ME7zqHkw~+5tfgXCuzLyqv6by=Qjj&Q5uxl`X*XyehhDb#TrWddn%bg8UUH*SkF{1Ed@^JrY=D;_~fSFiDYE^P5MlO#JG%&=rUTMBu2 zhRhf8=f-s)kV#w4MiCi4gcg2(PVAy)Q30cYdEShr5sG!gkmI^bM<)eJ4bt7CD@|>< z<=|Daq3ePiKpNz5Lu^+V}JDkRH1rc9e4HjGoo#!XkzK-E%D-D zW6W}nMj`+p<3gKX$itut_7fcS3>DS@{LaPiOlV2o5u$zL2>x9c^fzD1{MG_Dt-@Fz zOn9D9d=Z=WCk~)AC9zdDYzOsBgP=CF^XwQ_v~knNa4U+QGk~Agwl;O3^)RqvLFU>?8@mA6*_PD8lEQ@xd9gZ%&v(c& zuTf%^=FX~&VtEc#$@*0J6P=#7O1%XSwb%O;Ei%%5`1r9pLRP;K@m?b|)&5e-(zfNf zrLE6TYjD4{l%i8XV(DHXB@WY5CV?UGA$pI(9}?ox9#$|-`EYHHmo=*^%GqewHO7Xm zXyw_NJ)^AP_IuqNG8wtwTV8@u#HP%6*S_5`-64kgZhvV!jT{7#o+*saZEmi5rTbFe z+P*_n!~BPIybiqF3KuI4nE$95#TA`-|7o$|k)uGIgz5IWQFyhP-1$GTw_kXMnVMeO z^wXOCFK07;%+zYvTH7c8`(1X{FjpeaBm9vLqEOS}I=>VtbR~Ph!@9ciqn5)*Ot{Ry z)Ywm+Ei7-?GosqK@P__V+WFnEiSXBm?+V8q56_seRs$4hYkPWBr!KaR7c5y)Y@@{BF^SzY7vhFEq5)>hl9d zMotL|420mK7=4Iogvg&ApV+*l-+O2K)_rYlicF>*g?EO%?nfodGH(+$vV3lav0Cpmgr6X;I#6yrKga(5tpu1y zRO*QH6NC~yF>_ATa90gZKRP6sUh0juV)$c4V^s%3=pIgHkA>#tS8i^G%|}r|sUqs- zO~XQql;hEjEiEs(Gl;t_g1qekmKq3ki69x=?{ zt2;K(CqwtPx~A$_&D2&Ew=-7xwmF4GJ(w%0rCl7CtQ#aAn6x;WuUqRO`0dJO7S*oM z19NMOiFK~<9Fv*T(T9VtGjea=ob3EMFsXGIh3xVt88qv0Ic3`V%X-=7Q9f;$|0o$T zFpQ2*RJYp+$h-VhNo(2UG-u7cRztMf_V??!@(QI3jcIlVNH%bv-@^ChU*s%zmeja6 z`#6MuKXK+v6i^@08+=>nGi8GwUSL8*-P>?3?rKI^-*Mq{Pkf&m-{3l zuCxvxR((4PI*oVrn(eI1{pLXlgFcVfIvW@o5>n^mq_t9s$HCF@0q;5vP=Kg!N$Z6-6JJ^Y^IvFowk%Opnwy*7QoFznVO|Yz9hxe0J+Jv0WuOzm;GT|wZY4J3Z`uA25D64$@-p#%<{#I%0u8n2}f6tLQdUUON7c9he zCmf7wa$SOqndxRfVBb5Re;yIKCOWuu(L->LvAo6eji-(9KN$zzA8*Xttt!~+ZqcXe zHN`3ZPB7Et>HRU$_!=8j8EqdVXX+}m{yu;rRQ)?mNgk$kDB*%O5aIyA%KGQt{24f} z^T+b%)~-!^7v5OHYtE}!!}nNxVgLL^99`h8q;Avf7rK{c^!rQU?&cO(&1WXG6$>UE zDI0g~)CsC^m=T;nRhJE=U9|21$D^8`ai^qJO$%HWm%NFMG$9PspP$Hp%b zyld9104@46nq7KtpX}lyk+ziqO^l#7Q~KqY;-ig zvK~psAaDY>qlPA<`B_?BJF{zgX>}(ojZF~2KxDR*k%RfrcpE%4Y5I8n%9X14?M?g$ zB3g5&d(LlXWy?Wn(}(0WFO0jh+MNJbR$JiMLKMjGCs|m`Ju<^Ju}aNZ2RVRYSEI6l zdltS{m+L$4UHJStaV&we^2^0E2@#wARB1Po6BTT0h%J>=*+b9HycJ-V_+WTieIHe( z^TjPpYznVz{FBBVMX&b+)_=0SyDTjjWvXA-Lu}N5;~OdRq{sgm=No z|H04Rs&RD{P6kVr3%YhK+NOhFU#vi$!xGz-a*lr4d|jT8YipvcWdCz#@#*=eL+$j} zPDJiT)R{r1?>-flq1C5_s;TuipE*eBg$wetM<913eZWZS$Ip+4sRPL`&CWrujcTyJ@xSaSPn3NzoItOhw(od}*wJ73pOOP_C18vJ@(JE93Z9!j#j?XHU+ zwQ%dCz{uj4oh6E?T(Y;NgYn_U)Fmx+e~i4n7-)4n*UxQj74P2dM_S?hg^-^_=)8CD z6{Ft8v0vBHZyRauhs_U|CfEQ4+g%BabV|y~A(@%ddT|mlzZC!hX~DP=LO>hXy;9r` z9x&Pxg2RZt08E;ot1{Of|*!7Az!Q_{*h#twP0SkN? z`e-q4*{ar7UQpjP1fXsE3nOY2py?i<<)q)aqwW&{iaA`-wiLj++Jy;ikPgJ{1=;K# z;ftnM`&I*;8R@IwNRf@Pr|$SLlY!|;NaQndLw-1 zjMekZ2Enps2Lq;0UgxAY)jZNvH+g8bL#+3}f#|jBTm_jQAI0_+*JXMQh*KvCGl^{V8R{+Zv3fn z7u>*9_4l*UTz6{WT_xUKPs^9$-9v2n%3ODXBvNkzd~n{~jsw&1*_k6OBeU ziOgHWrxg67Nc{`;gni5_R~r7BTh7SIS`c41miGH?P>xj@cL8*Qv1$|D)t9MFzK|n8 zaP~qkCj=zSCl;u|4vg?ZhkpfIv-ugcZFo@_o{)^Yp?`L6`mx^6g_-fpt1*K-`ueYr zM+1Z}lX!*Hl5K|aDI&nh#ZGOIshi&N{ml*I<`YP(Yrhv)kgFBKSa|LSBQy^ZqL&c1 zSc{>&>-%@&h$5mi{E}MSr=iovJp`^6LwiMA8=r>XiAR&RlC#I;c^lV$d1uM_Rfn+k zMS(&5D$N}6y%#2D)F9J*5Ucz#%Y0{-0#csPi6x{DwO#AgyIp4HQ~iE5W&CC{NAhCc zIlQEtJC_$<$s8$-i&dieIAF3ng}R({tz+i~0}zb$V0LC72i^YV%jqA?ZZ-Y&YyF5T zb#I8G-h0zWMh6X5)2v^7|6cH#DLkP&Bl~ikgv_Z>gy9!dhv&y)syv2iumN2ryc|&Y z(Z-hy6Z7Rmv6wy9P()%TOuryWI=Fkwid-+BW!|q(*=xj3pLpZuP3i4WES=-xfHhL& zR3e1yZMGdH9A!gB(1mIT=>0Xa^H4U8eb#J?O9`y)?8Kg|h2|cs{{Cxky#!A4T}5uX z%JJ6h?`+SUxzks)umKO0bEAx}+QM6bAJX?`Z*Fse{^yMYXcfPW`mO=q{c+gFHm29B_`gapPM0Y62?1yyayrI*2eV(iox zYK&zMl(7bB-eQKy5A}4opUFLJE({LXhxSWS`2d1Vcb()XLpJ3Via1u9PZHVbOOPz4~zF|^VQ&qcm4V$`NRs>GGP5mS#I|K@66)7 zCTFE?IE$hHZ!koP7{Qk41v%(k8Q;4?6&O-WG(AYG;vP@qWKNJb5C=@7H#7@mNiX^> z19S)S04xkU5L4Ck!;(cHBC{u5is2Z>GYs|H+>L~c^QeWzwb(N8+nzy2gs~QTpBi(2cyTyT*7nIkmE)DdF+yn2w(xXqjCHu&^49FtI8b8f3x3g^dx0$+8=b z4?LWP`RMt_`~cjzEKc|=HKg#9zf${_lBVLkmn%&~Sy?&er9qKu(P^h_oj*`1r2SMd zQ`8jinogxsS@~k}2A_<~3h(bB0o?9#y25@Z@?JZkz(1R9&70Dw-xIlZl>FuwdKNBRb9jqJ` zO+)WSoxq3@SO3=o))_-H1*p`UGtCctOx(W_v=N7Tf?gVt;v0<;Z!8oIeP5q%!85;~ zP{L{_P67=O`~-<_a49Mx6~9i!cdoF(9FkJYGG8IM@&Yl0HTaEm>wxiF4i4(Q^Gd@o z3N!>OH{L3u%NUy!_yW3FoY7uSH}wRIc0s+B`i12=ll68zGdc))Nk5pC9{fXg=T5Eo zqd`*dk`t8gB@&Bm1DhT-34V-d+3|y_d~JU6w_%NR ziDXXLaF})(ujH&HEd|%C7-pEQ8|h69`_2RL7Kl9JI7~nCd^1c`%OTO)xpOC!vOVa1 zEDD|B72nkP=T9V55fS;9i2Ud$5mAB%O##Nye{?BrNesRS4yS)35zGbEB4hj`kyr>H~n3e!ij zd%*QxS2`J`)w_DtD#GXL_~2{p$DAc!w4Qpcb%kU$TgL^~0d zPb(`c3${rRt19s6)6JB1fasuHsDxxGqz487h2yVSiBSucj)z195MeG6IjO6wBc>__ z@i$=$Lvs!;2WxBI3P^4GR$0HyPC9{Yyy$+O$?-#hwzz9T=T8Ick7c%TM|M_@(rQ}R zdnP>!mBJ_pGUib8$*Vi32ePxy=XBV5b~L6NKNVi!Q=>%}UWG6{0%tFqqa^`Hzsugp zWN6ABQ9B}bVn()$iL=XwuXlP^+(EeUtZT0Dy}G$Y&Uo~goah{<)eIzU^Xr4 zg{&(?;xH_>=|%yu+HI!5=81>n`puiPVXGhA-sHf3|3LD_ZSRn~i(5;}Q4X#uo!p<& z0i7wNg~T@Wv4l&62Je?bw}QTB!hJ+8D}p|V#(yW%$ApMYnB1--Koh>U>wVYDo&pn5 zGMs*KB`!m9dpLVwvEG+Iaojh}PV0~x7Z%2rWA=JD8#`y(WfPuLb~GudqN?X5mkft) z+-)`AIbIkagD^}H723U4ImlO_bqNE`-JR^Wz&7tfOTq9&U=chbxH7NyGkK7rxf%0k zHQKoNg-OWu{!K}VJMQ2Rj(Qum%n$GnasHX)92LUgiqNb6Ew7*&JcbG0FtiO>R~6~ddp1be1v^t7qj009>luBBm{Br_Q-IyBq`#v~11Z3AlZO!65vgeByWu3_$6%`@I`-saUR5~XQqgII>jkgk zQzUql!L+h*6*Lgq70yt#YolNQ`K;l53}Sf_YBslWQVMpYKn|=Q=Mx!bWCleSVGsa` za?LEvPg*T`G#lJO{JKmI9x+Uihxf$|>Syw$_Fm~8s^iEw)Gz!=yRBTOeM>|BGh`KfODCo6ee}7+>Vf%ERx03D96A5L6uH`eEmp>BZ!Qrc zdpaAPvxE1JJfwQ)#S0K?9rX%%mBio+FNt=A8MGEVPz@pJTNNj5zRIpXE&wWDSRU^p z3GVLQbwo7|5fjk`1HtduGw}U;Fw~He1smb9fpX>L`}YSJ3{c(I5KSFEjwpFxg@eS5 zXBbakyXP%JFrW@Lu?k(*-Elp~_(4=(K^9o_W6%)ULxcIyESnLA)}`wxhR1#xTb5qr zvnrkQl|7Kir?7mQt}?NH>_(w4Yma&Ew8L>`pb({G-#fd{JB<8P!J^n);$T# zCKS)P-gl88(uCKoqo7n1oE`mYbZcwXPz4d$;>0kw7lLLNAuT&VzXazy0mL8*0H`Ak zbB?OP1z}_vO7V`G>{%-Wt=^zAx?9f+o%|HY%Z{M(P zZ_=ifI7fFQa+8!3*flD^6%rj7zC6eBqui24bW8eyDE63(&_XN&^n~woXo?St2}al` zY_N){$;MD9s6f*|b|cCkUfnW$J$&gc*=bmevbdYrAVG}Mj);wT!;E~2X_O~2MslOq$Iy&O3807Z|=l(kT z;lqaz40C;n-(Q85UDNG{wd}Mw>ijRzZDgtcDU??G5w7nnR$MfeWWzL@zLo=}}^-YJ{ zL&2q_i2hzoagL6T%xkt=aCXvB&@vF;01gx3$%_~Y39=#&EGaE$Hy~zztQwMU!@$Uh z$Qb0kKtr|PJqt+*p!01`p4`w({%?O}PkSmsF3y(L@BLq~=Lw zCf*6MAQ|!U+{QJS(gHu+p4H@-wp9UxL2Z_K8eVHt;SDw*6ovr6|H;H;61O?LUH- zw#FsxZu5U~m9BS%qM38lrAxc00!R zKQgKy+hQdZsUH_3v76qzl+ErqWS0p3rT`)#kwa;V@{)(2%3zt^U(%}EIS~66Mi(P= zqHRio$ZW=LSV194zu;b90$TgG=`adNkUb0#f#*7u)Q~a8C+!r8xoetfMzDUI%y)hY ztbmcupA>ya+|EV+f~AyfVMI1byKMyRO%Pj@88drJXc|fVcEj<$@4{PqG_dke?-_Q$ zc&=S>=po$f62uvgvah$C)eIY(C;;%;@r-n{X>cMIo*!C+Gbh_X(;Q$$9-uvvbVyIn zH{cVD_uYlUDq%yw- zE!i72KalsyST`b#tt;9T&{T}D^^!dI8+kb+ZyUQ+uwc5cC zNb*H@;`{F6zIytaU}?Uq6XUMk#lQJvRca9}rnS@8uhe&D8dyV)Rq4wAU_j;1=IfqC z#uv-gTBT*DvVf~=Y&mM6$@;gwO-k9y-%pdOZb!Q=JpU$?G9-+?|0s#|s;l^9w$PDC zRDNixiAn+GZ7T%*;i&2l|4G=sCxWC%BBX@{E(CL24{VxnWG#i-mh_HX28xc6E|@G* zu-Q^_R;H+sq-iF;L!zHTU!a{va~aYvOcj-Nbt{*AK$fdkts-0}qO_s_vUP&&nSzvt z+5$Lj>Ar%EhY{8|7QPIn7pLugrG`}lQNZh+meM-(7ri622!)4(70H`tAr>1P}gW>FA&NE{UxN3tPSS^{d$+;v-+kp1c;Dx6P6!v#9D5e{w zMbv+SqwER>a@*gnd9B!kWIQ@>;JvmLEv#qw2}TkYK(Z`e?J1HHQ$E#P4j+85hxhS0 z2c%69c?SNO9BhT&<5Gxgis@5|mi#w_99Tc{QF04`iEb&>@Z{`@a^sH5B zrqco%4Tz>_)K<6|pL!AfOb#$E*P2OyABKn9)(IpJsprb2whA#&hJU6_&Hdyqx_kC+ zPBVX#{zBlicEG`*j*FU++dejp-)h+y{=s$2sc)2NrnNyH^fti@i7=lqxJ97KoH`6 zth`ZL>bJNGNM%(`P34y_f6KK zW|ii6Z8xZHB7BFhQh<0Dr$*Mpg#>>Y1As4HKE4!#6s;u>vpWWu6@-C6tBOM)5~im0 z=p*2ol2=j+DkyMIdfY)X()Jq{311y-F-41Ux)3K~Evc5P%kH5FQ=B zYL}Q1XNTkiCxoK0o6FtzyK!ao!I!vIaH)IVGm&r2c6_Gur#me|+Gl}&U%#Zwf~wS3 zKX0j{4^;-gjj^`l@M)U>2O|@5l?gh){Q&WV zq|*ZY1#|!-vyG!;ropApPk`@S!j(Ql53ZiFcW{ujELKv37=VNgdUzb{-E~nwK%lj? zmE4oj7K>mMV$nr`>B8hi@h5pX$i+0_{(FE(jp3UhNoAg%Rb@Or(|_%-YdayanDf3* ze3fz8N1nS0vH8XAXl`lQ3C1-g<;K%JGn7S5!EYB#?4eeq_-go8e$g7yC?#3nKTjt~ zZTgu~yZ(jW`c+wyXRP*Sl-zQ6sm6s{oW;-A$**H2R^9P?Qx&#)qNt;e5jm=Jd`t*rdFA3Er* z0^LVPY4f0Ya?7jdpd#WZ*G*pc`Mi3aqLF2E3ArA@>iw7zpl^EzM`6f~DV^h89FMF@ z0}hQlN^LrTJ2T(iapuCai+7Wc4bR9+&vw#na@}Cb#+G8qnn*64z+GOur!QZp$DM#S z@4in%FGist{J9zd_efIp3i;_Z)~RsDs!Y1yu{Epo&;$Vpwd2@Id<2qpTbi5W0+tfQ z$bZS0pPTCsB#!75(0A;Ril^c(AmZdLDe;N5#k8F3GwXphM$!Q&{&?C{z>es?c!4<6Rf+2`bZyft@(dOf*G!)I8u}TE ztWvo7UVdq^{56m02L2|C49i;Rv4jxp&pLjvw=NRO6zs&YV`pNRSZ;;|UHZB3*r4n0 z<7NT9rJtQCY@<5)WUO4xRI9Q6B#ap=ebQbh9=ow;3A)iBR%)O(K+-N1CoYk0 z78Nt_iV5xJyIxmDTTwVEWKgr*NFsWTprq zx2QsN&+> zcvXS+!;?f@HAMt{8T7K#kU*20f9XOaW>!Rt%5)UD(1;3Fpi}NUz zcl#bn-K4xm!I-C2C`rRK^HoP*h8#T`qyCa8vfm?D7y-WRib1JjKM>~5568rLS zagdWm9^a9}qN~_XeQjhToc*XT;;(~*OD+*9Ncft|QBgH9KD7-`Z8p5{l3evh61n9< z&kKy(+)WF}op!B-xQX-mmcfd=Gpe^#E;1@KDumbI@fEraDzi}OQ4tksP#uorkV?BrGm!>ODiSJ=p?O4!6dGyLpxSA;pYPtB z_W9p+?_KNOweGrW-FL6`p7(w88=v3ve235H^Zh=nz1kgq-Lq#;jKnPpj^R(%a$rh^PhFypA+Si&($yzHV zFWPzKWdwTYtaR-C;o^yf(gSDTVN;QH5aC{!Eq(Drui60<>x)Rl`Z)vdIlh6Pzx2*2 zI%srAttk_R1C&lU7xilYL{B zdxy>#aLM}mTi0K)mz69RVv|4`4Y}1zu{-xT?}duL>SF8JfwXVlPS`6B*z(Rr;mz8;3H_&ZlOYl;cPE3eWVB(Z7jpHxlzVO$kuKKX(5hITzJqh2u&*ejY&Eu0OF` z1!+bI^nopoMO`tSg`eppL-*p$*K==>=S``2UpX4= z#uluRIGvc-Ze+BU#-prO4v-K5Yp-7m5y^~NANO<9MTk9OxDyLqcCB{pG;{#N#?8C$ z7Q84Z(5Tw|ecqOdp!oC__K!kMqX>gaa+OaFyn{WD^-4fxFlOl#Yzxd#1!P0X9^@uB zo%OMGsq8#cjn$v_%t^6B+5VN-Ya8!{=0J|^QCXamnp1K%>%sUgF-7m#pcUvVm_?QMF?o;6vh?*4exCh$ z_GW+Um+=TyH=y7XJ7?{z&YBnZBIbvjX_Efa;&1Ltm6_nR;1`pExIGDx26aMf63!vk z98?$GduM}XRNp@68y3iCi%=|;;d|ty{m2aEtdT!J@aF!4-5OSP)fb)F)zC_W`HMuO zSu-Wi?`je=ik0xXC;X7K+75qHn}bJ9-Vrlq&z>TA{s*&+LgNFLzuWp3L;|Ib(Y}8{ z^j)5%u90G=%LK-yt;`sMYwQgqQlj=qWzNRi7rl8?SyH=c={$`9DW6Y@nhWN8yq**1 zr}6xYO6!0~xRdhZ4Bw_spdBtJo6f?&Y^}TAS*c>Q5SwNTO%1Nta(mxiSYrbNJ8UbZ zR+9eH-hA=Z*iTFz`x5>My(NP-MW13jPxRE(zg|#83$BY=_PzV~aT0)gP__B1D;_>1 z1sWym_Z&LBz~bG6@?g^@z0LTd#Stpg83gq$5O%9wndaPg1c~N5VAZc4IHvB1r0Vw= zqiki`o)n5A+JF+mroDaI+{nRK7X46eX(F;Rq-M>alYl(TK-o4{+;pg)$DwFcW3`VO z$PC)r5*^tDAsYB`$`snMjJgP;`F)JD6>l^9DB1>`tu@9N+o-+$p=jUQop}aJH3C)1 zE1mchlbzE02Vk4O9XGkH^KOoi>VEU;wQ-@!5`Uq!#=0v0iBVAg|704Y?t$&#BVD3W ze)8Vk(0x}oFA^Q>V|E=x(bVMV_i>rS*gH?$@2aLC_8d`Hs-0O5l?laY#}hWW6@VD0 z(&3$(XQ-h&IV}MoZ&yDr`vMWi0kazetsxk;sv=-{)c+L~OUuOiVgP_rbsUE5sKZ?I zYj?1}E3hDHny_H~iHqoPgP)(7X0RO9c0)MNK(q24Z#Cd%c7;%{^{Nz8s=vY}NRP)+ zf(>!RF5x$5atXv`90{#}5CAy&9Z=MJ9G{E^QGFGnvkr1o%^B~Z&jLHSN%vJpt2@#} zY#6&5dj~D5T!2k9K6iDUM&`|Uhvc1_R{^K_Vg352>Jzc_eP}aE2GwJ*0u+;=J%w}o z44{9L_ibgtr!A8Y+VL1)JpUr_#Nhjwf+a|>(IsgkUIn|5Y&wYEUo$}G z14)6d;M*_u6<|$x+}~liHiAiqRF3wi2EMvm_zn7tprltA<3652a(|+^fzI{U{Io)C z_xDE8!RsBQfL@GwQk#&1#$Hk59x<+f*MX#S_Mg*5rZEVhzjNd*wveH{jt@a&c?QL8 zXxe)E%9Sg`8ip^IlG1u10*nWpm>8GV*Zs32Y2J3kQ$R^(r8rVoafE!A@=R2O6|Mt3v#D)sgCT z3{~Jzy8FIsiA06JplFu#1Fr#8`+hyX01gw7_IT)4XAeO8d=Bg^d6_(mSm!bHq}P=( zlYU^J-`!PcPsAd!!Fn(lsKG%7`*(i6`dfW8X3kH;LOll%{WnnjG5c&>cA8%VW zD-6wT|MCOIHx30d8N+%|njpI)J4D+MAWV&O=FqUyuWB8Q=zRVB1F18Tghy_0}m zY&rHvFK8W-RnP>TWX9b+A^#bwK^afXfAj!P7!YvCO=%D^(MUTiM;}{Pe+O!yO~1&C zHXx1ep6pzl>qhXYjqN!4Yw0OkOO z_9mRwqgAr=c23xp_?uv8_9^wlO(mw`v!WA)clVk8vN$puv+fWYCY+0i5P@{T6BF>) z#Xd16?dTs)k{Sjzo`SE8)Av_x=2qsD@*l<2BVR0h zE#1qKBz=Oa zgUpp9T5JuQs!QzJISap>d*)C)M&PpKomY$Rx29-NdjY0=TiWxKm)iQKRi=SqH@^Vf z$@mm^Go|y{eDhB)VmbA`+igpKooEWm(C*aV^CZ39WUviiGO*)@B^VIlOCirAHMbT?g`@<7Vo{*x8IFVD_3W+3#79ZRr()Iw`~e@FoPbW zdoU3S^7?_6q>^B(#2|ECQDLPNt$AZ|zwM5=QwXE9_0;D_6CGO;(zCu8CH_pe*Zqt+ zm>NG_)J~QrYCQWg)AJBmK!H`OyS))DCKKFgW@Ctm9y3PIuWB z5nOJC{O5<`>}pT-Ma9L%HMiZmbWCO*gP~^`<=@rwe7kwyTZf1`FAg#!6CX7J&a*EN z6x+>ybm-6_TU?{*{W+20XNv{twbA@_eaD0Dau9vYJbjx?e-G)bli?Mx67q8m?RQ!t zHTZ1c^f5}!#^dZI-$x9_R<=Y&Jv-z5pYywJ8V!j#)Shyvap^8X>y@k?w{2NkP-n{G z^P(}NWG$!IA?59fH>P<@10|d^7(2AX9UFt77&)An_4Q+PL}NpsQL)JI=W`EwUt7t) z|6^84t?So0nd9+P?gBSqmgprl#wXg9D6K_Xmu=7hG!6)lbgPW`!s#|v=-X(rLvcmU z`{{0%QFtzpsU)yC-WC$nPb{WIZdYFkbIrDJRUaylqu(#|OU2tF8QvrPm z^U!0PEtO4jFDVXwe%#&T1(Vs>npz@N)REa&gGqFqF0>P~n!yk~Xq9kVTTEPB7VKSN z@amh*>*zX?uM_XL2OqD{i{{Qn6c4Q2-th_(EtS(464;m{;^wk!d$>VQh3;0-Gh=Xq z-z|8cJvut(m@R_qPgs9{9PF1(3o14yW-D9qYinx8mkRwRj#4(K$fg{4IWupfh|47$ z5RFH=wBy?&l_>QYd}?Xgy>ln#liu}jk5to_)tA|dMIObmj1CI``&RsXy|(UwvxBYX zD&j@bupkeHdf%woc%D@~UWSJfEuL@lz$f9(>$tq6i;x=Z4L{Xe^#b~_OjCx%PTat* za8(YI1@1HmfT4i!B)o?^Gkf4=k}amRIpPJu)ra;ZKIqC{fDeIjJm@9AY;DcSgUbEy z&VadSqH#~m?0p@XzWI?KX6wFWP3cV~QKvxM6fz&1Ae4u@i$zIgFscBUc2qQc2_sbK*F zyR@)2{krGNr`46F$s;XaYe!tYlzLx9rqkUYT6?Js_g5c%JXKJ|)dd?FMPOH074xv; z6^fY$)8C#bn4@FhV;*s84OTXXQ{omrapB2d=FFL+X5)RDzJHoAY@$oJ)2GsJ?$4zaU5$~-z^Cu76B)VN zEJ<(l*^QQ5Mo+m$H|&oYW@W~^)01tB<(mqhuV3@Z_>A?frve#$HO{_Fyw@EW-N`r5 zpoYOHof3cN^`Y*bAxs-F%*~^8Y%9#Oo@0Z5cV@?_q~^nP{golX!ND{wb;f~92pQH`CllSV`)47b`&7iis-_#DT`SRt9 zd|$PlhZn8Ihk>s!AyNxv3Ye@8b7zNdwqrOK~MR7BYX^PgwhvkgvNhgCN1%) zT4G{iwpcE+g}HW2dgt!lzSq1I;^eYn3htv)(TOMY=i(TTq zc;TwwHev1Q^#qK$^3)LnN6UoCGL5R8`~&W&JMT8~nVbc7+dlr0tb!}b5BJxlzSyNv z^xT23U%#49g zH|_Ax#of~V!uV?ye4#HBdsXJcw}nGbLH_7xd{%b>>)5D({FbEb8fJ%b-9_1y{tq5a zw=4G-;E~yTbqsGew~bXzz4K(I#&w`{`U%u?NlU)L@0%%DOOwzJtJf4ce$cvOza?fg z2A99|v9tQxmR=(5+UenfXeKr(spO@OQJC*`D>n18dh)-9?uu+zjk zx*1u=P)+SStks2msEgnteek_Z%hcvd^8BC68F-BaM<0%f;w%j`kSWRjMc2?U?94QU z7`c(|h>>d1^e3f?>oB^QRD#%n&;l-FO=WEd>$(kn@$kZx7!SvyuV*xU zhm5X#abITk$WTXW<$mX=6x()bMk}O6cV7MGQH=$_Vq=W=3&x*4%5}+fGDBh$S%Dk( z2D)X#YfjdA(j@I3^n#DLDVNn0?%V}`KRZ)WfV=*(UI6;&el$_U1T8fYS(tYf%9hmE zjto2C!BZoz(KLW(y}eM$$>h!9bjhNmyB{t%y5=CrOW_%bmKIry(kqeaPshFs-pXL8 zFLZ9bW#I7kIHyTvBuHn!<5Jjbqk+t*!2Jq{wvKx#4>Y?pt*>37u!fPQE8_C`*BXSJ z2&R;QF=(>)Xlvg9b^Q_T5GdM=Ttc4dv!pP_OHlFTQFc8ZwGcb0q~wd;Ec`M6AJcm0 z%@O&L!ERq>dabMTG^L)>$aJ+?c4Vz-=A3wYyfQw$#)(E!_O~a0B}trKnnzLTV5hfR z;h~!e0r)iO&Haqh$cyvWN=1jA*oU0NADD{ynW>Owy*Uzs(8~}2L?t#9+gd#Si_3XE*qGnyLzn5DH9mECM6^yUSE z=Ak}#Qz}e#Z)|XZS~5M zj#ro~8g(Q4*RwdgC?k4=2rv{EV6rGVYYNam!b0!t#zSq(Laf>67CMPS2&N7-kJP>9 z$VhA-XWI5+_x2(rWaYpp6_Lov_mpl-Ems$h0&)-=KRD=p5>slIzJ3S{i&AvB)1mN_ zAI{r%)F@$Nvb-d>p_XqJO7Rd}5MpBQ^1L=NKX;<~pURgNAtGc|e6z!_uD zW^R84mMHE(6x zz1$#Q_63CT)_8&_4sta(TqrqJE#s$X*YMmNGA<@7!0<#B$X%D44ugH+155~+)ywq9 zx~KGDwc=m(%z_-1^HMP-ECC#`{j`3dNa2bjr$bYcWm{?JU|M*2RdR44b=ir+xv%YlU2&fYgR%UMVhp2O2brvJb&jRkne$};R6csyNY8@@98 zI%L%}4PlLK4s;i--Oddb5a47WWoWV-`g8sf-GQ@@N|e7!P<&seA9Ov+XH$ZX z4>zpcaq`{o6WL7h#Je99Ble7C6o#1w8m@Wy@+GIg&RZ7fvEx4H=bCGmat8O@dOFwW zL30A`2rIQ`@7{(jDg!$$UE6L~7C#?Qn`EnJpmTnR88^~)d&2h0u^9Pami#LtVP^>@wRax7ITc^JgL)FuYJhks_n} z06w6WX+zWAh)N?jCE)WB(O(D`P_Lupyt%D(6Ynzt95=;!VzFWE)*_mIV~~J$#I{m* z-oDyEK^fcip<~&n>CUghY@mgWA!u?ukR=vGjYKs%{5g_ zz=#D%7Q%eB6k>Tn?@AGB!26}rC|fQY^R3ZxsCD_7sUpgXxP_X{`DIrS;+Fm$hI9)R zTN+q5{+K&Uku5$8bnOrs>EA+&%KGm6KQU=cZF|bLn8VA)#>W>@kS&ot*u^t5&*s~N zKG#~0>4Pw*5$FNuZwCqzdRi)_F_8@uvj#rBc=^(l(&AHnZv)}lQ@%Z5L@$hYc|RMmm{bx@^BJX!Ly?9MVnDc$e4<5SOw8pvXG|b&>~_C)NJ=`Z z|2MoC+Bni@6naE{2IVJA&fv1Z`|bCv5-X)J7sw4-4g+oy$P^BCCzn=bam|y0zowXL zUip;o=a>Uw)M8`mMPYqWhL8#i-AF)?RZ?ca;X73~xX!H=sSH-Je&fkIXzAaCfUTIq ziU(iY?~|LPylKJl7dvC2%T)lH#b9_|NAdd`o9t7T%JA>NrJyY=@2@hCkaSuv_xq@8TgD1i4YHX#c(ZeQORB!vwH=LQ4!@E3MfD2SW3agUZ;LL+Ob2!Q7SoUq;PM3{?@ zp!+P_<8r&MadR*3fkn9#7wpy6i;~NRFKcvaYt>QnS|Fc-o9D)Pz`v>2rv^kM>nlC5Ai;%(p3CdaQ zxb~nW%%O&dgkZQ-PgEbv*(KOYND)B?bF_8e&##0aJV9>w-I?iF_nw8TeSnVWf)NlC zpWkFO3BbBX4L3t#sCwK2^kaJarM7qxK(6A9eQTWs3-gQ{ws6yf76cXOiyj1{k}(W) zK#Hs&miHAH@9WUaI3fw?{``14IVAeI3pAWl89ZRAm~NVkh$!$GG7>ST$=o+!G_sIZ zr#us~s=pwfm`mBYxAK4}mWv~r#p{DR6Sbn4p#;H}BHX*$7V==mi&i-cwqh2A-So{O z$54I};{ZxQ6a%vu1H;I=4Ii*_{^7Kp%7}ajfl|vED0k zl~HpNnXbBo0uke^!9LOV2s3Xsi!ydp4EN`#aA!I_Ae@sTq68uCZEh?NI6l<8lYu>y-^r4fw_ zGY^Sj=;pBu&3NnzA9CcLrBqh`htQ*cPhzpD=org@#on$mB*)Eys?0(N!>ocOw1ah& z|0II9V=>$KC|MC|6KYKjRkHU;gsjZp*=p(0m)(noMD0c zomSXne$keC@cC~FtO2X#V&`95I7CyRidvwwQ)$tV1wzp0;YFE~>XJAdMo3CTr4Bf?xsZCU7OLT(8@a5)EvQ8;+5 z1{iQnFn>5Ou^LfHY+PKvLQA|ac@kJ5eVFZCFs!20@@M?9ajU@fwOh7JD{QE$0&HuC zaA_%PAv`=b!%{+cIFR3=qdnZg*+~v6Cz?0LB?wZl5eh7iEJ&k-mPq8mF^rK~0_%}U zi95I5)}O%x;WGs+c1@ z1^`D~jHtq|j<0Y4^d`PI5nB%~g7+>#Sf^tz@e|iep7vZIF9Rd_QgP^64y9=n0o-cL zQ5imV2NseXwaaj2+i6Fbhs3hIaH_0w8Gy%&P$5BR7n!^nWB~X7yfPF#Z8AK0<=t zp;AX%yYbS-X?f%48tmxNX1HhU079CdpAX!%q;sMxqqICNE-qK^ep=DzbTgC2_{irB zM%uPHEbZofK^+o?d+q&FZecq5=NlusNl3lvSiMtUUV3;TqBB;=yubM>4t=k0^Mp2Hv2S14mqiWI51n zuucT#V{1F?0%l2Oj~;9-$_y^i1f%GJ3)UE5Uc+<$Bp!bW+=l;=vw{Ezkm5v054B@K zs2_Aw-m{lxi?i?Aij+i+>zF4&L9ss!bBE+V! z<`%=f>Di`esm)@~HFCMt!vS<04t7X+eOub@HB~0QC_}u*OMxX+k@>kIxgpYv7ONnj zHAFbFISytjb0m&wnq<&$8)q5V<}H=~6mu=5SfQmJGg=zpo?pVoY4S{DS4K#JF3X2Y z-+r+xYV51c^AG#sijmi7PuKn}W0GarC-1mzAOp%IUBnj(-h2+H2DH+<;6SBO&sHKa4;ki66mDA{BWoj@G5E+-XJ5+yv(BQ9`BHMfzkwQ5^% zam)!_SmkoybzlX|LEUp;Ov6@J8l?7c5P`#{N90q?wj-W0fVl=oAGyqdE2N+9I;{Uj z=i0CPEr-dgc4qiK$`2vxYu{aZ1lBGraiE<`=36KKiW>xc`jqii=P3ooggFyjf>h}@ zuM$N5K+1U+-4{A_&8uI$RfbQp>y1VRbXOp(!VMIt?tow-9K4~Vtu#+W*&0#U^v_BJ((2n zHOG7p&(L-(3KfEH$@95*qHVzH0tZ6Kli%I8sde?*ogI!@&dkFA?QyyTc+>p+(g0Bf zvG36tV2E-QjuD;(=U@9tn2-qe9-Xf`8Vn0{^2#^11$BAW-Y0;@`c*PvK6?0qS-tn+ zVJ(pto{dbW98YytGZgsy5oj|bjOJt@UN;9+7geHTc>HDLlJXx0=dLHb6(%wdYW~I+ z;d3IcCjrMKamg0|3s!JNIR>XX;3*1|4?O0w*Z|TIUesVJ5PQgAn8u;rIs^WcJ=l?F zoI0X+NqljEFNMzC;9JEY5|vQkCY%gL>PKwXWI32Go@9a{B8FC@q+*{{0zHcu#m3)$ za)}5DSD|Bs78{(N_;lrdV*)M8n^FUxii+@VONZ_-1AoxCZpaFKwh;${ZN=iJ=*z=v zD>n_}UC&+S1^VXj5)eXUE{o*xB^!qN*8WN#Sads55eJPr;yUC`sF`p`xJX(Npl`O;w%)FaP zS^Mnef-ms~OE#IuY$UR3ln||0rxvYIvzdWwx^M1@RvC(wBM{DxAS4!$-7q8olJLGO z6c%efq}-0dD0&DXPDtxp!UUb1=`V7T6nMk?SX;W{+Xv>8tE8Ar&wf+zUu3N5i9zEh z1tNpG#&NV&Pz z1S>)Q)E`ah%v|9B0~=L_JH4O7+LpTjLIl~p3++DbO?Ze2_j(LDyYy;Xjm=&Qf|!&o z0FhPBX#yK%jnIMZvVXyRu&CD?bJKNSt%27a zCHp7X$>=)rtqrqAfO#eGPWr(Sz`{RPXLMI45_V_VYLeV_oewoBq-Yzw#d21Fkt=5b zQmop$X5azEJjv#Q(Gtj<`fof2K%{;ED}TzJ^p+EcyvI%F&Fb+ zkMJQm>2vk&67$+Hw_WyA7{*fe`gX?JwVZC5S0UG;n{uskcrp*%Md&T^{2&6yqBum6 zL(&6132jP(aC_BxtHlLA;|3b#Vw({?J_wP|Olp5Sk+{Lux6AKKRG9j& zncTZvM{bXHdbqaUTeE#-ZqqHshzYgtzj-8Yt_P)3_|2{(-h}Wd?(zYe%7)slsM{Ik z%sYg{&-ja&CRc%1+O{?2aG5;!6#VC1_CnYy1B5P7oDu|p@i5A@pA0e2x#VK{<|TG~ zwB_|>+b&<)I9*8&ww}d?w`l?pGFZ_;^O59lyr28o7eIT1h@)arv}*v53c(nsqN4>- zBM}p>qc*~Lkv*J76L=ddOIPG1=1Bg-k{s?WOL@Taw{*>i8@RG8jQ^hCr0n4Vd3c5mwoyc#egWt883KamKq6$lMZb6_+5{ryQkkP2hArmDmm>>1a7@FFSJ4I#n{W`|1tV#a<@o^L}5F!8>m4f(+$A9hC^>``nx&cHBrxzkaJxG34Qx% zHY2w?MMUbZ&NyU_gjJC0FY?+>GcG*+?YM?d(2L|lp|?92+Cn8ndODP%;qJURg?pG> z3ee!e!6uawtm<7(6{s2`WdTLtlZzUIbNPqIGbSrbnGgx0B+FBQ-OCf8u}ed8uf1aa zWxiLUV7rO9Fv@hxQvpeuK`l#WxvZ02=qU86HTsXT!M&u|`1Z?xM9>SYc;U_;-kA7m zlKxFvPA!>9Yt6D6>bS@{9(>C?!B|-d^MnMA-4Nacgtx(wasa;t0|AL|K4}+&M!=qk z<~&B9s_-1;tfWFf=N3b!6EAjdX+X5O9lbLyc=e4>jhLbhIL>~7(9UlmG~MZne6aE< zC#`kw{~*PHgp$?Ud&dbRCPGCm=c~Hb1aYI$#kJBKmky&O-CL>Tak|Rj(F!BCA^8i&R`ItsFLY|YdiLIr0VH+`IoOVg z%C}hT4gh`Rd`WT(BeI=bk5VmZX-Oapm`iQT;AYS;VFwejZup|L+IMdqYKstj214By z7=9Z?ZP`O@Hn1FDXA9dT!gCg_a?mt!Oigbl4D=Kcl1U7G#TRh6*JPbSuIGxgzrmN=`2G?=CU3;#PrkBeweNdc_0MoJ> z$Q8b7Zs_RTP7NB8ORn-TMx)65h~b`S@*YJfOC+jM+!K4o>EinY@#0;2z;`1=b9bdi z3`ro1uU}A-4hq!(44vMg*r*F$>x{_Nl)w5$f$(0*UBHIKc%T2COsXe{`LcSj1$3`G z36&6!2?*cXDSF0#?WiT+hrMIW8i0Nmk>O-%@d*UQ;&j*eG_2uGFC4`KG0%qYrk6>h9IQ};Tk5R`keJ^-zT*7y1 zEBQ1M;eenDM{XbuZ*AvNgA|xJuqtHu9$xoUPvN^qb#m9)aKNe{u`i~$4ApDW( zWfG7`#L1n&WE%BxS6Bq^kUVC2gbSO7UBqqh7(qQ3L0^C~Fr%yssr0EjXIP1Hpg7j| zKHjK;g1Gx_hiiIs!fFkIH!Z8N7uCN<>9X{8l{Io(I*vOL8Um#WHqBjxE=VO1)5rp5 zw)s-*uf1;tafNGMTU@gSii>6dr^?<~_fjHNt}7U?IQrdJ8a?x#AA*FgT|dbP;(`=l z3B{c{=&X04Mq4lT=FI{$dD5U|ng~q599$B{KH(#eW$BQ2fq{ROV5)0MhB0lFXd+PjLU3%0&iP|CDb+W96$H%yTgRkgHS1!U4Gyvz9DuORD1g69jKXdQmcMIjS zW*`h%Y&bD2t&5{WtSaaXpg_Hj1_ze_uEo{Y@SYC;h+tgfpw6ZMKGQ0C5DO%$JFH{2 z@VRAt*`%p;uhn9N-!%q|pLpk&KH2Q{*=*tck$`PUNgCOnCvt7Y`-Yya78z-38*Ir6 zkxe+FUQzSz+{fGbuWRo%dIp|r3UaFn(@uC}GL&$lWTv6>?9AhgpYYIWV8+)+aZ&E? zrU_tGM^T{dz@z!muCHG(^OV*a`5i14^n3Z$rq|zUxVxrB{>$=~=p&!%%Y&Dc++9?w z@Af9ip!-GgdfPHf=Nktq;+c81Hks?P2OclcJ5)8tV_2V|`ul!Be2_LRVeJCTNA((n zgMD}T?#t|J59|0{FlK==xmfM^*wN^IHvS$_Q_BbPPDswb&+NmPh zVtr5JPHjtnl419drbFHqkq1E&hfnPN6s^gkZzrW9|HZ1Z^}&d=jA?nl;!yJ??1;B4 zq|JUuh49K`ddVv=MWjlN;y7e~eH*!?>c?yY!?GD$%dd6!NK_p$&`iXv5EGkB%XUXcsX-Q77*S6& ze+2=Qk_4tvZ{~du2VBo}4+E-%R)*z9I3O!nDbJ*@mAF!kIKVQau{k zFA(^J&l=4l`32W*yFkt=tuXUn=bcA>bXRUa>b(e<2cXPVywjlJm;a>IG}&{_>zL+s z62_yk!j1_@sCf8szOkhD_q9$qBz=F8Q$0JmvUn2A(NOZrFSu7UHSQH(jA_?#1F9+- zmC+E$U1sIiIX^0|=p zmPp3vqm~xFrA)SyxC2ymL@wc#NARv}(P$=p<}gw&wVbQJ@SSs!KyIq(V>1b`N9GTg z@#{av0G!}ghy`{ECEa=^M5v|5fBY*gZe&YX+++D{xa?|Z+DIE4l_8^Bz&9pEF9>HP z*#s+vc>1E%l@baw$B%{r?q;8mKI>aPK;hx_lwHCxOS`j`d9Q%=xshbN2zNBPk+Q(W z&K?bW8NpYH(&3^iq~X-#O5rylVEKOb6~TL;yw&Rv)vN*diRC(;?~9mmi)*OD!cJ!s zn{U`t?Lg*IJs20`G)E?eGbi);b_+AJ?8UZVg>BWCEWX?6k#>T_Ipr=;SI}gfG&)r+ zHdEnd`5Ye<=;uNSkGZoU@*Y4s?oX|`^vkwWWal3ka@6Am`smIu;!E*IiJX-+sa8;lx)Xh1|xL zK;;MVaC8LU>&mXe%%qRBd*}O4cEhBp`1X?u513^i||bNL+gHd`oZ_Haa+%)w3S-;WemUu$NZ4_a$Qb1irQyz``^FubV zBodT%<7ilBp_(#^Xc}FK>!aV&c+qjl-0Ddq%!ct?hQ~!tq5Ie9lz+J46X;9{PX`FN z(SswSd8@=XUqjDmyhsjp160!+#X;t*W6u<@9bJUUKi@-v7Z|!rOud7itGk9eP!zCL zqMrZ$-N+ivprVN40C*ybj&B0$<`RO*o%tqUkaMT4Z{6`9NBl)gj+k~!S5zHGq5MQG zCsJ@UuyjA03zCp3t(aU%9yf(OZ?=K&3oZ^cJfUYvVUIbLrYg)V;=`XLm;}>O@E^rN zuD%bq=c8hDw}!e6bD}z_;sXHQT;cO(d>%kxEF}+ge)%9Y!z*y*i+o@22O@#WPogL= zf4b}EGjQUCeAD&OeynpUoQvX2S{_QT%{MPZUqN6IfxLSui^4+_$hol$i}-xW>}CQp zpc_-~CP|II{~@Kq^&>C7Pb}5nsrZI#uF8>Gv|{ZcAPwR(SEnC97EYpA^=`@@DPqm} z_W@LF8w6OAeI4`w<9Hk{< zc!5y9;V~xtCD+Of60aa8zSr@x%J3v28R_Rfz7-lDQ!Xk9d;(V08lgu%pU;s{B^4Dk zUi*S0Q13@zp@M)ZA0s3{YWhZi5G#t-QpI5wrxEYCZS*4m#YBY+p$;rIl0150jM7E^ z16Bg`9vrC5mMS7)6xEsF68}2lDBN}fY6jHVXhyvTc_1=hQQ6T&Qu>6-RSf_sMsXs*S`Cdz_NCs8jJv(IdimPqAW+pMRnFmcm@(bC(5bRU)so993C=! zvqSS8?x6vpBAdjuRHX?I9K`3hHlM&Hn!xQz76EpyC8Y?4ws6!ksV)&huU9Ym#>?d{ zpo@zD>*(4`?BkYh@gCJ;6O|{D6p;yJt%ziK_RM`hj@nlX=66Y8`5>_BT*#!MF#OjM zKOn{_A~e1H0qm>%J9l_A>{>7i%6Yy+;X?@`H^2JsmPI3<&#Az@E1eskqjLEBh^>tH zbBZ8a|42rZ+{jaDS0AwX_{0KyZlv)sm0c}>sfgl`h!>Kt>&?1ps-x57vn0xB#EF@J zf;gn!E$Li5#3u?ul6;HaMT4R;Qq0IVlIp1Jmc`XkTl-G`Boug5XcX8;+VvB_Ln9BC zogN+0hl*%?x{G9b&`E)*g!+zK?Vl5QmhZzjmcpPDibKc*`#L1Hq}Tx%rDHF*Z{6tN zUYbx4II-yGkN z#il|N&@+d*>Hh@2h~(XTO?Ux9h6rG00z$}xw(k;ZOXmttRp%Rl`6T0~LNN7iQi@Q? zr-DEsF9YFztgL64B$rSmkSGUQeJB+m(QM(#S$sw#>u>lIyMd3)IVsOGogz zWta0*QcShUTn_1DRjT-hCmKY_!w_}LJo!&4(q4_udM9d{LxRX z$_6=V0^ti)y6ZTdpz3v^B+AN+6Gzu^>4%pP%)t;Bf{WF2LRss%25|S13POdeM?dtw zG}uuy1d~)vqT_T22{^Z%0gjr*cT_xZz*eCD98~mZJM~@@84?XNuw`*E$%)<9|A`Ti z>ivPSG9{1~PR!8h4vc$!N`aJyX_Rb$1s{cJag4M_)cpa)xspn6P?tojbHLphI{Lel)V~QTnJ!QuXMB^kzAhsVO=$?dQD*=o2k2 z1u~J(d$|j|hXgVoUcUaZnh|m8U4hf8jx6W3mp zQe-<)^Ol9o7M?B?HtIr6n=sl1R4|PRd7~#DwWCy2f!^4L_KFrWr4*?cjDA<*2f>zT zVu3b^o<^%6DvG|ec%G@|KQquhWd{UNkd?}i&Y2|J zbO4rYLJw7U7GThVQ5$`=`DC&x-z2lv#Pn|pwbLT%2#9-Xj|;^gPv>y&`iPyTtw%DWqw3E)}2)7l13J4ikoxCXJjM{oZGZH<~bgVJN2X((Q_!n-Cl|_PY@9 z4uy9FVh{x|M`8+65`Lri5Pbq_xvO;JbdoZYN*j8=?9X9Q0sF+xEwJ56()lz*HO;xn zJ4Stg@$Z*mW+3HvGbN!L^7{P{WJ`xKdX3ONI}Zw?aIDH3q`}*A&TQ1ax^u7*(_Io& zlVljBrd{wg4Jh%+qk1gDPSF*B1Ib2KuQ0@;_9qZK5lzmOvDeI$TR%FQg@IT5rsO|c z-ugA6;sn~uP?-l}sK9ur6lf%!)|Xn3n!DtO$X(=dlM|$hvra49{1H|4WjjaIF`^D55Bj79PxKv+jnEG4nEq<1eyjafF|q97RQLp9EZd^V+iM~;v}*6mxwX7BmqBAO>+H%&Mj2+gQ=1x+bRzVEvj;ld z8t&HCgab{6E@3S63%nLOuuA}*yWT%CI@z1_)h4KEwfEb;sa6Yv?KIIAW1;iAipcco z42jVYNi!zy-pje+*nFt5rE<@yW17cXY;?PqszeXWcb&%AF)EP=E+sq`EjQ%D4-Y;* zm3^XUf__{0g?#xDyY^)&)gvu##TCv)tCHg5^EZz)2{Lxnj0y*K6g%}9sw@w6+X5{2 z9`I|Wp|v%uYHRJAk_=NfM7VM|XZz|Otxr@xHIx>g`Dl9g8A;ck+g1~-M|{%`-?s=K zZZeEbJmNoQ{esJL271JuzM`SQKY&5$6Kz8)-}rkK%evEU^a-hK?O5%)va*&q%jtIp zkGUo{cze`HF?Rf1x#@cv^nX4>{;y2K|GEj+(f&_XZH$xx_Yj8ajDU}q({8R%7|&33 zj5+*#hy4%VaZT09FK^vhBy8oO1x&y6B%gNSe{$hbEClA@zrMr&>O1^j8Ln?1`{7#0 zMvhK*$mB$Eg$bsI26_Z}5@YwxSGoD6M9)Hz;SKaZJf4LiaEO}CF&F~s$c7~q#!Bu0 z`^wF{xYDwIZJt;}!f)ETckV7om(bVXmjcu`LP z%ePwuPyH|B!kr`k;Yk1ASvkD=a)zqiXXvxcp&a80{2A}W9{__?+ahY91_xP24gHL- z?%BEXm>t{DqZk$g%$nl;dsnZ%caPjCVZohA?n%9R-#GGq?S1tT9rw66Pu6Qamcr0T zHg_N_7AkAm?k`((f!o7|`zNqp|N4LaHv*IYcgFMoG{pM<;7tZqn&<9)@wV;uD}4T? z_G-HJCI{?I6%N{(;(sw@R?5gpu39B2vue*O8HLrWuy>F6%9RQ$SAI~)U-hpO4%?Vm znmPWDPw+Lj^~4FH|J=de^029${ei73m*X7&yTY)u&s j4h~Y5N384)9yn?$Wn*g|(yl0kgBUC|txbs=j8FeJe4=xh literal 0 HcmV?d00001 diff --git a/docs/spark-connect-overview.md b/docs/spark-connect-overview.md index 1cc409bfbc007..723bae9fd9be5 100644 --- a/docs/spark-connect-overview.md +++ b/docs/spark-connect-overview.md @@ -370,6 +370,8 @@ one may implement their own class extending `ClassFinder` for customized search +For more information on application development with Spark Connect as well as extending Spark Connect +with custom functionality, see [Application Development with Spark Connect](app-dev-spark-connect.html). # Client application authentication While Spark Connect does not have built-in authentication, it is designed to From 7b4922ea90d19d7e0510a205740b4c150057e988 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 25 Nov 2024 19:22:43 +0900 Subject: [PATCH 013/438] [SPARK-50408][PYTHON][DOCS] Refine the docstring for datetime functions - part 4 ### What changes were proposed in this pull request? Refine the docstring for datetime functions ### Why are the changes needed? to improve the docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? new doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48953 from zhengruifeng/py_doc_9. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 541 ++++++++++++++---------- 1 file changed, 313 insertions(+), 228 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 68b51440278cb..31f3e88c46c08 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -8932,6 +8932,10 @@ def current_timezone() -> Column: :class:`~pyspark.sql.Column` current session local timezone. + See Also + -------- + :meth:`pyspark.sql.functions.convert_timezone` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") @@ -22640,7 +22644,7 @@ def convert_timezone( the current session time zone is used as the source time zone. targetTz : :class:`~pyspark.sql.Column` The time zone to which the input timestamp should be converted. - sourceTs : :class:`~pyspark.sql.Column` + sourceTs : :class:`~pyspark.sql.Column` or column name A timestamp without time zone. Returns @@ -22648,35 +22652,43 @@ def convert_timezone( :class:`~pyspark.sql.Column` A new column that contains a timestamp for converted time zone. + See Also + -------- + :meth:`pyspark.sql.functions.current_timezone` + Examples -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - Example 1: Converts the timestamp without time zone `sourceTs`, - the source time zone `sourceTz` is None. + Example 1: Converts the timestamp without time zone `sourceTs`. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([('2015-04-08',)], ['dt']) - >>> df.select(sf.convert_timezone( # doctest: +SKIP - ... None, sf.lit('Asia/Hong_Kong'), 'dt') - ... ).show() - +--------------------------------------------------------+ - |convert_timezone(current_timezone(), Asia/Hong_Kong, dt)| - +--------------------------------------------------------+ - | 2015-04-08 00:00:00| - +--------------------------------------------------------+ + >>> df = spark.createDataFrame([('2015-04-08 00:00:00',)], ['ts']) + >>> df.select( + ... '*', + ... sf.convert_timezone(None, sf.lit('Asia/Hong_Kong'), 'ts') + ... ).show() # doctest: +SKIP + +-------------------+--------------------------------------------------------+ + | ts|convert_timezone(current_timezone(), Asia/Hong_Kong, ts)| + +-------------------+--------------------------------------------------------+ + |2015-04-08 00:00:00| 2015-04-08 15:00:00| + +-------------------+--------------------------------------------------------+ - Example 2: Converts the timestamp without time zone `sourceTs`. + Example 2: Converts the timestamp with time zone `sourceTs`. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([('2015-04-08',)], ['dt']) - >>> df.select(sf.convert_timezone( - ... sf.lit('America/Los_Angeles'), sf.lit('Asia/Hong_Kong'), 'dt') + >>> df = spark.createDataFrame([('2015-04-08 15:00:00',)], ['ts']) + >>> df.select( + ... '*', + ... sf.convert_timezone(sf.lit('Asia/Hong_Kong'), sf.lit('America/Los_Angeles'), df.ts) ... ).show() - +---------------------------------------------------------+ - |convert_timezone(America/Los_Angeles, Asia/Hong_Kong, dt)| - +---------------------------------------------------------+ - | 2015-04-08 15:00:00| - +---------------------------------------------------------+ + +-------------------+---------------------------------------------------------+ + | ts|convert_timezone(Asia/Hong_Kong, America/Los_Angeles, ts)| + +-------------------+---------------------------------------------------------+ + |2015-04-08 15:00:00| 2015-04-08 00:00:00| + +-------------------+---------------------------------------------------------+ + + >>> spark.conf.unset("spark.sql.session.timeZone") """ if sourceTz is None: return _invoke_function_over_columns("convert_timezone", targetTz, sourceTs) @@ -22698,13 +22710,13 @@ def make_dt_interval( Parameters ---------- - days : :class:`~pyspark.sql.Column` or str, optional + days : :class:`~pyspark.sql.Column` or column name, optional The number of days, positive or negative. - hours : :class:`~pyspark.sql.Column` or str, optional + hours : :class:`~pyspark.sql.Column` or column name, optional The number of hours, positive or negative. - mins : :class:`~pyspark.sql.Column` or str, optional + mins : :class:`~pyspark.sql.Column` or column name, optional The number of minutes, positive or negative. - secs : :class:`~pyspark.sql.Column` or str, optional + secs : :class:`~pyspark.sql.Column` or column name, optional The number of seconds with the fractional part in microsecond precision. Returns @@ -22712,63 +22724,62 @@ def make_dt_interval( :class:`~pyspark.sql.Column` A new column that contains a DayTimeIntervalType duration. - Examples + See Also -------- + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.make_ym_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples + -------- Example 1: Make DayTimeIntervalType duration from days, hours, mins and secs. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], - ... ["day", "hour", "min", "sec"]) - >>> df.select(sf.make_dt_interval(df.day, df.hour, df.min, df.sec)).show(truncate=False) - +------------------------------------------+ - |make_dt_interval(day, hour, min, sec) | - +------------------------------------------+ - |INTERVAL '1 12:30:01.001001' DAY TO SECOND| - +------------------------------------------+ + >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], ['day', 'hour', 'min', 'sec']) + >>> df.select('*', sf.make_dt_interval(df.day, df.hour, df.min, df.sec)).show(truncate=False) + +---+----+---+--------+------------------------------------------+ + |day|hour|min|sec |make_dt_interval(day, hour, min, sec) | + +---+----+---+--------+------------------------------------------+ + |1 |12 |30 |1.001001|INTERVAL '1 12:30:01.001001' DAY TO SECOND| + +---+----+---+--------+------------------------------------------+ Example 2: Make DayTimeIntervalType duration from days, hours and mins. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], - ... ["day", "hour", "min", "sec"]) - >>> df.select(sf.make_dt_interval(df.day, df.hour, df.min)).show(truncate=False) - +-----------------------------------+ - |make_dt_interval(day, hour, min, 0)| - +-----------------------------------+ - |INTERVAL '1 12:30:00' DAY TO SECOND| - +-----------------------------------+ + >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], ['day', 'hour', 'min', 'sec']) + >>> df.select('*', sf.make_dt_interval(df.day, 'hour', df.min)).show(truncate=False) + +---+----+---+--------+-----------------------------------+ + |day|hour|min|sec |make_dt_interval(day, hour, min, 0)| + +---+----+---+--------+-----------------------------------+ + |1 |12 |30 |1.001001|INTERVAL '1 12:30:00' DAY TO SECOND| + +---+----+---+--------+-----------------------------------+ Example 3: Make DayTimeIntervalType duration from days and hours. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], - ... ["day", "hour", "min", "sec"]) - >>> df.select(sf.make_dt_interval(df.day, df.hour)).show(truncate=False) - +-----------------------------------+ - |make_dt_interval(day, hour, 0, 0) | - +-----------------------------------+ - |INTERVAL '1 12:00:00' DAY TO SECOND| - +-----------------------------------+ + >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], ['day', 'hour', 'min', 'sec']) + >>> df.select('*', sf.make_dt_interval(df.day, df.hour)).show(truncate=False) + +---+----+---+--------+-----------------------------------+ + |day|hour|min|sec |make_dt_interval(day, hour, 0, 0) | + +---+----+---+--------+-----------------------------------+ + |1 |12 |30 |1.001001|INTERVAL '1 12:00:00' DAY TO SECOND| + +---+----+---+--------+-----------------------------------+ Example 4: Make DayTimeIntervalType duration from days. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], - ... ["day", "hour", "min", "sec"]) - >>> df.select(sf.make_dt_interval(df.day)).show(truncate=False) - +-----------------------------------+ - |make_dt_interval(day, 0, 0, 0) | - +-----------------------------------+ - |INTERVAL '1 00:00:00' DAY TO SECOND| - +-----------------------------------+ + >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], ['day', 'hour', 'min', 'sec']) + >>> df.select('*', sf.make_dt_interval('day')).show(truncate=False) + +---+----+---+--------+-----------------------------------+ + |day|hour|min|sec |make_dt_interval(day, 0, 0, 0) | + +---+----+---+--------+-----------------------------------+ + |1 |12 |30 |1.001001|INTERVAL '1 00:00:00' DAY TO SECOND| + +---+----+---+--------+-----------------------------------+ - Example 5: Make DayTimeIntervalType duration. + Example 5: Make empty interval. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([[1, 12, 30, 01.001001]], - ... ["day", "hour", "min", "sec"]) - >>> df.select(sf.make_dt_interval()).show(truncate=False) + >>> spark.range(1).select(sf.make_dt_interval()).show(truncate=False) +-----------------------------------+ |make_dt_interval(0, 0, 0, 0) | +-----------------------------------+ @@ -22800,19 +22811,19 @@ def try_make_interval( Parameters ---------- - years : :class:`~pyspark.sql.Column` or str, optional + years : :class:`~pyspark.sql.Column` or column name, optional The number of years, positive or negative. - months : :class:`~pyspark.sql.Column` or str, optional + months : :class:`~pyspark.sql.Column` or column name, optional The number of months, positive or negative. - weeks : :class:`~pyspark.sql.Column` or str, optional + weeks : :class:`~pyspark.sql.Column` or column name, optional The number of weeks, positive or negative. - days : :class:`~pyspark.sql.Column` or str, optional + days : :class:`~pyspark.sql.Column` or column name, optional The number of days, positive or negative. - hours : :class:`~pyspark.sql.Column` or str, optional + hours : :class:`~pyspark.sql.Column` or column name, optional The number of hours, positive or negative. - mins : :class:`~pyspark.sql.Column` or str, optional + mins : :class:`~pyspark.sql.Column` or column name, optional The number of minutes, positive or negative. - secs : :class:`~pyspark.sql.Column` or str, optional + secs : :class:`~pyspark.sql.Column` or column name, optional The number of seconds with the fractional part in microsecond precision. Returns @@ -22820,16 +22831,21 @@ def try_make_interval( :class:`~pyspark.sql.Column` A new column that contains an interval. - Examples + See Also -------- + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.make_dt_interval` + :meth:`pyspark.sql.functions.make_ym_interval` + Examples + -------- Example 1: Try make interval from years, months, weeks, days, hours, mins and secs. >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_interval( - ... df.year, df.month, df.week, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.try_make_interval(df.year, df.month, 'week', df.day, 'hour', df.min, df.sec) ... ).show(truncate=False) +---------------------------------------------------------------+ |try_make_interval(year, month, week, day, hour, min, sec) | @@ -22841,9 +22857,9 @@ def try_make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_interval( - ... df.year, df.month, df.week, df.day, df.hour, df.min) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.try_make_interval(df.year, df.month, 'week', df.day, df.hour, df.min) ... ).show(truncate=False) +-------------------------------------------------------+ |try_make_interval(year, month, week, day, hour, min, 0)| @@ -22855,9 +22871,9 @@ def try_make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_interval( - ... df.year, df.month, df.week, df.day, df.hour) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.try_make_interval(df.year, df.month, 'week', df.day, df.hour) ... ).show(truncate=False) +-----------------------------------------------------+ |try_make_interval(year, month, week, day, hour, 0, 0)| @@ -22869,8 +22885,8 @@ def try_make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_interval(df.year, df.month, df.week, df.day)).show(truncate=False) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select(sf.try_make_interval(df.year, 'month', df.week, df.day)).show(truncate=False) +--------------------------------------------------+ |try_make_interval(year, month, week, day, 0, 0, 0)| +--------------------------------------------------+ @@ -22881,8 +22897,8 @@ def try_make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_interval(df.year, df.month, df.week)).show(truncate=False) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select(sf.try_make_interval(df.year, 'month', df.week)).show(truncate=False) +------------------------------------------------+ |try_make_interval(year, month, week, 0, 0, 0, 0)| +------------------------------------------------+ @@ -22893,8 +22909,8 @@ def try_make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_interval(df.year, df.month)).show(truncate=False) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select(sf.try_make_interval(df.year, 'month')).show(truncate=False) +---------------------------------------------+ |try_make_interval(year, month, 0, 0, 0, 0, 0)| +---------------------------------------------+ @@ -22905,7 +22921,7 @@ def try_make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) >>> df.select(sf.try_make_interval(df.year)).show(truncate=False) +-----------------------------------------+ |try_make_interval(year, 0, 0, 0, 0, 0, 0)| @@ -22913,18 +22929,25 @@ def try_make_interval( |100 years | +-----------------------------------------+ - Example 8: Try make interval from years with overflow. + Example 8: Try make empty interval. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([[2147483647, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_interval(df.year)).show(truncate=False) - +-----------------------------------------+ - |try_make_interval(year, 0, 0, 0, 0, 0, 0)| - +-----------------------------------------+ - |NULL | - +-----------------------------------------+ + >>> spark.range(1).select(sf.try_make_interval()).show(truncate=False) + +--------------------------------------+ + |try_make_interval(0, 0, 0, 0, 0, 0, 0)| + +--------------------------------------+ + |0 seconds | + +--------------------------------------+ + Example 9: Try make interval from years with overflow. + + >>> import pyspark.sql.functions as sf + >>> spark.range(1).select(sf.try_make_interval(sf.lit(2147483647))).show(truncate=False) + +-----------------------------------------------+ + |try_make_interval(2147483647, 0, 0, 0, 0, 0, 0)| + +-----------------------------------------------+ + |NULL | + +-----------------------------------------------+ """ _years = lit(0) if years is None else years _months = lit(0) if months is None else months @@ -22955,19 +22978,19 @@ def make_interval( Parameters ---------- - years : :class:`~pyspark.sql.Column` or str, optional + years : :class:`~pyspark.sql.Column` or column name, optional The number of years, positive or negative. - months : :class:`~pyspark.sql.Column` or str, optional + months : :class:`~pyspark.sql.Column` or column name, optional The number of months, positive or negative. - weeks : :class:`~pyspark.sql.Column` or str, optional + weeks : :class:`~pyspark.sql.Column` or column name, optional The number of weeks, positive or negative. - days : :class:`~pyspark.sql.Column` or str, optional + days : :class:`~pyspark.sql.Column` or column name, optional The number of days, positive or negative. - hours : :class:`~pyspark.sql.Column` or str, optional + hours : :class:`~pyspark.sql.Column` or column name, optional The number of hours, positive or negative. - mins : :class:`~pyspark.sql.Column` or str, optional + mins : :class:`~pyspark.sql.Column` or column name, optional The number of minutes, positive or negative. - secs : :class:`~pyspark.sql.Column` or str, optional + secs : :class:`~pyspark.sql.Column` or column name, optional The number of seconds with the fractional part in microsecond precision. Returns @@ -22975,16 +22998,21 @@ def make_interval( :class:`~pyspark.sql.Column` A new column that contains an interval. - Examples + See Also -------- + :meth:`pyspark.sql.functions.make_dt_interval` + :meth:`pyspark.sql.functions.make_ym_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples + -------- Example 1: Make interval from years, months, weeks, days, hours, mins and secs. >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.make_interval( - ... df.year, df.month, df.week, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.make_interval(df.year, df.month, 'week', df.day, df.hour, df.min, df.sec) ... ).show(truncate=False) +---------------------------------------------------------------+ |make_interval(year, month, week, day, hour, min, sec) | @@ -22996,9 +23024,9 @@ def make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.make_interval( - ... df.year, df.month, df.week, df.day, df.hour, df.min) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.make_interval(df.year, df.month, 'week', df.day, df.hour, df.min) ... ).show(truncate=False) +---------------------------------------------------+ |make_interval(year, month, week, day, hour, min, 0)| @@ -23010,9 +23038,9 @@ def make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.make_interval( - ... df.year, df.month, df.week, df.day, df.hour) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.make_interval(df.year, df.month, 'week', df.day, df.hour) ... ).show(truncate=False) +-------------------------------------------------+ |make_interval(year, month, week, day, hour, 0, 0)| @@ -23024,8 +23052,8 @@ def make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.make_interval(df.year, df.month, df.week, df.day)).show(truncate=False) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select(sf.make_interval(df.year, df.month, 'week', df.day)).show(truncate=False) +----------------------------------------------+ |make_interval(year, month, week, day, 0, 0, 0)| +----------------------------------------------+ @@ -23036,8 +23064,8 @@ def make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.make_interval(df.year, df.month, df.week)).show(truncate=False) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) + >>> df.select(sf.make_interval(df.year, df.month, 'week')).show(truncate=False) +--------------------------------------------+ |make_interval(year, month, week, 0, 0, 0, 0)| +--------------------------------------------+ @@ -23048,7 +23076,7 @@ def make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) >>> df.select(sf.make_interval(df.year, df.month)).show(truncate=False) +-----------------------------------------+ |make_interval(year, month, 0, 0, 0, 0, 0)| @@ -23060,7 +23088,7 @@ def make_interval( >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) + ... ['year', 'month', 'week', 'day', 'hour', 'min', 'sec']) >>> df.select(sf.make_interval(df.year)).show(truncate=False) +-------------------------------------+ |make_interval(year, 0, 0, 0, 0, 0, 0)| @@ -23068,12 +23096,10 @@ def make_interval( |100 years | +-------------------------------------+ - Example 8: Make interval. + Example 8: Make empty interval. >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([[100, 11, 1, 1, 12, 30, 01.001001]], - ... ["year", "month", "week", "day", "hour", "min", "sec"]) - >>> df.select(sf.make_interval()).show(truncate=False) + >>> spark.range(1).select(sf.make_interval()).show(truncate=False) +----------------------------------+ |make_interval(0, 0, 0, 0, 0, 0, 0)| +----------------------------------+ @@ -23112,22 +23138,22 @@ def make_timestamp( Parameters ---------- - years : :class:`~pyspark.sql.Column` or str + years : :class:`~pyspark.sql.Column` or column name The year to represent, from 1 to 9999 - months : :class:`~pyspark.sql.Column` or str + months : :class:`~pyspark.sql.Column` or column name The month-of-year to represent, from 1 (January) to 12 (December) - days : :class:`~pyspark.sql.Column` or str + days : :class:`~pyspark.sql.Column` or column name The day-of-month to represent, from 1 to 31 - hours : :class:`~pyspark.sql.Column` or str + hours : :class:`~pyspark.sql.Column` or column name The hour-of-day to represent, from 0 to 23 - mins : :class:`~pyspark.sql.Column` or str + mins : :class:`~pyspark.sql.Column` or column name The minute-of-hour to represent, from 0 to 59 - secs : :class:`~pyspark.sql.Column` or str + secs : :class:`~pyspark.sql.Column` or column name The second-of-minute and its micro-fraction to represent, from 0 to 60. The value can be either an integer like 13 , or a fraction like 13.123. If the sec argument equals to 60, the seconds field is set to 0 and 1 minute is added to the final timestamp. - timezone : :class:`~pyspark.sql.Column` or str, optional + timezone : :class:`~pyspark.sql.Column` or column name, optional The time zone identifier. For example, CET, UTC and etc. Returns @@ -23135,38 +23161,48 @@ def make_timestamp( :class:`~pyspark.sql.Column` A new column that contains a timestamp. + See Also + -------- + :meth:`pyspark.sql.functions.make_timestamp_ltz` + :meth:`pyspark.sql.functions.make_timestamp_ntz` + :meth:`pyspark.sql.functions.try_make_timestamp` + :meth:`pyspark.sql.functions.try_make_timestamp_ltz` + :meth:`pyspark.sql.functions.try_make_timestamp_ntz` + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") Example 1: Make timestamp from years, months, days, hours, mins and secs. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.make_timestamp( - ... df.year, df.month, df.day, df.hour, df.min, df.sec, df.timezone) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.make_timestamp(df.year, df.month, df.day, 'hour', df.min, df.sec, 'tz') ... ).show(truncate=False) - +----------------------------------------------------------+ - |make_timestamp(year, month, day, hour, min, sec, timezone)| - +----------------------------------------------------------+ - |2014-12-27 21:30:45.887 | - +----------------------------------------------------------+ + +----------------------------------------------------+ + |make_timestamp(year, month, day, hour, min, sec, tz)| + +----------------------------------------------------+ + |2014-12-27 21:30:45.887 | + +----------------------------------------------------+ Example 2: Make timestamp without timezone. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.make_timestamp( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.make_timestamp(df.year, df.month, df.day, 'hour', df.min, df.sec) ... ).show(truncate=False) +------------------------------------------------+ |make_timestamp(year, month, day, hour, min, sec)| +------------------------------------------------+ |2014-12-28 06:30:45.887 | +------------------------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ if timezone is not None: @@ -23221,17 +23257,27 @@ def try_make_timestamp( :class:`~pyspark.sql.Column` A new column that contains a timestamp or NULL in case of an error. + See Also + -------- + :meth:`pyspark.sql.functions.make_timestamp` + :meth:`pyspark.sql.functions.make_timestamp_ltz` + :meth:`pyspark.sql.functions.make_timestamp_ntz` + :meth:`pyspark.sql.functions.try_make_timestamp_ltz` + :meth:`pyspark.sql.functions.try_make_timestamp_ntz` + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") Example 1: Make timestamp from years, months, days, hours, mins and secs. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.try_make_timestamp( - ... df.year, df.month, df.day, df.hour, df.min, df.sec, df.timezone) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.try_make_timestamp(df.year, df.month, df.day, 'hour', df.min, df.sec, 'tz') ... ).show(truncate=False) +----------------------------------------------------+ |try_make_timestamp(year, month, day, hour, min, sec)| @@ -23242,11 +23288,10 @@ def try_make_timestamp( Example 2: Make timestamp without timezone. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.try_make_timestamp( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.try_make_timestamp(df.year, df.month, df.day, 'hour', df.min, df.sec) ... ).show(truncate=False) +----------------------------------------------------+ |try_make_timestamp(year, month, day, hour, min, sec)| @@ -23258,17 +23303,17 @@ def try_make_timestamp( Example 3: Make timestamp with invalid input. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 13, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.try_make_timestamp( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.try_make_timestamp(df.year, df.month, df.day, 'hour', df.min, df.sec) ... ).show(truncate=False) +----------------------------------------------------+ |try_make_timestamp(year, month, day, hour, min, sec)| +----------------------------------------------------+ |NULL | +----------------------------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ if timezone is not None: @@ -23323,38 +23368,48 @@ def make_timestamp_ltz( :class:`~pyspark.sql.Column` A new column that contains a current timestamp. + See Also + -------- + :meth:`pyspark.sql.functions.make_timestamp` + :meth:`pyspark.sql.functions.make_timestamp_ntz` + :meth:`pyspark.sql.functions.try_make_timestamp` + :meth:`pyspark.sql.functions.try_make_timestamp_ltz` + :meth:`pyspark.sql.functions.try_make_timestamp_ntz` + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") Example 1: Make the current timestamp from years, months, days, hours, mins and secs. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.make_timestamp_ltz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec, df.timezone) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.make_timestamp_ltz(df.year, df.month, 'day', df.hour, df.min, df.sec, 'tz') ... ).show(truncate=False) - +--------------------------------------------------------------+ - |make_timestamp_ltz(year, month, day, hour, min, sec, timezone)| - +--------------------------------------------------------------+ - |2014-12-27 21:30:45.887 | - +--------------------------------------------------------------+ + +--------------------------------------------------------+ + |make_timestamp_ltz(year, month, day, hour, min, sec, tz)| + +--------------------------------------------------------+ + |2014-12-27 21:30:45.887 | + +--------------------------------------------------------+ Example 2: Make the current timestamp without timezone. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.make_timestamp_ltz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.make_timestamp_ltz(df.year, df.month, 'day', df.hour, df.min, df.sec) ... ).show(truncate=False) +----------------------------------------------------+ |make_timestamp_ltz(year, month, day, hour, min, sec)| +----------------------------------------------------+ |2014-12-28 06:30:45.887 | +----------------------------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ if timezone is not None: @@ -23409,54 +23464,62 @@ def try_make_timestamp_ltz( :class:`~pyspark.sql.Column` A new column that contains a current timestamp, or NULL in case of an error. + See Also + -------- + :meth:`pyspark.sql.functions.make_timestamp` + :meth:`pyspark.sql.functions.make_timestamp_ltz` + :meth:`pyspark.sql.functions.make_timestamp_ntz` + :meth:`pyspark.sql.functions.try_make_timestamp` + :meth:`pyspark.sql.functions.try_make_timestamp_ntz` + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") Example 1: Make the current timestamp from years, months, days, hours, mins and secs. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.try_make_timestamp_ltz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec, df.timezone) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.try_make_timestamp_ltz('year', 'month', df.day, df.hour, df.min, df.sec, 'tz') ... ).show(truncate=False) - +------------------------------------------------------------------+ - |try_make_timestamp_ltz(year, month, day, hour, min, sec, timezone)| - +------------------------------------------------------------------+ - |2014-12-27 21:30:45.887 | - +------------------------------------------------------------------+ + +------------------------------------------------------------+ + |try_make_timestamp_ltz(year, month, day, hour, min, sec, tz)| + +------------------------------------------------------------+ + |2014-12-27 21:30:45.887 | + +------------------------------------------------------------+ Example 2: Make the current timestamp without timezone. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.try_make_timestamp_ltz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.try_make_timestamp_ltz('year', 'month', df.day, df.hour, df.min, df.sec) ... ).show(truncate=False) +--------------------------------------------------------+ |try_make_timestamp_ltz(year, month, day, hour, min, sec)| +--------------------------------------------------------+ |2014-12-28 06:30:45.887 | +--------------------------------------------------------+ - >>> spark.conf.unset("spark.sql.session.timeZone") Example 3: Make the current timestamp with invalid input. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 13, 28, 6, 30, 45.887, 'CET']], - ... ["year", "month", "day", "hour", "min", "sec", "timezone"]) - >>> df.select(sf.try_make_timestamp_ltz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec', 'tz']) + >>> df.select( + ... sf.try_make_timestamp_ltz('year', 'month', df.day, df.hour, df.min, df.sec) ... ).show(truncate=False) +--------------------------------------------------------+ |try_make_timestamp_ltz(year, month, day, hour, min, sec)| +--------------------------------------------------------+ |NULL | +--------------------------------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ if timezone is not None: @@ -23487,17 +23550,17 @@ def make_timestamp_ntz( Parameters ---------- - years : :class:`~pyspark.sql.Column` or str + years : :class:`~pyspark.sql.Column` or column name The year to represent, from 1 to 9999 - months : :class:`~pyspark.sql.Column` or str + months : :class:`~pyspark.sql.Column` or column name The month-of-year to represent, from 1 (January) to 12 (December) - days : :class:`~pyspark.sql.Column` or str + days : :class:`~pyspark.sql.Column` or column name The day-of-month to represent, from 1 to 31 - hours : :class:`~pyspark.sql.Column` or str + hours : :class:`~pyspark.sql.Column` or column name The hour-of-day to represent, from 0 to 23 - mins : :class:`~pyspark.sql.Column` or str + mins : :class:`~pyspark.sql.Column` or column name The minute-of-hour to represent, from 0 to 59 - secs : :class:`~pyspark.sql.Column` or str + secs : :class:`~pyspark.sql.Column` or column name The second-of-minute and its micro-fraction to represent, from 0 to 60. The value can be either an integer like 13 , or a fraction like 13.123. If the sec argument equals to 60, the seconds field is set @@ -23508,23 +23571,32 @@ def make_timestamp_ntz( :class:`~pyspark.sql.Column` A new column that contains a local date-time. - Examples + See Also -------- + :meth:`pyspark.sql.functions.make_timestamp` + :meth:`pyspark.sql.functions.make_timestamp_ltz` + :meth:`pyspark.sql.functions.try_make_timestamp` + :meth:`pyspark.sql.functions.try_make_timestamp_ltz` + :meth:`pyspark.sql.functions.try_make_timestamp_ntz` + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.try_make_interval` - Example 1: Make local date-time from years, months, days, hours, mins, secs. + Examples + -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887]], - ... ["year", "month", "day", "hour", "min", "sec"]) - >>> df.select(sf.make_timestamp_ntz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.make_timestamp_ntz('year', 'month', df.day, df.hour, df.min, df.sec) ... ).show(truncate=False) +----------------------------------------------------+ |make_timestamp_ntz(year, month, day, hour, min, sec)| +----------------------------------------------------+ |2014-12-28 06:30:45.887 | +----------------------------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns( @@ -23570,39 +23642,48 @@ def try_make_timestamp_ntz( :class:`~pyspark.sql.Column` A new column that contains a local date-time, or NULL in case of an error. + See Also + -------- + :meth:`pyspark.sql.functions.make_timestamp` + :meth:`pyspark.sql.functions.make_timestamp_ltz` + :meth:`pyspark.sql.functions.make_timestamp_ntz` + :meth:`pyspark.sql.functions.try_make_timestamp` + :meth:`pyspark.sql.functions.try_make_timestamp_ltz` + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") Example 1: Make local date-time from years, months, days, hours, mins, secs. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887]], - ... ["year", "month", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_timestamp_ntz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.try_make_timestamp_ntz('year', 'month', df.day, df.hour, df.min, df.sec) ... ).show(truncate=False) +--------------------------------------------------------+ |try_make_timestamp_ntz(year, month, day, hour, min, sec)| +--------------------------------------------------------+ |2014-12-28 06:30:45.887 | +--------------------------------------------------------+ - >>> spark.conf.unset("spark.sql.session.timeZone") Example 2: Make local date-time with invalid input >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") >>> df = spark.createDataFrame([[2014, 13, 28, 6, 30, 45.887]], - ... ["year", "month", "day", "hour", "min", "sec"]) - >>> df.select(sf.try_make_timestamp_ntz( - ... df.year, df.month, df.day, df.hour, df.min, df.sec) + ... ['year', 'month', 'day', 'hour', 'min', 'sec']) + >>> df.select( + ... sf.try_make_timestamp_ntz('year', 'month', df.day, df.hour, df.min, df.sec) ... ).show(truncate=False) +--------------------------------------------------------+ |try_make_timestamp_ntz(year, month, day, hour, min, sec)| +--------------------------------------------------------+ |NULL | +--------------------------------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns( @@ -23622,9 +23703,9 @@ def make_ym_interval( Parameters ---------- - years : :class:`~pyspark.sql.Column` or str, optional + years : :class:`~pyspark.sql.Column` or column name, optional The number of years, positive or negative - months : :class:`~pyspark.sql.Column` or str, optional + months : :class:`~pyspark.sql.Column` or column name, optional The number of months, positive or negative Returns @@ -23632,44 +23713,48 @@ def make_ym_interval( :class:`~pyspark.sql.Column` A new column that contains a year-month interval. + See Also + -------- + :meth:`pyspark.sql.functions.make_interval` + :meth:`pyspark.sql.functions.make_dt_interval` + :meth:`pyspark.sql.functions.try_make_interval` + Examples -------- + >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") Example 1: Make year-month interval from years, months. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> df = spark.createDataFrame([[2014, 12]], ["year", "month"]) - >>> df.select(sf.make_ym_interval(df.year, df.month)).show(truncate=False) - +-------------------------------+ - |make_ym_interval(year, month) | - +-------------------------------+ - |INTERVAL '2015-0' YEAR TO MONTH| - +-------------------------------+ + >>> df = spark.createDataFrame([[2014, 12]], ['year', 'month']) + >>> df.select('*', sf.make_ym_interval('year', df.month)).show(truncate=False) + +----+-----+-------------------------------+ + |year|month|make_ym_interval(year, month) | + +----+-----+-------------------------------+ + |2014|12 |INTERVAL '2015-0' YEAR TO MONTH| + +----+-----+-------------------------------+ Example 2: Make year-month interval from years. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> df = spark.createDataFrame([[2014, 12]], ["year", "month"]) - >>> df.select(sf.make_ym_interval(df.year)).show(truncate=False) - +-------------------------------+ - |make_ym_interval(year, 0) | - +-------------------------------+ - |INTERVAL '2014-0' YEAR TO MONTH| - +-------------------------------+ + >>> df = spark.createDataFrame([[2014, 12]], ['year', 'month']) + >>> df.select('*', sf.make_ym_interval(df.year)).show(truncate=False) + +----+-----+-------------------------------+ + |year|month|make_ym_interval(year, 0) | + +----+-----+-------------------------------+ + |2014|12 |INTERVAL '2014-0' YEAR TO MONTH| + +----+-----+-------------------------------+ - Example 3: Make year-month interval. + Example 3: Make empty interval. >>> import pyspark.sql.functions as sf - >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> df = spark.createDataFrame([[2014, 12]], ["year", "month"]) - >>> df.select(sf.make_ym_interval()).show(truncate=False) + >>> spark.range(1).select(sf.make_ym_interval()).show(truncate=False) +----------------------------+ |make_ym_interval(0, 0) | +----------------------------+ |INTERVAL '0-0' YEAR TO MONTH| +----------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ _years = lit(0) if years is None else years From da4bcb732faaf382d1c015665912db92e91fde16 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 25 Nov 2024 19:30:43 +0800 Subject: [PATCH 014/438] [MINOR][PYTHON] Remove some unused third-party library imports ### What changes were proposed in this pull request? Remove some unused third-part library imports ### Why are the changes needed? these imports are never used ### 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 #48954 from zhengruifeng/fix_has_numpy. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/connect/session.py | 7 ------- python/pyspark/sql/pandas/types.py | 5 +---- python/pyspark/sql/session.py | 6 ------ 3 files changed, 1 insertion(+), 17 deletions(-) diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index bfd79092ccf4d..2d544f3f0eca7 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -113,13 +113,6 @@ from pyspark.sql.connect.shell.progress import ProgressHandler from pyspark.sql.connect.datasource import DataSourceRegistration -try: - import memory_profiler # noqa: F401 - - has_memory_profiler = True -except Exception: - has_memory_profiler = False - class SparkSession: # The active SparkSession for the current thread diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py index 648af21502864..d65126bb3db9e 100644 --- a/python/pyspark/sql/pandas/types.py +++ b/python/pyspark/sql/pandas/types.py @@ -53,14 +53,11 @@ ) from pyspark.errors import PySparkTypeError, UnsupportedOperationException, PySparkValueError from pyspark.loose_version import LooseVersion -from pyspark.sql.utils import has_numpy - -if has_numpy: - import numpy as np if TYPE_CHECKING: import pandas as pd import pyarrow as pa + import numpy as np from pyspark.sql.pandas._typing import SeriesLike as PandasSeriesLike from pyspark.sql.pandas._typing import DataFrameLike as PandasDataFrameLike diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index e97b844564100..e35a1cc82d795 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -90,12 +90,6 @@ from pyspark.sql.connect.client import SparkConnectClient from pyspark.sql.connect.shell.progress import ProgressHandler -try: - import memory_profiler # noqa: F401 - - has_memory_profiler = True -except Exception: - has_memory_profiler = False __all__ = ["SparkSession"] From 976f8875edd4669439880a06be041e262a2427f4 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 21:52:12 +0800 Subject: [PATCH 015/438] [SPARK-50286][SQL] Correctly propagate SQL options to WriteBuilder ### What changes were proposed in this pull request? SPARK-49098 introduced a SQL syntax to allow users to set table options on DSv2 write cases, but unfortunately, the options set by SQL are not propagated correctly to the underlying DSv2 `WriteBuilder` ``` INSERT INTO $t1 WITH (`write.split-size` = 10) SELECT ... ``` ``` df.writeTo(t1).option("write.split-size", "10").append() ``` From the user's perspective, the above two are equivalent, but internal implementations differ slightly. Both of them are going to construct an ``` AppendData(r: DataSourceV2Relation, ..., writeOptions, ...) ``` but the SQL `options` are carried by `r.options`, and the `DataFrame` API `options` are carried by `writeOptions`. Currently, only the latter is propagated to the `WriteBuilder`, and the former is silently dropped. This PR fixes the above issue by merging those two `options`. Currently, the `options` propagation is inconsistent in `DataFrame`, `DataFrameV2`, and SQL: - DataFrame API, the same `options` are carried by both `writeOptions` and `DataSourceV2Relation` - DataFrameV2 API cases, options are only carried by `write options` - SQL, `options` are only carried by `DataSourceV2Relation` BTW, `SessionConfigSupport` only takes effect on `DataFrame` and `DataFrameV2` API, it is not considered in the `SQL` read/write path entirely in the current codebase. ### Why are the changes needed? Correctly propagate SQL options to `WriteBuilder`, to complete the feature added in SPARK-49098, so that DSv2 implementations like Iceberg can benefit. ### Does this PR introduce _any_ user-facing change? No, it's an unreleased feature. ### How was this patch tested? UTs added by SPARK-36680 and SPARK-49098 are updated also to check SQL `options` are correctly propagated to the physical plan ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48822 from pan3793/SPARK-50286. Authored-by: Cheng Pan Signed-off-by: Wenchen Fan --- .../connector/catalog/InMemoryBaseTable.scala | 35 +- .../InMemoryRowLevelOperationTable.scala | 4 +- .../sql/connector/catalog/InMemoryTable.scala | 14 +- .../catalog/InMemoryTableWithV2Filter.scala | 27 +- .../execution/datasources/v2/V2Writes.scala | 36 +- .../org/apache/spark/sql/QueryTest.scala | 10 +- .../connector/DataSourceV2OptionSuite.scala | 327 ++++++++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 93 +---- .../sql/connector/V1WriteFallbackSuite.scala | 6 +- 9 files changed, 406 insertions(+), 146 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2OptionSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala index 497ef848ac78f..ab17b93ad6146 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala @@ -295,7 +295,7 @@ abstract class InMemoryBaseTable( TableCapability.TRUNCATE) override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { - new InMemoryScanBuilder(schema) + new InMemoryScanBuilder(schema, options) } private def canEvaluate(filter: Filter): Boolean = { @@ -309,8 +309,10 @@ abstract class InMemoryBaseTable( } } - class InMemoryScanBuilder(tableSchema: StructType) extends ScanBuilder - with SupportsPushDownRequiredColumns with SupportsPushDownFilters { + class InMemoryScanBuilder( + tableSchema: StructType, + options: CaseInsensitiveStringMap) extends ScanBuilder + with SupportsPushDownRequiredColumns with SupportsPushDownFilters { private var schema: StructType = tableSchema private var postScanFilters: Array[Filter] = Array.empty private var evaluableFilters: Array[Filter] = Array.empty @@ -318,7 +320,7 @@ abstract class InMemoryBaseTable( override def build: Scan = { val scan = InMemoryBatchScan( - data.map(_.asInstanceOf[InputPartition]).toImmutableArraySeq, schema, tableSchema) + data.map(_.asInstanceOf[InputPartition]).toImmutableArraySeq, schema, tableSchema, options) if (evaluableFilters.nonEmpty) { scan.filter(evaluableFilters) } @@ -442,7 +444,8 @@ abstract class InMemoryBaseTable( case class InMemoryBatchScan( var _data: Seq[InputPartition], readSchema: StructType, - tableSchema: StructType) + tableSchema: StructType, + options: CaseInsensitiveStringMap) extends BatchScanBaseClass(_data, readSchema, tableSchema) with SupportsRuntimeFiltering { override def filterAttributes(): Array[NamedReference] = { @@ -474,17 +477,17 @@ abstract class InMemoryBaseTable( } } - abstract class InMemoryWriterBuilder() extends SupportsTruncate with SupportsDynamicOverwrite - with SupportsStreamingUpdateAsAppend { + abstract class InMemoryWriterBuilder(val info: LogicalWriteInfo) + extends SupportsTruncate with SupportsDynamicOverwrite with SupportsStreamingUpdateAsAppend { - protected var writer: BatchWrite = Append - protected var streamingWriter: StreamingWrite = StreamingAppend + protected var writer: BatchWrite = new Append(info) + protected var streamingWriter: StreamingWrite = new StreamingAppend(info) override def overwriteDynamicPartitions(): WriteBuilder = { - if (writer != Append) { + if (!writer.isInstanceOf[Append]) { throw new IllegalArgumentException(s"Unsupported writer type: $writer") } - writer = DynamicOverwrite + writer = new DynamicOverwrite(info) streamingWriter = new StreamingNotSupportedOperation("overwriteDynamicPartitions") this } @@ -529,13 +532,13 @@ abstract class InMemoryBaseTable( override def abort(messages: Array[WriterCommitMessage]): Unit = {} } - protected object Append extends TestBatchWrite { + class Append(val info: LogicalWriteInfo) extends TestBatchWrite { override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized { withData(messages.map(_.asInstanceOf[BufferedRows])) } } - private object DynamicOverwrite extends TestBatchWrite { + class DynamicOverwrite(val info: LogicalWriteInfo) extends TestBatchWrite { override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized { val newData = messages.map(_.asInstanceOf[BufferedRows]) dataMap --= newData.flatMap(_.rows.map(getKey)) @@ -543,7 +546,7 @@ abstract class InMemoryBaseTable( } } - protected object TruncateAndAppend extends TestBatchWrite { + class TruncateAndAppend(val info: LogicalWriteInfo) extends TestBatchWrite { override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized { dataMap.clear() withData(messages.map(_.asInstanceOf[BufferedRows])) @@ -572,7 +575,7 @@ abstract class InMemoryBaseTable( s"${operation} isn't supported for streaming query.") } - private object StreamingAppend extends TestStreamingWrite { + class StreamingAppend(val info: LogicalWriteInfo) extends TestStreamingWrite { override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { dataMap.synchronized { withData(messages.map(_.asInstanceOf[BufferedRows])) @@ -580,7 +583,7 @@ abstract class InMemoryBaseTable( } } - protected object StreamingTruncateAndAppend extends TestStreamingWrite { + class StreamingTruncateAndAppend(val info: LogicalWriteInfo) extends TestStreamingWrite { override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { dataMap.synchronized { dataMap.clear() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala index 4abe4c8b3e3fb..3a684dc57c02f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala @@ -59,7 +59,7 @@ class InMemoryRowLevelOperationTable( } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { - new InMemoryScanBuilder(schema) { + new InMemoryScanBuilder(schema, options) { override def build: Scan = { val scan = super.build() configuredScan = scan.asInstanceOf[InMemoryBatchScan] @@ -115,7 +115,7 @@ class InMemoryRowLevelOperationTable( override def rowId(): Array[NamedReference] = Array(PK_COLUMN_REF) override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { - new InMemoryScanBuilder(schema) + new InMemoryScanBuilder(schema, options) } override def newWriteBuilder(info: LogicalWriteInfo): DeltaWriteBuilder = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala index af04816e6b6f0..c27b8fea059f7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTable.scala @@ -84,23 +84,23 @@ class InMemoryTable( InMemoryBaseTable.maybeSimulateFailedTableWrite(new CaseInsensitiveStringMap(properties)) InMemoryBaseTable.maybeSimulateFailedTableWrite(info.options) - new InMemoryWriterBuilderWithOverWrite() + new InMemoryWriterBuilderWithOverWrite(info) } - private class InMemoryWriterBuilderWithOverWrite() extends InMemoryWriterBuilder - with SupportsOverwrite { + class InMemoryWriterBuilderWithOverWrite(override val info: LogicalWriteInfo) + extends InMemoryWriterBuilder(info) with SupportsOverwrite { override def truncate(): WriteBuilder = { - if (writer != Append) { + if (!writer.isInstanceOf[Append]) { throw new IllegalArgumentException(s"Unsupported writer type: $writer") } - writer = TruncateAndAppend - streamingWriter = StreamingTruncateAndAppend + writer = new TruncateAndAppend(info) + streamingWriter = new StreamingTruncateAndAppend(info) this } override def overwrite(filters: Array[Filter]): WriteBuilder = { - if (writer != Append) { + if (!writer.isInstanceOf[Append]) { throw new IllegalArgumentException(s"Unsupported writer type: $writer") } writer = new Overwrite(filters) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala index 20ada0d622bca..9b7a90774f91c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2Filter.scala @@ -47,19 +47,22 @@ class InMemoryTableWithV2Filter( } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { - new InMemoryV2FilterScanBuilder(schema) + new InMemoryV2FilterScanBuilder(schema, options) } - class InMemoryV2FilterScanBuilder(tableSchema: StructType) - extends InMemoryScanBuilder(tableSchema) { + class InMemoryV2FilterScanBuilder( + tableSchema: StructType, + options: CaseInsensitiveStringMap) + extends InMemoryScanBuilder(tableSchema, options) { override def build: Scan = InMemoryV2FilterBatchScan( - data.map(_.asInstanceOf[InputPartition]).toImmutableArraySeq, schema, tableSchema) + data.map(_.asInstanceOf[InputPartition]).toImmutableArraySeq, schema, tableSchema, options) } case class InMemoryV2FilterBatchScan( var _data: Seq[InputPartition], readSchema: StructType, - tableSchema: StructType) + tableSchema: StructType, + options: CaseInsensitiveStringMap) extends BatchScanBaseClass(_data, readSchema, tableSchema) with SupportsRuntimeV2Filtering { override def filterAttributes(): Array[NamedReference] = { @@ -93,21 +96,21 @@ class InMemoryTableWithV2Filter( InMemoryBaseTable.maybeSimulateFailedTableWrite(new CaseInsensitiveStringMap(properties)) InMemoryBaseTable.maybeSimulateFailedTableWrite(info.options) - new InMemoryWriterBuilderWithOverWrite() + new InMemoryWriterBuilderWithOverWrite(info) } - private class InMemoryWriterBuilderWithOverWrite() extends InMemoryWriterBuilder - with SupportsOverwriteV2 { + class InMemoryWriterBuilderWithOverWrite(override val info: LogicalWriteInfo) + extends InMemoryWriterBuilder(info) with SupportsOverwriteV2 { override def truncate(): WriteBuilder = { - assert(writer == Append) - writer = TruncateAndAppend - streamingWriter = StreamingTruncateAndAppend + assert(writer.isInstanceOf[Append]) + writer = new TruncateAndAppend(info) + streamingWriter = new StreamingTruncateAndAppend(info) this } override def overwrite(predicates: Array[Predicate]): WriteBuilder = { - assert(writer == Append) + assert(writer.isInstanceOf[Append]) writer = new Overwrite(predicates) streamingWriter = new StreamingNotSupportedOperation( s"overwrite (${predicates.mkString("filters(", ", ", ")")})") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala index 319cc1c731577..17b2579ca873a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util.{Optional, UUID} +import scala.jdk.CollectionConverters._ + import org.apache.spark.sql.catalyst.expressions.PredicateHelper import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Project, ReplaceData, WriteDelta} import org.apache.spark.sql.catalyst.rules.Rule @@ -44,7 +46,8 @@ object V2Writes extends Rule[LogicalPlan] with PredicateHelper { override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case a @ AppendData(r: DataSourceV2Relation, query, options, _, None, _) => - val writeBuilder = newWriteBuilder(r.table, options, query.schema) + val writeOptions = mergeOptions(options, r.options.asScala.toMap) + val writeBuilder = newWriteBuilder(r.table, writeOptions, query.schema) val write = writeBuilder.build() val newQuery = DistributionAndOrderingUtils.prepareQuery(write, query, r.funCatalog) a.copy(write = Some(write), query = newQuery) @@ -61,7 +64,8 @@ object V2Writes extends Rule[LogicalPlan] with PredicateHelper { }.toArray val table = r.table - val writeBuilder = newWriteBuilder(table, options, query.schema) + val writeOptions = mergeOptions(options, r.options.asScala.toMap) + val writeBuilder = newWriteBuilder(table, writeOptions, query.schema) val write = writeBuilder match { case builder: SupportsTruncate if isTruncate(predicates) => builder.truncate().build() @@ -76,7 +80,8 @@ object V2Writes extends Rule[LogicalPlan] with PredicateHelper { case o @ OverwritePartitionsDynamic(r: DataSourceV2Relation, query, options, _, None) => val table = r.table - val writeBuilder = newWriteBuilder(table, options, query.schema) + val writeOptions = mergeOptions(options, r.options.asScala.toMap) + val writeBuilder = newWriteBuilder(table, writeOptions, query.schema) val write = writeBuilder match { case builder: SupportsDynamicOverwrite => builder.overwriteDynamicPartitions().build() @@ -87,31 +92,44 @@ object V2Writes extends Rule[LogicalPlan] with PredicateHelper { o.copy(write = Some(write), query = newQuery) case WriteToMicroBatchDataSource( - relation, table, query, queryId, writeOptions, outputMode, Some(batchId)) => - + relationOpt, table, query, queryId, options, outputMode, Some(batchId)) => + val writeOptions = mergeOptions( + options, relationOpt.map(r => r.options.asScala.toMap).getOrElse(Map.empty)) val writeBuilder = newWriteBuilder(table, writeOptions, query.schema, queryId) val write = buildWriteForMicroBatch(table, writeBuilder, outputMode) val microBatchWrite = new MicroBatchWrite(batchId, write.toStreaming) val customMetrics = write.supportedCustomMetrics.toImmutableArraySeq - val funCatalogOpt = relation.flatMap(_.funCatalog) + val funCatalogOpt = relationOpt.flatMap(_.funCatalog) val newQuery = DistributionAndOrderingUtils.prepareQuery(write, query, funCatalogOpt) - WriteToDataSourceV2(relation, microBatchWrite, newQuery, customMetrics) + WriteToDataSourceV2(relationOpt, microBatchWrite, newQuery, customMetrics) case rd @ ReplaceData(r: DataSourceV2Relation, _, query, _, _, None) => val rowSchema = DataTypeUtils.fromAttributes(rd.dataInput) - val writeBuilder = newWriteBuilder(r.table, Map.empty, rowSchema) + val writeOptions = mergeOptions(Map.empty, r.options.asScala.toMap) + val writeBuilder = newWriteBuilder(r.table, writeOptions, rowSchema) val write = writeBuilder.build() val newQuery = DistributionAndOrderingUtils.prepareQuery(write, query, r.funCatalog) // project away any metadata columns that could be used for distribution and ordering rd.copy(write = Some(write), query = Project(rd.dataInput, newQuery)) case wd @ WriteDelta(r: DataSourceV2Relation, _, query, _, projections, None) => - val deltaWriteBuilder = newDeltaWriteBuilder(r.table, Map.empty, projections) + val writeOptions = mergeOptions(Map.empty, r.options.asScala.toMap) + val deltaWriteBuilder = newDeltaWriteBuilder(r.table, writeOptions, projections) val deltaWrite = deltaWriteBuilder.build() val newQuery = DistributionAndOrderingUtils.prepareQuery(deltaWrite, query, r.funCatalog) wd.copy(write = Some(deltaWrite), query = newQuery) } + private def mergeOptions( + commandOptions: Map[String, String], + dsOptions: Map[String, String]): Map[String, String] = { + // for DataFrame API cases, same options are carried by both Command and DataSourceV2Relation + // for DataFrameV2 API cases, options are only carried by Command + // for SQL cases, options are only carried by DataSourceV2Relation + assert(commandOptions == dsOptions || commandOptions.isEmpty || dsOptions.isEmpty) + commandOptions ++ dsOptions + } + private def buildWriteForMicroBatch( table: SupportsWrite, writeBuilder: WriteBuilder, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 30180d48da71a..b59c83c23d3c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -27,7 +27,7 @@ import org.scalatest.Assertions import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.storage.StorageLevel @@ -449,12 +449,12 @@ object QueryTest extends Assertions { } } - def withPhysicalPlansCaptured(spark: SparkSession, thunk: => Unit): Seq[SparkPlan] = { - var capturedPlans = Seq.empty[SparkPlan] + def withQueryExecutionsCaptured(spark: SparkSession)(thunk: => Unit): Seq[QueryExecution] = { + var capturedQueryExecutions = Seq.empty[QueryExecution] val listener = new QueryExecutionListener { override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { - capturedPlans = capturedPlans :+ qe.executedPlan + capturedQueryExecutions = capturedQueryExecutions :+ qe } override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} } @@ -468,7 +468,7 @@ object QueryTest extends Assertions { spark.listenerManager.unregister(listener) } - capturedPlans + capturedQueryExecutions } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2OptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2OptionSuite.scala new file mode 100644 index 0000000000000..70291336ba317 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2OptionSuite.scala @@ -0,0 +1,327 @@ +/* + * 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.connector + +import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.QueryTest.withQueryExecutionsCaptured +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.catalog.InMemoryBaseTable +import org.apache.spark.sql.execution.CommandResultExec +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.functions.lit + +class DataSourceV2OptionSuite extends DatasourceV2SQLBase { + import testImplicits._ + + private val catalogAndNamespace = "testcat.ns1.ns2." + + test("SPARK-36680: Supports Dynamic Table Options for SQL Select") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") + + var df = sql(s"SELECT * FROM $t1") + var collected = df.queryExecution.optimizedPlan.collect { + case scan: DataSourceV2ScanRelation => + assert(scan.relation.options.isEmpty) + } + assert (collected.size == 1) + checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) + + df = sql(s"SELECT * FROM $t1 WITH (`split-size` = 5)") + collected = df.queryExecution.optimizedPlan.collect { + case scan: DataSourceV2ScanRelation => + assert(scan.relation.options.get("split-size") == "5") + } + assert (collected.size == 1) + checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) + + collected = df.queryExecution.executedPlan.collect { + case BatchScanExec(_, scan: InMemoryBaseTable#InMemoryBatchScan, _, _, _, _) => + assert(scan.options.get("split-size") === "5") + } + assert (collected.size == 1) + + val noValues = intercept[AnalysisException]( + sql(s"SELECT * FROM $t1 WITH (`split-size`)")) + assert(noValues.message.contains( + "Operation not allowed: Values must be specified for key(s): [split-size]")) + } + } + + test("SPARK-50286: Propagate options for DataFrameReader") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") + + var df = spark.table(t1) + var collected = df.queryExecution.optimizedPlan.collect { + case scan: DataSourceV2ScanRelation => + assert(scan.relation.options.isEmpty) + } + assert (collected.size == 1) + checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) + + df = spark.read.option("split-size", "5").table(t1) + collected = df.queryExecution.optimizedPlan.collect { + case scan: DataSourceV2ScanRelation => + assert(scan.relation.options.get("split-size") == "5") + } + assert (collected.size == 1) + checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) + + collected = df.queryExecution.executedPlan.collect { + case BatchScanExec(_, scan: InMemoryBaseTable#InMemoryBatchScan, _, _, _, _) => + assert(scan.options.get("split-size") === "5") + } + assert (collected.size == 1) + } + } + + test("SPARK-49098, SPARK-50286: Supports Dynamic Table Options for SQL Insert") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + val df = sql(s"INSERT INTO $t1 WITH (`write.split-size` = 10) VALUES (1, 'a'), (2, 'b')") + + var collected = df.queryExecution.optimizedPlan.collect { + case CommandResult(_, AppendData(relation: DataSourceV2Relation, _, _, _, _, _), _, _) => + assert(relation.options.get("write.split-size") == "10") + } + assert (collected.size == 1) + + collected = df.queryExecution.executedPlan.collect { + case CommandResultExec( + _, AppendDataExec(_, _, write), + _) => + val append = write.toBatch.asInstanceOf[InMemoryBaseTable#Append] + assert(append.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + + val insertResult = sql(s"SELECT * FROM $t1") + checkAnswer(insertResult, Seq(Row(1, "a"), Row(2, "b"))) + } + } + + test("SPARK-50286: Propagate options for DataFrameWriter Append") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + val captured = withQueryExecutionsCaptured(spark) { + Seq(1 -> "a", 2 -> "b").toDF("id", "data") + .write + .option("write.split-size", "10") + .mode("append") + .insertInto(t1) + } + assert(captured.size === 1) + val qe = captured.head + var collected = qe.optimizedPlan.collect { + case AppendData(_: DataSourceV2Relation, _, writeOptions, _, _, _) => + assert(writeOptions("write.split-size") == "10") + } + assert (collected.size == 1) + + collected = qe.executedPlan.collect { + case AppendDataExec(_, _, write) => + val append = write.toBatch.asInstanceOf[InMemoryBaseTable#Append] + assert(append.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + } + } + + test("SPARK-50286: Propagate options for DataFrameWriterV2 Append") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + val captured = withQueryExecutionsCaptured(spark) { + Seq(1 -> "a", 2 -> "b").toDF("id", "data") + .writeTo(t1) + .option("write.split-size", "10") + .append() + } + assert(captured.size === 1) + val qe = captured.head + var collected = qe.optimizedPlan.collect { + case AppendData(_: DataSourceV2Relation, _, writeOptions, _, _, _) => + assert(writeOptions("write.split-size") == "10") + } + assert (collected.size == 1) + + collected = qe.executedPlan.collect { + case AppendDataExec(_, _, write) => + val append = write.toBatch.asInstanceOf[InMemoryBaseTable#Append] + assert(append.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + } + } + + test("SPARK-49098, SPARK-50286: Supports Dynamic Table Options for SQL Insert Overwrite") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") + + val df = sql(s"INSERT OVERWRITE $t1 WITH (`write.split-size` = 10) " + + s"VALUES (3, 'c'), (4, 'd')") + var collected = df.queryExecution.optimizedPlan.collect { + case CommandResult(_, + OverwriteByExpression(relation: DataSourceV2Relation, _, _, _, _, _, _), + _, _) => + assert(relation.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + + collected = df.queryExecution.executedPlan.collect { + case CommandResultExec( + _, OverwriteByExpressionExec(_, _, write), + _) => + val append = write.toBatch.asInstanceOf[InMemoryBaseTable#TruncateAndAppend] + assert(append.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + + val insertResult = sql(s"SELECT * FROM $t1") + checkAnswer(insertResult, Seq(Row(3, "c"), Row(4, "d"))) + } + } + + test("SPARK-50286: Propagate options for DataFrameWriterV2 OverwritePartitions") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") + + val captured = withQueryExecutionsCaptured(spark) { + Seq(3 -> "c", 4 -> "d").toDF("id", "data") + .writeTo(t1) + .option("write.split-size", "10") + .overwritePartitions() + } + assert(captured.size === 1) + val qe = captured.head + var collected = qe.optimizedPlan.collect { + case OverwritePartitionsDynamic(_: DataSourceV2Relation, _, writeOptions, _, _) => + assert(writeOptions("write.split-size") === "10") + } + assert (collected.size == 1) + + collected = qe.executedPlan.collect { + case OverwritePartitionsDynamicExec(_, _, write) => + val dynOverwrite = write.toBatch.asInstanceOf[InMemoryBaseTable#DynamicOverwrite] + assert(dynOverwrite.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + } + } + + test("SPARK-49098, SPARK-50286: Supports Dynamic Table Options for SQL Insert Replace") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") + + val df = sql(s"INSERT INTO $t1 WITH (`write.split-size` = 10) " + + s"REPLACE WHERE TRUE " + + s"VALUES (3, 'c'), (4, 'd')") + var collected = df.queryExecution.optimizedPlan.collect { + case CommandResult(_, + OverwriteByExpression(relation: DataSourceV2Relation, _, _, _, _, _, _), + _, _) => + assert(relation.options.get("write.split-size") == "10") + } + assert (collected.size == 1) + + collected = df.queryExecution.executedPlan.collect { + case CommandResultExec( + _, OverwriteByExpressionExec(_, _, write), + _) => + val append = write.toBatch.asInstanceOf[InMemoryBaseTable#TruncateAndAppend] + assert(append.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + + val insertResult = sql(s"SELECT * FROM $t1") + checkAnswer(insertResult, Seq(Row(3, "c"), Row(4, "d"))) + } + } + + test("SPARK-50286: Propagate options for DataFrameWriter Overwrite") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + val captured = withQueryExecutionsCaptured(spark) { + Seq(1 -> "a", 2 -> "b").toDF("id", "data") + .write + .option("write.split-size", "10") + .mode("overwrite") + .insertInto(t1) + } + assert(captured.size === 1) + + val qe = captured.head + var collected = qe.optimizedPlan.collect { + case OverwriteByExpression(_: DataSourceV2Relation, _, _, writeOptions, _, _, _) => + assert(writeOptions("write.split-size") === "10") + } + assert (collected.size == 1) + + collected = qe.executedPlan.collect { + case OverwriteByExpressionExec(_, _, write) => + val append = write.toBatch.asInstanceOf[InMemoryBaseTable#TruncateAndAppend] + assert(append.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + } + } + + test("SPARK-50286: Propagate options for DataFrameWriterV2 Overwrite") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") + + val captured = withQueryExecutionsCaptured(spark) { + Seq(3 -> "c", 4 -> "d").toDF("id", "data") + .writeTo(t1) + .option("write.split-size", "10") + .overwrite(lit(true)) + } + assert(captured.size === 1) + val qe = captured.head + + var collected = qe.optimizedPlan.collect { + case OverwriteByExpression(_: DataSourceV2Relation, _, _, writeOptions, _, _, _) => + assert(writeOptions("write.split-size") === "10") + } + assert (collected.size == 1) + + collected = qe.executedPlan.collect { + case OverwriteByExpressionExec(_, _, write) => + val append = write.toBatch.asInstanceOf[InMemoryBaseTable#TruncateAndAppend] + assert(append.info.options.get("write.split-size") === "10") + } + assert (collected.size == 1) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 510ea49b58418..6a659fa6e3ee9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.CurrentUserContext.CURRENT_USER import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchNamespaceException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, ColumnStat, CommandResult, OverwriteByExpression} +import org.apache.spark.sql.catalyst.plans.logical.ColumnStat import org.apache.spark.sql.catalyst.statsEstimation.StatsEstimationTestBase import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.connector.catalog.{Column => ColumnV2, _} @@ -44,7 +44,6 @@ import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelationWithTable} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} @@ -3634,96 +3633,6 @@ class DataSourceV2SQLSuiteV1Filter } } - - test("SPARK-36680: Supports Dynamic Table Options for Spark SQL") { - val t1 = s"${catalogAndNamespace}table" - withTable(t1) { - sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") - sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") - - var df = sql(s"SELECT * FROM $t1") - var collected = df.queryExecution.optimizedPlan.collect { - case scan: DataSourceV2ScanRelation => - assert(scan.relation.options.isEmpty) - } - assert (collected.size == 1) - checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) - - df = sql(s"SELECT * FROM $t1 WITH (`split-size` = 5)") - collected = df.queryExecution.optimizedPlan.collect { - case scan: DataSourceV2ScanRelation => - assert(scan.relation.options.get("split-size") == "5") - } - assert (collected.size == 1) - checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) - - val noValues = intercept[AnalysisException]( - sql(s"SELECT * FROM $t1 WITH (`split-size`)")) - assert(noValues.message.contains( - "Operation not allowed: Values must be specified for key(s): [split-size]")) - } - } - - test("SPARK-36680: Supports Dynamic Table Options for Insert") { - val t1 = s"${catalogAndNamespace}table" - withTable(t1) { - sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") - val df = sql(s"INSERT INTO $t1 WITH (`write.split-size` = 10) VALUES (1, 'a'), (2, 'b')") - - val collected = df.queryExecution.optimizedPlan.collect { - case CommandResult(_, AppendData(relation: DataSourceV2Relation, _, _, _, _, _), _, _) => - assert(relation.options.get("write.split-size") == "10") - } - assert (collected.size == 1) - - val insertResult = sql(s"SELECT * FROM $t1") - checkAnswer(insertResult, Seq(Row(1, "a"), Row(2, "b"))) - } - } - - test("SPARK-36680: Supports Dynamic Table Options for Insert Overwrite") { - val t1 = s"${catalogAndNamespace}table" - withTable(t1) { - sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") - sql(s"INSERT INTO $t1 WITH (`write.split-size` = 10) VALUES (1, 'a'), (2, 'b')") - - val df = sql(s"INSERT OVERWRITE $t1 WITH (`write.split-size` = 10) " + - s"VALUES (3, 'c'), (4, 'd')") - val collected = df.queryExecution.optimizedPlan.collect { - case CommandResult(_, - OverwriteByExpression(relation: DataSourceV2Relation, _, _, _, _, _, _), - _, _) => - assert(relation.options.get("write.split-size") == "10") - } - assert (collected.size == 1) - - val insertResult = sql(s"SELECT * FROM $t1") - checkAnswer(insertResult, Seq(Row(3, "c"), Row(4, "d"))) - } - } - - test("SPARK-36680: Supports Dynamic Table Options for Insert Replace") { - val t1 = s"${catalogAndNamespace}table" - withTable(t1) { - sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") - sql(s"INSERT INTO $t1 WITH (`write.split-size` = 10) VALUES (1, 'a'), (2, 'b')") - - val df = sql(s"INSERT INTO $t1 WITH (`write.split-size` = 10) " + - s"REPLACE WHERE TRUE " + - s"VALUES (3, 'c'), (4, 'd')") - val collected = df.queryExecution.optimizedPlan.collect { - case CommandResult(_, - OverwriteByExpression(relation: DataSourceV2Relation, _, _, _, _, _, _), - _, _) => - assert(relation.options.get("write.split-size") == "10") - } - assert (collected.size == 1) - - val insertResult = sql(s"SELECT * FROM $t1") - checkAnswer(insertResult, Seq(Row(3, "c"), Row(4, "d"))) - } - } - test("SPARK-49183: custom spark_catalog generates location for managed tables") { // Reset CatalogManager to clear the materialized `spark_catalog` instance, so that we can // configure a new implementation. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index 04fc7e23ebb24..68c2a01c69aea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession, SQLContext} -import org.apache.spark.sql.QueryTest.withPhysicalPlansCaptured +import org.apache.spark.sql.QueryTest.withQueryExecutionsCaptured import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule @@ -213,8 +213,8 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before .getOrCreate() def captureWrite(sparkSession: SparkSession)(thunk: => Unit): SparkPlan = { - val physicalPlans = withPhysicalPlansCaptured(sparkSession, thunk) - val v1FallbackWritePlans = physicalPlans.filter { + val queryExecutions = withQueryExecutionsCaptured(sparkSession)(thunk) + val v1FallbackWritePlans = queryExecutions.map(_.executedPlan).filter { case _: AppendDataExecV1 | _: OverwriteByExpressionExecV1 => true case _ => false } From 5dc65a5416b51b9a41d36881cb919a9d10834511 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 21:54:32 +0800 Subject: [PATCH 016/438] [SPARK-50287][SQL] Merge options of table and relation when creating WriteBuilder in FileTable ### What changes were proposed in this pull request? Merge `options` of table and relation when creating WriteBuilder in FileTable. ### Why are the changes needed? Similar to SPARK-49519 which fixes the read path. ### Does this PR introduce _any_ user-facing change? FileTable's options are accounted on the V2 write path now, but given the built-in file formats use V1 by default, it has no real effect. ### How was this patch tested? UT is updated to cover the case. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48821 from pan3793/SPARK-50287. Authored-by: Cheng Pan Signed-off-by: Wenchen Fan --- .../apache/spark/sql/v2/avro/AvroTable.scala | 6 +++-- .../execution/datasources/v2/FileTable.scala | 14 ++++++++++ .../execution/datasources/v2/FileWrite.scala | 2 +- .../datasources/v2/csv/CSVTable.scala | 6 +++-- .../datasources/v2/json/JsonTable.scala | 6 +++-- .../datasources/v2/orc/OrcTable.scala | 6 +++-- .../datasources/v2/parquet/ParquetTable.scala | 6 +++-- .../datasources/v2/text/TextTable.scala | 6 +++-- .../datasources/v2/FileTableSuite.scala | 27 ++++++++++++------- 9 files changed, 57 insertions(+), 22 deletions(-) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala b/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala index 8ec711b2757f5..e898253be1168 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala @@ -42,10 +42,12 @@ case class AvroTable( override def inferSchema(files: Seq[FileStatus]): Option[StructType] = AvroUtils.inferSchema(sparkSession, options.asScala.toMap, files) - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { new WriteBuilder { - override def build(): Write = AvroWrite(paths, formatName, supportsDataType, info) + override def build(): Write = + AvroWrite(paths, formatName, supportsDataType, mergedWriteInfo(info)) } + } override def supportsDataType(dataType: DataType): Boolean = AvroUtils.supportsDataType(dataType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 4eee731e0b2d6..863104da80c2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex} @@ -159,6 +160,19 @@ abstract class FileTable( options.asCaseSensitiveMap().asScala new CaseInsensitiveStringMap(finalOptions.asJava) } + + /** + * Merge the options of FileTable and the LogicalWriteInfo while respecting the + * keys of the options carried by LogicalWriteInfo. + */ + protected def mergedWriteInfo(writeInfo: LogicalWriteInfo): LogicalWriteInfo = { + LogicalWriteInfoImpl( + writeInfo.queryId(), + writeInfo.schema(), + mergedOptions(writeInfo.options()), + writeInfo.rowIdSchema(), + writeInfo.metadataSchema()) + } } object FileTable { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala index f4cabcb69d08c..77e1ade44780f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala @@ -49,7 +49,7 @@ trait FileWrite extends Write { private val schema = info.schema() private val queryId = info.queryId() - private val options = info.options() + val options = info.options() override def description(): String = formatName diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala index 4c201ca66cf6c..df8df37b711fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala @@ -49,10 +49,12 @@ case class CSVTable( CSVDataSource(parsedOptions).inferSchema(sparkSession, files, parsedOptions) } - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { new WriteBuilder { - override def build(): Write = CSVWrite(paths, formatName, supportsDataType, info) + override def build(): Write = + CSVWrite(paths, formatName, supportsDataType, mergedWriteInfo(info)) } + } override def supportsDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala index 54244c4d95e77..1c1d3393b95a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala @@ -49,10 +49,12 @@ case class JsonTable( sparkSession, files, parsedOptions) } - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { new WriteBuilder { - override def build(): Write = JsonWrite(paths, formatName, supportsDataType, info) + override def build(): Write = + JsonWrite(paths, formatName, supportsDataType, mergedWriteInfo(info)) } + } override def supportsDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala index 1037370967c87..81c347ae9c59c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala @@ -43,10 +43,12 @@ case class OrcTable( override def inferSchema(files: Seq[FileStatus]): Option[StructType] = OrcUtils.inferSchema(sparkSession, files, options.asScala.toMap) - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { new WriteBuilder { - override def build(): Write = OrcWrite(paths, formatName, supportsDataType, info) + override def build(): Write = + OrcWrite(paths, formatName, supportsDataType, mergedWriteInfo(info)) } + } override def supportsDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala index 8463a05569c05..28c5a62f91ecb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala @@ -43,10 +43,12 @@ case class ParquetTable( override def inferSchema(files: Seq[FileStatus]): Option[StructType] = ParquetUtils.inferSchema(sparkSession, options.asScala.toMap, files) - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { new WriteBuilder { - override def build(): Write = ParquetWrite(paths, formatName, supportsDataType, info) + override def build(): Write = + ParquetWrite(paths, formatName, supportsDataType, mergedWriteInfo(info)) } + } override def supportsDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala index 87ae34532f88a..d8880b84c6211 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala @@ -39,10 +39,12 @@ case class TextTable( override def inferSchema(files: Seq[FileStatus]): Option[StructType] = Some(StructType(Array(StructField("value", StringType)))) - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { new WriteBuilder { - override def build(): Write = TextWrite(paths, formatName, supportsDataType, info) + override def build(): Write = + TextWrite(paths, formatName, supportsDataType, mergedWriteInfo(info)) } + } override def supportsDataType(dataType: DataType): Boolean = dataType == StringType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala index 0316f09e42ce3..0d18e3bf809e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.{QueryTest, SparkSession} import org.apache.spark.sql.connector.read.ScanBuilder -import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl, WriteBuilder} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.text.TextFileFormat @@ -96,8 +96,8 @@ class FileTableSuite extends QueryTest with SharedSparkSession { } allFileBasedDataSources.foreach { format => - test(s"SPARK-49519: Merge options of table and relation when constructing FileScanBuilder" + - s" - $format") { + test("SPARK-49519, SPARK-50287: Merge options of table and relation when " + + s"constructing ScanBuilder and WriteBuilder in FileFormat - $format") { withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { val userSpecifiedSchema = StructType(Seq(StructField("c1", StringType))) @@ -108,20 +108,29 @@ class FileTableSuite extends QueryTest with SharedSparkSession { val table = provider.getTable( userSpecifiedSchema, Array.empty, - dsOptions.asCaseSensitiveMap()) + dsOptions.asCaseSensitiveMap()).asInstanceOf[FileTable] val tableOptions = new CaseInsensitiveStringMap( Map("k2" -> "table_v2", "k3" -> "v3").asJava) - val mergedOptions = table.asInstanceOf[FileTable].newScanBuilder(tableOptions) match { + + val mergedReadOptions = table.newScanBuilder(tableOptions) match { case csv: CSVScanBuilder => csv.options case json: JsonScanBuilder => json.options case orc: OrcScanBuilder => orc.options case parquet: ParquetScanBuilder => parquet.options case text: TextScanBuilder => text.options } - assert(mergedOptions.size() == 3) - assert("v1".equals(mergedOptions.get("k1"))) - assert("table_v2".equals(mergedOptions.get("k2"))) - assert("v3".equals(mergedOptions.get("k3"))) + assert(mergedReadOptions.size === 3) + assert(mergedReadOptions.get("k1") === "v1") + assert(mergedReadOptions.get("k2") === "table_v2") + assert(mergedReadOptions.get("k3") === "v3") + + val writeInfo = LogicalWriteInfoImpl("query-id", userSpecifiedSchema, tableOptions) + val mergedWriteOptions = table.newWriteBuilder(writeInfo).build() + .asInstanceOf[FileWrite].options + assert(mergedWriteOptions.size === 3) + assert(mergedWriteOptions.get("k1") === "v1") + assert(mergedWriteOptions.get("k2") === "table_v2") + assert(mergedWriteOptions.get("k3") === "v3") case _ => throw new IllegalArgumentException(s"Failed to get table provider for $format") } From 214fd53af7fbf32b30829340e330f93f7132674a Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 09:33:00 -0800 Subject: [PATCH 017/438] [SPARK-45151][UI][FOLLOWUP] Add missing space before date value ### What changes were proposed in this pull request? Add missing space before date value image ### Why are the changes needed? A minor UI fix, and make it consistent with https://github.com/apache/spark/blob/28951ed6681f9f0ad28908fd92b2c4a871fa5958/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala#L75 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48952 from pan3793/SPARK-45151-space. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/ui/jobs/TaskThreadDumpPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskThreadDumpPage.scala index 49b919ce0de97..5fe542d4fa21e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskThreadDumpPage.scala @@ -71,7 +71,7 @@ private[spark] class TaskThreadDumpPage(

-

Updated at{UIUtils.formatDate(time)}

+

Updated at {UIUtils.formatDate(time)}

From 6667ba27cbad33bc2ad53f2d651b70168ada8b1a Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 09:36:29 -0800 Subject: [PATCH 018/438] [SPARK-50412][BUILD] Rename `ojdbc11.version` to `ojdbc17.version` ### What changes were proposed in this pull request? A super nit, renaming `ojdbc11.version` to `ojdbc17.version` to make the property name more intuitive. ### Why are the changes needed? SPARK-50268 upgrades the Oracle JDBC driver from ojdbc11 to ojdbc17, but leaves the maven property name as `ojdbc11.version`, this looks too weird. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? GHA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48957 from pan3793/SPARK-50412. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 4482f3f49badf..cfd5584e9861d 100644 --- a/pom.xml +++ b/pom.xml @@ -334,7 +334,7 @@ 42.7.4 11.5.9.0 12.8.1.jre11 - 23.6.0.24.10 + 23.6.0.24.10 ${project.version} @@ -1348,7 +1348,7 @@ com.oracle.database.jdbc ojdbc17 - ${ojdbc11.version} + ${ojdbc17.version} test From f7122137006e941393c8be619fb51b3b713a24cb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 26 Nov 2024 09:22:22 +0900 Subject: [PATCH 019/438] [SPARK-50415][BUILD] Upgrade `zstd-jni` to 1.5.6-8 ### What changes were proposed in this pull request? This PR aims to upgrade `zstd-jni` to `1.5.6-8`. ### Why are the changes needed? To bring the following bug fix, - [Guard all native code execution behind the shared lock in the ZstdCompressionCtx and ZstdDecompressionCtx](https://github.com/luben/zstd-jni/commit/cec96538e692336eae11b033d9f09e7be30ff075) Here is the release tag. - https://github.com/luben/zstd-jni/releases/tag/v1.5.6-8 ### 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 #48959 from dongjoon-hyun/SPARK-50415. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- 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 902c9787adc56..aef06b8a20f53 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -280,4 +280,4 @@ xz/1.10//xz-1.10.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar zookeeper-jute/3.9.3//zookeeper-jute-3.9.3.jar zookeeper/3.9.3//zookeeper-3.9.3.jar -zstd-jni/1.5.6-7//zstd-jni-1.5.6-7.jar +zstd-jni/1.5.6-8//zstd-jni-1.5.6-8.jar diff --git a/pom.xml b/pom.xml index cfd5584e9861d..65c644cc0690a 100644 --- a/pom.xml +++ b/pom.xml @@ -839,7 +839,7 @@ com.github.luben zstd-jni - 1.5.6-7 + 1.5.6-8 com.clearspring.analytics From 331d0bf30092be62191476e4a679b403e1a369b9 Mon Sep 17 00:00:00 2001 From: Eric Marnadi Date: Tue, 26 Nov 2024 13:33:04 +0900 Subject: [PATCH 020/438] [SPARK-50017][SS] Support Avro encoding for TransformWithState operator ### What changes were proposed in this pull request? Currently, we use the internal byte representation to store state for stateful streaming operators in the StateStore. This PR introduces Avro serialization and deserialization capabilities in the RocksDBStateEncoder so that we can instead use Avro encoding to store state. This is currently enabled for the TransformWithState operator via SQLConf to support all functionality supported by TWS ### Why are the changes needed? UnsafeRow is an inherently unstable format that makes no guarantees of being backwards-compatible. Therefore, if the format changes between Spark releases, this could cause StateStore corruptions. Avro is more stable, and inherently enables schema evolution. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Amended and added to unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #48401 from ericm-db/avro. Lead-authored-by: Eric Marnadi Co-authored-by: Eric Marnadi <132308037+ericm-db@users.noreply.github.com> Signed-off-by: Jungtaek Lim --- .../apache/spark/sql/internal/SQLConf.scala | 13 + .../StateStoreColumnFamilySchemaUtils.scala | 41 +- .../execution/streaming/StreamExecution.scala | 1 + .../streaming/state/RocksDBStateEncoder.scala | 538 ++++++++++++++++-- .../state/RocksDBStateStoreProvider.scala | 120 +++- .../StateSchemaCompatibilityChecker.scala | 25 + .../streaming/state/StateStore.scala | 15 +- .../streaming/state/StateStoreConf.scala | 3 + ...ateDataSourceTransformWithStateSuite.scala | 4 +- .../state/RocksDBStateStoreSuite.scala | 1 + .../streaming/state/RocksDBSuite.scala | 13 + .../streaming/state/ValueStateSuite.scala | 2 +- .../TransformWithListStateSuite.scala | 5 +- .../TransformWithMapStateSuite.scala | 5 +- .../streaming/TransformWithStateSuite.scala | 17 +- .../streaming/TransformWithStateTTLTest.scala | 5 +- .../TransformWithValueStateTTLSuite.scala | 3 +- 17 files changed, 744 insertions(+), 67 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index ba0a37541e490..378eca09097f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2221,6 +2221,17 @@ object SQLConf { .intConf .createWithDefault(1) + val STREAMING_STATE_STORE_ENCODING_FORMAT = + buildConf("spark.sql.streaming.stateStore.encodingFormat") + .doc("The encoding format used for stateful operators to store information " + + "in the state store") + .version("4.0.0") + .stringConf + .transform(_.toLowerCase(Locale.ROOT)) + .checkValue(v => Set("unsaferow", "avro").contains(v), + "Valid values are 'unsaferow' and 'avro'") + .createWithDefault("unsaferow") + val STATE_STORE_COMPRESSION_CODEC = buildConf("spark.sql.streaming.stateStore.compression.codec") .internal() @@ -5596,6 +5607,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def stateStoreCheckpointFormatVersion: Int = getConf(STATE_STORE_CHECKPOINT_FORMAT_VERSION) + def stateStoreEncodingFormat: String = getConf(STREAMING_STATE_STORE_ENCODING_FORMAT) + def checkpointRenamedFileCheck: Boolean = getConf(CHECKPOINT_RENAMEDFILE_CHECK_ENABLED) def parquetFilterPushDown: Boolean = getConf(PARQUET_FILTER_PUSHDOWN_ENABLED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala index 7da8408f98b0f..585298fa4c993 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala @@ -20,10 +20,49 @@ import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, StateStoreColFamilySchema} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ object StateStoreColumnFamilySchemaUtils { + /** + * Avro uses zig-zag encoding for some fixed-length types, like Longs and Ints. For range scans + * we want to use big-endian encoding, so we need to convert the source schema to replace these + * types with BinaryType. + * + * @param schema The schema to convert + * @param ordinals If non-empty, only convert fields at these ordinals. + * If empty, convert all fields. + */ + def convertForRangeScan(schema: StructType, ordinals: Seq[Int] = Seq.empty): StructType = { + val ordinalSet = ordinals.toSet + + StructType(schema.fields.zipWithIndex.flatMap { case (field, idx) => + if ((ordinals.isEmpty || ordinalSet.contains(idx)) && isFixedSize(field.dataType)) { + // For each numeric field, create two fields: + // 1. Byte marker for null, positive, or negative values + // 2. The original numeric value in big-endian format + // Byte type is converted to Int in Avro, which doesn't work for us as Avro + // uses zig-zag encoding as opposed to big-endian for Ints + Seq( + StructField(s"${field.name}_marker", BinaryType, nullable = false), + field.copy(name = s"${field.name}_value", BinaryType) + ) + } else { + Seq(field) + } + }) + } + + private def isFixedSize(dataType: DataType): Boolean = dataType match { + case _: ByteType | _: BooleanType | _: ShortType | _: IntegerType | _: LongType | + _: FloatType | _: DoubleType => true + case _ => false + } + + def getTtlColFamilyName(stateName: String): String = { + "$ttl_" + stateName + } + def getValueStateSchema[T]( stateName: String, keyEncoder: ExpressionEncoder[Any], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index bd501c9357234..44202bb0d2944 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -715,6 +715,7 @@ abstract class StreamExecution( object StreamExecution { val QUERY_ID_KEY = "sql.streaming.queryId" + val RUN_ID_KEY = "sql.streaming.runId" val IS_CONTINUOUS_PROCESSING = "__is_continuous_processing" val IO_EXCEPTION_NAMES = Seq( classOf[InterruptedException].getName, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index 4c7a226e0973f..f39022c1f53a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -17,13 +17,21 @@ package org.apache.spark.sql.execution.streaming.state +import java.io.ByteArrayOutputStream import java.lang.Double.{doubleToRawLongBits, longBitsToDouble} import java.lang.Float.{floatToRawIntBits, intBitsToFloat} import java.nio.{ByteBuffer, ByteOrder} +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWriter, GenericRecord} +import org.apache.avro.io.{DecoderFactory, EncoderFactory} + import org.apache.spark.internal.Logging +import org.apache.spark.sql.avro.{AvroDeserializer, AvroSerializer, SchemaConverters} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BoundReference, JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter +import org.apache.spark.sql.execution.streaming.StateStoreColumnFamilySchemaUtils import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider.{STATE_ENCODING_NUM_VERSION_BYTES, STATE_ENCODING_VERSION, VIRTUAL_COL_FAMILY_PREFIX_BYTES} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform @@ -49,6 +57,7 @@ abstract class RocksDBKeyStateEncoderBase( def offsetForColFamilyPrefix: Int = if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + val out = new ByteArrayOutputStream /** * Get Byte Array for the virtual column family id that is used as prefix for * key state rows. @@ -89,23 +98,24 @@ abstract class RocksDBKeyStateEncoderBase( } } -object RocksDBStateEncoder { +object RocksDBStateEncoder extends Logging { def getKeyEncoder( keyStateEncoderSpec: KeyStateEncoderSpec, useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short] = None): RocksDBKeyStateEncoder = { + virtualColFamilyId: Option[Short] = None, + avroEnc: Option[AvroEncoder] = None): RocksDBKeyStateEncoder = { // Return the key state encoder based on the requested type keyStateEncoderSpec match { case NoPrefixKeyStateEncoderSpec(keySchema) => - new NoPrefixKeyStateEncoder(keySchema, useColumnFamilies, virtualColFamilyId) + new NoPrefixKeyStateEncoder(keySchema, useColumnFamilies, virtualColFamilyId, avroEnc) case PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) => new PrefixKeyScanStateEncoder(keySchema, numColsPrefixKey, - useColumnFamilies, virtualColFamilyId) + useColumnFamilies, virtualColFamilyId, avroEnc) case RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals) => new RangeKeyScanStateEncoder(keySchema, orderingOrdinals, - useColumnFamilies, virtualColFamilyId) + useColumnFamilies, virtualColFamilyId, avroEnc) case _ => throw new IllegalArgumentException(s"Unsupported key state encoder spec: " + @@ -115,11 +125,12 @@ object RocksDBStateEncoder { def getValueEncoder( valueSchema: StructType, - useMultipleValuesPerKey: Boolean): RocksDBValueStateEncoder = { + useMultipleValuesPerKey: Boolean, + avroEnc: Option[AvroEncoder] = None): RocksDBValueStateEncoder = { if (useMultipleValuesPerKey) { - new MultiValuedStateEncoder(valueSchema) + new MultiValuedStateEncoder(valueSchema, avroEnc) } else { - new SingleValueStateEncoder(valueSchema) + new SingleValueStateEncoder(valueSchema, avroEnc) } } @@ -145,6 +156,26 @@ object RocksDBStateEncoder { encodedBytes } + /** + * This method takes an UnsafeRow, and serializes to a byte array using Avro encoding. + */ + def encodeUnsafeRowToAvro( + row: UnsafeRow, + avroSerializer: AvroSerializer, + valueAvroType: Schema, + out: ByteArrayOutputStream): Array[Byte] = { + // InternalRow -> Avro.GenericDataRecord + val avroData = + avroSerializer.serialize(row) + out.reset() + val encoder = EncoderFactory.get().directBinaryEncoder(out, null) + val writer = new GenericDatumWriter[Any]( + valueAvroType) // Defining Avro writer for this struct type + writer.write(avroData, encoder) // Avro.GenericDataRecord -> byte array + encoder.flush() + out.toByteArray + } + def decodeToUnsafeRow(bytes: Array[Byte], numFields: Int): UnsafeRow = { if (bytes != null) { val row = new UnsafeRow(numFields) @@ -154,6 +185,26 @@ object RocksDBStateEncoder { } } + /** + * This method takes a byte array written using Avro encoding, and + * deserializes to an UnsafeRow using the Avro deserializer + */ + def decodeFromAvroToUnsafeRow( + valueBytes: Array[Byte], + avroDeserializer: AvroDeserializer, + valueAvroType: Schema, + valueProj: UnsafeProjection): UnsafeRow = { + val reader = new GenericDatumReader[Any](valueAvroType) + val decoder = DecoderFactory.get().binaryDecoder(valueBytes, 0, valueBytes.length, null) + // bytes -> Avro.GenericDataRecord + val genericData = reader.read(null, decoder) + // Avro.GenericDataRecord -> InternalRow + val internalRow = avroDeserializer.deserialize( + genericData).orNull.asInstanceOf[InternalRow] + // InternalRow -> UnsafeRow + valueProj.apply(internalRow) + } + def decodeToUnsafeRow(bytes: Array[Byte], reusedRow: UnsafeRow): UnsafeRow = { if (bytes != null) { // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. @@ -174,16 +225,20 @@ object RocksDBStateEncoder { * @param keySchema - schema of the key to be encoded * @param numColsPrefixKey - number of columns to be used for prefix key * @param useColumnFamilies - if column family is enabled for this encoder + * @param avroEnc - if Avro encoding is specified for this StateEncoder, this encoder will + * be defined */ class PrefixKeyScanStateEncoder( keySchema: StructType, numColsPrefixKey: Int, useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None) + virtualColFamilyId: Option[Short] = None, + avroEnc: Option[AvroEncoder] = None) extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) { import RocksDBStateEncoder._ + private val usingAvroEncoding = avroEnc.isDefined private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { keySchema.zipWithIndex.take(numColsPrefixKey) } @@ -203,6 +258,18 @@ class PrefixKeyScanStateEncoder( UnsafeProjection.create(refs) } + // Prefix Key schema and projection definitions used by the Avro Serializers + // and Deserializers + private val prefixKeySchema = StructType(keySchema.take(numColsPrefixKey)) + private lazy val prefixKeyAvroType = SchemaConverters.toAvroType(prefixKeySchema) + private val prefixKeyProj = UnsafeProjection.create(prefixKeySchema) + + // Remaining Key schema and projection definitions used by the Avro Serializers + // and Deserializers + private val remainingKeySchema = StructType(keySchema.drop(numColsPrefixKey)) + private lazy val remainingKeyAvroType = SchemaConverters.toAvroType(remainingKeySchema) + private val remainingKeyProj = UnsafeProjection.create(remainingKeySchema) + // This is quite simple to do - just bind sequentially, as we don't change the order. private val restoreKeyProjection: UnsafeProjection = UnsafeProjection.create(keySchema) @@ -210,9 +277,24 @@ class PrefixKeyScanStateEncoder( private val joinedRowOnKey = new JoinedRow() override def encodeKey(row: UnsafeRow): Array[Byte] = { - val prefixKeyEncoded = encodeUnsafeRow(extractPrefixKey(row)) - val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) - + val (prefixKeyEncoded, remainingEncoded) = if (usingAvroEncoding) { + ( + encodeUnsafeRowToAvro( + extractPrefixKey(row), + avroEnc.get.keySerializer, + prefixKeyAvroType, + out + ), + encodeUnsafeRowToAvro( + remainingKeyProjection(row), + avroEnc.get.suffixKeySerializer.get, + remainingKeyAvroType, + out + ) + ) + } else { + (encodeUnsafeRow(extractPrefixKey(row)), encodeUnsafeRow(remainingKeyProjection(row))) + } val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( prefixKeyEncoded.length + remainingEncoded.length + 4 ) @@ -243,9 +325,25 @@ class PrefixKeyScanStateEncoder( Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4 + prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) - val prefixKeyDecoded = decodeToUnsafeRow(prefixKeyEncoded, numFields = numColsPrefixKey) - val remainingKeyDecoded = decodeToUnsafeRow(remainingKeyEncoded, - numFields = keySchema.length - numColsPrefixKey) + val (prefixKeyDecoded, remainingKeyDecoded) = if (usingAvroEncoding) { + ( + decodeFromAvroToUnsafeRow( + prefixKeyEncoded, + avroEnc.get.keyDeserializer, + prefixKeyAvroType, + prefixKeyProj + ), + decodeFromAvroToUnsafeRow( + remainingKeyEncoded, + avroEnc.get.suffixKeyDeserializer.get, + remainingKeyAvroType, + remainingKeyProj + ) + ) + } else { + (decodeToUnsafeRow(prefixKeyEncoded, numFields = numColsPrefixKey), + decodeToUnsafeRow(remainingKeyEncoded, numFields = keySchema.length - numColsPrefixKey)) + } restoreKeyProjection(joinedRowOnKey.withLeft(prefixKeyDecoded).withRight(remainingKeyDecoded)) } @@ -255,7 +353,11 @@ class PrefixKeyScanStateEncoder( } override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { - val prefixKeyEncoded = encodeUnsafeRow(prefixKey) + val prefixKeyEncoded = if (usingAvroEncoding) { + encodeUnsafeRowToAvro(prefixKey, avroEnc.get.keySerializer, prefixKeyAvroType, out) + } else { + encodeUnsafeRow(prefixKey) + } val (prefix, startingOffset) = encodeColumnFamilyPrefix( prefixKeyEncoded.length + 4 ) @@ -299,13 +401,16 @@ class PrefixKeyScanStateEncoder( * @param keySchema - schema of the key to be encoded * @param orderingOrdinals - the ordinals for which the range scan is constructed * @param useColumnFamilies - if column family is enabled for this encoder + * @param avroEnc - if Avro encoding is specified for this StateEncoder, this encoder will + * be defined */ class RangeKeyScanStateEncoder( keySchema: StructType, orderingOrdinals: Seq[Int], useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None) - extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) { + virtualColFamilyId: Option[Short] = None, + avroEnc: Option[AvroEncoder] = None) + extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { import RocksDBStateEncoder._ @@ -374,6 +479,22 @@ class RangeKeyScanStateEncoder( UnsafeProjection.create(refs) } + private val rangeScanAvroSchema = StateStoreColumnFamilySchemaUtils.convertForRangeScan( + StructType(rangeScanKeyFieldsWithOrdinal.map(_._1).toArray)) + + private lazy val rangeScanAvroType = SchemaConverters.toAvroType(rangeScanAvroSchema) + + private val rangeScanAvroProjection = UnsafeProjection.create(rangeScanAvroSchema) + + // Existing remainder key schema stuff + private val remainingKeySchema = StructType( + 0.to(keySchema.length - 1).diff(orderingOrdinals).map(keySchema(_)) + ) + + private lazy val remainingKeyAvroType = SchemaConverters.toAvroType(remainingKeySchema) + + private val remainingKeyAvroProjection = UnsafeProjection.create(remainingKeySchema) + // Reusable objects private val joinedRowOnKey = new JoinedRow() @@ -563,13 +684,272 @@ class RangeKeyScanStateEncoder( writer.getRow() } + /** + * Encodes an UnsafeRow into an Avro-compatible byte array format for range scan operations. + * + * This method transforms row data into a binary format that preserves ordering when + * used in range scans. + * For each field in the row: + * - A marker byte is written to indicate null status or sign (for numeric types) + * - The value is written in big-endian format + * + * Special handling is implemented for: + * - Null values: marked with nullValMarker followed by zero bytes + * - Negative numbers: marked with negativeValMarker + * - Floating point numbers: bit manipulation to handle sign and NaN values correctly + * + * @param row The UnsafeRow to encode + * @param avroType The Avro schema defining the structure for encoding + * @return Array[Byte] containing the Avro-encoded data that preserves ordering for range scans + * @throws UnsupportedOperationException if a field's data type is not supported for range + * scan encoding + */ + def encodePrefixKeyForRangeScan( + row: UnsafeRow, + avroType: Schema): Array[Byte] = { + val record = new GenericData.Record(avroType) + var fieldIdx = 0 + rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => + val field = fieldWithOrdinal._1 + val value = row.get(idx, field.dataType) + + // Create marker byte buffer + val markerBuffer = ByteBuffer.allocate(1) + markerBuffer.order(ByteOrder.BIG_ENDIAN) + + if (value == null) { + markerBuffer.put(nullValMarker) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + record.put(fieldIdx + 1, ByteBuffer.wrap(new Array[Byte](field.dataType.defaultSize))) + } else { + field.dataType match { + case BooleanType => + markerBuffer.put(positiveValMarker) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + val valueBuffer = ByteBuffer.allocate(1) + valueBuffer.put(if (value.asInstanceOf[Boolean]) 1.toByte else 0.toByte) + record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) + + case ByteType => + val byteVal = value.asInstanceOf[Byte] + markerBuffer.put(if (byteVal < 0) negativeValMarker else positiveValMarker) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + + val valueBuffer = ByteBuffer.allocate(1) + valueBuffer.order(ByteOrder.BIG_ENDIAN) + valueBuffer.put(byteVal) + record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) + + case ShortType => + val shortVal = value.asInstanceOf[Short] + markerBuffer.put(if (shortVal < 0) negativeValMarker else positiveValMarker) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + + val valueBuffer = ByteBuffer.allocate(2) + valueBuffer.order(ByteOrder.BIG_ENDIAN) + valueBuffer.putShort(shortVal) + record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) + + case IntegerType => + val intVal = value.asInstanceOf[Int] + markerBuffer.put(if (intVal < 0) negativeValMarker else positiveValMarker) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + + val valueBuffer = ByteBuffer.allocate(4) + valueBuffer.order(ByteOrder.BIG_ENDIAN) + valueBuffer.putInt(intVal) + record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) + + case LongType => + val longVal = value.asInstanceOf[Long] + markerBuffer.put(if (longVal < 0) negativeValMarker else positiveValMarker) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + + val valueBuffer = ByteBuffer.allocate(8) + valueBuffer.order(ByteOrder.BIG_ENDIAN) + valueBuffer.putLong(longVal) + record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) + + case FloatType => + val floatVal = value.asInstanceOf[Float] + val rawBits = floatToRawIntBits(floatVal) + markerBuffer.put(if ((rawBits & floatSignBitMask) != 0) { + negativeValMarker + } else { + positiveValMarker + }) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + + val valueBuffer = ByteBuffer.allocate(4) + valueBuffer.order(ByteOrder.BIG_ENDIAN) + if ((rawBits & floatSignBitMask) != 0) { + val updatedVal = rawBits ^ floatFlipBitMask + valueBuffer.putFloat(intBitsToFloat(updatedVal)) + } else { + valueBuffer.putFloat(floatVal) + } + record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) + + case DoubleType => + val doubleVal = value.asInstanceOf[Double] + val rawBits = doubleToRawLongBits(doubleVal) + markerBuffer.put(if ((rawBits & doubleSignBitMask) != 0) { + negativeValMarker + } else { + positiveValMarker + }) + record.put(fieldIdx, ByteBuffer.wrap(markerBuffer.array())) + + val valueBuffer = ByteBuffer.allocate(8) + valueBuffer.order(ByteOrder.BIG_ENDIAN) + if ((rawBits & doubleSignBitMask) != 0) { + val updatedVal = rawBits ^ doubleFlipBitMask + valueBuffer.putDouble(longBitsToDouble(updatedVal)) + } else { + valueBuffer.putDouble(doubleVal) + } + record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) + + case _ => throw new UnsupportedOperationException( + s"Range scan encoding not supported for data type: ${field.dataType}") + } + } + fieldIdx += 2 + } + + out.reset() + val writer = new GenericDatumWriter[GenericRecord](rangeScanAvroType) + val encoder = EncoderFactory.get().binaryEncoder(out, null) + writer.write(record, encoder) + encoder.flush() + out.toByteArray + } + + /** + * Decodes an Avro-encoded byte array back into an UnsafeRow for range scan operations. + * + * This method reverses the encoding process performed by encodePrefixKeyForRangeScan: + * - Reads the marker byte to determine null status or sign + * - Reconstructs the original values from big-endian format + * - Handles special cases for floating point numbers by reversing bit manipulations + * + * The decoding process preserves the original data types and values, including: + * - Null values marked by nullValMarker + * - Sign information for numeric types + * - Proper restoration of negative floating point values + * + * @param bytes The Avro-encoded byte array to decode + * @param avroType The Avro schema defining the structure for decoding + * @return UnsafeRow containing the decoded data + * @throws UnsupportedOperationException if a field's data type is not supported for range + * scan decoding + */ + def decodePrefixKeyForRangeScan( + bytes: Array[Byte], + avroType: Schema): UnsafeRow = { + + val reader = new GenericDatumReader[GenericRecord](avroType) + val decoder = DecoderFactory.get().binaryDecoder(bytes, 0, bytes.length, null) + val record = reader.read(null, decoder) + + val rowWriter = new UnsafeRowWriter(rangeScanKeyFieldsWithOrdinal.length) + rowWriter.resetRowWriter() + + var fieldIdx = 0 + rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => + val field = fieldWithOrdinal._1 + + val markerBytes = record.get(fieldIdx).asInstanceOf[ByteBuffer].array() + val markerBuf = ByteBuffer.wrap(markerBytes) + markerBuf.order(ByteOrder.BIG_ENDIAN) + val marker = markerBuf.get() + + if (marker == nullValMarker) { + rowWriter.setNullAt(idx) + } else { + field.dataType match { + case BooleanType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + rowWriter.write(idx, bytes(0) == 1) + + case ByteType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.get()) + + case ShortType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.getShort()) + + case IntegerType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.getInt()) + + case LongType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.getLong()) + + case FloatType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + if (marker == negativeValMarker) { + val floatVal = valueBuf.getFloat + val updatedVal = floatToRawIntBits(floatVal) ^ floatFlipBitMask + rowWriter.write(idx, intBitsToFloat(updatedVal)) + } else { + rowWriter.write(idx, valueBuf.getFloat()) + } + + case DoubleType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + if (marker == negativeValMarker) { + val doubleVal = valueBuf.getDouble + val updatedVal = doubleToRawLongBits(doubleVal) ^ doubleFlipBitMask + rowWriter.write(idx, longBitsToDouble(updatedVal)) + } else { + rowWriter.write(idx, valueBuf.getDouble()) + } + + case _ => throw new UnsupportedOperationException( + s"Range scan decoding not supported for data type: ${field.dataType}") + } + } + fieldIdx += 2 + } + + rowWriter.getRow() + } + override def encodeKey(row: UnsafeRow): Array[Byte] = { // This prefix key has the columns specified by orderingOrdinals val prefixKey = extractPrefixKey(row) - val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) + val rangeScanKeyEncoded = if (avroEnc.isDefined) { + encodePrefixKeyForRangeScan(prefixKey, rangeScanAvroType) + } else { + encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) + } val result = if (orderingOrdinals.length < keySchema.length) { - val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) + val remainingEncoded = if (avroEnc.isDefined) { + encodeUnsafeRowToAvro( + remainingKeyProjection(row), + avroEnc.get.keySerializer, + remainingKeyAvroType, + out + ) + } else { + encodeUnsafeRow(remainingKeyProjection(row)) + } val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( rangeScanKeyEncoded.length + remainingEncoded.length + 4 ) @@ -606,9 +986,12 @@ class RangeKeyScanStateEncoder( Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4, prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) - val prefixKeyDecodedForRangeScan = decodeToUnsafeRow(prefixKeyEncoded, - numFields = orderingOrdinals.length) - val prefixKeyDecoded = decodePrefixKeyForRangeScan(prefixKeyDecodedForRangeScan) + val prefixKeyDecoded = if (avroEnc.isDefined) { + decodePrefixKeyForRangeScan(prefixKeyEncoded, rangeScanAvroType) + } else { + decodePrefixKeyForRangeScan(decodeToUnsafeRow(prefixKeyEncoded, + numFields = orderingOrdinals.length)) + } if (orderingOrdinals.length < keySchema.length) { // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes @@ -620,8 +1003,14 @@ class RangeKeyScanStateEncoder( remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) - val remainingKeyDecoded = decodeToUnsafeRow(remainingKeyEncoded, - numFields = keySchema.length - orderingOrdinals.length) + val remainingKeyDecoded = if (avroEnc.isDefined) { + decodeFromAvroToUnsafeRow(remainingKeyEncoded, + avroEnc.get.keyDeserializer, + remainingKeyAvroType, remainingKeyAvroProjection) + } else { + decodeToUnsafeRow(remainingKeyEncoded, + numFields = keySchema.length - orderingOrdinals.length) + } val joined = joinedRowOnKey.withLeft(prefixKeyDecoded).withRight(remainingKeyDecoded) val restored = restoreKeyProjection(joined) @@ -634,7 +1023,11 @@ class RangeKeyScanStateEncoder( } override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { - val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) + val rangeScanKeyEncoded = if (avroEnc.isDefined) { + encodePrefixKeyForRangeScan(prefixKey, rangeScanAvroType) + } else { + encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) + } val (prefix, startingOffset) = encodeColumnFamilyPrefix(rangeScanKeyEncoded.length + 4) Platform.putInt(prefix, startingOffset, rangeScanKeyEncoded.length) @@ -653,6 +1046,7 @@ class RangeKeyScanStateEncoder( * It uses the first byte of the generated byte array to store the version the describes how the * row is encoded in the rest of the byte array. Currently, the default version is 0, * + * If the avroEnc is specified, we are using Avro encoding for this column family's keys * VERSION 0: [ VERSION (1 byte) | ROW (N bytes) ] * The bytes of a UnsafeRow is written unmodified to starting from offset 1 * (offset 0 is the version byte of value 0). That is, if the unsafe row has N bytes, @@ -661,19 +1055,27 @@ class RangeKeyScanStateEncoder( class NoPrefixKeyStateEncoder( keySchema: StructType, useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None) - extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) { + virtualColFamilyId: Option[Short] = None, + avroEnc: Option[AvroEncoder] = None) + extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { import RocksDBStateEncoder._ // Reusable objects + private val usingAvroEncoding = avroEnc.isDefined private val keyRow = new UnsafeRow(keySchema.size) + private lazy val keyAvroType = SchemaConverters.toAvroType(keySchema) + private val keyProj = UnsafeProjection.create(keySchema) override def encodeKey(row: UnsafeRow): Array[Byte] = { if (!useColumnFamilies) { encodeUnsafeRow(row) } else { - val bytesToEncode = row.getBytes + // If avroEnc is defined, we know that we need to use Avro to + // encode this UnsafeRow to Avro bytes + val bytesToEncode = if (usingAvroEncoding) { + encodeUnsafeRowToAvro(row, avroEnc.get.keySerializer, keyAvroType, out) + } else row.getBytes val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( bytesToEncode.length + STATE_ENCODING_NUM_VERSION_BYTES @@ -697,11 +1099,21 @@ class NoPrefixKeyStateEncoder( if (useColumnFamilies) { if (keyBytes != null) { // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. - keyRow.pointTo( - keyBytes, - decodeKeyStartOffset + STATE_ENCODING_NUM_VERSION_BYTES, - keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - VIRTUAL_COL_FAMILY_PREFIX_BYTES) - keyRow + if (usingAvroEncoding) { + val dataLength = keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - + VIRTUAL_COL_FAMILY_PREFIX_BYTES + val avroBytes = new Array[Byte](dataLength) + Platform.copyMemory( + keyBytes, decodeKeyStartOffset + STATE_ENCODING_NUM_VERSION_BYTES, + avroBytes, Platform.BYTE_ARRAY_OFFSET, dataLength) + decodeFromAvroToUnsafeRow(avroBytes, avroEnc.get.keyDeserializer, keyAvroType, keyProj) + } else { + keyRow.pointTo( + keyBytes, + decodeKeyStartOffset + STATE_ENCODING_NUM_VERSION_BYTES, + keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - VIRTUAL_COL_FAMILY_PREFIX_BYTES) + keyRow + } } else { null } @@ -727,17 +1139,28 @@ class NoPrefixKeyStateEncoder( * This encoder supports RocksDB StringAppendOperator merge operator. Values encoded can be * merged in RocksDB using merge operation, and all merged values can be read using decodeValues * operation. + * If the avroEnc is specified, we are using Avro encoding for this column family's values */ -class MultiValuedStateEncoder(valueSchema: StructType) +class MultiValuedStateEncoder( + valueSchema: StructType, + avroEnc: Option[AvroEncoder] = None) extends RocksDBValueStateEncoder with Logging { import RocksDBStateEncoder._ + private val usingAvroEncoding = avroEnc.isDefined // Reusable objects + private val out = new ByteArrayOutputStream private val valueRow = new UnsafeRow(valueSchema.size) + private lazy val valueAvroType = SchemaConverters.toAvroType(valueSchema) + private val valueProj = UnsafeProjection.create(valueSchema) override def encodeValue(row: UnsafeRow): Array[Byte] = { - val bytes = encodeUnsafeRow(row) + val bytes = if (usingAvroEncoding) { + encodeUnsafeRowToAvro(row, avroEnc.get.valueSerializer, valueAvroType, out) + } else { + encodeUnsafeRow(row) + } val numBytes = bytes.length val encodedBytes = new Array[Byte](java.lang.Integer.BYTES + bytes.length) @@ -756,7 +1179,12 @@ class MultiValuedStateEncoder(valueSchema: StructType) val encodedValue = new Array[Byte](numBytes) Platform.copyMemory(valueBytes, java.lang.Integer.BYTES + Platform.BYTE_ARRAY_OFFSET, encodedValue, Platform.BYTE_ARRAY_OFFSET, numBytes) - decodeToUnsafeRow(encodedValue, valueRow) + if (usingAvroEncoding) { + decodeFromAvroToUnsafeRow( + encodedValue, avroEnc.get.valueDeserializer, valueAvroType, valueProj) + } else { + decodeToUnsafeRow(encodedValue, valueRow) + } } } @@ -782,7 +1210,12 @@ class MultiValuedStateEncoder(valueSchema: StructType) pos += numBytes pos += 1 // eat the delimiter character - decodeToUnsafeRow(encodedValue, valueRow) + if (usingAvroEncoding) { + decodeFromAvroToUnsafeRow( + encodedValue, avroEnc.get.valueDeserializer, valueAvroType, valueProj) + } else { + decodeToUnsafeRow(encodedValue, valueRow) + } } } } @@ -802,16 +1235,29 @@ class MultiValuedStateEncoder(valueSchema: StructType) * The bytes of a UnsafeRow is written unmodified to starting from offset 1 * (offset 0 is the version byte of value 0). That is, if the unsafe row has N bytes, * then the generated array byte will be N+1 bytes. + * If the avroEnc is specified, we are using Avro encoding for this column family's values */ -class SingleValueStateEncoder(valueSchema: StructType) - extends RocksDBValueStateEncoder { +class SingleValueStateEncoder( + valueSchema: StructType, + avroEnc: Option[AvroEncoder] = None) + extends RocksDBValueStateEncoder with Logging { import RocksDBStateEncoder._ + private val usingAvroEncoding = avroEnc.isDefined // Reusable objects + private val out = new ByteArrayOutputStream private val valueRow = new UnsafeRow(valueSchema.size) + private lazy val valueAvroType = SchemaConverters.toAvroType(valueSchema) + private val valueProj = UnsafeProjection.create(valueSchema) - override def encodeValue(row: UnsafeRow): Array[Byte] = encodeUnsafeRow(row) + override def encodeValue(row: UnsafeRow): Array[Byte] = { + if (usingAvroEncoding) { + encodeUnsafeRowToAvro(row, avroEnc.get.valueSerializer, valueAvroType, out) + } else { + encodeUnsafeRow(row) + } + } /** * Decode byte array for a value to a UnsafeRow. @@ -820,7 +1266,15 @@ class SingleValueStateEncoder(valueSchema: StructType) * the given byte array. */ override def decodeValue(valueBytes: Array[Byte]): UnsafeRow = { - decodeToUnsafeRow(valueBytes, valueRow) + if (valueBytes == null) { + return null + } + if (usingAvroEncoding) { + decodeFromAvroToUnsafeRow( + valueBytes, avroEnc.get.valueDeserializer, valueAvroType, valueProj) + } else { + decodeToUnsafeRow(valueBytes, valueRow) + } } override def supportsMultipleValuesPerKey: Boolean = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 1fc6ab5910c6c..e5a4175aeec1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.execution.streaming.state import java.io._ -import java.util.concurrent.ConcurrentHashMap +import java.util.UUID +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.util.control.NonFatal +import com.google.common.cache.CacheBuilder import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -29,11 +31,12 @@ import org.apache.spark.{SparkConf, SparkEnv, SparkException} import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec +import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, AvroSerializer, SchemaConverters} import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StreamExecution} import org.apache.spark.sql.types.StructType -import org.apache.spark.util.Utils +import org.apache.spark.util.{NonFateSharingCache, Utils} private[sql] class RocksDBStateStoreProvider extends StateStoreProvider with Logging with Closeable @@ -74,10 +77,17 @@ private[sql] class RocksDBStateStoreProvider isInternal: Boolean = false): Unit = { verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) val newColFamilyId = rocksDB.createColFamilyIfAbsent(colFamilyName) + // Create cache key using store ID to avoid collisions + val avroEncCacheKey = s"${getRunId(hadoopConf)}_${stateStoreId.operatorId}_" + + s"${stateStoreId.partitionId}_$colFamilyName" + + val avroEnc = getAvroEnc( + stateStoreEncoding, avroEncCacheKey, keyStateEncoderSpec, valueSchema) + keyValueEncoderMap.putIfAbsent(colFamilyName, (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, useColumnFamilies, - Some(newColFamilyId)), RocksDBStateEncoder.getValueEncoder(valueSchema, - useMultipleValuesPerKey))) + Some(newColFamilyId), avroEnc), RocksDBStateEncoder.getValueEncoder(valueSchema, + useMultipleValuesPerKey, avroEnc))) } override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { @@ -364,6 +374,7 @@ private[sql] class RocksDBStateStoreProvider this.storeConf = storeConf this.hadoopConf = hadoopConf this.useColumnFamilies = useColumnFamilies + this.stateStoreEncoding = storeConf.stateStoreEncodingFormat if (useMultipleValuesPerKey) { require(useColumnFamilies, "Multiple values per key support requires column families to be" + @@ -377,10 +388,17 @@ private[sql] class RocksDBStateStoreProvider defaultColFamilyId = Some(rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME)) } + val colFamilyName = StateStore.DEFAULT_COL_FAMILY_NAME + // Create cache key using store ID to avoid collisions + val avroEncCacheKey = s"${getRunId(hadoopConf)}_${stateStoreId.operatorId}_" + + s"${stateStoreId.partitionId}_$colFamilyName" + val avroEnc = getAvroEnc( + stateStoreEncoding, avroEncCacheKey, keyStateEncoderSpec, valueSchema) + keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, - useColumnFamilies, defaultColFamilyId), - RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) + useColumnFamilies, defaultColFamilyId, avroEnc), + RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey, avroEnc))) } override def stateStoreId: StateStoreId = stateStoreId_ @@ -458,6 +476,7 @@ private[sql] class RocksDBStateStoreProvider @volatile private var storeConf: StateStoreConf = _ @volatile private var hadoopConf: Configuration = _ @volatile private var useColumnFamilies: Boolean = _ + @volatile private var stateStoreEncoding: String = _ private[sql] lazy val rocksDB = { val dfsRootDir = stateStoreId.storeCheckpointLocation().toString @@ -593,6 +612,93 @@ object RocksDBStateStoreProvider { val STATE_ENCODING_VERSION: Byte = 0 val VIRTUAL_COL_FAMILY_PREFIX_BYTES = 2 + private val MAX_AVRO_ENCODERS_IN_CACHE = 1000 + // Add the cache at companion object level so it persists across provider instances + private val avroEncoderMap: NonFateSharingCache[String, AvroEncoder] = { + val guavaCache = CacheBuilder.newBuilder() + .maximumSize(MAX_AVRO_ENCODERS_IN_CACHE) // Adjust size based on your needs + .expireAfterAccess(1, TimeUnit.HOURS) // Optional: Add expiration if needed + .build[String, AvroEncoder]() + + new NonFateSharingCache(guavaCache) + } + + def getAvroEnc( + stateStoreEncoding: String, + avroEncCacheKey: String, + keyStateEncoderSpec: KeyStateEncoderSpec, + valueSchema: StructType): Option[AvroEncoder] = { + + stateStoreEncoding match { + case "avro" => Some( + RocksDBStateStoreProvider.avroEncoderMap.get( + avroEncCacheKey, + new java.util.concurrent.Callable[AvroEncoder] { + override def call(): AvroEncoder = createAvroEnc(keyStateEncoderSpec, valueSchema) + } + ) + ) + case "unsaferow" => None + } + } + + private def getRunId(hadoopConf: Configuration): String = { + val runId = hadoopConf.get(StreamExecution.RUN_ID_KEY) + if (runId != null) { + runId + } else { + assert(Utils.isTesting, "Failed to find query id/batch Id in task context") + UUID.randomUUID().toString + } + } + + private def getAvroSerializer(schema: StructType): AvroSerializer = { + val avroType = SchemaConverters.toAvroType(schema) + new AvroSerializer(schema, avroType, nullable = false) + } + + private def getAvroDeserializer(schema: StructType): AvroDeserializer = { + val avroType = SchemaConverters.toAvroType(schema) + val avroOptions = AvroOptions(Map.empty) + new AvroDeserializer(avroType, schema, + avroOptions.datetimeRebaseModeInRead, avroOptions.useStableIdForUnionType, + avroOptions.stableIdPrefixForUnionType, avroOptions.recursiveFieldMaxDepth) + } + + private def createAvroEnc( + keyStateEncoderSpec: KeyStateEncoderSpec, + valueSchema: StructType + ): AvroEncoder = { + val valueSerializer = getAvroSerializer(valueSchema) + val valueDeserializer = getAvroDeserializer(valueSchema) + val keySchema = keyStateEncoderSpec match { + case NoPrefixKeyStateEncoderSpec(schema) => + schema + case PrefixKeyScanStateEncoderSpec(schema, numColsPrefixKey) => + StructType(schema.take(numColsPrefixKey)) + case RangeKeyScanStateEncoderSpec(schema, orderingOrdinals) => + val remainingSchema = { + 0.until(schema.length).diff(orderingOrdinals).map { ordinal => + schema(ordinal) + } + } + StructType(remainingSchema) + } + val suffixKeySchema = keyStateEncoderSpec match { + case PrefixKeyScanStateEncoderSpec(schema, numColsPrefixKey) => + Some(StructType(schema.drop(numColsPrefixKey))) + case _ => None + } + AvroEncoder( + getAvroSerializer(keySchema), + getAvroDeserializer(keySchema), + valueSerializer, + valueDeserializer, + suffixKeySchema.map(getAvroSerializer), + suffixKeySchema.map(getAvroDeserializer) + ) + } + // Native operation latencies report as latency in microseconds // as SQLMetrics support millis. Convert the value to millis val CUSTOM_METRIC_GET_TIME = StateStoreCustomTimingMetric( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala index 721d72b6a0991..48b15ac04f40b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.sql.avro.{AvroDeserializer, AvroSerializer} import org.apache.spark.sql.catalyst.util.UnsafeRowUtils import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StatefulOperatorStateInfo} import org.apache.spark.sql.execution.streaming.state.SchemaHelper.{SchemaReader, SchemaWriter} @@ -37,6 +38,30 @@ case class StateSchemaValidationResult( schemaPath: String ) +/** + * An Avro-based encoder used for serializing between UnsafeRow and Avro + * byte arrays in RocksDB state stores. + * + * This encoder is primarily utilized by [[RocksDBStateStoreProvider]] and [[RocksDBStateEncoder]] + * to handle serialization and deserialization of state store data. + * + * @param keySerializer Serializer for converting state store keys to Avro format + * @param keyDeserializer Deserializer for converting Avro-encoded keys back to UnsafeRow + * @param valueSerializer Serializer for converting state store values to Avro format + * @param valueDeserializer Deserializer for converting Avro-encoded values back to UnsafeRow + * @param suffixKeySerializer Optional serializer for handling suffix keys in Avro format + * @param suffixKeyDeserializer Optional deserializer for converting Avro-encoded suffix + * keys back to UnsafeRow + */ +case class AvroEncoder( + keySerializer: AvroSerializer, + keyDeserializer: AvroDeserializer, + valueSerializer: AvroSerializer, + valueDeserializer: AvroDeserializer, + suffixKeySerializer: Option[AvroSerializer] = None, + suffixKeyDeserializer: Option[AvroDeserializer] = None +) extends Serializable + // Used to represent the schema of a column family in the state store case class StateStoreColFamilySchema( colFamilyName: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 72bc3ca33054d..e2b93c147891d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -37,10 +37,22 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.util.UnsafeRowUtils import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.{StatefulOperatorStateInfo, StreamExecution} import org.apache.spark.sql.types.StructType import org.apache.spark.util.{NextIterator, ThreadUtils, Utils} +sealed trait StateStoreEncoding { + override def toString: String = this match { + case StateStoreEncoding.UnsafeRow => "unsaferow" + case StateStoreEncoding.Avro => "avro" + } +} + +object StateStoreEncoding { + case object UnsafeRow extends StateStoreEncoding + case object Avro extends StateStoreEncoding +} + /** * Base trait for a versioned key-value store which provides read operations. Each instance of a * `ReadStateStore` represents a specific version of state data, and such instances are created @@ -769,6 +781,7 @@ object StateStore extends Logging { if (version < 0) { throw QueryExecutionErrors.unexpectedStateStoreVersion(version) } + hadoopConf.set(StreamExecution.RUN_ID_KEY, storeProviderId.queryRunId.toString) val storeProvider = getStateStoreProvider(storeProviderId, keySchema, valueSchema, keyStateEncoderSpec, useColumnFamilies, storeConf, hadoopConf, useMultipleValuesPerKey) storeProvider.getStore(version, stateStoreCkptId) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala index c8af395e996d8..9d26bf8fdf2e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala @@ -83,6 +83,9 @@ class StateStoreConf( /** The interval of maintenance tasks. */ val maintenanceInterval = sqlConf.streamingMaintenanceInterval + /** The interval of maintenance tasks. */ + val stateStoreEncodingFormat = sqlConf.stateStoreEncodingFormat + /** * When creating new state store checkpoint, which format version to use. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala index baab6327b35c1..af64f563cf7b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.{Encoders, Row} import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBFileManager, RocksDBStateStoreProvider, TestClass} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBFileManager, RocksDBStateStoreProvider, TestClass} import org.apache.spark.sql.functions.{col, explode, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{InputMapRow, ListState, MapInputEvent, MapOutputEvent, MapStateTTLProcessor, MaxEventTimeStatefulProcessor, OutputMode, RunningCountStatefulProcessor, RunningCountStatefulProcessorWithProcTimeTimerUpdates, StatefulProcessor, StateStoreMetricsTest, TestMapStateProcessor, TimeMode, TimerValues, TransformWithStateSuiteUtils, Trigger, TTLConfig, ValueState} @@ -126,7 +126,7 @@ class SessionGroupsStatefulProcessorWithTTL extends * Test suite to verify integration of state data source reader with the transformWithState operator */ class StateDataSourceTransformWithStateSuite extends StateStoreMetricsTest - with AlsoTestWithChangelogCheckpointingEnabled { + with AlsoTestWithChangelogCheckpointingEnabled with AlsoTestWithEncodingTypes { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala index e1bd9dd38066b..0abdcadefbd55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala @@ -43,6 +43,7 @@ import org.apache.spark.util.Utils @ExtendedSQLTest class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvider] with AlsoTestWithChangelogCheckpointingEnabled + with AlsoTestWithEncodingTypes with SharedSparkSession with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 637eb49130305..61ca8e7c32f61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -86,6 +86,19 @@ trait RocksDBStateStoreChangelogCheckpointingTestUtil { } } +trait AlsoTestWithEncodingTypes extends SQLTestUtils { + override protected def test(testName: String, testTags: Tag*)(testBody: => Any) + (implicit pos: Position): Unit = { + Seq("unsaferow", "avro").foreach { encoding => + super.test(s"$testName (encoding = $encoding)", testTags: _*) { + withSQLConf(SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT.key -> encoding) { + testBody + } + } + } + } +} + trait AlsoTestWithChangelogCheckpointingEnabled extends SQLTestUtils with RocksDBStateStoreChangelogCheckpointingTestUtil { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala index 55d08cd8f12a7..8984d9b0845b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala @@ -423,7 +423,7 @@ class ValueStateSuite extends StateVariableSuiteBase { * types (ValueState, ListState, MapState) used in arbitrary stateful operators. */ abstract class StateVariableSuiteBase extends SharedSparkSession - with BeforeAndAfter { + with BeforeAndAfter with AlsoTestWithEncodingTypes { before { StateStore.stop() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala index 88862e2ad0791..5d88db0d01ba4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Encoders import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf case class InputRow(key: String, action: String, value: String) @@ -127,7 +127,8 @@ class ToggleSaveAndEmitProcessor } class TransformWithListStateSuite extends StreamTest - with AlsoTestWithChangelogCheckpointingEnabled { + with AlsoTestWithChangelogCheckpointingEnabled + with AlsoTestWithEncodingTypes { import testImplicits._ test("test appending null value in list state throw exception") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala index 76c5cbeee424b..ec6ff4fcceb67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Encoders import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf case class InputMapRow(key: String, action: String, value: (String, String)) @@ -81,7 +81,8 @@ class TestMapStateProcessor * operators such as transformWithState. */ class TransformWithMapStateSuite extends StreamTest - with AlsoTestWithChangelogCheckpointingEnabled { + with AlsoTestWithChangelogCheckpointingEnabled + with AlsoTestWithEncodingTypes { import testImplicits._ private def testMapStateWithNullUserKey(inputMapRow: InputMapRow): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 505775d4f6a9b..91a47645f4179 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -429,11 +429,12 @@ class SleepingTimerProcessor extends StatefulProcessor[String, String, String] { * Class that adds tests for transformWithState stateful streaming operator */ class TransformWithStateSuite extends StateStoreMetricsTest - with AlsoTestWithChangelogCheckpointingEnabled { + with AlsoTestWithChangelogCheckpointingEnabled with AlsoTestWithEncodingTypes { import testImplicits._ - test("transformWithState - streaming with rocksdb and invalid processor should fail") { + test("transformWithState - streaming with rocksdb and" + + " invalid processor should fail") { withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, SQLConf.SHUFFLE_PARTITIONS.key -> @@ -688,7 +689,8 @@ class TransformWithStateSuite extends StateStoreMetricsTest } } - test("transformWithState - streaming with rocksdb and event time based timer") { + test("transformWithState - streaming with rocksdb and event " + + "time based timer") { val inputData = MemoryStream[(String, Int)] val result = inputData.toDS() @@ -778,7 +780,8 @@ class TransformWithStateSuite extends StateStoreMetricsTest ) } - test("Use statefulProcessor without transformWithState - handle should be absent") { + test("Use statefulProcessor without transformWithState -" + + " handle should be absent") { val processor = new RunningCountStatefulProcessor() val ex = intercept[Exception] { processor.getHandle @@ -1034,7 +1037,8 @@ class TransformWithStateSuite extends StateStoreMetricsTest } } - test("transformWithState - verify StateSchemaV3 writes correct SQL schema of key/value") { + test("transformWithState - verify StateSchemaV3 writes " + + "correct SQL schema of key/value") { withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, SQLConf.SHUFFLE_PARTITIONS.key -> @@ -1605,7 +1609,8 @@ class TransformWithStateSuite extends StateStoreMetricsTest } } - test("transformWithState - verify that schema file is kept after metadata is purged") { + test("transformWithState - verify that schema file " + + "is kept after metadata is purged") { withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, SQLConf.SHUFFLE_PARTITIONS.key -> diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala index 2ddf69aa49e04..75fda9630779e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala @@ -21,7 +21,7 @@ import java.sql.Timestamp import java.time.Duration import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock @@ -41,7 +41,8 @@ case class OutputEvent( * Test suite base for TransformWithState with TTL support. */ abstract class TransformWithStateTTLTest - extends StreamTest { + extends StreamTest with AlsoTestWithChangelogCheckpointingEnabled + with AlsoTestWithEncodingTypes { import testImplicits._ def getProcessor(ttlConfig: TTLConfig): StatefulProcessor[String, InputEvent, OutputEvent] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala index 21c3beb79314c..b19c126c7386b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala @@ -262,7 +262,8 @@ class TransformWithValueStateTTLSuite extends TransformWithStateTTLTest { } } - test("verify StateSchemaV3 writes correct SQL schema of key/value and with TTL") { + test("verify StateSchemaV3 writes correct SQL " + + "schema of key/value and with TTL") { withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, SQLConf.SHUFFLE_PARTITIONS.key -> From 69d433bcfd5a2d69f3cd7f8c4e310a3b5854fc74 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 26 Nov 2024 13:54:36 +0900 Subject: [PATCH 021/438] [SPARK-50387][SS] Update condition for timer expiry and relevant test ### What changes were proposed in this pull request? Update condition for timer expiry and relevant test ### Why are the changes needed? To ensure that the expiry and removal conditions are consistent. Also, we don't have to wait for an extra microbatch to expire timers in certain cases. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Added unit tests ``` [info] Run completed in 4 seconds, 638 milliseconds. [info] Total number of tests run: 12 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 12, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48927 from anishshri-db/task/SPARK-50387. Authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim --- .../sql/tests/pandas/test_pandas_transform_with_state.py | 8 +++++--- .../spark/sql/execution/streaming/TimerStateImpl.scala | 2 +- .../spark/sql/execution/streaming/state/TimerSuite.scala | 9 +++++---- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 8901f09e9272d..5143392498187 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -553,9 +553,11 @@ def check_results(batch_df, batch_id): Row(id="a-expired", timestamp="0"), } else: - # watermark has not progressed, so timer registered in batch 1(watermark = 10) - # has not yet expired - assert set(batch_df.sort("id").collect()) == {Row(id="a", timestamp="15")} + # verify that rows and expired timer produce the expected result + assert set(batch_df.sort("id").collect()) == { + Row(id="a", timestamp="15"), + Row(id="a-expired", timestamp="10000"), + } self._test_transform_with_state_in_pandas_event_time( EventTimeStatefulProcessor(), check_results diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala index d0fbaf6600609..5d20f53449c59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala @@ -178,7 +178,7 @@ class TimerStateImpl( val rowPair = iter.next() val keyRow = rowPair.key val result = getTimerRowFromSecIndex(keyRow) - if (result._2 < expiryTimestampMs) { + if (result._2 <= expiryTimestampMs) { result } else { finished = true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala index 24a120be9d9af..428845d5ebcbb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala @@ -72,8 +72,9 @@ class TimerSuite extends StateVariableSuiteBase { assert(timerState1.listTimers().toSet === Set(15000L, 1000L)) assert(timerState1.getExpiredTimers(Long.MaxValue).toSeq === Seq(("test_key", 1000L), ("test_key", 15000L))) - // if timestamp equals to expiryTimestampsMs, will not considered expired - assert(timerState1.getExpiredTimers(15000L).toSeq === Seq(("test_key", 1000L))) + // if timestamp equals to expiryTimestampsMs, it will be considered expired + assert(timerState1.getExpiredTimers(15000L).toSeq === + Seq(("test_key", 1000L), ("test_key", 15000L))) assert(timerState1.listTimers().toSet === Set(15000L, 1000L)) timerState1.registerTimer(20L * 1000) @@ -128,7 +129,7 @@ class TimerSuite extends StateVariableSuiteBase { timerTimerstamps.foreach(timerState.registerTimer) assert(timerState.getExpiredTimers(Long.MaxValue).toSeq.map(_._2) === timerTimerstamps.sorted) assert(timerState.getExpiredTimers(4200L).toSeq.map(_._2) === - timerTimerstamps.sorted.takeWhile(_ < 4200L)) + timerTimerstamps.sorted.takeWhile(_ <= 4200L)) assert(timerState.getExpiredTimers(Long.MinValue).toSeq === Seq.empty) ImplicitGroupingKeyTracker.removeImplicitKey() } @@ -162,7 +163,7 @@ class TimerSuite extends StateVariableSuiteBase { (timerTimestamps1 ++ timerTimestamps2 ++ timerTimerStamps3).sorted) assert(timerState1.getExpiredTimers(Long.MinValue).toSeq === Seq.empty) assert(timerState1.getExpiredTimers(8000L).toSeq.map(_._2) === - (timerTimestamps1 ++ timerTimestamps2 ++ timerTimerStamps3).sorted.takeWhile(_ < 8000L)) + (timerTimestamps1 ++ timerTimestamps2 ++ timerTimerStamps3).sorted.takeWhile(_ <= 8000L)) } } From afb5d6f23abfae8950068f3cf5460ca3913a9742 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 25 Nov 2024 21:40:07 -0800 Subject: [PATCH 022/438] [SPARK-50410][CONNECT] Refactor the `sql` function in `SparkSession` to eliminate duplicate code ### What changes were proposed in this pull request? There is duplicate code between https://github.com/apache/spark/blob/7b4922ea90d19d7e0510a205740b4c150057e988/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala#L214-L235 and https://github.com/apache/spark/blob/7b4922ea90d19d7e0510a205740b4c150057e988/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala#L245-L267 So this pr introduces a new `private` function named `sql` in `SparkSession`. This function takes the type of `proto.SqlCommand` as its input, and the above two functions are refactored to call the newly added function in order to remove duplicate code. ### Why are the changes needed? To eliminate duplicate code. ### 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 #48955 from LuciferYang/ref-sql-function. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/SparkSession.scala | 75 ++++++++----------- 1 file changed, 32 insertions(+), 43 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 7edb1f51f11b1..231c604b98bb5 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -211,27 +211,13 @@ class SparkSession private[sql] ( /** @inheritdoc */ @Experimental - def sql(sqlText: String, args: Array[_]): DataFrame = newDataFrame { builder => - // Send the SQL once to the server and then check the output. - val cmd = newCommand(b => - b.setSqlCommand( - proto.SqlCommand - .newBuilder() - .setSql(sqlText) - .addAllPosArguments(args.map(lit(_).expr).toImmutableArraySeq.asJava))) - val plan = proto.Plan.newBuilder().setCommand(cmd) - val responseIter = client.execute(plan.build()) - - try { - val response = responseIter - .find(_.hasSqlCommandResult) - .getOrElse(throw new RuntimeException("SQLCommandResult must be present")) - // Update the builder with the values from the result. - builder.mergeFrom(response.getSqlCommandResult.getRelation) - } finally { - // consume the rest of the iterator - responseIter.foreach(_ => ()) - } + def sql(sqlText: String, args: Array[_]): DataFrame = { + val sqlCommand = proto.SqlCommand + .newBuilder() + .setSql(sqlText) + .addAllPosArguments(args.map(lit(_).expr).toImmutableArraySeq.asJava) + .build() + sql(sqlCommand) } /** @inheritdoc */ @@ -242,28 +228,13 @@ class SparkSession private[sql] ( /** @inheritdoc */ @Experimental - override def sql(sqlText: String, args: java.util.Map[String, Any]): DataFrame = newDataFrame { - builder => - // Send the SQL once to the server and then check the output. - val cmd = newCommand(b => - b.setSqlCommand( - proto.SqlCommand - .newBuilder() - .setSql(sqlText) - .putAllNamedArguments(args.asScala.map { case (k, v) => (k, lit(v).expr) }.asJava))) - val plan = proto.Plan.newBuilder().setCommand(cmd) - val responseIter = client.execute(plan.build()) - - try { - val response = responseIter - .find(_.hasSqlCommandResult) - .getOrElse(throw new RuntimeException("SQLCommandResult must be present")) - // Update the builder with the values from the result. - builder.mergeFrom(response.getSqlCommandResult.getRelation) - } finally { - // consume the rest of the iterator - responseIter.foreach(_ => ()) - } + override def sql(sqlText: String, args: java.util.Map[String, Any]): DataFrame = { + val sqlCommand = proto.SqlCommand + .newBuilder() + .setSql(sqlText) + .putAllNamedArguments(args.asScala.map { case (k, v) => (k, lit(v).expr) }.asJava) + .build() + sql(sqlCommand) } /** @inheritdoc */ @@ -271,6 +242,24 @@ class SparkSession private[sql] ( sql(query, Array.empty) } + private def sql(sqlCommand: proto.SqlCommand): DataFrame = newDataFrame { builder => + // Send the SQL once to the server and then check the output. + val cmd = newCommand(b => b.setSqlCommand(sqlCommand)) + val plan = proto.Plan.newBuilder().setCommand(cmd) + val responseIter = client.execute(plan.build()) + + try { + val response = responseIter + .find(_.hasSqlCommandResult) + .getOrElse(throw new RuntimeException("SQLCommandResult must be present")) + // Update the builder with the values from the result. + builder.mergeFrom(response.getSqlCommandResult.getRelation) + } finally { + // consume the rest of the iterator + responseIter.foreach(_ => ()) + } + } + /** @inheritdoc */ def read: DataFrameReader = new DataFrameReader(this) From 9054aa287601ab0596f6ea2a43c9176b848c06fc Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Tue, 26 Nov 2024 14:26:14 +0800 Subject: [PATCH 023/438] [SPARK-50406][PYTHON][TESTS] Improve pyspark.sql.tests.test_udtf ### What changes were proposed in this pull request? Improve pyspark.sql.tests.test_udtf by - extract `udtf_for_table_argument` for code reuse - use assertDataFrameEqual universally ### Why are the changes needed? Code cleanup. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Test changes only. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48949 from xinrong-meng/impr_udtf_test. Authored-by: Xinrong Meng Signed-off-by: Xinrong Meng --- python/pyspark/sql/tests/test_udtf.py | 212 +++++++++++--------------- 1 file changed, 91 insertions(+), 121 deletions(-) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index f3f993fc6a787..206cfd7dc4885 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -74,8 +74,7 @@ def eval(self): yield "hello", "world" func = udtf(TestUDTF, returnType="c1: string, c2: string") - rows = func().collect() - self.assertEqual(rows, [Row(c1="hello", c2="world")]) + assertDataFrameEqual(func(), [Row(c1="hello", c2="world")]) def test_udtf_yield_single_row_col(self): class TestUDTF: @@ -83,8 +82,7 @@ def eval(self, a: int): yield a, func = udtf(TestUDTF, returnType="a: int") - rows = func(lit(1)).collect() - self.assertEqual(rows, [Row(a=1)]) + assertDataFrameEqual(func(lit(1)), [Row(a=1)]) def test_udtf_yield_multi_cols(self): class TestUDTF: @@ -92,8 +90,7 @@ def eval(self, a: int): yield a, a + 1 func = udtf(TestUDTF, returnType="a: int, b: int") - rows = func(lit(1)).collect() - self.assertEqual(rows, [Row(a=1, b=2)]) + assertDataFrameEqual(func(lit(1)), [Row(a=1, b=2)]) def test_udtf_yield_multi_rows(self): class TestUDTF: @@ -102,8 +99,7 @@ def eval(self, a: int): yield a + 1, func = udtf(TestUDTF, returnType="a: int") - rows = func(lit(1)).collect() - self.assertEqual(rows, [Row(a=1), Row(a=2)]) + assertDataFrameEqual(func(lit(1)), [Row(a=1), Row(a=2)]) def test_udtf_yield_multi_row_col(self): class TestUDTF: @@ -113,8 +109,8 @@ def eval(self, a: int, b: int): yield a, b, b - a func = udtf(TestUDTF, returnType="a: int, b: int, c: int") - rows = func(lit(1), lit(2)).collect() - self.assertEqual(rows, [Row(a=1, b=2, c=3), Row(a=1, b=2, c=-1), Row(a=1, b=2, c=1)]) + res = func(lit(1), lit(2)) + assertDataFrameEqual(res, [Row(a=1, b=2, c=3), Row(a=1, b=2, c=-1), Row(a=1, b=2, c=1)]) def test_udtf_decorator(self): @udtf(returnType="a: int, b: int") @@ -122,8 +118,7 @@ class TestUDTF: def eval(self, a: int): yield a, a + 1 - rows = TestUDTF(lit(1)).collect() - self.assertEqual(rows, [Row(a=1, b=2)]) + assertDataFrameEqual(TestUDTF(lit(1)), [Row(a=1, b=2)]) def test_udtf_registration(self): class TestUDTF: @@ -135,9 +130,7 @@ def eval(self, a: int, b: int): func = udtf(TestUDTF, returnType="a: int, b: int, c: int") self.spark.udtf.register("testUDTF", func) df = self.spark.sql("SELECT * FROM testUDTF(1, 2)") - self.assertEqual( - df.collect(), [Row(a=1, b=2, c=3), Row(a=1, b=2, c=-1), Row(a=1, b=2, c=1)] - ) + assertDataFrameEqual(df, [Row(a=1, b=2, c=3), Row(a=1, b=2, c=-1), Row(a=1, b=2, c=1)]) def test_udtf_with_lateral_join(self): class TestUDTF: @@ -150,10 +143,17 @@ def eval(self, a: int, b: int) -> Iterator: df = self.spark.sql( "SELECT f.* FROM values (0, 1), (1, 2) t(a, b), LATERAL testUDTF(a, b) f" ) + schema = StructType( + [ + StructField("a", IntegerType(), True), + StructField("b", IntegerType(), True), + StructField("c", IntegerType(), True), + ] + ) expected = self.spark.createDataFrame( - [(0, 1, 1), (0, 1, -1), (1, 2, 3), (1, 2, -1)], schema=["a", "b", "c"] + [(0, 1, 1), (0, 1, -1), (1, 2, 3), (1, 2, -1)], schema=schema ) - self.assertEqual(df.collect(), expected.collect()) + assertDataFrameEqual(df, expected) def test_udtf_eval_with_return_stmt(self): class TestUDTF: @@ -161,8 +161,8 @@ def eval(self, a: int, b: int): return [(a, a + 1), (b, b + 1)] func = udtf(TestUDTF, returnType="a: int, b: int") - rows = func(lit(1), lit(2)).collect() - self.assertEqual(rows, [Row(a=1, b=2), Row(a=2, b=3)]) + res = func(lit(1), lit(2)) + assertDataFrameEqual(res, [Row(a=1, b=2), Row(a=2, b=3)]) def test_udtf_eval_returning_non_tuple(self): @udtf(returnType="a: int") @@ -217,14 +217,14 @@ class TestUDTF: def eval(self, a: int): ... - self.assertEqual(TestUDTF(lit(1)).collect(), []) + assertDataFrameEqual(TestUDTF(lit(1)), []) @udtf(returnType="a: int") class TestUDTF: def eval(self, a: int): return - self.assertEqual(TestUDTF(lit(1)).collect(), []) + assertDataFrameEqual(TestUDTF(lit(1)), []) def test_udtf_with_conditional_return(self): class TestUDTF: @@ -234,8 +234,8 @@ def eval(self, a: int): func = udtf(TestUDTF, returnType="a: int") self.spark.udtf.register("test_udtf", func) - self.assertEqual( - self.spark.sql("SELECT * FROM range(0, 8) JOIN LATERAL test_udtf(id)").collect(), + assertDataFrameEqual( + self.spark.sql("SELECT * FROM range(0, 8) JOIN LATERAL test_udtf(id)"), [Row(id=6, a=6), Row(id=7, a=7)], ) @@ -254,9 +254,9 @@ def eval(self, a: int): yield a, yield None, - self.assertEqual(TestUDTF(lit(1)).collect(), [Row(a=1), Row(a=None)]) + assertDataFrameEqual(TestUDTF(lit(1)), [Row(a=1), Row(a=None)]) df = self.spark.createDataFrame([(0, 1), (1, 2)], schema=["a", "b"]) - self.assertEqual(TestUDTF(lit(1)).join(df, "a", "inner").collect(), [Row(a=1, b=2)]) + assertDataFrameEqual(TestUDTF(lit(1)).join(df, "a", "inner"), [Row(a=1, b=2)]) assertDataFrameEqual( TestUDTF(lit(1)).join(df, "a", "left"), [Row(a=None, b=None), Row(a=1, b=2)] ) @@ -267,10 +267,10 @@ class TestUDTF: def eval(self, a: int): yield a, - self.assertEqual(TestUDTF(lit(None)).collect(), [Row(a=None)]) + assertDataFrameEqual(TestUDTF(lit(None)), [Row(a=None)]) self.spark.udtf.register("testUDTF", TestUDTF) df = self.spark.sql("SELECT * FROM testUDTF(null)") - self.assertEqual(df.collect(), [Row(a=None)]) + assertDataFrameEqual(df, [Row(a=None)]) # These are expected error message substrings to be used in test cases below. tooManyPositionalArguments = "too many positional arguments" @@ -366,8 +366,8 @@ def __init__(self): def eval(self, a: int): yield a, a + 1, self.key - rows = TestUDTF(lit(1)).collect() - self.assertEqual(rows, [Row(a=1, b=2, c="test")]) + res = TestUDTF(lit(1)) + assertDataFrameEqual(res, [Row(a=1, b=2, c="test")]) def test_udtf_terminate(self): @udtf(returnType="key: string, value: float") @@ -385,8 +385,8 @@ def terminate(self): yield "count", float(self._count) yield "avg", self._sum / self._count - self.assertEqual( - TestUDTF(lit(1)).collect(), + assertDataFrameEqual( + TestUDTF(lit(1)), [Row(key="input", value=1), Row(key="count", value=1.0), Row(key="avg", value=1.0)], ) @@ -395,8 +395,8 @@ def terminate(self): "SELECT id, key, value FROM range(0, 10, 1, 2), " "LATERAL test_udtf(id) WHERE key != 'input'" ) - self.assertEqual( - df.collect(), + assertDataFrameEqual( + df, [ Row(id=4, key="count", value=5.0), Row(id=4, key="avg", value=2.0), @@ -608,10 +608,8 @@ def eval(self, person): yield f"{person.name}: {person.age}", self.spark.udtf.register("test_udtf", TestUDTF) - self.assertEqual( - self.spark.sql( - "select * from test_udtf(named_struct('name', 'Alice', 'age', 1))" - ).collect(), + assertDataFrameEqual( + self.spark.sql("select * from test_udtf(named_struct('name', 'Alice', 'age', 1))"), [Row(x="Alice: 1")], ) @@ -634,8 +632,8 @@ def eval(self, m): yield str(m), self.spark.udtf.register("test_udtf", TestUDTF) - self.assertEqual( - self.spark.sql("select * from test_udtf(map('key', 'value'))").collect(), + assertDataFrameEqual( + self.spark.sql("select * from test_udtf(map('key', 'value'))"), [Row(x="{'key': 'value'}")], ) @@ -645,7 +643,7 @@ class TestUDTF: def eval(self, x: int): yield {"a": x, "b": x + 1}, - self.assertEqual(TestUDTF(lit(1)).collect(), [Row(x=Row(a=1, b=2))]) + assertDataFrameEqual(TestUDTF(lit(1)), [Row(x=Row(a=1, b=2))]) def test_udtf_with_array_output_types(self): @udtf(returnType="x: array") @@ -653,7 +651,7 @@ class TestUDTF: def eval(self, x: int): yield [x, x + 1, x + 2], - self.assertEqual(TestUDTF(lit(1)).collect(), [Row(x=[1, 2, 3])]) + assertDataFrameEqual(TestUDTF(lit(1)), [Row(x=[1, 2, 3])]) def test_udtf_with_map_output_types(self): @udtf(returnType="x: map") @@ -661,7 +659,7 @@ class TestUDTF: def eval(self, x: int): yield {x: str(x)}, - self.assertEqual(TestUDTF(lit(1)).collect(), [Row(x={1: "1"})]) + assertDataFrameEqual(TestUDTF(lit(1)), [Row(x={1: "1"})]) def test_udtf_with_empty_output_types(self): @udtf(returnType=StructType()) @@ -1019,17 +1017,21 @@ def test_udtf(a: int): ) def test_udtf_with_table_argument_query(self): + func = self.udtf_for_table_argument() + self.spark.udtf.register("test_udtf", func) + assertDataFrameEqual( + self.spark.sql("SELECT * FROM test_udtf(TABLE (SELECT id FROM range(0, 8)))"), + [Row(a=6), Row(a=7)], + ) + + def udtf_for_table_argument(self): class TestUDTF: def eval(self, row: Row): if row["id"] > 5: yield row["id"], func = udtf(TestUDTF, returnType="a: int") - self.spark.udtf.register("test_udtf", func) - self.assertEqual( - self.spark.sql("SELECT * FROM test_udtf(TABLE (SELECT id FROM range(0, 8)))").collect(), - [Row(a=6), Row(a=7)], - ) + return func def test_udtf_with_int_and_table_argument_query(self): class TestUDTF: @@ -1039,26 +1041,19 @@ def eval(self, i: int, row: Row): func = udtf(TestUDTF, returnType="a: int") self.spark.udtf.register("test_udtf", func) - self.assertEqual( - self.spark.sql( - "SELECT * FROM test_udtf(5, TABLE (SELECT id FROM range(0, 8)))" - ).collect(), + assertDataFrameEqual( + self.spark.sql("SELECT * FROM test_udtf(5, TABLE (SELECT id FROM range(0, 8)))"), [Row(a=6), Row(a=7)], ) def test_udtf_with_table_argument_identifier(self): - class TestUDTF: - def eval(self, row: Row): - if row["id"] > 5: - yield row["id"], - - func = udtf(TestUDTF, returnType="a: int") + func = self.udtf_for_table_argument() self.spark.udtf.register("test_udtf", func) with self.tempView("v"): self.spark.sql("CREATE OR REPLACE TEMPORARY VIEW v as SELECT id FROM range(0, 8)") - self.assertEqual( - self.spark.sql("SELECT * FROM test_udtf(TABLE (v))").collect(), + assertDataFrameEqual( + self.spark.sql("SELECT * FROM test_udtf(TABLE (v))"), [Row(a=6), Row(a=7)], ) @@ -1073,44 +1068,29 @@ def eval(self, i: int, row: Row): with self.tempView("v"): self.spark.sql("CREATE OR REPLACE TEMPORARY VIEW v as SELECT id FROM range(0, 8)") - self.assertEqual( - self.spark.sql("SELECT * FROM test_udtf(5, TABLE (v))").collect(), + assertDataFrameEqual( + self.spark.sql("SELECT * FROM test_udtf(5, TABLE (v))"), [Row(a=6), Row(a=7)], ) def test_udtf_with_table_argument_unknown_identifier(self): - class TestUDTF: - def eval(self, row: Row): - if row["id"] > 5: - yield row["id"], - - func = udtf(TestUDTF, returnType="a: int") + func = self.udtf_for_table_argument() self.spark.udtf.register("test_udtf", func) with self.assertRaisesRegex(AnalysisException, "TABLE_OR_VIEW_NOT_FOUND"): self.spark.sql("SELECT * FROM test_udtf(TABLE (v))").collect() def test_udtf_with_table_argument_malformed_query(self): - class TestUDTF: - def eval(self, row: Row): - if row["id"] > 5: - yield row["id"], - - func = udtf(TestUDTF, returnType="a: int") + func = self.udtf_for_table_argument() self.spark.udtf.register("test_udtf", func) with self.assertRaisesRegex(AnalysisException, "TABLE_OR_VIEW_NOT_FOUND"): self.spark.sql("SELECT * FROM test_udtf(TABLE (SELECT * FROM v))").collect() def test_udtf_with_table_argument_cte_inside(self): - class TestUDTF: - def eval(self, row: Row): - if row["id"] > 5: - yield row["id"], - - func = udtf(TestUDTF, returnType="a: int") + func = self.udtf_for_table_argument() self.spark.udtf.register("test_udtf", func) - self.assertEqual( + assertDataFrameEqual( self.spark.sql( """ SELECT * FROM test_udtf(TABLE ( @@ -1120,19 +1100,14 @@ def eval(self, row: Row): SELECT * FROM t )) """ - ).collect(), + ), [Row(a=6), Row(a=7)], ) def test_udtf_with_table_argument_cte_outside(self): - class TestUDTF: - def eval(self, row: Row): - if row["id"] > 5: - yield row["id"], - - func = udtf(TestUDTF, returnType="a: int") + func = self.udtf_for_table_argument() self.spark.udtf.register("test_udtf", func) - self.assertEqual( + assertDataFrameEqual( self.spark.sql( """ WITH t AS ( @@ -1140,11 +1115,11 @@ def eval(self, row: Row): ) SELECT * FROM test_udtf(TABLE (SELECT id FROM t)) """ - ).collect(), + ), [Row(a=6), Row(a=7)], ) - self.assertEqual( + assertDataFrameEqual( self.spark.sql( """ WITH t AS ( @@ -1152,28 +1127,23 @@ def eval(self, row: Row): ) SELECT * FROM test_udtf(TABLE (t)) """ - ).collect(), + ), [Row(a=6), Row(a=7)], ) # TODO(SPARK-44233): Fix the subquery resolution. @unittest.skip("Fails to resolve the subquery.") def test_udtf_with_table_argument_lateral_join(self): - class TestUDTF: - def eval(self, row: Row): - if row["id"] > 5: - yield row["id"], - - func = udtf(TestUDTF, returnType="a: int") + func = self.udtf_for_table_argument() self.spark.udtf.register("test_udtf", func) - self.assertEqual( + assertDataFrameEqual( self.spark.sql( """ SELECT * FROM range(0, 8) AS t, LATERAL test_udtf(TABLE (t)) """ - ).collect(), + ), [Row(a=6), Row(a=7)], ) @@ -1198,8 +1168,8 @@ def eval(self, a: Row, b: Row): self.spark.sql(query).collect() with self.sql_conf({"spark.sql.tvf.allowMultipleTableArguments.enabled": True}): - self.assertEqual( - self.spark.sql(query).collect(), + assertDataFrameEqual( + self.spark.sql(query), [ Row(a=0, b=0), Row(a=1, b=0), @@ -2539,8 +2509,10 @@ def eval(self, v): yield i, v.toJson() self.spark.udtf.register("test_udtf", TestUDTF) - rows = self.spark.sql('select i, s from test_udtf(parse_json(\'{"a":"b"}\'))').collect() - self.assertEqual(rows, [Row(i=n, s='{"a":"b"}') for n in range(10)]) + assertDataFrameEqual( + self.spark.sql('select i, s from test_udtf(parse_json(\'{"a":"b"}\'))'), + [Row(i=n, s='{"a":"b"}') for n in range(10)], + ) def test_udtf_with_nested_variant_input(self): # struct @@ -2551,10 +2523,10 @@ def eval(self, v): yield i, v["v"].toJson() self.spark.udtf.register("test_udtf_struct", TestUDTFStruct) - rows = self.spark.sql( + res = self.spark.sql( "select i, s from test_udtf_struct(named_struct('v', parse_json('{\"a\":\"c\"}')))" - ).collect() - self.assertEqual(rows, [Row(i=n, s='{"a":"c"}') for n in range(10)]) + ) + assertDataFrameEqual(res, [Row(i=n, s='{"a":"c"}') for n in range(10)]) # array @udtf(returnType="i int, s: string") @@ -2564,10 +2536,8 @@ def eval(self, v): yield i, v[0].toJson() self.spark.udtf.register("test_udtf_array", TestUDTFArray) - rows = self.spark.sql( - 'select i, s from test_udtf_array(array(parse_json(\'{"a":"d"}\')))' - ).collect() - self.assertEqual(rows, [Row(i=n, s='{"a":"d"}') for n in range(10)]) + res = self.spark.sql('select i, s from test_udtf_array(array(parse_json(\'{"a":"d"}\')))') + assertDataFrameEqual(res, [Row(i=n, s='{"a":"d"}') for n in range(10)]) # map @udtf(returnType="i int, s: string") @@ -2577,10 +2547,10 @@ def eval(self, v): yield i, v["v"].toJson() self.spark.udtf.register("test_udtf_map", TestUDTFMap) - rows = self.spark.sql( + res = self.spark.sql( "select i, s from test_udtf_map(map('v', parse_json('{\"a\":\"e\"}')))" - ).collect() - self.assertEqual(rows, [Row(i=n, s='{"a":"e"}') for n in range(10)]) + ) + assertDataFrameEqual(res, [Row(i=n, s='{"a":"e"}') for n in range(10)]) def test_udtf_with_variant_output(self): @udtf(returnType="i int, v: variant") @@ -2591,8 +2561,8 @@ def eval(self, n): yield i, VariantVal(bytes([2, 1, 0, 0, 2, 5, 97 + i]), bytes([1, 1, 0, 1, 97])) self.spark.udtf.register("test_udtf", TestUDTF) - rows = self.spark.sql("select i, to_json(v) from test_udtf(8)").collect() - self.assertEqual(rows, [Row(i=n, s=f'{{"a":"{chr(97 + n)}"}}') for n in range(8)]) + res = self.spark.sql("select i, to_json(v) from test_udtf(8)") + assertDataFrameEqual(res, [Row(i=n, s=f'{{"a":"{chr(97 + n)}"}}') for n in range(8)]) def test_udtf_with_nested_variant_output(self): # struct @@ -2606,8 +2576,8 @@ def eval(self, n): } self.spark.udtf.register("test_udtf_struct", TestUDTFStruct) - rows = self.spark.sql("select i, to_json(v.v1) from test_udtf_struct(8)").collect() - self.assertEqual(rows, [Row(i=n, s=f'{{"a":"{chr(97 + n)}"}}') for n in range(8)]) + res = self.spark.sql("select i, to_json(v.v1) from test_udtf_struct(8)") + assertDataFrameEqual(res, [Row(i=n, s=f'{{"a":"{chr(97 + n)}"}}') for n in range(8)]) # array @udtf(returnType="i int, v: array") @@ -2620,8 +2590,8 @@ def eval(self, n): ] self.spark.udtf.register("test_udtf_array", TestUDTFArray) - rows = self.spark.sql("select i, to_json(v[0]) from test_udtf_array(8)").collect() - self.assertEqual(rows, [Row(i=n, s=f'{{"a":"{chr(98 + n)}"}}') for n in range(8)]) + res = self.spark.sql("select i, to_json(v[0]) from test_udtf_array(8)") + assertDataFrameEqual(res, [Row(i=n, s=f'{{"a":"{chr(98 + n)}"}}') for n in range(8)]) # map @udtf(returnType="i int, v: map") @@ -2634,8 +2604,8 @@ def eval(self, n): } self.spark.udtf.register("test_udtf_struct", TestUDTFStruct) - rows = self.spark.sql("select i, to_json(v['v1']) from test_udtf_struct(8)").collect() - self.assertEqual(rows, [Row(i=n, s=f'{{"a":"{chr(99 + n)}"}}') for n in range(8)]) + res = self.spark.sql("select i, to_json(v['v1']) from test_udtf_struct(8)") + assertDataFrameEqual(res, [Row(i=n, s=f'{{"a":"{chr(99 + n)}"}}') for n in range(8)]) class UDTFTests(BaseUDTFTestsMixin, ReusedSQLTestCase): From 9bab16a854d8c4e6decbb8812b332ae5e8825993 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 26 Nov 2024 18:03:49 +0900 Subject: [PATCH 024/438] [MINOR][TESTS] Use SparkContext.setLogLevel in QuietTest ### What changes were proposed in this pull request? This PR proposes to use SparkContext.setLogLevel in QuietTest ### Why are the changes needed? To avoid the direct update of logger but reuses SparkContext.setLogLevel. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48966 from HyukjinKwon/minor-log-level. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/testing/utils.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index 1dd15666382f6..9ac0c9ba73995 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -192,14 +192,16 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: class QuietTest: def __init__(self, sc): - self.log4j = sc._jvm.org.apache.log4j + self.sc = sc def __enter__(self): - self.old_level = self.log4j.LogManager.getRootLogger().getLevel() - self.log4j.LogManager.getRootLogger().setLevel(self.log4j.Level.FATAL) + self.old_level = ( + self.sc._jvm.org.apache.log4j.LogManager.getRootLogger().getLevel().toString() + ) + self.sc.setLogLevel("FATAL") def __exit__(self, exc_type, exc_val, exc_tb): - self.log4j.LogManager.getRootLogger().setLevel(self.old_level) + self.sc.setLogLevel(self.old_level) class PySparkTestCase(unittest.TestCase): From f7a264375890d8dc3a605457e4aff90f879be412 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 26 Nov 2024 18:08:52 +0900 Subject: [PATCH 025/438] [SPARK-50423][PYTHON][DOCS] Refine the docstring for misc and hash functions ### What changes were proposed in this pull request? Refine the docstring for misc and hash functions ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? updated doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48969 from zhengruifeng/py_doc_10. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 551 +++++++++++++++++------- 1 file changed, 398 insertions(+), 153 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 31f3e88c46c08..ac4f2fc9881b0 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -12691,9 +12691,15 @@ def current_catalog() -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.current_database` + :meth:`pyspark.sql.functions.current_schema` + Examples -------- - >>> spark.range(1).select(current_catalog()).show() + >>> import pyspark.sql.functions as sf + >>> spark.range(1).select(sf.current_catalog()).show() +-----------------+ |current_catalog()| +-----------------+ @@ -12709,9 +12715,15 @@ def current_database() -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.current_catalog` + :meth:`pyspark.sql.functions.current_schema` + Examples -------- - >>> spark.range(1).select(current_database()).show() + >>> import pyspark.sql.functions as sf + >>> spark.range(1).select(sf.current_database()).show() +----------------+ |current_schema()| +----------------+ @@ -12727,6 +12739,11 @@ def current_schema() -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.current_catalog` + :meth:`pyspark.sql.functions.current_database` + Examples -------- >>> import pyspark.sql.functions as sf @@ -12746,9 +12763,15 @@ def current_user() -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.user` + :meth:`pyspark.sql.functions.session_user` + Examples -------- - >>> spark.range(1).select(current_user()).show() # doctest: +SKIP + >>> import pyspark.sql.functions as sf + >>> spark.range(1).select(sf.current_user()).show() # doctest: +SKIP +--------------+ |current_user()| +--------------+ @@ -12764,6 +12787,11 @@ def user() -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.current_user` + :meth:`pyspark.sql.functions.session_user` + Examples -------- >>> import pyspark.sql.functions as sf @@ -12783,6 +12811,11 @@ def session_user() -> Column: .. versionadded:: 4.0.0 + See Also + -------- + :meth:`pyspark.sql.functions.user` + :meth:`pyspark.sql.functions.current_user` + Examples -------- >>> import pyspark.sql.functions as sf @@ -12799,7 +12832,7 @@ def session_user() -> Column: @_try_remote_functions def crc32(col: "ColumnOrName") -> Column: """ - Calculates the cyclic redundancy check value (CRC32) of a binary column and + Calculates the cyclic redundancy check value (CRC32) of a binary column and returns the value as a bigint. .. versionchanged:: 3.4.0 @@ -12807,7 +12840,7 @@ def crc32(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to compute on. Returns @@ -12819,8 +12852,14 @@ def crc32(col: "ColumnOrName") -> Column: Examples -------- - >>> spark.createDataFrame([('ABC',)], ['a']).select(crc32('a').alias('crc32')).collect() - [Row(crc32=2743272264)] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('ABC',)], ['a']) + >>> df.select('*', sf.crc32('a')).show(truncate=False) + +---+----------+ + |a |crc32(a) | + +---+----------+ + |ABC|2743272264| + +---+----------+ """ return _invoke_function_over_columns("crc32", col) @@ -12836,7 +12875,7 @@ def md5(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to compute on. Returns @@ -12846,8 +12885,14 @@ def md5(col: "ColumnOrName") -> Column: Examples -------- - >>> spark.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect() - [Row(hash='902fbdd2b1df0c4f70b4a5d23525e932')] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('ABC',)], ['a']) + >>> df.select('*', sf.md5('a')).show(truncate=False) + +---+--------------------------------+ + |a |md5(a) | + +---+--------------------------------+ + |ABC|902fbdd2b1df0c4f70b4a5d23525e932| + +---+--------------------------------+ """ return _invoke_function_over_columns("md5", col) @@ -12863,7 +12908,7 @@ def sha1(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to compute on. Returns @@ -12871,10 +12916,21 @@ def sha1(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the column for computed results. + See Also + -------- + :meth:`pyspark.sql.functions.sha` + :meth:`pyspark.sql.functions.sha2` + Examples -------- - >>> spark.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect() - [Row(hash='3c01bdbb26f358bab27f267924aa2c9a03fcfdb8')] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('ABC',)], ['a']) + >>> df.select('*', sf.sha1('a')).show(truncate=False) + +---+----------------------------------------+ + |a |sha1(a) | + +---+----------------------------------------+ + |ABC|3c01bdbb26f358bab27f267924aa2c9a03fcfdb8| + +---+----------------------------------------+ """ return _invoke_function_over_columns("sha1", col) @@ -12892,7 +12948,7 @@ def sha2(col: "ColumnOrName", numBits: int) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to compute on. numBits : int the desired bit length of the result, which must have a @@ -12903,12 +12959,18 @@ def sha2(col: "ColumnOrName", numBits: int) -> Column: :class:`~pyspark.sql.Column` the column for computed results. + See Also + -------- + :meth:`pyspark.sql.functions.sha` + :meth:`pyspark.sql.functions.sha1` + Examples -------- - >>> df = spark.createDataFrame([["Alice"], ["Bob"]], ["name"]) - >>> df.withColumn("sha2", sha2(df.name, 256)).show(truncate=False) + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([['Alice'], ['Bob']], ['name']) + >>> df.select('*', sf.sha2('name', 256)).show(truncate=False) +-----+----------------------------------------------------------------+ - |name |sha2 | + |name |sha2(name, 256) | +-----+----------------------------------------------------------------+ |Alice|3bc51062973c458d5a6f2d8d64a023246354ad7e064b1e4e009ec8a0699a3043| |Bob |cd9fb1e148ccd8442e5aa74904cc73bf6fb54d1d54d333bd596aa9bb4bb4e961| @@ -12938,7 +13000,7 @@ def hash(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name one or more columns to compute on. Returns @@ -12946,27 +13008,34 @@ def hash(*cols: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` hash value as int column. + See Also + -------- + :meth:`pyspark.sql.functions.xxhash64` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([('ABC', 'DEF')], ['c1', 'c2']) + >>> df.select('*', sf.hash('c1')).show() + +---+---+----------+ + | c1| c2| hash(c1)| + +---+---+----------+ + |ABC|DEF|-757602832| + +---+---+----------+ - Hash for one column - - >>> df.select(hash('c1').alias('hash')).show() - +----------+ - | hash| - +----------+ - |-757602832| - +----------+ - - Two or more columns + >>> df.select('*', sf.hash('c1', df.c2)).show() + +---+---+------------+ + | c1| c2|hash(c1, c2)| + +---+---+------------+ + |ABC|DEF| 599895104| + +---+---+------------+ - >>> df.select(hash('c1', 'c2').alias('hash')).show() - +---------+ - | hash| - +---------+ - |599895104| - +---------+ + >>> df.select('*', sf.hash('*')).show() + +---+---+------------+ + | c1| c2|hash(c1, c2)| + +---+---+------------+ + |ABC|DEF| 599895104| + +---+---+------------+ """ return _invoke_function_over_seq_of_columns("hash", cols) @@ -12983,7 +13052,7 @@ def xxhash64(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name one or more columns to compute on. Returns @@ -12991,27 +13060,34 @@ def xxhash64(*cols: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` hash value as long column. + See Also + -------- + :meth:`pyspark.sql.functions.hash` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([('ABC', 'DEF')], ['c1', 'c2']) - - Hash for one column - - >>> df.select(xxhash64('c1').alias('hash')).show() - +-------------------+ - | hash| - +-------------------+ - |4105715581806190027| - +-------------------+ - - Two or more columns - - >>> df.select(xxhash64('c1', 'c2').alias('hash')).show() - +-------------------+ - | hash| - +-------------------+ - |3233247871021311208| - +-------------------+ + >>> df.select('*', sf.xxhash64('c1')).show() + +---+---+-------------------+ + | c1| c2| xxhash64(c1)| + +---+---+-------------------+ + |ABC|DEF|4105715581806190027| + +---+---+-------------------+ + + >>> df.select('*', sf.xxhash64('c1', df.c2)).show() + +---+---+-------------------+ + | c1| c2| xxhash64(c1, c2)| + +---+---+-------------------+ + |ABC|DEF|3233247871021311208| + +---+---+-------------------+ + + >>> df.select('*', sf.xxhash64('*')).show() + +---+---+-------------------+ + | c1| c2| xxhash64(c1, c2)| + +---+---+-------------------+ + |ABC|DEF|3233247871021311208| + +---+---+-------------------+ """ return _invoke_function_over_seq_of_columns("xxhash64", cols) @@ -13029,9 +13105,9 @@ def assert_true(col: "ColumnOrName", errMsg: Optional[Union[Column, str]] = None Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name column name or column that represents the input column to test - errMsg : :class:`~pyspark.sql.Column` or str, optional + errMsg : :class:`~pyspark.sql.Column` or literal string, optional A Python string literal or column containing the error message Returns @@ -13039,16 +13115,36 @@ def assert_true(col: "ColumnOrName", errMsg: Optional[Union[Column, str]] = None :class:`~pyspark.sql.Column` `null` if the input column is `true` otherwise throws an error with specified message. + See Also + -------- + :meth:`pyspark.sql.functions.raise_error` + Examples -------- - >>> df = spark.createDataFrame([(0,1)], ['a', 'b']) - >>> df.select(assert_true(df.a < df.b).alias('r')).collect() - [Row(r=None)] - >>> df.select(assert_true(df.a < df.b, df.a).alias('r')).collect() - [Row(r=None)] - >>> df.select(assert_true(df.a < df.b, 'error').alias('r')).collect() - [Row(r=None)] - >>> df.select(assert_true(df.a > df.b, 'My error msg').alias('r')).collect() # doctest: +SKIP + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(0, 1)], ['a', 'b']) + >>> df.select('*', sf.assert_true(df.a < df.b)).show() # doctest: +SKIP + +------------------------------------------------------+ + |assert_true((a < b), '(a#788L < b#789L)' is not true!)| + +------------------------------------------------------+ + | NULL| + +------------------------------------------------------+ + + >>> df.select('*', sf.assert_true(df.a < df.b, df.a)).show() + +---+---+-----------------------+ + | a| b|assert_true((a < b), a)| + +---+---+-----------------------+ + | 0| 1| NULL| + +---+---+-----------------------+ + + >>> df.select('*', sf.assert_true(df.a < df.b, 'error')).show() + +---+---+---------------------------+ + | a| b|assert_true((a < b), error)| + +---+---+---------------------------+ + | 0| 1| NULL| + +---+---+---------------------------+ + + >>> df.select('*', sf.assert_true(df.a > df.b, 'My error msg')).show() # doctest: +SKIP ... java.lang.RuntimeException: My error msg ... @@ -13076,7 +13172,7 @@ def raise_error(errMsg: Union[Column, str]) -> Column: Parameters ---------- - errMsg : :class:`~pyspark.sql.Column` or str + errMsg : :class:`~pyspark.sql.Column` or literal string A Python string literal or column containing the error message Returns @@ -13084,10 +13180,14 @@ def raise_error(errMsg: Union[Column, str]) -> Column: :class:`~pyspark.sql.Column` throws an error with specified message. + See Also + -------- + :meth:`pyspark.sql.functions.assert_true` + Examples -------- - >>> df = spark.range(1) - >>> df.select(raise_error("My error message")).show() # doctest: +SKIP + >>> import pyspark.sql.functions as sf + >>> spark.range(1).select(sf.raise_error("My error message")).show() # doctest: +SKIP ... java.lang.RuntimeException: My error message ... @@ -24359,21 +24459,21 @@ def aes_encrypt( Parameters ---------- - input : :class:`~pyspark.sql.Column` or str + input : :class:`~pyspark.sql.Column` or column name The binary value to encrypt. - key : :class:`~pyspark.sql.Column` or str + key : :class:`~pyspark.sql.Column` or column name The passphrase to use to encrypt the data. mode : :class:`~pyspark.sql.Column` or str, optional Specifies which block cipher mode should be used to encrypt messages. Valid modes: ECB, GCM, CBC. - padding : :class:`~pyspark.sql.Column` or str, optional + padding : :class:`~pyspark.sql.Column` or column name, optional Specifies how to pad messages whose length is not a multiple of the block size. Valid values: PKCS, NONE, DEFAULT. The DEFAULT padding means PKCS for ECB, NONE for GCM and PKCS for CBC. - iv : :class:`~pyspark.sql.Column` or str, optional + iv : :class:`~pyspark.sql.Column` or column name, optional Optional initialization vector. Only supported for CBC and GCM modes. Valid values: None or "". 16-byte array for CBC mode. 12-byte array for GCM mode. - aad : :class:`~pyspark.sql.Column` or str, optional + aad : :class:`~pyspark.sql.Column` or column name, optional Optional additional authenticated data. Only supported for GCM mode. This can be any free-form input and must be provided for both encryption and decryption. @@ -24382,6 +24482,11 @@ def aes_encrypt( :class:`~pyspark.sql.Column` A new column that contains an encrypted value. + See Also + -------- + :meth:`pyspark.sql.functions.aes_decrypt` + :meth:`pyspark.sql.functions.try_aes_decrypt` + Examples -------- @@ -24394,7 +24499,7 @@ def aes_encrypt( ... ["input", "key", "mode", "padding", "iv", "aad"] ... ) >>> df.select(sf.base64(sf.aes_encrypt( - ... df.input, df.key, df.mode, df.padding, sf.to_binary(df.iv, sf.lit("hex")), df.aad) + ... df.input, df.key, "mode", df.padding, sf.to_binary(df.iv, sf.lit("hex")), df.aad) ... )).show(truncate=False) +-----------------------------------------------------------------------+ |base64(aes_encrypt(input, key, mode, padding, to_binary(iv, hex), aad))| @@ -24411,7 +24516,7 @@ def aes_encrypt( ... ["input", "key", "mode", "padding", "iv", "aad"] ... ) >>> df.select(sf.base64(sf.aes_encrypt( - ... df.input, df.key, df.mode, df.padding, sf.to_binary(df.iv, sf.lit("hex"))) + ... df.input, df.key, "mode", df.padding, sf.to_binary(df.iv, sf.lit("hex"))) ... )).show(truncate=False) +--------------------------------------------------------------------+ |base64(aes_encrypt(input, key, mode, padding, to_binary(iv, hex), ))| @@ -24426,7 +24531,7 @@ def aes_encrypt( ... "Spark SQL", "1234567890abcdef", "ECB", "PKCS",)], ... ["input", "key", "mode", "padding"] ... ) - >>> df.select(sf.aes_decrypt(sf.aes_encrypt(df.input, df.key, df.mode, df.padding), + >>> df.select(sf.aes_decrypt(sf.aes_encrypt(df.input, df.key, "mode", df.padding), ... df.key, df.mode, df.padding ... ).cast("STRING")).show(truncate=False) +---------------------------------------------------------------------------------------------+ @@ -24442,7 +24547,7 @@ def aes_encrypt( ... "Spark SQL", "0000111122223333", "ECB",)], ... ["input", "key", "mode"] ... ) - >>> df.select(sf.aes_decrypt(sf.aes_encrypt(df.input, df.key, df.mode), + >>> df.select(sf.aes_decrypt(sf.aes_encrypt(df.input, df.key, "mode"), ... df.key, df.mode ... ).cast("STRING")).show(truncate=False) +---------------------------------------------------------------------------------------------+ @@ -24493,18 +24598,18 @@ def aes_decrypt( Parameters ---------- - input : :class:`~pyspark.sql.Column` or str + input : :class:`~pyspark.sql.Column` or column name The binary value to decrypt. - key : :class:`~pyspark.sql.Column` or str + key : :class:`~pyspark.sql.Column` or column name The passphrase to use to decrypt the data. - mode : :class:`~pyspark.sql.Column` or str, optional + mode : :class:`~pyspark.sql.Column` or column name, optional Specifies which block cipher mode should be used to decrypt messages. Valid modes: ECB, GCM, CBC. - padding : :class:`~pyspark.sql.Column` or str, optional + padding : :class:`~pyspark.sql.Column` or column name, optional Specifies how to pad messages whose length is not a multiple of the block size. Valid values: PKCS, NONE, DEFAULT. The DEFAULT padding means PKCS for ECB, NONE for GCM and PKCS for CBC. - aad : :class:`~pyspark.sql.Column` or str, optional + aad : :class:`~pyspark.sql.Column` or column name, optional Optional additional authenticated data. Only supported for GCM mode. This can be any free-form input and must be provided for both encryption and decryption. @@ -24513,6 +24618,11 @@ def aes_decrypt( :class:`~pyspark.sql.Column` A new column that contains a decrypted value. + See Also + -------- + :meth:`pyspark.sql.functions.aes_encrypt` + :meth:`pyspark.sql.functions.try_aes_decrypt` + Examples -------- @@ -24526,7 +24636,7 @@ def aes_decrypt( ... ["input", "key", "mode", "padding", "aad"] ... ) >>> df.select(sf.aes_decrypt( - ... sf.unbase64(df.input), df.key, df.mode, df.padding, df.aad + ... sf.unbase64(df.input), df.key, "mode", df.padding, df.aad ... ).cast("STRING")).show(truncate=False) +---------------------------------------------------------------------+ |CAST(aes_decrypt(unbase64(input), key, mode, padding, aad) AS STRING)| @@ -24543,7 +24653,7 @@ def aes_decrypt( ... ["input", "key", "mode", "padding"] ... ) >>> df.select(sf.aes_decrypt( - ... sf.unbase64(df.input), df.key, df.mode, df.padding + ... sf.unbase64(df.input), df.key, "mode", df.padding ... ).cast("STRING")).show(truncate=False) +------------------------------------------------------------------+ |CAST(aes_decrypt(unbase64(input), key, mode, padding, ) AS STRING)| @@ -24560,7 +24670,7 @@ def aes_decrypt( ... ["input", "key", "mode", "padding"] ... ) >>> df.select(sf.aes_decrypt( - ... sf.unbase64(df.input), df.key, df.mode + ... sf.unbase64(df.input), df.key, "mode" ... ).cast("STRING")).show(truncate=False) +------------------------------------------------------------------+ |CAST(aes_decrypt(unbase64(input), key, mode, DEFAULT, ) AS STRING)| @@ -24612,18 +24722,18 @@ def try_aes_decrypt( Parameters ---------- - input : :class:`~pyspark.sql.Column` or str + input : :class:`~pyspark.sql.Column` or column name The binary value to decrypt. - key : :class:`~pyspark.sql.Column` or str + key : :class:`~pyspark.sql.Column` or column name The passphrase to use to decrypt the data. - mode : :class:`~pyspark.sql.Column` or str, optional + mode : :class:`~pyspark.sql.Column` or column name, optional Specifies which block cipher mode should be used to decrypt messages. Valid modes: ECB, GCM, CBC. - padding : :class:`~pyspark.sql.Column` or str, optional + padding : :class:`~pyspark.sql.Column` or column name, optional Specifies how to pad messages whose length is not a multiple of the block size. Valid values: PKCS, NONE, DEFAULT. The DEFAULT padding means PKCS for ECB, NONE for GCM and PKCS for CBC. - aad : :class:`~pyspark.sql.Column` or str, optional + aad : :class:`~pyspark.sql.Column` or column name, optional Optional additional authenticated data. Only supported for GCM mode. This can be any free-form input and must be provided for both encryption and decryption. @@ -24632,6 +24742,11 @@ def try_aes_decrypt( :class:`~pyspark.sql.Column` A new column that contains a decrypted value or a NULL value. + See Also + -------- + :meth:`pyspark.sql.functions.aes_encrypt` + :meth:`pyspark.sql.functions.aes_decrypt` + Examples -------- @@ -24645,7 +24760,7 @@ def try_aes_decrypt( ... ["input", "key", "mode", "padding", "aad"] ... ) >>> df.select(sf.try_aes_decrypt( - ... sf.unbase64(df.input), df.key, df.mode, df.padding, df.aad + ... sf.unbase64(df.input), df.key, "mode", df.padding, df.aad ... ).cast("STRING")).show(truncate=False) +-------------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, padding, aad) AS STRING)| @@ -24663,7 +24778,7 @@ def try_aes_decrypt( ... ["input", "key", "mode", "padding", "aad"] ... ) >>> df.select(sf.try_aes_decrypt( - ... sf.unbase64(df.input), df.key, df.mode, df.padding, df.aad + ... sf.unbase64(df.input), df.key, "mode", df.padding, df.aad ... ).cast("STRING")).show(truncate=False) +-------------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, padding, aad) AS STRING)| @@ -24680,7 +24795,7 @@ def try_aes_decrypt( ... ["input", "key", "mode", "padding"] ... ) >>> df.select(sf.try_aes_decrypt( - ... sf.unbase64(df.input), df.key, df.mode, df.padding + ... sf.unbase64(df.input), df.key, "mode", df.padding ... ).cast("STRING")).show(truncate=False) +----------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, padding, ) AS STRING)| @@ -24697,7 +24812,7 @@ def try_aes_decrypt( ... ["input", "key", "mode", "padding"] ... ) >>> df.select(sf.try_aes_decrypt( - ... sf.unbase64(df.input), df.key, df.mode + ... sf.unbase64(df.input), df.key, "mode" ... ).cast("STRING")).show(truncate=False) +----------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, DEFAULT, ) AS STRING)| @@ -24737,7 +24852,12 @@ def sha(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name + + See Also + -------- + :meth:`pyspark.sql.functions.sha1` + :meth:`pyspark.sql.functions.sha2` Examples -------- @@ -24827,18 +24947,28 @@ def reflect(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str - the first element should be a literal string for the class name, - and the second element should be a literal string for the method name, - and the remaining are input arguments to the Java method. + cols : :class:`~pyspark.sql.Column` or column name + the first element should be a Column representing literal string for the class name, + and the second element should be a Column representing literal string for the method name, + and the remaining are input arguments (Columns or column names) to the Java method. + + See Also + -------- + :meth:`pyspark.sql.functions.java_method` + :meth:`pyspark.sql.functions.try_reflect` Examples -------- - >>> df = spark.createDataFrame([("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2",)], ["a"]) + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('a5cf6c42-0c85-418f-af6c-3e4e5b1328f2',)], ['a']) >>> df.select( - ... reflect(lit("java.util.UUID"), lit("fromString"), df.a).alias('r') - ... ).collect() - [Row(r='a5cf6c42-0c85-418f-af6c-3e4e5b1328f2')] + ... sf.reflect(sf.lit('java.util.UUID'), sf.lit('fromString'), 'a') + ... ).show(truncate=False) + +--------------------------------------+ + |reflect(java.util.UUID, fromString, a)| + +--------------------------------------+ + |a5cf6c42-0c85-418f-af6c-3e4e5b1328f2 | + +--------------------------------------+ """ return _invoke_function_over_seq_of_columns("reflect", cols) @@ -24852,13 +24982,20 @@ def java_method(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str - the first element should be a literal string for the class name, - and the second element should be a literal string for the method name, - and the remaining are input arguments to the Java method. + cols : :class:`~pyspark.sql.Column` or column name + the first element should be a Column representing literal string for the class name, + and the second element should be a Column representing literal string for the method name, + and the remaining are input arguments (Columns or column names) to the Java method. + + See Also + -------- + :meth:`pyspark.sql.functions.reflect` + :meth:`pyspark.sql.functions.try_reflect` Examples -------- + Example 1: Reflecting a method call with a column argument + >>> import pyspark.sql.functions as sf >>> spark.range(1).select( ... sf.java_method( @@ -24872,6 +25009,19 @@ def java_method(*cols: "ColumnOrName") -> Column: +-----------------------------------------------------------------------------+ |a5cf6c42-0c85-418f-af6c-3e4e5b1328f2 | +-----------------------------------------------------------------------------+ + + Example 2: Reflecting a method call with a column name argument + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('a5cf6c42-0c85-418f-af6c-3e4e5b1328f2',)], ['a']) + >>> df.select( + ... sf.java_method(sf.lit('java.util.UUID'), sf.lit('fromString'), 'a') + ... ).show(truncate=False) + +------------------------------------------+ + |java_method(java.util.UUID, fromString, a)| + +------------------------------------------+ + |a5cf6c42-0c85-418f-af6c-3e4e5b1328f2 | + +------------------------------------------+ """ return _invoke_function_over_seq_of_columns("java_method", cols) @@ -24887,10 +25037,15 @@ def try_reflect(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str - the first element should be a literal string for the class name, - and the second element should be a literal string for the method name, - and the remaining are input arguments to the Java method. + cols : :class:`~pyspark.sql.Column` or column name + the first element should be a Column representing literal string for the class name, + and the second element should be a Column representing literal string for the method name, + and the remaining are input arguments (Columns or column names) to the Java method. + + See Also + -------- + :meth:`pyspark.sql.functions.reflect` + :meth:`pyspark.sql.functions.java_method` Examples -------- @@ -24899,25 +25054,24 @@ def try_reflect(*cols: "ColumnOrName") -> Column: >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2",)], ["a"]) >>> df.select( - ... sf.try_reflect(sf.lit("java.util.UUID"), sf.lit("fromString"), df.a) - ... ).show() + ... sf.try_reflect(sf.lit("java.util.UUID"), sf.lit("fromString"), "a") + ... ).show(truncate=False) +------------------------------------------+ |try_reflect(java.util.UUID, fromString, a)| +------------------------------------------+ - | a5cf6c42-0c85-418...| + |a5cf6c42-0c85-418f-af6c-3e4e5b1328f2 | +------------------------------------------+ Example 2: Exception in the reflection call, resulting in null >>> from pyspark.sql import functions as sf - >>> df = spark.range(1) - >>> df.select( + >>> spark.range(1).select( ... sf.try_reflect(sf.lit("scala.Predef"), sf.lit("require"), sf.lit(False)) - ... ).show() + ... ).show(truncate=False) +-----------------------------------------+ |try_reflect(scala.Predef, require, false)| +-----------------------------------------+ - | NULL| + |NULL | +-----------------------------------------+ """ return _invoke_function_over_seq_of_columns("try_reflect", cols) @@ -24933,12 +25087,12 @@ def version() -> Column: Examples -------- - >>> df = spark.range(1) - >>> df.select(version()).show(truncate=False) # doctest: +SKIP + >>> from pyspark.sql import functions as sf + >>> spark.range(1).select(sf.version()).show(truncate=False) # doctest: +SKIP +----------------------------------------------+ |version() | +----------------------------------------------+ - |3.5.0 cafbea5b13623276517a9d716f75745eff91f616| + |4.0.0 4f8d1f575e99aeef8990c63a9614af0fc5479330| +----------------------------------------------+ """ return _invoke_function_over_columns("version") @@ -24953,13 +25107,18 @@ def typeof(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name Examples -------- - >>> df = spark.createDataFrame([(1,)], ["a"]) - >>> df.select(typeof(df.a).alias('r')).collect() - [Row(r='bigint')] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(True, 1, 1.0, 'xyz',)], ['a', 'b', 'c', 'd']) + >>> df.select(sf.typeof(df.a), sf.typeof(df.b), sf.typeof('c'), sf.typeof('d')).show() + +---------+---------+---------+---------+ + |typeof(a)|typeof(b)|typeof(c)|typeof(d)| + +---------+---------+---------+---------+ + | boolean| bigint| double| string| + +---------+---------+---------+---------+ """ return _invoke_function_over_columns("typeof", col) @@ -24974,20 +25133,48 @@ def stack(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name the first element should be a literal int for the number of rows to be separated, and the remaining are input elements to be separated. Examples -------- - >>> df = spark.createDataFrame([(1, 2, 3)], ["a", "b", "c"]) - >>> df.select(stack(lit(2), df.a, df.b, df.c)).show(truncate=False) - +----+----+ - |col0|col1| - +----+----+ - |1 |2 | - |3 |NULL| - +----+----+ + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(1, 2, 3)], ['a', 'b', 'c']) + >>> df.select('*', sf.stack(sf.lit(2), df.a, df.b, 'c')).show() + +---+---+---+----+----+ + | a| b| c|col0|col1| + +---+---+---+----+----+ + | 1| 2| 3| 1| 2| + | 1| 2| 3| 3|NULL| + +---+---+---+----+----+ + + >>> df.select('*', sf.stack(sf.lit(2), df.a, df.b, 'c').alias('x', 'y')).show() + +---+---+---+---+----+ + | a| b| c| x| y| + +---+---+---+---+----+ + | 1| 2| 3| 1| 2| + | 1| 2| 3| 3|NULL| + +---+---+---+---+----+ + + >>> df.select('*', sf.stack(sf.lit(3), df.a, df.b, 'c')).show() + +---+---+---+----+ + | a| b| c|col0| + +---+---+---+----+ + | 1| 2| 3| 1| + | 1| 2| 3| 2| + | 1| 2| 3| 3| + +---+---+---+----+ + + >>> df.select('*', sf.stack(sf.lit(4), df.a, df.b, 'c')).show() + +---+---+---+----+ + | a| b| c|col0| + +---+---+---+----+ + | 1| 2| 3| 1| + | 1| 2| 3| 2| + | 1| 2| 3| 3| + | 1| 2| 3|NULL| + +---+---+---+----+ """ return _invoke_function_over_seq_of_columns("stack", cols) @@ -25001,14 +25188,26 @@ def bitmap_bit_position(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The input column. + See Also + -------- + :meth:`pyspark.sql.functions.bitmap_bucket_number` + :meth:`pyspark.sql.functions.bitmap_construct_agg` + :meth:`pyspark.sql.functions.bitmap_count` + :meth:`pyspark.sql.functions.bitmap_or_agg` + Examples -------- - >>> df = spark.createDataFrame([(123,)], ["a"]) - >>> df.select(bitmap_bit_position(df.a).alias("r")).collect() - [Row(r=122)] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(123,)], ['a']) + >>> df.select('*', sf.bitmap_bit_position('a')).show() + +---+----------------------+ + | a|bitmap_bit_position(a)| + +---+----------------------+ + |123| 122| + +---+----------------------+ """ return _invoke_function_over_columns("bitmap_bit_position", col) @@ -25022,14 +25221,26 @@ def bitmap_bucket_number(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The input column. + See Also + -------- + :meth:`pyspark.sql.functions.bitmap_bit_position` + :meth:`pyspark.sql.functions.bitmap_construct_agg` + :meth:`pyspark.sql.functions.bitmap_count` + :meth:`pyspark.sql.functions.bitmap_or_agg` + Examples -------- - >>> df = spark.createDataFrame([(123,)], ["a"]) - >>> df.select(bitmap_bucket_number(df.a).alias("r")).collect() - [Row(r=1)] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(123,)], ['a']) + >>> df.select('*', sf.bitmap_bucket_number('a')).show() + +---+-----------------------+ + | a|bitmap_bucket_number(a)| + +---+-----------------------+ + |123| 1| + +---+-----------------------+ """ return _invoke_function_over_columns("bitmap_bucket_number", col) @@ -25044,16 +25255,28 @@ def bitmap_construct_agg(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The input column will most likely be bitmap_bit_position(). + See Also + -------- + :meth:`pyspark.sql.functions.bitmap_bit_position` + :meth:`pyspark.sql.functions.bitmap_bucket_number` + :meth:`pyspark.sql.functions.bitmap_count` + :meth:`pyspark.sql.functions.bitmap_or_agg` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(1,),(2,),(3,)], ["a"]) - >>> df.select(substring(hex( - ... bitmap_construct_agg(bitmap_bit_position(df.a)) - ... ), 0, 6).alias("r")).collect() - [Row(r='070000')] + >>> df.select( + ... sf.bitmap_construct_agg(sf.bitmap_bit_position('a')) + ... ).show() + +--------------------------------------------+ + |bitmap_construct_agg(bitmap_bit_position(a))| + +--------------------------------------------+ + | [07 00 00 00 00 0...| + +--------------------------------------------+ """ return _invoke_function_over_columns("bitmap_construct_agg", col) @@ -25067,14 +25290,26 @@ def bitmap_count(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The input bitmap. + See Also + -------- + :meth:`pyspark.sql.functions.bitmap_bit_position` + :meth:`pyspark.sql.functions.bitmap_bucket_number` + :meth:`pyspark.sql.functions.bitmap_construct_agg` + :meth:`pyspark.sql.functions.bitmap_or_agg` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("FFFF",)], ["a"]) - >>> df.select(bitmap_count(to_binary(df.a, lit("hex"))).alias('r')).collect() - [Row(r=16)] + >>> df.select(sf.bitmap_count(sf.to_binary(df.a, sf.lit("hex")))).show() + +-------------------------------+ + |bitmap_count(to_binary(a, hex))| + +-------------------------------+ + | 16| + +-------------------------------+ """ return _invoke_function_over_columns("bitmap_count", col) @@ -25087,18 +25322,28 @@ def bitmap_or_agg(col: "ColumnOrName") -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.bitmap_bit_position` + :meth:`pyspark.sql.functions.bitmap_bucket_number` + :meth:`pyspark.sql.functions.bitmap_construct_agg` + :meth:`pyspark.sql.functions.bitmap_count` + Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The input column should be bitmaps created from bitmap_construct_agg(). Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("10",),("20",),("40",)], ["a"]) - >>> df.select(substring(hex( - ... bitmap_or_agg(to_binary(df.a, lit("hex"))) - ... ), 0, 6).alias("r")).collect() - [Row(r='700000')] + >>> df.select(sf.bitmap_or_agg(sf.to_binary(df.a, sf.lit("hex")))).show() + +--------------------------------+ + |bitmap_or_agg(to_binary(a, hex))| + +--------------------------------+ + | [70 00 00 00 00 0...| + +--------------------------------+ """ return _invoke_function_over_columns("bitmap_or_agg", col) From 7cbfc2cad078f40c290eff6ffd90e4902397314a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 26 Nov 2024 10:56:38 +0100 Subject: [PATCH 026/438] [SPARK-50422][SQL] Make `Parameterized SQL queries` of `SparkSession.sql` API GA ### What changes were proposed in this pull request? This PR aims to make `Parameterized SQL queries` of `SparkSession.sql` API GA in Apache Spark 4.0.0. ### Why are the changes needed? Apache Spark has been supported `Parameterized SQL queries` because they are very convenient usage for the users . - https://github.com/apache/spark/pull/38864 (Since Spark 3.4.0) - https://github.com/apache/spark/pull/41568 (Since Spark 3.5.0) It's time to make it GA by removing `Experimental` tags since this feature has been serving well for a long time. ### Does this PR introduce _any_ user-facing change? No, there is no behavior change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48965 from dongjoon-hyun/SPARK-50422. Authored-by: Dongjoon Hyun Signed-off-by: Max Gekk --- .../jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala | 3 --- .../src/main/scala/org/apache/spark/sql/api/SparkSession.scala | 3 --- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 3 --- 3 files changed, 9 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 231c604b98bb5..b74d0c2ff2243 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -210,7 +210,6 @@ class SparkSession private[sql] ( throw ConnectClientUnsupportedErrors.executeCommand() /** @inheritdoc */ - @Experimental def sql(sqlText: String, args: Array[_]): DataFrame = { val sqlCommand = proto.SqlCommand .newBuilder() @@ -221,13 +220,11 @@ class SparkSession private[sql] ( } /** @inheritdoc */ - @Experimental def sql(sqlText: String, args: Map[String, Any]): DataFrame = { sql(sqlText, args.asJava) } /** @inheritdoc */ - @Experimental override def sql(sqlText: String, args: java.util.Map[String, Any]): DataFrame = { val sqlCommand = proto.SqlCommand .newBuilder() diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala index 64b0a87c573d3..35f74497b96f4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala @@ -470,7 +470,6 @@ abstract class SparkSession extends Serializable with Closeable { * is. * @since 3.5.0 */ - @Experimental def sql(sqlText: String, args: Array[_]): Dataset[Row] /** @@ -488,7 +487,6 @@ abstract class SparkSession extends Serializable with Closeable { * `array()`, `struct()`, in that case it is taken as is. * @since 3.4.0 */ - @Experimental def sql(sqlText: String, args: Map[String, Any]): Dataset[Row] /** @@ -506,7 +504,6 @@ abstract class SparkSession extends Serializable with Closeable { * `array()`, `struct()`, in that case it is taken as is. * @since 3.4.0 */ - @Experimental def sql(sqlText: String, args: util.Map[String, Any]): Dataset[Row] = { sql(sqlText, args.asScala.toMap) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index a7f85db12b214..8cf30fb39f310 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -460,7 +460,6 @@ class SparkSession private( } /** @inheritdoc */ - @Experimental def sql(sqlText: String, args: Array[_]): DataFrame = { sql(sqlText, args, new QueryPlanningTracker) } @@ -498,13 +497,11 @@ class SparkSession private( } /** @inheritdoc */ - @Experimental def sql(sqlText: String, args: Map[String, Any]): DataFrame = { sql(sqlText, args, new QueryPlanningTracker) } /** @inheritdoc */ - @Experimental override def sql(sqlText: String, args: java.util.Map[String, Any]): DataFrame = { sql(sqlText, args.asScala.toMap) } From 02bfce6056ec3d0e99753c8f318588390c52baee Mon Sep 17 00:00:00 2001 From: Neil Ramaswamy Date: Wed, 27 Nov 2024 08:29:35 +0900 Subject: [PATCH 027/438] [SPARK-50302][SS] Ensure secondary index sizes equal primary index sizes for TransformWithState stateful variables with TTL MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR ensures that the secondary indexes that state variables with TTL use are at most the size of the corresponding state variable's primary index. This change will eliminate unnecessary work done during the cleanup of stateful variables with TTL. ### Why are the changes needed? #### Context The `TransformWithState` operator (hereon out known as "TWS") will allow users write procedural logic over streams of records. To store state between micro-batches, Spark will provide users _stateful variables_, which persist between micro-batches. For example, you might want to emit an average of the past 5 records, every 5 records. You might only receive 2 records in the first micro-batch, so you have to _buffer_ these 2 records until you get 3 more in a subsequent batch. TWS supports 3 different types of stateful variables: single values, lists, and maps. The TWS operator also supports stateful variables with Time To Live; this allows you to say, "keep a certain record in state for `d` units of time". This TTL is per-record. This means that every record in a list (or map) can expiry at a different point in time, depending on when the element in the list is inserted. A record inserted into a stateful list (or map) at time `t1` will expire at `t1 + d`, and a second that expires at `t2 + d` will expire at `t2 + d`. (For value state, there's only one value, so "everything" expires at the same time.) A very natural question to now ask is, how do we efficiently determine which elements have expired in the list, without having to do a full scan of every record in state? The idea here is to keep a secondary index from expiration timestamp, to the specific record that needs to be evicted. Not so hard, right? #### The state cleanup strategy today Today's cleanup strategy is about as simple as I indicated earlier: for every insert to a value/map/list, you: 1. Write to the primary index 2. Using the current timestamp, you write into the secondary index The issue with this approach is that we do two _unconditional_ writes. This means that if the same state variable is written to with different timestamps, there will exist one element in the primary index, while there exists two elements in the secondary index. Consider the following example for a state variable `foo` with value `v1`, and TTL delay of 500: For batch 0, `batchTimestampMs = 100`, `foo` updates to `v1`: - Primary index: `[foo -> (v1, 600)]` - Secondary index: `[(600, foo) -> EMPTY]` Note that the state variable is included in the secondary index key because we might have several elements with the same expiration timestamp; we want `(600, foo)` to not overwrite a `(600, bar)`, just because they both expire at 600. Batch 1: `batchTimestampMs = 200`, `foo` updates to `v2`. Primary index: `[foo -> (v2, 700)]` Secondary index: `[(600, foo) -> EMPTY, (700, foo) -> EMPTY]` Now, we have two entries in our secondary index. If the current timestamp advanced to something like 800, we'd take the following steps: 1. We'd take the first element from the secondary index `(600, foo)`, and lookup `foo` in the primary index. That would yield `(v2, 700)`. The value of 700 in the primary index is still less than 800, so we would remove `foo` from the primary index. 2. Then, we would look at `(700, foo)`. We'd look up `foo` in the primary index and see nothing, so we'd do nothing. You'll notice here that step 2 is _entirely_ redundant. We read `(700, foo)` and did a get to the primary index, for something that was doomed—it would have never returned anything. While this isn't great, the story is unfortunately significantly worse for lists. The way that we store lists is by having a single key in RocksDB, whose value is the concatenated bytes of all the values in that list. When we do cleanup for a list, we go through _all_ of its records and Thus, it's possible for us to have a list that looks something like: - Primary index: `[foo -> [(v1, 600), (v2, 700), (v3, 900)]]` - Secondary index: `[(600, foo) -> EMPTY, (700, foo) -> EMPTY, (900, foo) -> EMPTY]` Now, suppose that the current timestamp is 800. We need to expire the records in the list. So, we do the following: 1. We take the first element from the secondary index, `(600, foo)`. This tells us that the list `foo` needs cleaning up. We clean up everything in `foo` less than 800. Since we store lists as a single key, we issue a RocksDB `clear` operation, iterate through all of the existing values, eliminate `(v1, 600)` and `(v2, 700)`, and write back `(v3, 900)`. 2. But we still have things left in our secondary index! We now get `(700, foo)`, and we unknowingly do cleanup on `foo` _again_. This consists of clearing `foo`, iterating through its elements, and writing back `(v3, 900)`. But since cleanup already happened, this step is _entirely_ redundant. 3. We encounter `(900, foo)` from the secondary index, and since 900 > 800, we can bail out of cleanup. Step 2 here is extremely wasteful. If we have `n` elements in our secondary index for the same key, then, in the worst case, we will do the extra cleanup `n-1` times; and each time is a _linear_ time operation! Thus, for a list that has `n` elements, `d` of which need to be cleaned up, the worst-case time complexity is in `O(d*(n-d))`, instead of `O(n)`. And it's _completely_ unnecessary work. #### How does this PR fix the issue? It's pretty simple to fix this for value state and map state. This is because every key in value or map state maps to exactly one element in the secondary index. We can maintain a one-to-one correspondence. Any time we modify value/map state, we make sure that we delete the previous entry in the secondary index. This logic is implemented by OneToOneTTLState. The trickier aspect is handling this for ListState, where the secondary index goes from grouping key to the map that needs to be cleaned up. There's a one to many mapping here; one grouping key maps to multiple records, all of which could expire at a different time. The trick to making sure that secondary indexes don't explode is by making your secondary index store only the minimum expiration timestamp in a list. The rough intuition is that you don't need to store anything larger than that, since when you clean up due to the minimum expiration timestamp, you'll go through the list anyway, and you can find the next minimum timestamp; you can then put _that_ into your secondary index. This logic is implemented by OneToManyTTLState. ### How should reviewers review this PR? - Start by reading this long description. If you have questions, please ping me in the comments. I would be more than happy to explain. - Then, understand the class doc comments for `OneToOneTTLState` and `OneToManyTTLState` in `TTLState.scala`. - Then, I'd recommend going through the unit tests, and making sure that the _behavior_ makes sense to you. If it doesn't, please leave a question. - Finally, you can look at the actual stateful variable implementations. ### Does this PR introduce _any_ user-facing change? No, but it is a format difference in the way TWS represents its internal state. However, since TWS is currently `private[sql]` and not publicly available, this is not an issue. ### How was this patch tested? - Existing UTs have been modified to conform with this new behavior. - New UTs added to verify that the new indices we added ### Was this patch authored or co-authored using generative AI tooling? Generated-by: GitHub Copilot Closes #48853 from neilramaswamy/spark-50302. Authored-by: Neil Ramaswamy Signed-off-by: Jungtaek Lim --- .../streaming/ListStateImplWithTTL.scala | 125 ++-- .../streaming/MapStateImplWithTTL.scala | 94 +-- .../streaming/StateTypesEncoderUtils.scala | 77 +- .../StatefulProcessorHandleImpl.scala | 2 +- .../sql/execution/streaming/TTLState.scala | 679 ++++++++++++------ .../streaming/ValueStateImplWithTTL.scala | 61 +- .../streaming/state/ListStateSuite.scala | 15 +- .../streaming/state/ValueStateSuite.scala | 15 +- .../TransformWithListStateTTLSuite.scala | 277 ++++++- .../TransformWithMapStateTTLSuite.scala | 3 +- .../streaming/TransformWithStateTTLTest.scala | 79 +- .../TransformWithValueStateTTLSuite.scala | 6 +- 12 files changed, 965 insertions(+), 468 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala index 4c8dd6a193c25..4e32b80578155 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors} import org.apache.spark.sql.streaming.{ListState, TTLConfig} -import org.apache.spark.sql.types.StructType import org.apache.spark.util.NextIterator /** @@ -45,21 +44,13 @@ class ListStateImplWithTTL[S]( valEncoder: ExpressionEncoder[Any], ttlConfig: TTLConfig, batchTimestampMs: Long, - metrics: Map[String, SQLMetric] = Map.empty) - extends SingleKeyTTLStateImpl(stateName, store, keyExprEnc, batchTimestampMs) - with ListStateMetricsImpl - with ListState[S] { - - override def stateStore: StateStore = store - override def baseStateName: String = stateName - override def exprEncSchema: StructType = keyExprEnc.schema + metrics: Map[String, SQLMetric]) + extends OneToManyTTLState( + stateName, store, keyExprEnc.schema, ttlConfig, batchTimestampMs, metrics) with ListState[S] { private lazy val stateTypesEncoder = StateTypesEncoder(keyExprEnc, valEncoder, stateName, hasTtl = true) - private lazy val ttlExpirationMs = - StateTTL.calculateExpirationTimeForDuration(ttlConfig.ttlDuration, batchTimestampMs) - initialize() private def initialize(): Unit = { @@ -106,35 +97,22 @@ class ListStateImplWithTTL[S]( validateNewState(newState) val encodedKey = stateTypesEncoder.encodeGroupingKey() - var isFirst = true - var entryCount = 0L - TWSMetricsUtils.resetMetric(metrics, "numUpdatedStateRows") - - newState.foreach { v => - val encodedValue = stateTypesEncoder.encodeValue(v, ttlExpirationMs) - if (isFirst) { - store.put(encodedKey, encodedValue, stateName) - isFirst = false - } else { - store.merge(encodedKey, encodedValue, stateName) - } - entryCount += 1 - TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows") + val newStateUnsafeRows = newState.iterator.map { v => + stateTypesEncoder.encodeValue(v, ttlExpirationMs) } - upsertTTLForStateKey(encodedKey) - updateEntryCount(encodedKey, entryCount) + + updatePrimaryAndSecondaryIndices(true, encodedKey, newStateUnsafeRows, ttlExpirationMs) } /** Append an entry to the list. */ override def appendValue(newState: S): Unit = { StateStoreErrors.requireNonNullStateValue(newState, stateName) + val encodedKey = stateTypesEncoder.encodeGroupingKey() - val entryCount = getEntryCount(encodedKey) - store.merge(encodedKey, - stateTypesEncoder.encodeValue(newState, ttlExpirationMs), stateName) - TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows") - upsertTTLForStateKey(encodedKey) - updateEntryCount(encodedKey, entryCount + 1) + val newStateUnsafeRow = stateTypesEncoder.encodeValue(newState, ttlExpirationMs) + + updatePrimaryAndSecondaryIndices(false, encodedKey, + Iterator.single(newStateUnsafeRow), ttlExpirationMs) } /** Append an entire list to the existing value. */ @@ -142,25 +120,21 @@ class ListStateImplWithTTL[S]( validateNewState(newState) val encodedKey = stateTypesEncoder.encodeGroupingKey() - var entryCount = getEntryCount(encodedKey) - newState.foreach { v => - val encodedValue = stateTypesEncoder.encodeValue(v, ttlExpirationMs) - store.merge(encodedKey, encodedValue, stateName) - entryCount += 1 - TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows") + // The UnsafeRows created here are reused: we do NOT copy them. As a result, + // this iterator must only be used lazily, and it should never be materialized, + // unless you call newStateUnsafeRows.map(_.copy()). + val newStateUnsafeRows = newState.iterator.map { v => + stateTypesEncoder.encodeValue(v, ttlExpirationMs) } - upsertTTLForStateKey(encodedKey) - updateEntryCount(encodedKey, entryCount) + + updatePrimaryAndSecondaryIndices(false, encodedKey, + newStateUnsafeRows, ttlExpirationMs) } /** Remove this state. */ override def clear(): Unit = { - val encodedKey = stateTypesEncoder.encodeGroupingKey() - store.remove(encodedKey, stateName) - val entryCount = getEntryCount(encodedKey) - TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows", entryCount) - removeEntryCount(encodedKey) - clearTTLState() + val groupingKey = stateTypesEncoder.encodeGroupingKey() + clearAllStateForElementKey(groupingKey) } private def validateNewState(newState: Array[S]): Unit = { @@ -175,36 +149,41 @@ class ListStateImplWithTTL[S]( /** * Loops through all the values associated with the grouping key, and removes * the expired elements from the list. - * @param groupingKey grouping key for which cleanup should be performed. + * @param elementKey grouping key for which cleanup should be performed. */ - override def clearIfExpired(groupingKey: UnsafeRow): Long = { + override def clearExpiredValues(elementKey: UnsafeRow): ValueExpirationResult = { var numValuesExpired = 0L - val unsafeRowValuesIterator = store.valuesIterator(groupingKey, stateName) + val unsafeRowValuesIterator = store.valuesIterator(elementKey, stateName) // We clear the list, and use the iterator to put back all of the non-expired values - store.remove(groupingKey, stateName) - removeEntryCount(groupingKey) + store.remove(elementKey, stateName) + + var newMinExpirationMsOpt: Option[Long] = None var isFirst = true - var entryCount = 0L unsafeRowValuesIterator.foreach { encodedValue => if (!stateTypesEncoder.isExpired(encodedValue, batchTimestampMs)) { if (isFirst) { - store.put(groupingKey, encodedValue, stateName) isFirst = false + store.put(elementKey, encodedValue, stateName) } else { - store.merge(groupingKey, encodedValue, stateName) + store.merge(elementKey, encodedValue, stateName) + } + + // If it is not expired, it needs to be reinserted (either via put or merge), but + // it also has an expiration time that might be the new minimum. + val currentExpirationMs = stateTypesEncoder.decodeTtlExpirationMs(encodedValue) + + newMinExpirationMsOpt = newMinExpirationMsOpt match { + case Some(minExpirationMs) => + Some(math.min(minExpirationMs, currentExpirationMs.get)) + case None => + Some(currentExpirationMs.get) } - entryCount += 1 } else { numValuesExpired += 1 } } - updateEntryCount(groupingKey, entryCount) - TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows", numValuesExpired) - numValuesExpired - } - private def upsertTTLForStateKey(encodedGroupingKey: UnsafeRow): Unit = { - upsertTTLForStateKey(ttlExpirationMs, encodedGroupingKey) + ValueExpirationResult(numValuesExpired, newMinExpirationMsOpt) } /* @@ -238,11 +217,23 @@ class ListStateImplWithTTL[S]( } } + private[sql] def getMinValues(): Iterator[Long] = { + val groupingKey = stateTypesEncoder.encodeGroupingKey() + minIndexIterator() + .filter(_._1 == groupingKey) + .map(_._2) + } + /** - * Get all ttl values stored in ttl state for current implicit - * grouping key. + * Get the TTL value stored in TTL state for the current implicit grouping key, + * if it exists. */ - private[sql] def getValuesInTTLState(): Iterator[Long] = { - getValuesInTTLState(stateTypesEncoder.encodeGroupingKey()) + private[sql] def getValueInTTLState(): Option[Long] = { + val groupingKey = stateTypesEncoder.encodeGroupingKey() + val ttlRowsForGroupingKey = getTTLRows().filter(_.elementKey == groupingKey).toSeq + + assert(ttlRowsForGroupingKey.size <= 1, "Multiple TTLRows found for grouping key " + + s"$groupingKey. Expected at most 1. Found: ${ttlRowsForGroupingKey.mkString(", ")}.") + ttlRowsForGroupingKey.headOption.map(_.expirationMs) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala index 19704b6d1bd59..64581006555e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ import org.apache.spark.sql.execution.streaming.state.{PrefixKeyScanStateEncoderSpec, StateStore, StateStoreErrors} @@ -48,17 +47,14 @@ class MapStateImplWithTTL[K, V]( valEncoder: ExpressionEncoder[Any], ttlConfig: TTLConfig, batchTimestampMs: Long, - metrics: Map[String, SQLMetric] = Map.empty) - extends CompositeKeyTTLStateImpl[K](stateName, store, - keyExprEnc, userKeyEnc, batchTimestampMs) - with MapState[K, V] with Logging { +metrics: Map[String, SQLMetric]) + extends OneToOneTTLState( + stateName, store, getCompositeKeySchema(keyExprEnc.schema, userKeyEnc.schema), ttlConfig, + batchTimestampMs, metrics) with MapState[K, V] with Logging { private val stateTypesEncoder = new CompositeKeyStateEncoder( keyExprEnc, userKeyEnc, valEncoder, stateName, hasTtl = true) - private val ttlExpirationMs = - StateTTL.calculateExpirationTimeForDuration(ttlConfig.ttlDuration, batchTimestampMs) - initialize() private def initialize(): Unit = { @@ -102,15 +98,12 @@ class MapStateImplWithTTL[K, V]( StateStoreErrors.requireNonNullStateValue(key, stateName) StateStoreErrors.requireNonNullStateValue(value, stateName) - val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey() - val encodedUserKey = stateTypesEncoder.encodeUserKey(key) - - val encodedValue = stateTypesEncoder.encodeValue(value, ttlExpirationMs) val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key) - store.put(encodedCompositeKey, encodedValue, stateName) - TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows") + val ttlExpirationMs = StateTTL + .calculateExpirationTimeForDuration(ttlConfig.ttlDuration, batchTimestampMs) + val encodedValue = stateTypesEncoder.encodeValue(value, ttlExpirationMs) - upsertTTLForStateKey(ttlExpirationMs, encodedGroupingKey, encodedUserKey) + updatePrimaryAndSecondaryIndices(encodedCompositeKey, encodedValue, ttlExpirationMs) } /** Get the map associated with grouping key */ @@ -161,41 +154,12 @@ class MapStateImplWithTTL[K, V]( /** Remove this state. */ override def clear(): Unit = { - keys().foreach { itr => - removeKey(itr) - } - clearTTLState() - } - - /** - * Clears the user state associated with this grouping key - * if it has expired. This function is called by Spark to perform - * cleanup at the end of transformWithState processing. - * - * Spark uses a secondary index to determine if the user state for - * this grouping key has expired. However, its possible that the user - * has updated the TTL and secondary index is out of date. Implementations - * must validate that the user State has actually expired before cleanup based - * on their own State data. - * - * @param groupingKey grouping key for which cleanup should be performed. - * @param userKey user key for which cleanup should be performed. - */ - override def clearIfExpired( - groupingKeyRow: UnsafeRow, - userKeyRow: UnsafeRow): Long = { - val compositeKeyRow = stateTypesEncoder.encodeCompositeKey(groupingKeyRow, userKeyRow) + val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey() + val unsafeRowPairIterator = store.prefixScan(encodedGroupingKey, stateName) - val retRow = store.get(compositeKeyRow, stateName) - var numRemovedElements = 0L - if (retRow != null) { - if (stateTypesEncoder.isExpired(retRow, batchTimestampMs)) { - store.remove(compositeKeyRow, stateName) - numRemovedElements += 1 - TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows") - } + unsafeRowPairIterator.foreach { rowPair => + clearAllStateForElementKey(rowPair.key) } - numRemovedElements } /* @@ -243,30 +207,18 @@ class MapStateImplWithTTL[K, V]( * grouping key. */ private[sql] def getKeyValuesInTTLState(): Iterator[(K, Long)] = { - val ttlIterator = ttlIndexIterator() val implicitGroupingKey = stateTypesEncoder.encodeGroupingKey() - var nextValue: Option[(K, Long)] = None - - new Iterator[(K, Long)] { - override def hasNext: Boolean = { - while (nextValue.isEmpty && ttlIterator.hasNext) { - val nextTtlValue = ttlIterator.next() - val groupingKey = nextTtlValue.groupingKey - if (groupingKey equals implicitGroupingKey.getStruct( - 0, keyExprEnc.schema.length)) { - val userKey = stateTypesEncoder.decodeUserKey( - nextTtlValue.userKey) - nextValue = Some(userKey.asInstanceOf[K], nextTtlValue.expirationMs) - } - } - nextValue.isDefined - } - - override def next(): (K, Long) = { - val result = nextValue.get - nextValue = None - result - } + .getStruct(0, keyExprEnc.schema.length) + + // We're getting composite rows back + getTTLRows().filter { ttlRow => + val compositeKey = ttlRow.elementKey + val groupingKey = compositeKey.getStruct(0, keyExprEnc.schema.length) + groupingKey == implicitGroupingKey + }.map { ttlRow => + val compositeKey = ttlRow.elementKey + val userKey = stateTypesEncoder.decodeCompositeKey(compositeKey) + (userKey.asInstanceOf[K], ttlRow.expirationMs) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala index d87de4c69c40a..a2b7ee4ba7916 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala @@ -30,6 +30,11 @@ import org.apache.spark.sql.types._ * files and to be passed into `RocksDBStateKey(/Value)Encoder`. */ object TransformWithStateKeyValueRowSchemaUtils { + /** + * Creates a schema that is the concatenation of the grouping key and a user-defined + * key. This is used by MapState to create a composite key that is then treated as + * an "elementKey" by OneToOneTTLState. + */ def getCompositeKeySchema( groupingKeySchema: StructType, userKeySchema: StructType): StructType = { @@ -38,24 +43,37 @@ object TransformWithStateKeyValueRowSchemaUtils { .add("userKey", new StructType(userKeySchema.fields)) } - def getSingleKeyTTLRowSchema(keySchema: StructType): StructType = + /** + * Represents the schema of keys in the TTL index, managed by TTLState implementations. + * There is no value associated with entries in the TTL index, so there is no method + * called, for example, getTTLValueSchema. + */ + def getTTLRowKeySchema(keySchema: StructType): StructType = new StructType() .add("expirationMs", LongType) - .add("groupingKey", keySchema) + .add("elementKey", keySchema) - def getCompositeKeyTTLRowSchema( - groupingKeySchema: StructType, - userKeySchema: StructType): StructType = + /** + * Represents the schema of a single long value, which is used to store the expiration + * timestamp of elements in the minimum index, managed by OneToManyTTLState. + */ + def getExpirationMsRowSchema(): StructType = new StructType() .add("expirationMs", LongType) - .add("groupingKey", new StructType(groupingKeySchema.fields)) - .add("userKey", new StructType(userKeySchema.fields)) + /** + * Represents the schema of an element with TTL in the primary index. We store the expiration + * of each value along with the value itself, since each value has its own TTL. It is used as + * the value schema of every value, for every stateful variable. + */ def getValueSchemaWithTTL(schema: StructType, hasTTL: Boolean): StructType = { if (hasTTL) { - new StructType().add("value", schema) + new StructType() + .add("value", schema) .add("ttlExpirationMs", LongType) - } else schema + } else { + schema + } } } @@ -118,7 +136,9 @@ class StateTypesEncoder[V]( def decodeValue(row: UnsafeRow): V = { if (hasTtl) { rowToObjDeserializer.apply(row.getStruct(0, valEncoder.schema.length)) - } else rowToObjDeserializer.apply(row) + } else { + rowToObjDeserializer.apply(row) + } } /** @@ -225,10 +245,6 @@ class CompositeKeyStateEncoder[K, V]( compositeKeyProjection(InternalRow(groupingKey, userKey)) } - def decodeUserKey(row: UnsafeRow): K = { - userKeyRowToObjDeserializer.apply(row) - } - /** * The input row is of composite Key schema. * Only user key is returned though grouping key also exist in the row. @@ -239,37 +255,14 @@ class CompositeKeyStateEncoder[K, V]( } /** Class for TTL with single key serialization */ -class SingleKeyTTLEncoder( - keyExprEnc: ExpressionEncoder[Any]) { - - private val ttlKeyProjection = UnsafeProjection.create( - getSingleKeyTTLRowSchema(keyExprEnc.schema)) - - def encodeTTLRow(expirationMs: Long, groupingKey: UnsafeRow): UnsafeRow = { - ttlKeyProjection.apply( - InternalRow(expirationMs, groupingKey.asInstanceOf[InternalRow])) - } -} - -/** Class for TTL with composite key serialization */ -class CompositeKeyTTLEncoder[K]( - keyExprEnc: ExpressionEncoder[Any], - userKeyEnc: ExpressionEncoder[Any]) { +class TTLEncoder(schema: StructType) { - private val ttlKeyProjection = UnsafeProjection.create( - getCompositeKeyTTLRowSchema(keyExprEnc.schema, userKeyEnc.schema)) + private val ttlKeyProjection = UnsafeProjection.create(getTTLRowKeySchema(schema)) - def encodeTTLRow( - expirationMs: Long, - groupingKey: UnsafeRow, - userKey: UnsafeRow): UnsafeRow = { + // Take a groupingKey UnsafeRow and turn it into a (expirationMs, groupingKey) UnsafeRow. + def encodeTTLRow(expirationMs: Long, elementKey: UnsafeRow): UnsafeRow = { ttlKeyProjection.apply( - InternalRow( - expirationMs, - groupingKey.getStruct(0, keyExprEnc.schema.length) - .asInstanceOf[InternalRow], - userKey.getStruct(0, userKeyEnc.schema.length) - .asInstanceOf[InternalRow])) + InternalRow(expirationMs, elementKey.asInstanceOf[InternalRow])) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala index 0f90fa8d9e490..5d13af0af7c43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala @@ -189,7 +189,7 @@ class StatefulProcessorHandleImpl( def doTtlCleanup(): Unit = { val numValuesRemovedDueToTTLExpiry = metrics.get("numValuesRemovedDueToTTLExpiry").get ttlStates.forEach { s => - numValuesRemovedDueToTTLExpiry += s.clearExpiredState() + numValuesRemovedDueToTTLExpiry += s.clearExpiredStateForAllKeys() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala index 87d1a15dff1a9..b4449f99d6ba5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala @@ -19,274 +19,529 @@ package org.apache.spark.sql.execution.streaming import java.time.Duration import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ -import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore} +import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, RangeKeyScanStateEncoderSpec, StateStore} +import org.apache.spark.sql.streaming.TTLConfig import org.apache.spark.sql.types._ -object StateTTLSchema { - val TTL_VALUE_ROW_SCHEMA: StructType = - StructType(Array(StructField("__dummy__", NullType))) -} - /** - * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]]. + * Any state variable that wants to support TTL must implement this trait, + * which they can do by extending [[OneToOneTTLState]] or [[OneToManyTTLState]]. * - * @param groupingKey grouping key for which ttl is set - * @param expirationMs expiration time for the grouping key - */ -case class SingleKeyTTLRow( - groupingKey: UnsafeRow, - expirationMs: Long) - -/** - * Encapsulates the ttl row information stored in [[CompositeKeyTTLStateImpl]]. + * The only required methods here are ones relating to evicting expired and all + * state, via clearExpiredStateForAllKeys and clearAllStateForElementKey, + * respectively. How classes do this is implementation detail, but the general + * pattern is to use secondary indexes to make sure cleanup scans + * theta(records to evict), not theta(all records). * - * @param groupingKey grouping key for which ttl is set - * @param userKey user key for which ttl is set - * @param expirationMs expiration time for the grouping key - */ -case class CompositeKeyTTLRow( - groupingKey: UnsafeRow, - userKey: UnsafeRow, - expirationMs: Long) - -/** - * Represents the underlying state for secondary TTL Index for a user defined - * state variable. + * There are two broad patterns of implementing stateful variables, and thus + * there are two broad patterns for implementing TTL. The first is when there + * is a one-to-one mapping between an element key [1] and a value; the primary + * and secondary index management for this case is implemented by + * [[OneToOneTTLState]]. When a single element key can have multiple values, + * all of which can expire at their own, unique times, then + * [[OneToManyTTLState]] should be used. + * + * In either case, implementations need to use some sort of secondary index + * that orders element keys by expiration time. This base functionality + * is provided by methods in this trait that read/write/delete to the + * so-called "TTL index". It is a secondary index with the layout of + * (expirationMs, elementKey) -> EMPTY_ROW. The expirationMs is big-endian + * encoded to allow for efficient range scans to find all expired keys. + * + * TTLState (or any abstract sub-classes) should never deal with encoding or + * decoding UnsafeRows to and from their user-facing types. The stateful variable + * themselves should be doing this; all other TTLState sub-classes should be concerned + * only with writing, reading, and deleting UnsafeRows and their associated + * expirations from the primary and secondary indexes. [2] + * + * [1]. You might ask, why call it "element key" instead of "grouping key"? + * This is because a single grouping key might have multiple elements, as in + * the case of a map, which has composite keys of the form (groupingKey, mapKey). + * In the case of ValueState, though, the element key is the grouping key. + * To generalize to both cases, this class should always use the term elementKey.) * - * This state allows Spark to query ttl values based on expiration time - * allowing efficient ttl cleanup. + * [2]. You might also ask, why design it this way? We want the TTLState abstract + * sub-classes to write to both the primary and secondary indexes, since they + * both need to stay in sync; co-locating the logic is cleanest. */ trait TTLState { + // Name of the state variable, e.g. the string the user passes to get{Value/List/Map}State + // in the init() method of a StatefulProcessor. + private[sql] def stateName: String + + // The StateStore instance used to store the state. There is only one instance shared + // among the primary and secondary indexes, since it uses virtual column families + // to keep the indexes separate. + private[sql] def store: StateStore + + // The schema of the primary key for the state variable. For value and list state, this + // is the grouping key. For map state, this is the composite key of the grouping key and + // a map key. + private[sql] def elementKeySchema: StructType + + // The timestamp at which the batch is being processed. All state variables that have + // an expiration at or before this timestamp must be cleaned up. + private[sql] def batchTimestampMs: Long + + // The configuration for this run of the streaming query. It may change between runs + // (e.g. user sets ttlConfig1, stops their query, updates to ttlConfig2, and then + // resumes their query). + private[sql] def ttlConfig: TTLConfig + + // A map from metric name to the underlying SQLMetric. This should not be updated + // by the underlying state variable, as the TTL state implementation should be + // handling all reads/writes/deletes to the indexes. + private[sql] def metrics: Map[String, SQLMetric] = Map.empty + + private final val TTL_INDEX = "$ttl_" + stateName + private final val TTL_INDEX_KEY_SCHEMA = getTTLRowKeySchema(elementKeySchema) + private final val TTL_EMPTY_VALUE_ROW_SCHEMA: StructType = + StructType(Array(StructField("__empty__", NullType))) + + private final val TTL_ENCODER = new TTLEncoder(elementKeySchema) + + // Empty row used for values + private final val TTL_EMPTY_VALUE_ROW = + UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null)) - /** - * Perform the user state clean up based on ttl values stored in - * this state. NOTE that its not safe to call this operation concurrently - * when the user can also modify the underlying State. Cleanup should be initiated - * after arbitrary state operations are completed by the user. - * - * @return number of values cleaned up. - */ - def clearExpiredState(): Long -} + private[sql] final def ttlExpirationMs = StateTTL + .calculateExpirationTimeForDuration(ttlConfig.ttlDuration, batchTimestampMs) -/** - * Manages the ttl information for user state keyed with a single key (grouping key). - */ -abstract class SingleKeyTTLStateImpl( - stateName: String, - store: StateStore, - keyExprEnc: ExpressionEncoder[Any], - ttlExpirationMs: Long) - extends TTLState { + store.createColFamilyIfAbsent( + TTL_INDEX, + TTL_INDEX_KEY_SCHEMA, + TTL_EMPTY_VALUE_ROW_SCHEMA, + RangeKeyScanStateEncoderSpec(TTL_INDEX_KEY_SCHEMA, Seq(0)), + isInternal = true + ) - import org.apache.spark.sql.execution.streaming.StateTTLSchema._ + private[sql] def insertIntoTTLIndex(expirationMs: Long, elementKey: UnsafeRow): Unit = { + val secondaryIndexKey = TTL_ENCODER.encodeTTLRow(expirationMs, elementKey) + store.put(secondaryIndexKey, TTL_EMPTY_VALUE_ROW, TTL_INDEX) + } - private val ttlColumnFamilyName = "$ttl_" + stateName - private val keySchema = getSingleKeyTTLRowSchema(keyExprEnc.schema) - private val keyTTLRowEncoder = new SingleKeyTTLEncoder(keyExprEnc) + // The deleteFromTTLIndex overload that takes an expiration time and elementKey as an + // argument is used when we need to _construct_ the key to delete from the TTL index. + // + // If we know the timestamp to delete and the elementKey, but don't have a pre-constructed + // UnsafeRow, then you should use this method to delete from the TTL index. + private[sql] def deleteFromTTLIndex(expirationMs: Long, elementKey: UnsafeRow): Unit = { + val secondaryIndexKey = TTL_ENCODER.encodeTTLRow(expirationMs, elementKey) + store.remove(secondaryIndexKey, TTL_INDEX) + } - // empty row used for values - private val EMPTY_ROW = - UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null)) + // The deleteFromTTLIndex overload that takes an UnsafeRow as an argument is used when + // we're deleting elements from the TTL index that we are iterating over. + // + // If we were to use the other deleteFromTTLIndex method, we would have to re-encode the + // components into an UnsafeRow. It is more efficient to just pass the UnsafeRow that we + // read from the iterator. + private[sql] def deleteFromTTLIndex(ttlKey: UnsafeRow): Unit = { + store.remove(ttlKey, TTL_INDEX) + } - store.createColFamilyIfAbsent(ttlColumnFamilyName, keySchema, TTL_VALUE_ROW_SCHEMA, - RangeKeyScanStateEncoderSpec(keySchema, Seq(0)), isInternal = true) + private[sql] def toTTLRow(ttlKey: UnsafeRow): TTLRow = { + val expirationMs = ttlKey.getLong(0) + val elementKey = ttlKey.getStruct(1, TTL_INDEX_KEY_SCHEMA.length) + TTLRow(elementKey, expirationMs) + } - /** - * This function will be called when clear() on State Variables - * with ttl enabled is called. This function should clear any - * associated ttlState, since we are clearing the user state. - */ - def clearTTLState(): Unit = { - val iterator = store.iterator(ttlColumnFamilyName) - iterator.foreach { kv => - store.remove(kv.key, ttlColumnFamilyName) - } + private[sql] def getTTLRows(): Iterator[TTLRow] = { + store.iterator(TTL_INDEX).map(kv => toTTLRow(kv.key)) } - def upsertTTLForStateKey( - expirationMs: Long, - groupingKey: UnsafeRow): Unit = { - val encodedTtlKey = keyTTLRowEncoder.encodeTTLRow( - expirationMs, groupingKey) - store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName) + // Returns an Iterator over all the keys in the TTL index that have expired. This method + // does not delete the keys from the TTL index; it is the responsibility of the caller + // to do so. + // + // The schema of the UnsafeRow returned by this iterator is (expirationMs, elementKey). + private[sql] def ttlEvictionIterator(): Iterator[UnsafeRow] = { + val ttlIterator = store.iterator(TTL_INDEX) + + // Recall that the format is (expirationMs, elementKey) -> TTL_EMPTY_VALUE_ROW, so + // kv.value doesn't ever need to be used. + ttlIterator.takeWhile { kv => + val expirationMs = kv.key.getLong(0) + StateTTL.isExpired(expirationMs, batchTimestampMs) + }.map(_.key) } + // Encapsulates a row stored in a TTL index. Exposed for testing. + private[sql] case class TTLRow(elementKey: UnsafeRow, expirationMs: Long) + /** - * Clears any state which has ttl older than [[ttlExpirationMs]]. + * Evicts the state associated with this stateful variable that has expired + * due to TTL. The eviction applies to all grouping keys, and to all indexes, + * primary or secondary. + * + * This method can be called at any time in the micro-batch execution, + * as long as it is allowed to complete before subsequent state operations are + * issued. Operations to the state variable should not be issued concurrently while + * this is running, since it may leave the state variable in an inconsistent state + * as it cleans up. + * + * @return number of values cleaned up. */ - override def clearExpiredState(): Long = { - val iterator = store.iterator(ttlColumnFamilyName) - var numValuesExpired = 0L + private[sql] def clearExpiredStateForAllKeys(): Long - iterator.takeWhile { kv => - val expirationMs = kv.key.getLong(0) - StateTTL.isExpired(expirationMs, ttlExpirationMs) - }.foreach { kv => - val groupingKey = kv.key.getStruct(1, keyExprEnc.schema.length) - numValuesExpired += clearIfExpired(groupingKey) - store.remove(kv.key, ttlColumnFamilyName) + /** + * When a user calls clear() on a stateful variable, this method is invoked to + * clear all of the state for the current (implicit) grouping key. It is responsible + * for deleting from the primary index as well as any secondary index(es). + * + * If a given state variable has to clean up multiple elementKeys (in MapState, for + * example, every key in the map is its own elementKey), then this method should + * be invoked for each of those keys. + */ + private[sql] def clearAllStateForElementKey(elementKey: UnsafeRow): Unit +} + +/** + * OneToOneTTLState is an implementation of [[TTLState]] that is used to manage + * TTL for state variables that need a single secondary index to efficiently manage + * records with an expiration. + * + * The primary index for state variables that can use a [[OneToOneTTLState]] have + * the form of: [elementKey -> (value, elementExpiration)]. You'll notice that, given + * a timestamp, it would take linear time to probe the primary index for all of its + * expired values. + * + * As a result, this class uses helper methods from [[TTLState]] to maintain the secondary + * index from [(elementExpiration, elementKey) -> EMPTY_ROW]. + * + * For an explanation of why this structure is not always sufficient (e.g. why the class + * [[OneToManyTTLState]] is needed), please visit its class-doc comment. + */ +abstract class OneToOneTTLState( + stateNameArg: String, + storeArg: StateStore, + elementKeySchemaArg: StructType, + ttlConfigArg: TTLConfig, + batchTimestampMsArg: Long, + metricsArg: Map[String, SQLMetric]) extends TTLState { + override private[sql] def stateName: String = stateNameArg + override private[sql] def store: StateStore = storeArg + override private[sql] def elementKeySchema: StructType = elementKeySchemaArg + override private[sql] def ttlConfig: TTLConfig = ttlConfigArg + override private[sql] def batchTimestampMs: Long = batchTimestampMsArg + override private[sql] def metrics: Map[String, SQLMetric] = metricsArg + + /** + * This method updates the TTL for the given elementKey to be expirationMs, + * updating both the primary and secondary indices if needed. + * + * Note that an elementKey may be the state variable's grouping key, _or_ it + * could be a composite key. MapState is an example of a state variable that + * has composite keys, which has the structure of the groupingKey followed by + * the specific key in the map. This method doesn't need to know what type of + * key is being used, though, since in either case, it's just an UnsafeRow. + * + * @param elementKey the key for which the TTL should be updated, which may + * either be the encoded grouping key, or the grouping key + * and some user-defined key. + * @param elementValue the value to update the primary index with. It is of the + * form (value, expirationMs). + * @param expirationMs the new expiration timestamp to use for elementKey. + */ + private[sql] def updatePrimaryAndSecondaryIndices( + elementKey: UnsafeRow, + elementValue: UnsafeRow, + expirationMs: Long): Unit = { + val existingPrimaryValue = store.get(elementKey, stateName) + + // Doesn't exist. Insert into the primary and TTL indexes. + if (existingPrimaryValue == null) { + store.put(elementKey, elementValue, stateName) + TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows") + insertIntoTTLIndex(expirationMs, elementKey) + } else { + // If the values are equal, then they must be equal in actual value and the expiration + // timestamp. We don't need to update any index in this case. + if (elementValue != existingPrimaryValue) { + store.put(elementKey, elementValue, stateName) + TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows") + + // Small optimization: the value could have changed, but the expirationMs could have + // stayed the same. We only put into the TTL index if the expirationMs has changed. + val existingExpirationMs = existingPrimaryValue.getLong(1) + if (existingExpirationMs != expirationMs) { + deleteFromTTLIndex(existingExpirationMs, elementKey) + insertIntoTTLIndex(expirationMs, elementKey) + } + } } - numValuesExpired } - private[sql] def ttlIndexIterator(): Iterator[SingleKeyTTLRow] = { - val ttlIterator = store.iterator(ttlColumnFamilyName) + override private[sql] def clearExpiredStateForAllKeys(): Long = { + var numValuesExpired = 0L - new Iterator[SingleKeyTTLRow] { - override def hasNext: Boolean = ttlIterator.hasNext + ttlEvictionIterator().foreach { ttlKey => + // Delete from secondary index + deleteFromTTLIndex(ttlKey) + // Delete from primary index + store.remove(toTTLRow(ttlKey).elementKey, stateName) - override def next(): SingleKeyTTLRow = { - val kv = ttlIterator.next() - SingleKeyTTLRow( - expirationMs = kv.key.getLong(0), - groupingKey = kv.key.getStruct(1, keyExprEnc.schema.length) - ) - } + numValuesExpired += 1 } + + TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows", numValuesExpired) + numValuesExpired } - private[sql] def getValuesInTTLState(groupingKey: UnsafeRow): Iterator[Long] = { - val ttlIterator = ttlIndexIterator() - var nextValue: Option[Long] = None - - new Iterator[Long] { - override def hasNext: Boolean = { - while (nextValue.isEmpty && ttlIterator.hasNext) { - val nextTtlValue = ttlIterator.next() - val valueGroupingKey = nextTtlValue.groupingKey - if (valueGroupingKey equals groupingKey) { - nextValue = Some(nextTtlValue.expirationMs) - } - } - nextValue.isDefined - } + override private[sql] def clearAllStateForElementKey(elementKey: UnsafeRow): Unit = { + val existingPrimaryValue = store.get(elementKey, stateName) + if (existingPrimaryValue != null) { + val existingExpirationMs = existingPrimaryValue.getLong(1) - override def next(): Long = { - val result = nextValue.get - nextValue = None - result - } + store.remove(elementKey, stateName) + TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows") + + deleteFromTTLIndex(existingExpirationMs, elementKey) } } - - /** - * Clears the user state associated with this grouping key - * if it has expired. This function is called by Spark to perform - * cleanup at the end of transformWithState processing. - * - * Spark uses a secondary index to determine if the user state for - * this grouping key has expired. However, its possible that the user - * has updated the TTL and secondary index is out of date. Implementations - * must validate that the user State has actually expired before cleanup based - * on their own State data. - * - * @param groupingKey grouping key for which cleanup should be performed. - * - * @return true if the state was cleared, false otherwise. - */ - def clearIfExpired(groupingKey: UnsafeRow): Long } /** - * Manages the ttl information for user state keyed with a single key (grouping key). + * [[OneToManyTTLState]] is an implementation of [[TTLState]] for stateful variables + * that associate a single key with multiple values; every value has its own expiration + * timestamp. + * + * We need an efficient way to find all the values that have expired, but we cannot + * issue point-wise deletes to the elements, since they are merged together using the + * RocksDB StringAppendOperator for merging. As such, we cannot keep a secondary index + * on the key (expirationMs, groupingKey, indexInList), since we have no way to delete a + * specific indexInList from the RocksDB value. (In the future, we could write a custom + * merge operator that can handle tombstones for deleted indexes, but RocksDB doesn't + * support custom merge operators written in Java/Scala.) + * + * Instead, we manage expiration per grouping key instead. Our secondary index will look + * like (expirationMs, groupingKey) -> EMPTY_ROW. This way, we can quickly find all the + * grouping keys that contain at least one element that has expired. + * + * To make sure that we aren't "late" in cleaning up expired values, this secondary index + * maps from the minimum expiration in a list and a grouping key to the EMPTY_VALUE. This + * index is called the "TTL index" in the code (to be consistent with [[OneToOneTTLState]]), + * though it behaves more like a work queue of lists that need to be cleaned up. + * + * Since a grouping key may have a large list and we need to quickly know what the + * minimum expiration is, we need to reverse this work queue index. This reversed index + * maps from key to the minimum expiration in the list, and it is called the "min-expiry" index. + * + * Note: currently, this is only used by ListState with TTL. */ -abstract class CompositeKeyTTLStateImpl[K]( - stateName: String, - store: StateStore, - keyExprEnc: ExpressionEncoder[Any], - userKeyEncoder: ExpressionEncoder[Any], - ttlExpirationMs: Long) - extends TTLState { - - import org.apache.spark.sql.execution.streaming.StateTTLSchema._ - - private val ttlColumnFamilyName = "$ttl_" + stateName - private val keySchema = getCompositeKeyTTLRowSchema( - keyExprEnc.schema, userKeyEncoder.schema +abstract class OneToManyTTLState( + stateNameArg: String, + storeArg: StateStore, + elementKeySchemaArg: StructType, + ttlConfigArg: TTLConfig, + batchTimestampMsArg: Long, + metricsArg: Map[String, SQLMetric]) extends TTLState { + override private[sql] def stateName: String = stateNameArg + override private[sql] def store: StateStore = storeArg + override private[sql] def elementKeySchema: StructType = elementKeySchemaArg + override private[sql] def ttlConfig: TTLConfig = ttlConfigArg + override private[sql] def batchTimestampMs: Long = batchTimestampMsArg + override private[sql] def metrics: Map[String, SQLMetric] = metricsArg + + // Schema of the min-expiry index: elementKey -> minExpirationMs + private val MIN_INDEX = "$min_" + stateName + private val MIN_INDEX_SCHEMA = elementKeySchema + private val MIN_INDEX_VALUE_SCHEMA = getExpirationMsRowSchema() + + // Projects a Long into an UnsafeRow + private val minIndexValueProjector = UnsafeProjection.create(MIN_INDEX_VALUE_SCHEMA) + + // Schema of the entry count index: elementKey -> count + private val COUNT_INDEX = "$count_" + stateName + private val COUNT_INDEX_VALUE_SCHEMA: StructType = + StructType(Seq(StructField("count", LongType, nullable = false))) + private val countIndexValueProjector = UnsafeProjection.create(COUNT_INDEX_VALUE_SCHEMA) + + // Reused internal row that we use to create an UnsafeRow with the schema of + // COUNT_INDEX_VALUE_SCHEMA and the desired value. It is not thread safe (although, anyway, + // this class is not thread safe). + private val reusedCountIndexValueRow = new GenericInternalRow(1) + + store.createColFamilyIfAbsent( + MIN_INDEX, + MIN_INDEX_SCHEMA, + MIN_INDEX_VALUE_SCHEMA, + NoPrefixKeyStateEncoderSpec(MIN_INDEX_SCHEMA), + isInternal = true ) - private val keyRowEncoder = new CompositeKeyTTLEncoder[K]( - keyExprEnc, userKeyEncoder) + store.createColFamilyIfAbsent( + COUNT_INDEX, + elementKeySchema, + COUNT_INDEX_VALUE_SCHEMA, + NoPrefixKeyStateEncoderSpec(elementKeySchema), + isInternal = true + ) - // empty row used for values - private val EMPTY_ROW = - UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null)) + // Helper method to get the number of entries in the list state for a given element key + private def getEntryCount(elementKey: UnsafeRow): Long = { + val countRow = store.get(elementKey, COUNT_INDEX) + if (countRow != null) { + countRow.getLong(0) + } else { + 0L + } + } - store.createColFamilyIfAbsent(ttlColumnFamilyName, keySchema, - TTL_VALUE_ROW_SCHEMA, RangeKeyScanStateEncoderSpec(keySchema, - Seq(0)), isInternal = true) + // Helper function to update the number of entries in the list state for a given element key + private def updateEntryCount(elementKey: UnsafeRow, updatedCount: Long): Unit = { + reusedCountIndexValueRow.setLong(0, updatedCount) + store.put(elementKey, + countIndexValueProjector(reusedCountIndexValueRow.asInstanceOf[InternalRow]), + COUNT_INDEX + ) + } - def clearTTLState(): Unit = { - val iterator = store.iterator(ttlColumnFamilyName) - iterator.foreach { kv => - store.remove(kv.key, ttlColumnFamilyName) - } + // Helper function to remove the number of entries in the list state for a given element key + private def removeEntryCount(elementKey: UnsafeRow): Unit = { + store.remove(elementKey, COUNT_INDEX) } - def upsertTTLForStateKey( - expirationMs: Long, - groupingKey: UnsafeRow, - userKey: UnsafeRow): Unit = { - val encodedTtlKey = keyRowEncoder.encodeTTLRow( - expirationMs, groupingKey, userKey) - store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName) + private def writePrimaryIndexEntries( + overwritePrimaryIndex: Boolean, + elementKey: UnsafeRow, + elementValues: Iterator[UnsafeRow]): Unit = { + val initialEntryCount = if (overwritePrimaryIndex) { + removeEntryCount(elementKey) + 0 + } else { + getEntryCount(elementKey) + } + + // Manually keep track of the count so that we can update the count index. We don't + // want to call elementValues.size since that will try to re-read the iterator. + var numNewElements = 0 + + // If we're overwriting the primary index, then we only need to put the first value, + // and then we can merge the rest. + var isFirst = true + elementValues.foreach { value => + numNewElements += 1 + if (isFirst && overwritePrimaryIndex) { + isFirst = false + store.put(elementKey, value, stateName) + } else { + store.merge(elementKey, value, stateName) + } + } + + TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows", numNewElements) + updateEntryCount(elementKey, initialEntryCount + numNewElements) } - /** - * Clears any state which has ttl older than [[ttlExpirationMs]]. - */ - override def clearExpiredState(): Long = { - val iterator = store.iterator(ttlColumnFamilyName) - var numRemovedElements = 0L - iterator.takeWhile { kv => - val expirationMs = kv.key.getLong(0) - StateTTL.isExpired(expirationMs, ttlExpirationMs) - }.foreach { kv => - numRemovedElements += clearIfExpired( - kv.key.getStruct(1, keyExprEnc.schema.length), - kv.key.getStruct(2, userKeyEncoder.schema.length)) - store.remove(kv.key, ttlColumnFamilyName) + private[sql] def updatePrimaryAndSecondaryIndices( + overwritePrimaryIndex: Boolean, + elementKey: UnsafeRow, + elementValues: Iterator[UnsafeRow], + expirationMs: Long): Unit = { + val existingMinExpirationUnsafeRow = store.get(elementKey, MIN_INDEX) + + writePrimaryIndexEntries(overwritePrimaryIndex, elementKey, elementValues) + + // If nothing exists in the minimum index, then we need to make sure to write + // the minimum and the TTL indices. There's nothing to clean-up from the + // secondary index, since it's empty. + if (existingMinExpirationUnsafeRow == null) { + // Insert into the min-expiry and TTL index, in no particular order. + store.put(elementKey, minIndexValueProjector(InternalRow(expirationMs)), MIN_INDEX) + insertIntoTTLIndex(expirationMs, elementKey) + } else { + val existingMinExpiration = existingMinExpirationUnsafeRow.getLong(0) + + if (overwritePrimaryIndex || expirationMs < existingMinExpiration) { + // We don't actually have to delete from the min-expiry index, since we're going + // to overwrite it on the next line. However, since the TTL index has the existing + // minimum expiration in it, we need to delete that. + deleteFromTTLIndex(existingMinExpiration, elementKey) + + // Insert into the min-expiry and TTL index, in no particular order. + store.put(elementKey, minIndexValueProjector(InternalRow(expirationMs)), MIN_INDEX) + insertIntoTTLIndex(expirationMs, elementKey) + } } - numRemovedElements } - private[sql] def ttlIndexIterator(): Iterator[CompositeKeyTTLRow] = { - val ttlIterator = store.iterator(ttlColumnFamilyName) + // The return type of clearExpiredValues. For a one-to-many stateful variable, cleanup + // must go through all of the values. numValuesExpired represents the number of entries + // that were removed (for metrics), and newMinExpirationMs is the new minimum expiration + // for the values remaining in the state variable. + case class ValueExpirationResult( + numValuesExpired: Long, + newMinExpirationMs: Option[Long]) - new Iterator[CompositeKeyTTLRow] { - override def hasNext: Boolean = ttlIterator.hasNext + // Clears all the expired values for the given elementKey. + protected def clearExpiredValues(elementKey: UnsafeRow): ValueExpirationResult - override def next(): CompositeKeyTTLRow = { - val kv = ttlIterator.next() - CompositeKeyTTLRow( - expirationMs = kv.key.getLong(0), - groupingKey = kv.key.getStruct(1, keyExprEnc.schema.length), - userKey = kv.key.getStruct(2, userKeyEncoder.schema.length) - ) + override private[sql] def clearExpiredStateForAllKeys(): Long = { + var totalNumValuesExpired = 0L + + ttlEvictionIterator().foreach { ttlKey => + val ttlRow = toTTLRow(ttlKey) + val elementKey = ttlRow.elementKey + + // Delete from TTL index and minimum index + deleteFromTTLIndex(ttlKey) + store.remove(elementKey, MIN_INDEX) + + // Now, we need the specific implementation to remove all the values associated with + // elementKey. + val valueExpirationResult = clearExpiredValues(elementKey) + + valueExpirationResult.newMinExpirationMs.foreach { newExpirationMs => + // Insert into the min-expiry and TTL index, in no particular order. + store.put(elementKey, minIndexValueProjector(InternalRow(newExpirationMs)), MIN_INDEX) + insertIntoTTLIndex(newExpirationMs, elementKey) } + + // If we have records [foo, bar, baz] and bar and baz are expiring, then, the + // entryCountBeforeExpirations would be 3. The numValuesExpired would be 2, and so the + // newEntryCount would be 3 - 2 = 1. + val entryCountBeforeExpirations = getEntryCount(elementKey) + val numValuesExpired = valueExpirationResult.numValuesExpired + val newEntryCount = entryCountBeforeExpirations - numValuesExpired + + TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows", numValuesExpired) + + if (newEntryCount == 0) { + removeEntryCount(elementKey) + } else { + updateEntryCount(elementKey, newEntryCount) + } + + totalNumValuesExpired += numValuesExpired } + + totalNumValuesExpired } - /** - * Clears the user state associated with this grouping key - * if it has expired. This function is called by Spark to perform - * cleanup at the end of transformWithState processing. - * - * Spark uses a secondary index to determine if the user state for - * this grouping key has expired. However, its possible that the user - * has updated the TTL and secondary index is out of date. Implementations - * must validate that the user State has actually expired before cleanup based - * on their own State data. - * - * @param groupingKey grouping key for which cleanup should be performed. - * @param userKey user key for which cleanup should be performed. - */ - def clearIfExpired(groupingKeyRow: UnsafeRow, - userKeyRow: UnsafeRow): Long + override private[sql] def clearAllStateForElementKey(elementKey: UnsafeRow): Unit = { + val existingMinExpirationUnsafeRow = store.get(elementKey, MIN_INDEX) + if (existingMinExpirationUnsafeRow != null) { + val existingMinExpiration = existingMinExpirationUnsafeRow.getLong(0) + + store.remove(elementKey, stateName) + TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows", getEntryCount(elementKey)) + removeEntryCount(elementKey) + + store.remove(elementKey, MIN_INDEX) + deleteFromTTLIndex(existingMinExpiration, elementKey) + } + } + + // Exposed for testing. + private[sql] def minIndexIterator(): Iterator[(UnsafeRow, Long)] = { + store + .iterator(MIN_INDEX) + .map(kv => (kv.key, kv.value.getLong(0))) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala index 60eea5842645e..87e4596f67309 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore} @@ -44,20 +43,20 @@ class ValueStateImplWithTTL[S]( ttlConfig: TTLConfig, batchTimestampMs: Long, metrics: Map[String, SQLMetric] = Map.empty) - extends SingleKeyTTLStateImpl( - stateName, store, keyExprEnc, batchTimestampMs) with ValueState[S] { + extends OneToOneTTLState( + stateName, store, keyExprEnc.schema, ttlConfig, batchTimestampMs, metrics) with ValueState[S] { - private val stateTypesEncoder = StateTypesEncoder(keyExprEnc, valEncoder, - stateName, hasTtl = true) - private val ttlExpirationMs = - StateTTL.calculateExpirationTimeForDuration(ttlConfig.ttlDuration, batchTimestampMs) + private val stateTypesEncoder = + StateTypesEncoder(keyExprEnc, valEncoder, stateName, hasTtl = true) initialize() private def initialize(): Unit = { store.createColFamilyIfAbsent(stateName, - keyExprEnc.schema, getValueSchemaWithTTL(valEncoder.schema, true), - NoPrefixKeyStateEncoderSpec(keyExprEnc.schema)) + keyExprEnc.schema, + getValueSchemaWithTTL(valEncoder.schema, true), + NoPrefixKeyStateEncoderSpec(keyExprEnc.schema) + ) } /** Function to check if state exists. Returns true if present and false otherwise */ @@ -76,6 +75,7 @@ class ValueStateImplWithTTL[S]( val retRow = store.get(encodedGroupingKey, stateName) if (retRow != null) { + // Getting the 0th ordinal of the struct using valEncoder val resState = stateTypesEncoder.decodeValue(retRow) if (!stateTypesEncoder.isExpired(retRow, batchTimestampMs)) { @@ -90,33 +90,19 @@ class ValueStateImplWithTTL[S]( /** Function to update and overwrite state associated with given key */ override def update(newState: S): Unit = { + val encodedKey = stateTypesEncoder.encodeGroupingKey() + + val ttlExpirationMs = StateTTL + .calculateExpirationTimeForDuration(ttlConfig.ttlDuration, batchTimestampMs) val encodedValue = stateTypesEncoder.encodeValue(newState, ttlExpirationMs) - val serializedGroupingKey = stateTypesEncoder.encodeGroupingKey() - store.put(serializedGroupingKey, - encodedValue, stateName) - TWSMetricsUtils.incrementMetric(metrics, "numUpdatedStateRows") - upsertTTLForStateKey(ttlExpirationMs, serializedGroupingKey) + + updatePrimaryAndSecondaryIndices(encodedKey, encodedValue, ttlExpirationMs) } /** Function to remove state for given key */ override def clear(): Unit = { - store.remove(stateTypesEncoder.encodeGroupingKey(), stateName) - TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows") - clearTTLState() - } - - def clearIfExpired(groupingKey: UnsafeRow): Long = { - val retRow = store.get(groupingKey, stateName) - - var result = 0L - if (retRow != null) { - if (stateTypesEncoder.isExpired(retRow, batchTimestampMs)) { - store.remove(groupingKey, stateName) - TWSMetricsUtils.incrementMetric(metrics, "numRemovedStateRows") - result = 1L - } - } - result + val groupingKey = stateTypesEncoder.encodeGroupingKey() + clearAllStateForElementKey(groupingKey) } /* @@ -161,11 +147,16 @@ class ValueStateImplWithTTL[S]( } /** - * Get all ttl values stored in ttl state for current implicit - * grouping key. + * Get the TTL value stored in TTL state for the current implicit grouping key, + * if it exists. */ - private[sql] def getValuesInTTLState(): Iterator[Long] = { - getValuesInTTLState(stateTypesEncoder.encodeGroupingKey()) + private[sql] def getValueInTTLState(): Option[Long] = { + val groupingKey = stateTypesEncoder.encodeGroupingKey() + val ttlRowsForGroupingKey = getTTLRows().filter(_.elementKey == groupingKey).toSeq + + assert(ttlRowsForGroupingKey.size <= 1, "Multiple TTLRows found for grouping key " + + s"$groupingKey. Expected at most 1. Found: ${ttlRowsForGroupingKey.mkString(", ")}.") + ttlRowsForGroupingKey.headOption.map(_.expirationMs) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala index 22876831c00d1..bb4343bf32159 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala @@ -190,8 +190,8 @@ class ListStateSuite extends StateVariableSuiteBase { var ttlValues = testState.getTTLValues() assert(ttlValues.nonEmpty) assert(ttlValues.forall(_._2 === ttlExpirationMs)) - var ttlStateValueIterator = testState.getValuesInTTLState() - assert(ttlStateValueIterator.hasNext) + var ttlStateValue = testState.getValueInTTLState() + assert(ttlStateValue.isDefined) // increment batchProcessingTime, or watermark and ensure expired value is not returned val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), @@ -212,10 +212,9 @@ class ListStateSuite extends StateVariableSuiteBase { ttlValues = nextBatchTestState.getTTLValues() assert(ttlValues.nonEmpty) assert(ttlValues.forall(_._2 === ttlExpirationMs)) - ttlStateValueIterator = nextBatchTestState.getValuesInTTLState() - assert(ttlStateValueIterator.hasNext) - assert(ttlStateValueIterator.next() === ttlExpirationMs) - assert(ttlStateValueIterator.isEmpty) + ttlStateValue = nextBatchTestState.getValueInTTLState() + assert(ttlStateValue.isDefined) + assert(ttlStateValue.get === ttlExpirationMs) // getWithoutTTL should still return the expired value assert(nextBatchTestState.getWithoutEnforcingTTL().toSeq === Seq("v1", "v2", "v3")) @@ -276,8 +275,8 @@ class ListStateSuite extends StateVariableSuiteBase { val ttlValues = testState.getTTLValues() assert(ttlValues.nonEmpty) assert(ttlValues.forall(_._2 === ttlExpirationMs)) - val ttlStateValueIterator = testState.getValuesInTTLState() - assert(ttlStateValueIterator.hasNext) + val ttlStateValue = testState.getValueInTTLState() + assert(ttlStateValue.isDefined) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala index 8984d9b0845b7..037fed045e8ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala @@ -327,8 +327,8 @@ class ValueStateSuite extends StateVariableSuiteBase { var ttlValue = testState.getTTLValue() assert(ttlValue.isDefined) assert(ttlValue.get._2 === ttlExpirationMs) - var ttlStateValueIterator = testState.getValuesInTTLState() - assert(ttlStateValueIterator.hasNext) + var ttlStateValueIterator = testState.getValueInTTLState() + assert(ttlStateValueIterator.isDefined) // increment batchProcessingTime, or watermark and ensure expired value is not returned val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), @@ -349,10 +349,9 @@ class ValueStateSuite extends StateVariableSuiteBase { ttlValue = nextBatchTestState.getTTLValue() assert(ttlValue.isDefined) assert(ttlValue.get._2 === ttlExpirationMs) - ttlStateValueIterator = nextBatchTestState.getValuesInTTLState() - assert(ttlStateValueIterator.hasNext) - assert(ttlStateValueIterator.next() === ttlExpirationMs) - assert(ttlStateValueIterator.isEmpty) + ttlStateValueIterator = nextBatchTestState.getValueInTTLState() + assert(ttlStateValueIterator.isDefined) + assert(ttlStateValueIterator.get === ttlExpirationMs) // getWithoutTTL should still return the expired value assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1") @@ -412,8 +411,8 @@ class ValueStateSuite extends StateVariableSuiteBase { val ttlValue = testState.getTTLValue() assert(ttlValue.isDefined) assert(ttlValue.get._2 === ttlExpirationMs) - val ttlStateValueIterator = testState.getValuesInTTLState() - assert(ttlStateValueIterator.hasNext) + val ttlStateValueIterator = testState.getValueInTTLState() + assert(ttlStateValueIterator.isDefined) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala index 409a255ae3e64..b188b92bdbb7c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala @@ -20,11 +20,77 @@ package org.apache.spark.sql.streaming import java.time.Duration import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, MemoryStream} +import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, MapStateImplWithTTL, MemoryStream, ValueStateImplWithTTL} import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock +// MultiStatefulVariableTTLProcessor is a StatefulProcessor that consumes a stream of +// strings and returns a stream of pairs. +// +// Internally, it uses several stateful variables to store the count of each string, for +// the sole purpose of verifying that these stateful variables all stay in sync and do not +// interfere with each other. +// +// The pattern of calling appendValue is to simulate the old behavior of appendValue, which +// used to add a record into the secondary index for every appendList call. +class MultiStatefulVariableTTLProcessor(ttlConfig: TTLConfig) + extends StatefulProcessor[String, String, (String, Long)]{ + @transient private var _listState: ListStateImplWithTTL[String] = _ + // Map from index to count + @transient private var _mapState: MapStateImplWithTTL[Long, Long] = _ + // Counts the number of times the string has occurred. It should always be + // equal to the size of the list state at the start and end of handleInputRows. + @transient private var _valueState: ValueStateImplWithTTL[Long] = _ + + override def init( + outputMode: OutputMode, + timeMode: TimeMode): Unit = { + _listState = getHandle + .getListState("listState", Encoders.STRING, ttlConfig) + .asInstanceOf[ListStateImplWithTTL[String]] + _mapState = getHandle + .getMapState("mapState", Encoders.scalaLong, Encoders.scalaLong, ttlConfig) + .asInstanceOf[MapStateImplWithTTL[Long, Long]] + _valueState = getHandle + .getValueState("valueState", Encoders.scalaLong, ttlConfig) + .asInstanceOf[ValueStateImplWithTTL[Long]] + } + override def handleInputRows( + key: String, + inputRows: Iterator[String], + timerValues: TimerValues): Iterator[(String, Long)] = { + assertSanity() + val iter = inputRows.map { row => + // Update the list state + _listState.appendValue(row) + + // Update the map state + val mapStateCurrentSize = _mapState.iterator().size + _mapState.updateValue(mapStateCurrentSize + 1, mapStateCurrentSize + 1) + + // Update the value state + val currentCountFromValueState = _valueState.get() + _valueState.update(currentCountFromValueState + 1) + + assertSanity() + + (key, _listState.get().size.toLong) + } + + iter + } + + // Asserts that the list state, map state, and value state are all in sync. + private def assertSanity(): Unit = { + val listSize = _listState.get().size + val mapSize = _mapState.iterator().size + val valueState = _valueState.get() + assert(listSize == mapSize) + assert(listSize == valueState) + } +} + class ListStateTTLProcessor(ttlConfig: TTLConfig) extends StatefulProcessor[String, InputEvent, OutputEvent] { @@ -80,10 +146,17 @@ class ListStateTTLProcessor(ttlConfig: TTLConfig) } else if (row.action == "append") { listState.appendValue(row.value) } else if (row.action == "get_values_in_ttl_state") { - val ttlValues = listState.getValuesInTTLState() + val ttlValues = listState.getValueInTTLState() ttlValues.foreach { v => results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results } + } else if (row.action == "get_values_in_min_state") { + val minValues = listState.getMinValues() + minValues.foreach { minExpirationMs => + results = OutputEvent(key, -1, isTTLValue = true, ttlValue = minExpirationMs) :: results + } + } else if (row.action == "clear") { + listState.clear() } results.iterator @@ -94,7 +167,8 @@ class ListStateTTLProcessor(ttlConfig: TTLConfig) * Test suite for testing list state with TTL. * We use the base TTL suite with a list state processor. */ -class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { +class TransformWithListStateTTLSuite extends TransformWithStateTTLTest + with StateStoreMetricsTest { import testImplicits._ @@ -105,6 +179,68 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { override def getStateTTLMetricName: String = "numListStateWithTTLVars" + test("verify the list state secondary index has at most one record per key") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(10)) + val inputStream = MemoryStream[String] + val result = inputStream.toDS() + .groupByKey(x => x) + .transformWithState( + new MultiStatefulVariableTTLProcessor(ttlConfig), + TimeMode.ProcessingTime(), + OutputMode.Append()) + val clock = new StreamManualClock + + testStream(result)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + + // We want all of the inputs to have different timestamps, so that each record + // gets its own unique TTL, and thus, its own unique secondary index record. Each + // is also processed in its own microbatch to ensure a unique batchTimestampMs. + AddData(inputStream, "k1"), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("k1", 1)), + + AddData(inputStream, "k2"), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("k2", 1)), + + AddData(inputStream, "k1"), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("k1", 2)), + + AddData(inputStream, "k2"), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("k2", 2)), + + AddData(inputStream, "k1"), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("k1", 3)), + + AddData(inputStream, "k2"), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("k2", 3)), + + // For each unique key that occurs t times, the MultiStatefulVariableTTLProcessor maintains: + // - Map state: t records in the primary, and t records in the TTL index + // - List state: 1 record in the primary, TTL, min, and count indexes + // - Value state: 1 record in the primary, and 1 record in the TTL index + // + // So in total, that amounts to 2t + 4 + 2 = 2t + 6 records. + // + // In this test, we have 2 unique keys, and each key occurs 3 times. Thus, the total number + // of keys in state is 2 * (2t + 6) where t = 3, which is 24. + // + // The number of updated rows is the total across the last time assertNumStateRows + // was called, and we only update numRowsUpdated for primary key updates. We ran 6 batches + // and each wrote 3 primary keys, so the total number of updated rows is 6 * 3 = 18. + assertNumStateRows(total = 24, updated = 18) + ) + } + } + test("verify iterator works with expired values in beginning of list") { withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, @@ -223,6 +359,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { // advance clock to trigger processing AdvanceManualClock(1 * 1000), CheckNewAnswer(), + // get ttl values AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), AdvanceManualClock(1 * 1000), @@ -231,6 +368,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { OutputEvent("k1", 2, isTTLValue = true, 182000), OutputEvent("k1", 3, isTTLValue = true, 182000) ), + AddData(inputStream, InputEvent("k1", "get", -1, null)), AdvanceManualClock(1 * 1000), CheckNewAnswer( @@ -262,6 +400,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { // advance clock to trigger processing AdvanceManualClock(1 * 1000), CheckNewAnswer(), + // get all elements without enforcing ttl AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)), AdvanceManualClock(1 * 1000), @@ -273,6 +412,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { OutputEvent("k1", 5, isTTLValue = false, -1), OutputEvent("k1", 6, isTTLValue = false, -1) ), + AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), AdvanceManualClock(1 * 1000), CheckNewAnswer( @@ -297,15 +437,14 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { // advance clock to trigger processing AdvanceManualClock(1 * 1000), CheckNewAnswer(), + // advance clock to expire the middle three elements AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)), AdvanceManualClock(1 * 1000), CheckNewAnswer( - OutputEvent("k1", -1, isTTLValue = true, 20000), - OutputEvent("k1", -1, isTTLValue = true, 181000), - OutputEvent("k1", -1, isTTLValue = true, 182000), - OutputEvent("k1", -1, isTTLValue = true, 188000) + OutputEvent("k1", -1, isTTLValue = true, 20000) ), + // progress batch timestamp from 9000 to 54000, expiring the middle // three elements. AdvanceManualClock(45 * 1000), @@ -320,6 +459,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { OutputEvent("k1", 8, isTTLValue = false, -1), OutputEvent("k1", 9, isTTLValue = false, -1) ), + AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)), AdvanceManualClock(1 * 1000), CheckNewAnswer( @@ -330,12 +470,11 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { OutputEvent("k1", 8, isTTLValue = false, -1), OutputEvent("k1", 9, isTTLValue = false, -1) ), + AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)), AdvanceManualClock(1 * 1000), CheckNewAnswer( - OutputEvent("k1", -1, isTTLValue = true, 181000), - OutputEvent("k1", -1, isTTLValue = true, 182000), - OutputEvent("k1", -1, isTTLValue = true, 188000) + OutputEvent("k1", -1, isTTLValue = true, 181000) ), StopStream ) @@ -343,6 +482,104 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { } } + // If we have a list for a key k1 -> [(v1, t1), (v2, t2), (v3, t3)] and they _all_ expire, + // then there should be no remaining records in any primary (or secondary index) for that key. + // However, if we have a separate key k2 -> [(v1, t4)] and the time is less than t4, then it + // should still be present after the clearing for k1. + test("verify min-expiry index doesn't insert when the new minimum is None") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + withTempDir { checkpointLocation => + val inputStream = MemoryStream[InputEvent] + val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(1)) + val result1 = inputStream + .toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig1), + TimeMode.ProcessingTime(), + OutputMode.Append() + ) + + val clock = new StreamManualClock + testStream(result1)( + StartStream( + Trigger.ProcessingTime("1 second"), + triggerClock = clock, + checkpointLocation = checkpointLocation.getAbsolutePath + ), + + // Add 3 elements all with different eviction timestamps. + AddData(inputStream, InputEvent("k1", "append", 1)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + + AddData(inputStream, InputEvent("k1", "append", 2)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + + AddData(inputStream, InputEvent("k1", "append", 3)), + AdvanceManualClock(1 * 1000), // Time is 3000 + CheckNewAnswer(), + + // Add a separate key; this should not be affected by k1 expiring. + // It will have an expiration of 64000. + AddData(inputStream, InputEvent("k2", "put", 1)), + + // Now, we should have: k1 -> [1, 2, 3] with TTLs [61000, 62000, 63000] respectively + AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( // Time is 4000 for this micro-batch + OutputEvent("k1", 1, isTTLValue = true, 61000), + OutputEvent("k1", 2, isTTLValue = true, 62000), + OutputEvent("k1", 3, isTTLValue = true, 63000) + ), + + AddData(inputStream, InputEvent("k1", "get_values_in_min_state", -1, null)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( // Time is 5000 for this micro-batch + OutputEvent("k1", -1, isTTLValue = true, 61000) + ), + + // The k1 records expire at 63000, and the current time is 5000. So, we advance the + // clock by 63 - 5 = 58 seconds to expire those. + AdvanceManualClock((63 - 5) * 1000), + CheckNewAnswer(), + + // There should be 4 state rows left over: the primary, TTL, min-expiry, and count + // indexes for k2. + // + // It's important to check with assertNumStateRows, since the InputEvents + // only return values for the current grouping key, not the entirety of RocksDB. + assertNumStateRows(total = 4, updated = 4), + + // The k1 calls should both return no values. However, the k2 calls should return + // one record each. We put these into one AddData call since we want them all to + // run when the batchTimestampMs is 65000. + AddData(inputStream, + // These should both return no values, since all of k1 has been expired. + InputEvent("k1", "get_values_in_ttl_state", -1, null), + InputEvent("k1", "get_values_in_min_state", -1, null), + + // However, k2 still has a record. + InputEvent("k2", "get_values_in_ttl_state", -1, null), + InputEvent("k2", "get_values_in_min_state", -1, null) + ), + AdvanceManualClock(1 * 1000), + CheckNewAnswer( // Time is 65000 for this micro-batch + OutputEvent("k2", -1, isTTLValue = true, 64000), + OutputEvent("k2", -1, isTTLValue = true, 64000) + ), + + assertNumStateRows(total = 0, updated = 0), + + StopStream + ) + } + } + } + test("verify iterator works with expired values in end of list") { withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, @@ -380,14 +617,23 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { // advance clock to trigger processing AdvanceManualClock(1 * 1000), CheckNewAnswer(), + // get ttl values - AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), + AddData(inputStream, + InputEvent("k1", "get_ttl_value_from_state", -1, null), + InputEvent("k1", "get_values_in_min_state", -1) + ), AdvanceManualClock(1 * 1000), CheckNewAnswer( + // From the get_ttl_value_from_state call OutputEvent("k1", 1, isTTLValue = true, 121000), OutputEvent("k1", 2, isTTLValue = true, 122000), - OutputEvent("k1", 3, isTTLValue = true, 122000) + OutputEvent("k1", 3, isTTLValue = true, 122000), + + // From the get_values_in_min_state call + OutputEvent("k1", -1, isTTLValue = true, 121000) ), + AddData(inputStream, InputEvent("k1", "get", -1, null)), AdvanceManualClock(1 * 1000), CheckNewAnswer( @@ -410,6 +656,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { // advance clock to trigger processing AdvanceManualClock(1 * 1000), CheckNewAnswer(), + // get ttl values AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)), AdvanceManualClock(1 * 1000), @@ -423,9 +670,8 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { ), AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)), AdvanceManualClock(1 * 1000), + CheckNewAnswer( - OutputEvent("k1", -1, isTTLValue = true, 121000), - OutputEvent("k1", -1, isTTLValue = true, 122000), OutputEvent("k1", -1, isTTLValue = true, 65000) ), // expire end values, batch timestamp from 7000 to 67000 @@ -447,8 +693,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest { AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)), AdvanceManualClock(1 * 1000), CheckNewAnswer( - OutputEvent("k1", -1, isTTLValue = true, 121000), - OutputEvent("k1", -1, isTTLValue = true, 122000) + OutputEvent("k1", -1, isTTLValue = true, 121000) ), StopStream ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala index 022280eb3bcef..2cb15263459ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala @@ -83,6 +83,8 @@ class MapStateSingleKeyTTLProcessor(ttlConfig: TTLConfig) ttlValues.foreach { v => results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v._2) :: results } + } else if (row.action == "clear") { + mapState.clear() } results.iterator @@ -308,7 +310,6 @@ class TransformWithMapStateTTLSuite extends TransformWithStateTTLTest { AddData(inputStream, MapInputEvent("k1", "", "get_values_in_ttl_state", -1)), AdvanceManualClock(1 * 1000), CheckNewAnswer( - MapOutputEvent("k1", "key3", -1, isTTLValue = true, 123000), MapOutputEvent("k1", "key3", -1, isTTLValue = true, 126000), MapOutputEvent("k1", "key4", -1, isTTLValue = true, 123000), MapOutputEvent("k1", "key5", -1, isTTLValue = true, 123000) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala index 75fda9630779e..55a46f51f9f6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala @@ -144,18 +144,24 @@ abstract class TransformWithStateTTLTest AddData(inputStream, InputEvent("k1", "put", 1)), // advance clock to trigger processing AdvanceManualClock(1 * 1000), + // In the primary index, we should have that k1 -> [(1, 61000)]. + // The TTL index has (61000, k1) -> empty. The min-expiry index has k1 -> 61000. CheckNewAnswer(), + // get this state, and make sure we get unexpired value AddData(inputStream, InputEvent("k1", "get", -1)), AdvanceManualClock(1 * 1000), CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)), + // ensure ttl values were added correctly AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1)), AdvanceManualClock(1 * 1000), CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = true, 61000)), + AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1)), AdvanceManualClock(1 * 1000), CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)), + // advance clock and update expiration time AdvanceManualClock(30 * 1000), AddData(inputStream, InputEvent("k1", "put", 1)), @@ -163,24 +169,30 @@ abstract class TransformWithStateTTLTest // advance clock to trigger processing AdvanceManualClock(1 * 1000), // validate value is not expired + // + // In the primary index, we still get that k1 -> [(1, 95000)]. + // The TTL index should now have (95000, k1) -> empty, and the min-expiry index + // should have k1 -> 95000. CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)), + // validate ttl value is updated in the state AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1)), AdvanceManualClock(1 * 1000), CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = true, 95000)), - // validate ttl state has both ttl values present + + // validate ttl state has only the newer ttl value present AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1)), AdvanceManualClock(1 * 1000), - CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000), - OutputEvent("k1", -1, isTTLValue = true, 95000) - ), - // advance clock after older expiration value + CheckNewAnswer( OutputEvent("k1", -1, isTTLValue = true, 95000)), + + // advance clock after original expiration value; this shouldn't do anything AdvanceManualClock(30 * 1000), // ensure unexpired value is still present in the state AddData(inputStream, InputEvent("k1", "get", -1)), AdvanceManualClock(1 * 1000), CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)), - // validate that the older expiration value is removed from ttl state + + // validate that the ttl index still has the newer value AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1)), AdvanceManualClock(1 * 1000), CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 95000)) @@ -286,4 +298,59 @@ abstract class TransformWithStateTTLTest ) } } + + test("validate that clear only clears the current grouping key") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + val inputStream = MemoryStream[InputEvent] + val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1)) + val result = inputStream.toDS() + .groupByKey(x => x.key) + .transformWithState( + getProcessor(ttlConfig), + TimeMode.ProcessingTime(), + OutputMode.Append()) + + val clock = new StreamManualClock + testStream(result)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + AddData(inputStream, + InputEvent("k1", "put", 1), + InputEvent("k2", "put", 2), + InputEvent("k3", "put", 3) + ), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + + AddData( + inputStream, + InputEvent("k1", "clear", -1), + InputEvent("k1", "get_ttl_value_from_state", -1), + InputEvent("k1", "get_values_in_ttl_state", -1) + ), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer(), + + AddData(inputStream, + InputEvent("k2", "get_ttl_value_from_state", -1), + InputEvent("k2", "get_values_in_ttl_state", -1), + + InputEvent("k3", "get_ttl_value_from_state", -1), + InputEvent("k3", "get_values_in_ttl_state", -1) + ), + // advance clock to trigger processing + AdvanceManualClock(1 * 1000), + CheckNewAnswer( + OutputEvent("k2", 2, isTTLValue = true, 61000), + OutputEvent("k2", -1, isTTLValue = true, 61000), + + OutputEvent("k3", 3, isTTLValue = true, 61000), + OutputEvent("k3", -1, isTTLValue = true, 61000) + ) + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala index b19c126c7386b..4c7f3a06ea7b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala @@ -55,10 +55,12 @@ object TTLInputProcessFunction { } else if (row.action == "put") { valueState.update(row.value) } else if (row.action == "get_values_in_ttl_state") { - val ttlValues = valueState.getValuesInTTLState() + val ttlValues = valueState.getValueInTTLState() ttlValues.foreach { v => results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results } + } else if (row.action == "clear") { + valueState.clear() } results.iterator @@ -76,6 +78,8 @@ object TTLInputProcessFunction { } } else if (row.action == "put") { valueState.update(row.value) + } else if (row.action == "clear") { + valueState.clear() } results.iterator From 4ec9ebf243d0f071cbc176b64f61117375031088 Mon Sep 17 00:00:00 2001 From: Niranjan Jayakar Date: Wed, 27 Nov 2024 08:48:31 +0900 Subject: [PATCH 028/438] [SPARK-50427][CONNECT][PYTHON] Expose configure_logging as a public API MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Expose `configure_logging` as a public API that can be used to configure the log level for Pyspark connect component. ### Why are the changes needed? We currently offer the mechanism to configure the connect-specific logger based on the environment variable `SPARK_CONNECT_LOG_LEVEL`. The logger is configured once at the the time of "module load". In some cases, Python frameworks (eg. IPythonKernel) can modify the Python log level after the fact leading to unintended log output. There is no good way to restore the logger to restore its previous functionality to honor the environment variable configured.  ### Does this PR introduce _any_ user-facing change? Yes. Provide a new API `configure_logging` in the module `pyspark.sql.connect.logging`. ### How was this patch tested? Local testing by calling `configure_logging` with different log levels. Further tested with IPythonKernel instance which changes the log level and confirmed that calling this API during app startup fixes it back to the correct log level. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48802 from nija-at/expose-log-method. Authored-by: Niranjan Jayakar Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/logging.py | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/connect/logging.py b/python/pyspark/sql/connect/logging.py index b80342cf99743..099193fd7ce45 100644 --- a/python/pyspark/sql/connect/logging.py +++ b/python/pyspark/sql/connect/logging.py @@ -21,13 +21,18 @@ import os from typing import Optional -__all__ = [ - "getLogLevel", -] +__all__ = ["configureLogging", "getLogLevel"] -def _configure_logging() -> logging.Logger: - """Configure logging for the Spark Connect clients.""" +def configureLogging(level: Optional[str] = None) -> logging.Logger: + """ + Configure log level for Spark Connect components. + When not specified as a parameter, log level will be configured based on + the SPARK_CONNECT_LOG_LEVEL environment variable. + When both are absent, logging is disabled. + + .. versionadded:: 4.0.0 + """ logger = PySparkLogger.getLogger(__name__) handler = logging.StreamHandler() handler.setFormatter( @@ -35,8 +40,9 @@ def _configure_logging() -> logging.Logger: ) logger.addHandler(handler) - # Check the environment variables for log levels: - if "SPARK_CONNECT_LOG_LEVEL" in os.environ: + if level is not None: + logger.setLevel(level.upper()) + elif "SPARK_CONNECT_LOG_LEVEL" in os.environ: logger.setLevel(os.environ["SPARK_CONNECT_LOG_LEVEL"].upper()) else: logger.disabled = True @@ -44,7 +50,7 @@ def _configure_logging() -> logging.Logger: # Instantiate the logger based on the environment configuration. -logger = _configure_logging() +logger = configureLogging() def getLogLevel() -> Optional[int]: From 5425d453d2442079377ce0fbcf667f70d35e256e Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 27 Nov 2024 09:42:25 +0900 Subject: [PATCH 029/438] [SPARK-50394][PYTHON][INFRA][FOLLOW-UP] Reduce parallelism further in Pure Python library builds ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/48932 that reduces parallelism further ### Why are the changes needed? In order to make the tests more robust: https://github.com/apache/spark/actions/workflows/build_python_connect.yml It fails because of OOM. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Will monitor the build. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48977 from HyukjinKwon/SPARK-50394-followup2. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_python_connect.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index d57a0c2b91623..471ad31279da4 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -93,7 +93,7 @@ jobs: # Several tests related to catalog requires to run them sequencially, e.g., writing a table in a listener. ./python/run-tests --parallelism=1 --python-executables=python3 --modules pyspark-connect,pyspark-ml-connect # None of tests are dependent on each other in Pandas API on Spark so run them in parallel - ./python/run-tests --parallelism=2 --python-executables=python3 --modules pyspark-pandas-connect-part0,pyspark-pandas-connect-part1,pyspark-pandas-connect-part2,pyspark-pandas-connect-part3 + ./python/run-tests --parallelism=1 --python-executables=python3 --modules pyspark-pandas-connect-part0,pyspark-pandas-connect-part1,pyspark-pandas-connect-part2,pyspark-pandas-connect-part3 # Stop Spark Connect server. ./sbin/stop-connect-server.sh From e55511c98e14115dd023f9014372fc7f23d9ef6c Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 27 Nov 2024 08:43:06 +0800 Subject: [PATCH 030/438] [SPARK-50388][PYTHON][TESTS][FOLLOW-UP] Move `have_flameprof` to `pyspark.testing.utils` ### What changes were proposed in this pull request? Move `have_flameprof` to `pyspark.testing.utils` ### Why are the changes needed? to centralize the import check ### Does this PR introduce _any_ user-facing change? no, test only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #48973 from zhengruifeng/fix_has_flameprof. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../tests/connect/test_parity_udf_profiler.py | 4 +- python/pyspark/sql/tests/test_udf_profiler.py | 38 ++++++++----------- python/pyspark/testing/utils.py | 2 + 3 files changed, 20 insertions(+), 24 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py b/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py index 274364b181441..5c46130c5b50d 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py +++ b/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py @@ -21,9 +21,9 @@ from pyspark.sql.tests.test_udf_profiler import ( UDFProfiler2TestsMixin, _do_computation, - has_flameprof, ) from pyspark.testing.connectutils import ReusedConnectTestCase +from pyspark.testing.utils import have_flameprof class UDFProfilerParityTests(UDFProfiler2TestsMixin, ReusedConnectTestCase): @@ -65,7 +65,7 @@ def action(df): io.getvalue(), f"10.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) diff --git a/python/pyspark/sql/tests/test_udf_profiler.py b/python/pyspark/sql/tests/test_udf_profiler.py index 7e752b2edca1f..7c741bce51f77 100644 --- a/python/pyspark/sql/tests/test_udf_profiler.py +++ b/python/pyspark/sql/tests/test_udf_profiler.py @@ -31,21 +31,15 @@ from pyspark.sql.functions import col, pandas_udf, udf from pyspark.sql.window import Window from pyspark.profiler import UDFBasicProfiler -from pyspark.testing.sqlutils import ( - ReusedSQLTestCase, +from pyspark.testing.sqlutils import ReusedSQLTestCase +from pyspark.testing.utils import ( have_pandas, have_pyarrow, + have_flameprof, pandas_requirement_message, pyarrow_requirement_message, ) -try: - import flameprof # noqa: F401 - - has_flameprof = True -except ImportError: - has_flameprof = False - def _do_computation(spark, *, action=lambda df: df.collect(), use_arrow=False): @udf("long", useArrow=use_arrow) @@ -208,7 +202,7 @@ def test_perf_profiler_udf(self): ) self.assertTrue(f"udf_{id}_perf.pstats" in os.listdir(d)) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -230,7 +224,7 @@ def test_perf_profiler_udf_with_arrow(self): io.getvalue(), f"10.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) def test_perf_profiler_udf_multiple_actions(self): @@ -252,7 +246,7 @@ def action(df): io.getvalue(), f"20.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) def test_perf_profiler_udf_registered(self): @@ -276,7 +270,7 @@ def add1(x): io.getvalue(), f"10.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -309,7 +303,7 @@ def add2(x): io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -345,7 +339,7 @@ def add2(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -395,7 +389,7 @@ def mean_udf(v: pd.Series) -> float: io.getvalue(), f"5.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -427,7 +421,7 @@ def min_udf(v: pd.Series) -> float: io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -458,7 +452,7 @@ def normalize(pdf): io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -496,7 +490,7 @@ def asof_join(left, right): io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -530,7 +524,7 @@ def normalize(table): io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) @unittest.skipIf( @@ -562,7 +556,7 @@ def summarize(left, right): io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) def test_perf_profiler_render(self): @@ -572,7 +566,7 @@ def test_perf_profiler_render(self): id = list(self.profile_results.keys())[0] - if has_flameprof: + if have_flameprof: self.assertIn("svg", self.spark.profile.render(id)) self.assertIn("svg", self.spark.profile.render(id, type="perf")) self.assertIn("svg", self.spark.profile.render(id, renderer="flameprof")) diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index 9ac0c9ba73995..e8c658ff05f87 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -91,6 +91,8 @@ def have_package(name: str) -> bool: have_graphviz = have_package("graphviz") graphviz_requirement_message = None if have_graphviz else "No module named 'graphviz'" +have_flameprof = have_package("flameprof") +flameprof_requirement_message = None if have_flameprof else "No module named 'flameprof'" pandas_requirement_message = None try: From 30cba12a51fcfda7fe42089e077ae53504be946e Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Wed, 27 Nov 2024 12:40:50 +0800 Subject: [PATCH 031/438] [SPARK-48344][SQL] Add SQL Scripting Execution Framework ### What changes were proposed in this pull request? This PR is second in series of refactoring Initial refactoring of SQL Scripting to prepare it for addition of **Execution Framework**: - Introducing `SqlScriptExecution`, new iterator that collects and returns only results. - Enabling execution of SQL Scripting using `sql()` API. - Enabling named parameters to be used with SQL Scripting. ### Why are the changes needed? This changes are needed to enable execution of SQL Scripts. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New `SqlScriptingExecutionSuite` to test behavior of newly added component. New `SqlScriptingE2eSuite` to test end to end behavior using `sql()` API. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48950 from miland-db/milan-dankovic_data/refactor-execution-2. Authored-by: Milan Dankovic Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 5 + .../spark/sql/errors/SqlScriptingErrors.scala | 8 + .../org/apache/spark/sql/SparkSession.scala | 92 +- .../sql/scripting/SqlScriptingExecution.scala | 92 ++ .../scripting/SqlScriptingExecutionNode.scala | 30 +- .../scripting/SqlScriptingInterpreter.scala | 67 +- .../sql/scripting/SqlScriptingE2eSuite.scala | 188 +++ .../SqlScriptingExecutionNodeSuite.scala | 2 + .../SqlScriptingExecutionSuite.scala | 1059 +++++++++++++++++ .../SqlScriptingInterpreterSuite.scala | 9 +- 10 files changed, 1513 insertions(+), 39 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 94513cca1023f..3c494704fd715 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5381,6 +5381,11 @@ "SQL Scripting is under development and not all features are supported. SQL Scripting enables users to write procedural SQL including control flow and error handling. To enable existing features set to `true`." ] }, + "SQL_SCRIPTING_WITH_POSITIONAL_PARAMETERS" : { + "message" : [ + "Positional parameters are not supported with SQL Scripting." + ] + }, "STATE_STORE_MULTIPLE_COLUMN_FAMILIES" : { "message" : [ "Creating multiple column families with is not supported." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala index f1c07200d503b..2a4b8fde6989c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala @@ -103,6 +103,14 @@ private[sql] object SqlScriptingErrors { messageParameters = Map("invalidStatement" -> toSQLStmt(stmt))) } + def positionalParametersAreNotSupportedWithSqlScripting(): Throwable = { + new SqlScriptingException( + origin = null, + errorClass = "UNSUPPORTED_FEATURE.SQL_SCRIPTING_WITH_POSITIONAL_PARAMETERS", + cause = null, + messageParameters = Map.empty) + } + def labelDoesNotExist( origin: Origin, labelName: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 8cf30fb39f310..dbe4543c33101 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -44,17 +44,19 @@ import org.apache.spark.sql.catalyst.analysis.{NameParameterizedQuery, PosParame import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, NamedExpression} import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range} +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan, Range} +import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.ExternalCommandRunner -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, SqlScriptingErrors} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ExternalCommandExecutor import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal._ import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.scripting.SqlScriptingExecution import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.streaming._ import org.apache.spark.sql.types.{DataType, StructType} @@ -431,6 +433,42 @@ class SparkSession private( | Everything else | * ----------------- */ + /** + * Executes given script and return the result of the last statement. + * If script contains no queries, an empty `DataFrame` is returned. + * + * @param script A SQL script to execute. + * @param args A map of parameter names to SQL literal expressions. + * + * @return The result as a `DataFrame`. + */ + private def executeSqlScript( + script: CompoundBody, + args: Map[String, Expression] = Map.empty): DataFrame = { + val sse = new SqlScriptingExecution(script, this, args) + var result: Option[Seq[Row]] = None + + while (sse.hasNext) { + sse.withErrorHandling { + val df = sse.next() + if (sse.hasNext) { + df.write.format("noop").mode("overwrite").save() + } else { + // Collect results from the last DataFrame. + result = Some(df.collect().toSeq) + } + } + } + + if (result.isEmpty) { + emptyDataFrame + } else { + val attributes = DataTypeUtils.toAttributes(result.get.head.schema) + Dataset.ofRows( + self, LocalRelation.fromExternalRows(attributes, result.get)) + } + } + /** * Executes a SQL query substituting positional parameters by the given arguments, * returning the result as a `DataFrame`. @@ -450,13 +488,30 @@ class SparkSession private( withActive { val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { val parsedPlan = sessionState.sqlParser.parsePlan(sqlText) - if (args.nonEmpty) { - PosParameterizedQuery(parsedPlan, args.map(lit(_).expr).toImmutableArraySeq) - } else { - parsedPlan + parsedPlan match { + case compoundBody: CompoundBody => + if (args.nonEmpty) { + // Positional parameters are not supported for SQL scripting. + throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() + } + compoundBody + case logicalPlan: LogicalPlan => + if (args.nonEmpty) { + PosParameterizedQuery(logicalPlan, args.map(lit(_).expr).toImmutableArraySeq) + } else { + logicalPlan + } } } - Dataset.ofRows(self, plan, tracker) + + plan match { + case compoundBody: CompoundBody => + // Execute the SQL script. + executeSqlScript(compoundBody) + case logicalPlan: LogicalPlan => + // Execute the standalone SQL statement. + Dataset.ofRows(self, plan, tracker) + } } /** @inheritdoc */ @@ -487,13 +542,26 @@ class SparkSession private( withActive { val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { val parsedPlan = sessionState.sqlParser.parsePlan(sqlText) - if (args.nonEmpty) { - NameParameterizedQuery(parsedPlan, args.transform((_, v) => lit(v).expr)) - } else { - parsedPlan + parsedPlan match { + case compoundBody: CompoundBody => + compoundBody + case logicalPlan: LogicalPlan => + if (args.nonEmpty) { + NameParameterizedQuery(logicalPlan, args.transform((_, v) => lit(v).expr)) + } else { + logicalPlan + } } } - Dataset.ofRows(self, plan, tracker) + + plan match { + case compoundBody: CompoundBody => + // Execute the SQL script. + executeSqlScript(compoundBody, args.transform((_, v) => lit(v).expr)) + case logicalPlan: LogicalPlan => + // Execute the standalone SQL statement. + Dataset.ofRows(self, plan, tracker) + } } /** @inheritdoc */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala new file mode 100644 index 0000000000000..59252f6229180 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala @@ -0,0 +1,92 @@ +/* + * 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.scripting + +import org.apache.spark.SparkException +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, CompoundBody} + +/** + * SQL scripting executor - executes script and returns result statements. + * This supports returning multiple result statements from a single script. + * + * @param sqlScript CompoundBody which need to be executed. + * @param session Spark session that SQL script is executed within. + * @param args A map of parameter names to SQL literal expressions. + */ +class SqlScriptingExecution( + sqlScript: CompoundBody, + session: SparkSession, + args: Map[String, Expression]) extends Iterator[DataFrame] { + + // Build the execution plan for the script. + private val executionPlan: Iterator[CompoundStatementExec] = + SqlScriptingInterpreter(session).buildExecutionPlan(sqlScript, args) + + private var current = getNextResult + + override def hasNext: Boolean = current.isDefined + + override def next(): DataFrame = { + if (!hasNext) throw SparkException.internalError("No more elements to iterate through.") + val nextDataFrame = current.get + current = getNextResult + nextDataFrame + } + + /** Helper method to iterate through statements until next result statement is encountered. */ + private def getNextResult: Option[DataFrame] = { + + def getNextStatement: Option[CompoundStatementExec] = + if (executionPlan.hasNext) Some(executionPlan.next()) else None + + var currentStatement = getNextStatement + // While we don't have a result statement, execute the statements. + while (currentStatement.isDefined) { + currentStatement match { + case Some(stmt: SingleStatementExec) if !stmt.isExecuted => + withErrorHandling { + val df = stmt.buildDataFrame(session) + df.logicalPlan match { + case _: CommandResult => // pass + case _ => return Some(df) // If the statement is a result, return it to the caller. + } + } + case _ => // pass + } + currentStatement = getNextStatement + } + None + } + + private def handleException(e: Throwable): Unit = { + // Rethrow the exception. + // TODO: SPARK-48353 Add error handling for SQL scripts + throw e + } + + def withErrorHandling(f: => Unit): Unit = { + try { + f + } catch { + case e: Throwable => + handleException(e) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index 9129fc6ab00f3..94284ec514f55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.scripting import org.apache.spark.SparkException import org.apache.spark.internal.Logging -import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.analysis.NameParameterizedQuery +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.{Origin, WithOrigin} import org.apache.spark.sql.errors.SqlScriptingErrors @@ -77,7 +79,7 @@ trait NonLeafStatementExec extends CompoundStatementExec { // DataFrame evaluates to True if it is single row, single column // of boolean type with value True. - val df = Dataset.ofRows(session, statement.parsedPlan) + val df = statement.buildDataFrame(session) df.schema.fields match { case Array(field) if field.dataType == BooleanType => df.limit(2).collect() match { @@ -105,6 +107,8 @@ trait NonLeafStatementExec extends CompoundStatementExec { * Logical plan of the parsed statement. * @param origin * Origin descriptor for the statement. + * @param args + * A map of parameter names to SQL literal expressions. * @param isInternal * Whether the statement originates from the SQL script or it is created during the * interpretation. Example: DropVariable statements are automatically created at the end of each @@ -113,6 +117,7 @@ trait NonLeafStatementExec extends CompoundStatementExec { class SingleStatementExec( var parsedPlan: LogicalPlan, override val origin: Origin, + val args: Map[String, Expression], override val isInternal: Boolean) extends LeafStatementExec with WithOrigin { @@ -122,6 +127,17 @@ class SingleStatementExec( */ var isExecuted = false + /** + * Plan with named parameters. + */ + private lazy val preparedPlan: LogicalPlan = { + if (args.nonEmpty) { + NameParameterizedQuery(parsedPlan, args) + } else { + parsedPlan + } + } + /** * Get the SQL query text corresponding to this statement. * @return @@ -132,6 +148,16 @@ class SingleStatementExec( origin.sqlText.get.substring(origin.startIndex.get, origin.stopIndex.get + 1) } + /** + * Builds a DataFrame from the parsedPlan of this SingleStatementExec + * @param session The SparkSession on which the parsedPlan is built. + * @return + * The DataFrame. + */ + def buildDataFrame(session: SparkSession): DataFrame = { + Dataset.ofRows(session, preparedPlan) + } + override def reset(): Unit = isExecuted = false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala index 1be75cb61c8b0..387ae36b881f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala @@ -19,13 +19,17 @@ package org.apache.spark.sql.scripting import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.{CaseStatement, CompoundBody, CompoundPlanStatement, CreateVariable, DropVariable, IfElseStatement, IterateStatement, LeaveStatement, LogicalPlan, LoopStatement, RepeatStatement, SingleStatement, WhileStatement} import org.apache.spark.sql.catalyst.trees.Origin /** * SQL scripting interpreter - builds SQL script execution plan. + * + * @param session + * Spark session that SQL script is executed within. */ -case class SqlScriptingInterpreter() { +case class SqlScriptingInterpreter(session: SparkSession) { /** * Build execution plan and return statements that need to be executed, @@ -33,15 +37,16 @@ case class SqlScriptingInterpreter() { * * @param compound * CompoundBody for which to build the plan. - * @param session - * Spark session that SQL script is executed within. + * @param args + * A map of parameter names to SQL literal expressions. * @return * Iterator through collection of statements to be executed. */ def buildExecutionPlan( compound: CompoundBody, - session: SparkSession): Iterator[CompoundStatementExec] = { - transformTreeIntoExecutable(compound, session).asInstanceOf[CompoundBodyExec].getTreeIterator + args: Map[String, Expression]): Iterator[CompoundStatementExec] = { + transformTreeIntoExecutable(compound, args) + .asInstanceOf[CompoundBodyExec].getTreeIterator } /** @@ -62,13 +67,14 @@ case class SqlScriptingInterpreter() { * * @param node * Root node of the parsed tree. - * @param session - * Spark session that SQL script is executed within. + * @param args + * A map of parameter names to SQL literal expressions. * @return * Executable statement. */ private def transformTreeIntoExecutable( - node: CompoundPlanStatement, session: SparkSession): CompoundStatementExec = + node: CompoundPlanStatement, + args: Map[String, Expression]): CompoundStatementExec = node match { case CompoundBody(collection, label) => // TODO [SPARK-48530]: Current logic doesn't support scoped variables and shadowing. @@ -78,49 +84,65 @@ case class SqlScriptingInterpreter() { } val dropVariables = variables .map(varName => DropVariable(varName, ifExists = true)) - .map(new SingleStatementExec(_, Origin(), isInternal = true)) + .map(new SingleStatementExec(_, Origin(), args, isInternal = true)) .reverse new CompoundBodyExec( - collection.map(st => transformTreeIntoExecutable(st, session)) ++ dropVariables, + collection.map(st => transformTreeIntoExecutable(st, args)) ++ dropVariables, label) case IfElseStatement(conditions, conditionalBodies, elseBody) => val conditionsExec = conditions.map(condition => - new SingleStatementExec(condition.parsedPlan, condition.origin, isInternal = false)) + new SingleStatementExec( + condition.parsedPlan, + condition.origin, + args, + isInternal = false)) val conditionalBodiesExec = conditionalBodies.map(body => - transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) val unconditionalBodiesExec = elseBody.map(body => - transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) new IfElseStatementExec( conditionsExec, conditionalBodiesExec, unconditionalBodiesExec, session) case CaseStatement(conditions, conditionalBodies, elseBody) => val conditionsExec = conditions.map(condition => - // todo: what to put here for isInternal, in case of simple case statement - new SingleStatementExec(condition.parsedPlan, condition.origin, isInternal = false)) + new SingleStatementExec( + condition.parsedPlan, + condition.origin, + args, + isInternal = false)) val conditionalBodiesExec = conditionalBodies.map(body => - transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) val unconditionalBodiesExec = elseBody.map(body => - transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) new CaseStatementExec( conditionsExec, conditionalBodiesExec, unconditionalBodiesExec, session) case WhileStatement(condition, body, label) => val conditionExec = - new SingleStatementExec(condition.parsedPlan, condition.origin, isInternal = false) + new SingleStatementExec( + condition.parsedPlan, + condition.origin, + args, + isInternal = false) val bodyExec = - transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec] + transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec] new WhileStatementExec(conditionExec, bodyExec, label, session) case RepeatStatement(condition, body, label) => val conditionExec = - new SingleStatementExec(condition.parsedPlan, condition.origin, isInternal = false) + new SingleStatementExec( + condition.parsedPlan, + condition.origin, + args, + isInternal = false) val bodyExec = - transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec] + transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec] new RepeatStatementExec(conditionExec, bodyExec, label, session) case LoopStatement(body, label) => - val bodyExec = transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec] + val bodyExec = transformTreeIntoExecutable(body, args) + .asInstanceOf[CompoundBodyExec] new LoopStatementExec(bodyExec, label) case leaveStatement: LeaveStatement => @@ -133,6 +155,7 @@ case class SqlScriptingInterpreter() { new SingleStatementExec( sparkStatement.parsedPlan, sparkStatement.origin, + args, isInternal = false) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala new file mode 100644 index 0000000000000..afcdfd343e33b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingE2eSuite.scala @@ -0,0 +1,188 @@ +/* + * 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.scripting + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.plans.logical.CompoundBody +import org.apache.spark.sql.catalyst.util.QuotingUtils.toSQLConf +import org.apache.spark.sql.exceptions.SqlScriptingException +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + + +/** + * End-to-end tests for SQL Scripting. + * This suite is not intended to heavily test the SQL scripting (parser & interpreter) logic. + * It is rather focused on testing the sql() API - whether it can handle SQL scripts correctly, + * results are returned in expected manner, config flags are applied properly, etc. + * For full functionality tests, see SqlScriptingParserSuite and SqlScriptingInterpreterSuite. + */ +class SqlScriptingE2eSuite extends QueryTest with SharedSparkSession { + // Helpers + private def verifySqlScriptResult(sqlText: String, expected: Seq[Row]): Unit = { + val df = spark.sql(sqlText) + checkAnswer(df, expected) + } + + private def verifySqlScriptResultWithNamedParams( + sqlText: String, + expected: Seq[Row], + args: Map[String, Any]): Unit = { + val df = spark.sql(sqlText, args) + checkAnswer(df, expected) + } + + // Tests setup + override protected def sparkConf: SparkConf = { + super.sparkConf.set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true") + } + + // Tests + test("SQL Scripting not enabled") { + withSQLConf(SQLConf.SQL_SCRIPTING_ENABLED.key -> "false") { + val sqlScriptText = + """ + |BEGIN + | SELECT 1; + |END""".stripMargin + checkError( + exception = intercept[SqlScriptingException] { + spark.sql(sqlScriptText).asInstanceOf[CompoundBody] + }, + condition = "UNSUPPORTED_FEATURE.SQL_SCRIPTING", + parameters = Map("sqlScriptingEnabled" -> toSQLConf(SQLConf.SQL_SCRIPTING_ENABLED.key))) + } + } + + test("single select") { + val sqlText = "SELECT 1;" + verifySqlScriptResult(sqlText, Seq(Row(1))) + } + + test("multiple selects") { + val sqlText = + """ + |BEGIN + | SELECT 1; + | SELECT 2; + |END""".stripMargin + verifySqlScriptResult(sqlText, Seq(Row(2))) + } + + test("multi statement - simple") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + | INSERT INTO t VALUES (1, 'a', 1.0); + | SELECT a FROM t; + |END + |""".stripMargin + verifySqlScriptResult(sqlScript, Seq(Row(1))) + } + } + + test("script without result statement") { + val sqlScript = + """ + |BEGIN + | DECLARE x INT; + | SET x = 1; + | DROP TEMPORARY VARIABLE x; + |END + |""".stripMargin + verifySqlScriptResult(sqlScript, Seq.empty) + } + + test("empty script") { + val sqlScript = + """ + |BEGIN + |END + |""".stripMargin + verifySqlScriptResult(sqlScript, Seq.empty) + } + + test("named params") { + val sqlScriptText = + """ + |BEGIN + | SELECT 1; + | IF :param_1 > 10 THEN + | SELECT :param_2; + | ELSE + | SELECT :param_3; + | END IF; + |END""".stripMargin + // Define a map with SQL parameters + val args: Map[String, Any] = Map( + "param_1" -> 5, + "param_2" -> "greater", + "param_3" -> "smaller" + ) + verifySqlScriptResultWithNamedParams(sqlScriptText, Seq(Row("smaller")), args) + } + + test("positional params") { + val sqlScriptText = + """ + |BEGIN + | SELECT 1; + | IF ? > 10 THEN + | SELECT ?; + | ELSE + | SELECT ?; + | END IF; + |END""".stripMargin + // Define an array with SQL parameters in the correct order. + val args: Array[Any] = Array(5, "greater", "smaller") + checkError( + exception = intercept[SqlScriptingException] { + spark.sql(sqlScriptText, args).asInstanceOf[CompoundBody] + }, + condition = "UNSUPPORTED_FEATURE.SQL_SCRIPTING_WITH_POSITIONAL_PARAMETERS", + parameters = Map.empty) + } + + test("named params with positional params - should fail") { + val sqlScriptText = + """ + |BEGIN + | SELECT ?; + | IF :param > 10 THEN + | SELECT 1; + | ELSE + | SELECT 2; + | END IF; + |END""".stripMargin + // Define a map with SQL parameters. + val args: Map[String, Any] = Map("param" -> 5) + checkError( + exception = intercept[AnalysisException] { + spark.sql(sqlScriptText, args).asInstanceOf[CompoundBody] + }, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "_16"), + context = ExpectedContext( + fragment = "?", + start = 16, + stop = 16)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala index baad5702f4f22..4874ea3d2795f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala @@ -39,6 +39,7 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi extends SingleStatementExec( parsedPlan = Project(Seq(Alias(Literal(condVal), description)()), OneRowRelation()), Origin(startIndex = Some(0), stopIndex = Some(description.length)), + Map.empty, isInternal = false) case class DummyLogicalPlan() extends LeafNode { @@ -50,6 +51,7 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi extends SingleStatementExec( parsedPlan = DummyLogicalPlan(), Origin(startIndex = Some(0), stopIndex = Some(description.length)), + Map.empty, isInternal = false) class LoopBooleanConditionEvaluator(condition: TestLoopCondition) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala new file mode 100644 index 0000000000000..bbeae942f9fe7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala @@ -0,0 +1,1059 @@ +/* + * 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.scripting + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.CompoundBody +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +/** + * SQL Scripting interpreter tests. + * Output from the parser is provided to the interpreter. + * Output from the interpreter (iterator over executable statements) is then checked - statements + * are executed and output DataFrames are compared with expected outputs. + */ +class SqlScriptingExecutionSuite extends QueryTest with SharedSparkSession { + + // Tests setup + override protected def sparkConf: SparkConf = { + super.sparkConf.set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true") + } + + // Helpers + private def runSqlScript( + sqlText: String, + args: Map[String, Expression] = Map.empty): Seq[Array[Row]] = { + val compoundBody = spark.sessionState.sqlParser.parsePlan(sqlText).asInstanceOf[CompoundBody] + val sse = new SqlScriptingExecution(compoundBody, spark, args) + sse.map { df => df.collect() }.toList + } + + private def verifySqlScriptResult(sqlText: String, expected: Seq[Seq[Row]]): Unit = { + val result = runSqlScript(sqlText) + assert(result.length == expected.length) + result.zip(expected).foreach { + case (actualAnswer, expectedAnswer) => + assert(actualAnswer.sameElements(expectedAnswer)) + } + } + + // Tests + test("multi statement - simple") { + withTable("t") { + val sqlScript = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |INSERT INTO t VALUES (1, 'a', 1.0); + |SELECT a, b FROM t WHERE a = 12; + |SELECT a FROM t; + |END + |""".stripMargin + val expected = Seq( + Seq.empty[Row], // select + Seq(Row(1)) // select + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("multi statement - count") { + withTable("t") { + val sqlScript = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |INSERT INTO t VALUES (1, 'a', 1.0); + |INSERT INTO t VALUES (1, 'a', 1.0); + |SELECT + | CASE WHEN COUNT(*) > 10 THEN true + | ELSE false + | END AS MoreThanTen + |FROM t; + |END + |""".stripMargin + val expected = Seq(Seq(Row(false))) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("session vars - set and read (SET VAR)") { + val sqlScript = + """ + |BEGIN + |DECLARE var = 1; + |SET VAR var = var + 1; + |SELECT var; + |END + |""".stripMargin + val expected = Seq(Seq(Row(2))) + verifySqlScriptResult(sqlScript, expected) + } + + test("session vars - set and read (SET)") { + val sqlScript = + """ + |BEGIN + |DECLARE var = 1; + |SET var = var + 1; + |SELECT var; + |END + |""".stripMargin + val expected = Seq(Seq(Row(2))) + verifySqlScriptResult(sqlScript, expected) + } + + test("session vars - set and read scoped") { + val sqlScript = + """ + |BEGIN + | BEGIN + | DECLARE var = 1; + | SELECT var; + | END; + | BEGIN + | DECLARE var = 2; + | SELECT var; + | END; + | BEGIN + | DECLARE var = 3; + | SET VAR var = var + 1; + | SELECT var; + | END; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(1)), // select + Seq(Row(2)), // select + Seq(Row(4)) // select + ) + verifySqlScriptResult(sqlScript, expected) + } + + test("session vars - drop var statement") { + val sqlScript = + """ + |BEGIN + |DECLARE var = 1; + |SET VAR var = var + 1; + |SELECT var; + |DROP TEMPORARY VARIABLE var; + |END + |""".stripMargin + val expected = Seq(Seq(Row(2))) + verifySqlScriptResult(sqlScript, expected) + } + + test("if") { + val commands = + """ + |BEGIN + | IF 1=1 THEN + | SELECT 42; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("if nested") { + val commands = + """ + |BEGIN + | IF 1=1 THEN + | IF 2=1 THEN + | SELECT 41; + | ELSE + | SELECT 42; + | END IF; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("if else going in if") { + val commands = + """ + |BEGIN + | IF 1=1 + | THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("if else if going in else if") { + val commands = + """ + |BEGIN + | IF 1=2 + | THEN + | SELECT 42; + | ELSE IF 1=1 + | THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + + test("if else going in else") { + val commands = + """ + |BEGIN + | IF 1=2 + | THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + + test("if else if going in else") { + val commands = + """ + |BEGIN + | IF 1=2 + | THEN + | SELECT 42; + | ELSE IF 1=3 + | THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(44))) + verifySqlScriptResult(commands, expected) + } + + test("if with count") { + withTable("t") { + val commands = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |INSERT INTO t VALUES (1, 'a', 1.0); + |INSERT INTO t VALUES (1, 'a', 1.0); + |IF (SELECT COUNT(*) > 2 FROM t) THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + + test("if else if with count") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + | INSERT INTO t VALUES (1, 'a', 1.0); + | INSERT INTO t VALUES (1, 'a', 1.0); + | IF (SELECT COUNT(*) > 2 FROM t) THEN + | SELECT 42; + | ELSE IF (SELECT COUNT(*) > 1 FROM t) THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END IF; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + + test("searched case") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = 1 THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("searched case nested") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1=1 THEN + | CASE + | WHEN 2=1 THEN + | SELECT 41; + | ELSE + | SELECT 42; + | END CASE; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("searched case second case") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = (SELECT 2) THEN + | SELECT 1; + | WHEN 2 = 2 THEN + | SELECT 42; + | WHEN (SELECT * FROM t) THEN + | SELECT * FROM b; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("searched case going in else") { + val commands = + """ + |BEGIN + | CASE + | WHEN 2 = 1 THEN + | SELECT 1; + | WHEN 3 IN (1,2) THEN + | SELECT 2; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + + test("searched case with count") { + withTable("t") { + val commands = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |INSERT INTO t VALUES (1, 'a', 1.0); + |INSERT INTO t VALUES (1, 'a', 1.0); + |CASE + | WHEN (SELECT COUNT(*) > 2 FROM t) THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + + test("searched case else with count") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + | INSERT INTO t VALUES (1, 'a', 1.0); + | INSERT INTO t VALUES (1, 'a', 1.0); + | CASE + | WHEN (SELECT COUNT(*) > 2 FROM t) THEN + | SELECT 42; + | WHEN (SELECT COUNT(*) > 1 FROM t) THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + + test("searched case no cases matched no else") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = 2 THEN + | SELECT 42; + | WHEN 1 = 3 THEN + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq.empty + verifySqlScriptResult(commands, expected) + } + + test("simple case") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("simple case nested") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | CASE 2 + | WHEN (SELECT 3) THEN + | SELECT 41; + | ELSE + | SELECT 42; + | END CASE; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("simple case second case") { + val commands = + """ + |BEGIN + | CASE (SELECT 2) + | WHEN 1 THEN + | SELECT 1; + | WHEN 2 THEN + | SELECT 42; + | WHEN (SELECT * FROM t) THEN + | SELECT * FROM b; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("simple case going in else") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 2 THEN + | SELECT 1; + | WHEN 3 THEN + | SELECT 2; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + + test("simple case with count") { + withTable("t") { + val commands = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |INSERT INTO t VALUES (1, 'a', 1.0); + |INSERT INTO t VALUES (1, 'a', 1.0); + |CASE (SELECT COUNT(*) FROM t) + | WHEN 1 THEN + | SELECT 41; + | WHEN 2 THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + } + + test("simple case else with count") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + | INSERT INTO t VALUES (1, 'a', 1.0); + | INSERT INTO t VALUES (2, 'b', 2.0); + | CASE (SELECT COUNT(*) FROM t) + | WHEN 1 THEN + | SELECT 42; + | WHEN 3 THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(44))) + verifySqlScriptResult(commands, expected) + } + } + + test("simple case no cases matched no else") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 2 THEN + | SELECT 42; + | WHEN 3 THEN + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq.empty + verifySqlScriptResult(commands, expected) + } + + test("simple case compare with null") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a INT) USING parquet; + | CASE (SELECT COUNT(*) FROM t) + | WHEN 1 THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + + test("while") { + val commands = + """ + |BEGIN + | DECLARE i = 0; + | WHILE i < 3 DO + | SELECT i; + | SET VAR i = i + 1; + | END WHILE; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(0)), // select i + Seq(Row(1)), // select i + Seq(Row(2)) // select i + ) + verifySqlScriptResult(commands, expected) + } + + test("while: not entering body") { + val commands = + """ + |BEGIN + | DECLARE i = 3; + | WHILE i < 3 DO + | SELECT i; + | SET VAR i = i + 1; + | END WHILE; + |END + |""".stripMargin + val expected = Seq.empty + verifySqlScriptResult(commands, expected) + } + + test("nested while") { + val commands = + """ + |BEGIN + | DECLARE i = 0; + | DECLARE j = 0; + | WHILE i < 2 DO + | SET VAR j = 0; + | WHILE j < 2 DO + | SELECT i, j; + | SET VAR j = j + 1; + | END WHILE; + | SET VAR i = i + 1; + | END WHILE; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(0, 0)), // select i, j + Seq(Row(0, 1)), // select i, j + Seq(Row(1, 0)), // select i, j + Seq(Row(1, 1)) // select i, j + ) + verifySqlScriptResult(commands, expected) + } + + test("while with count") { + withTable("t") { + val commands = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |WHILE (SELECT COUNT(*) < 2 FROM t) DO + | SELECT 42; + | INSERT INTO t VALUES (1, 'a', 1.0); + |END WHILE; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(42)), // select + Seq(Row(42)) // select + ) + verifySqlScriptResult(commands, expected) + } + } + + test("repeat") { + val commands = + """ + |BEGIN + | DECLARE i = 0; + | REPEAT + | SELECT i; + | SET VAR i = i + 1; + | UNTIL + | i = 3 + | END REPEAT; + |END + |""".stripMargin + val expected = Seq( + Seq(Row(0)), // select i + Seq(Row(1)), // select i + Seq(Row(2)) // select i + ) + verifySqlScriptResult(commands, expected) + } + + test("repeat: enters body only once") { + val commands = + """ + |BEGIN + | DECLARE i = 3; + | REPEAT + | SELECT i; + | SET VAR i = i + 1; + | UNTIL + | 1 = 1 + | END REPEAT; + |END + |""".stripMargin + + val expected = Seq(Seq(Row(3))) + verifySqlScriptResult(commands, expected) + } + + test("nested repeat") { + val commands = + """ + |BEGIN + | DECLARE i = 0; + | DECLARE j = 0; + | REPEAT + | SET VAR j = 0; + | REPEAT + | SELECT i, j; + | SET VAR j = j + 1; + | UNTIL j >= 2 + | END REPEAT; + | SET VAR i = i + 1; + | UNTIL i >= 2 + | END REPEAT; + |END + |""".stripMargin + + val expected = Seq( + Seq(Row(0, 0)), // select i, j + Seq(Row(0, 1)), // select i, j + Seq(Row(1, 0)), // select i, j + Seq(Row(1, 1)) // select i, j + ) + verifySqlScriptResult(commands, expected) + } + + test("repeat with count") { + withTable("t") { + val commands = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |REPEAT + | SELECT 42; + | INSERT INTO t VALUES (1, 'a', 1.0); + |UNTIL (SELECT COUNT(*) >= 2 FROM t) + |END REPEAT; + |END + |""".stripMargin + + val expected = Seq( + Seq(Row(42)), // select + Seq(Row(42)) // select + ) + verifySqlScriptResult(commands, expected) + } + } + + test("leave compound block") { + val sqlScriptText = + """ + |BEGIN + | lbl: BEGIN + | SELECT 1; + | LEAVE lbl; + | SELECT 2; + | END; + |END""".stripMargin + val expected = Seq(Seq(Row(1))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("leave while loop") { + val sqlScriptText = + """ + |BEGIN + | lbl: WHILE 1 = 1 DO + | SELECT 1; + | LEAVE lbl; + | END WHILE; + |END""".stripMargin + val expected = Seq(Seq(Row(1))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("leave repeat loop") { + val sqlScriptText = + """ + |BEGIN + | lbl: REPEAT + | SELECT 1; + | LEAVE lbl; + | UNTIL 1 = 2 + | END REPEAT; + |END""".stripMargin + val expected = Seq(Seq(Row(1))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("iterate while loop") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: WHILE x < 2 DO + | SET x = x + 1; + | ITERATE lbl; + | SET x = x + 2; + | END WHILE; + | SELECT x; + |END""".stripMargin + val expected = Seq(Seq(Row(2))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("iterate repeat loop") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: REPEAT + | SET x = x + 1; + | ITERATE lbl; + | SET x = x + 2; + | UNTIL x > 1 + | END REPEAT; + | SELECT x; + |END""".stripMargin + val expected = Seq(Seq(Row(2))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("leave outer loop from nested repeat loop") { + val sqlScriptText = + """ + |BEGIN + | lbl: REPEAT + | lbl2: REPEAT + | SELECT 1; + | LEAVE lbl; + | UNTIL 1 = 2 + | END REPEAT; + | UNTIL 1 = 2 + | END REPEAT; + |END""".stripMargin + val expected = Seq(Seq(Row(1))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("leave outer loop from nested while loop") { + val sqlScriptText = + """ + |BEGIN + | lbl: WHILE 1 = 1 DO + | lbl2: WHILE 2 = 2 DO + | SELECT 1; + | LEAVE lbl; + | END WHILE; + | END WHILE; + |END""".stripMargin + val expected = Seq(Seq(Row(1))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("iterate outer loop from nested while loop") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: WHILE x < 2 DO + | SET x = x + 1; + | lbl2: WHILE 2 = 2 DO + | SELECT 1; + | ITERATE lbl; + | END WHILE; + | END WHILE; + | SELECT x; + |END""".stripMargin + val expected = Seq( + Seq(Row(1)), // select 1 + Seq(Row(1)), // select 1 + Seq(Row(2)) // select x + ) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("nested compounds in loop - leave in inner compound") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: WHILE x < 2 DO + | SET x = x + 1; + | BEGIN + | SELECT 1; + | lbl2: BEGIN + | SELECT 2; + | LEAVE lbl2; + | SELECT 3; + | END; + | END; + | END WHILE; + | SELECT x; + |END""".stripMargin + val expected = Seq( + Seq(Row(1)), // select 1 + Seq(Row(2)), // select 2 + Seq(Row(1)), // select 1 + Seq(Row(2)), // select 2 + Seq(Row(2)) // select x + ) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("iterate outer loop from nested repeat loop") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: REPEAT + | SET x = x + 1; + | lbl2: REPEAT + | SELECT 1; + | ITERATE lbl; + | UNTIL 1 = 2 + | END REPEAT; + | UNTIL x > 1 + | END REPEAT; + | SELECT x; + |END""".stripMargin + val expected = Seq( + Seq(Row(1)), // select 1 + Seq(Row(1)), // select 1 + Seq(Row(2)) // select x + ) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("loop statement with leave") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: LOOP + | SET x = x + 1; + | SELECT x; + | IF x > 2 + | THEN + | LEAVE lbl; + | END IF; + | END LOOP; + | SELECT x; + |END""".stripMargin + val expected = Seq( + Seq(Row(1)), // select x + Seq(Row(2)), // select x + Seq(Row(3)), // select x + Seq(Row(3)) // select x + ) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("nested loop statement with leave") { + val commands = + """ + |BEGIN + | DECLARE x = 0; + | DECLARE y = 0; + | lbl1: LOOP + | SET VAR y = 0; + | lbl2: LOOP + | SELECT x, y; + | SET VAR y = y + 1; + | IF y >= 2 THEN + | LEAVE lbl2; + | END IF; + | END LOOP; + | SET VAR x = x + 1; + | IF x >= 2 THEN + | LEAVE lbl1; + | END IF; + | END LOOP; + |END + |""".stripMargin + + val expected = Seq( + Seq(Row(0, 0)), // select x, y + Seq(Row(0, 1)), // select x, y + Seq(Row(1, 0)), // select x, y + Seq(Row(1, 1)) // select x, y + ) + verifySqlScriptResult(commands, expected) + } + + test("iterate loop statement") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: LOOP + | SET x = x + 1; + | IF x > 1 THEN + | LEAVE lbl; + | END IF; + | ITERATE lbl; + | SET x = x + 2; + | END LOOP; + | SELECT x; + |END""".stripMargin + val expected = Seq(Seq(Row(2))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("leave outer loop from nested loop statement") { + val sqlScriptText = + """ + |BEGIN + | lbl: LOOP + | lbl2: LOOP + | SELECT 1; + | LEAVE lbl; + | END LOOP; + | END LOOP; + |END""".stripMargin + // Execution immediately leaves the outer loop after SELECT, + // so we expect only a single row in the result set. + val expected = Seq(Seq(Row(1))) + verifySqlScriptResult(sqlScriptText, expected) + } + + test("iterate outer loop from nested loop statement") { + val sqlScriptText = + """ + |BEGIN + | DECLARE x INT; + | SET x = 0; + | lbl: LOOP + | SET x = x + 1; + | IF x > 2 THEN + | LEAVE lbl; + | END IF; + | lbl2: LOOP + | SELECT 1; + | ITERATE lbl; + | SET x = 10; + | END LOOP; + | END LOOP; + | SELECT x; + |END""".stripMargin + val expected = Seq( + Seq(Row(1)), // select 1 + Seq(Row(1)), // select 1 + Seq(Row(3)) // select x + ) + verifySqlScriptResult(sqlScriptText, expected) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index b0b844d2b52ca..177ffc24d180a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.scripting import org.apache.spark.{SparkConf, SparkException, SparkNumberFormatException} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row} import org.apache.spark.sql.catalyst.QueryPlanningTracker +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.CompoundBody import org.apache.spark.sql.exceptions.SqlScriptingException import org.apache.spark.sql.internal.SQLConf @@ -39,10 +40,12 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { } // Helpers - private def runSqlScript(sqlText: String): Array[DataFrame] = { - val interpreter = SqlScriptingInterpreter() + private def runSqlScript( + sqlText: String, + args: Map[String, Expression] = Map.empty): Array[DataFrame] = { + val interpreter = SqlScriptingInterpreter(spark) val compoundBody = spark.sessionState.sqlParser.parsePlan(sqlText).asInstanceOf[CompoundBody] - val executionPlan = interpreter.buildExecutionPlan(compoundBody, spark) + val executionPlan = interpreter.buildExecutionPlan(compoundBody, args) executionPlan.flatMap { case statement: SingleStatementExec => if (statement.isExecuted) { From 7614819884ca192fab45ee2ace8a8e081ec8becc Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 27 Nov 2024 14:22:01 +0900 Subject: [PATCH 032/438] [SPARK-50430][CORE] Use the standard Properties.clone instead of manual clone ### What changes were proposed in this pull request? This PR proposes to use the standard Properties.clone instead of manual clone ### Why are the changes needed? In a very rare condition, when the properties were changed during the clone of Properties, it might throw an exception as below: ``` : java.util.ConcurrentModificationException at java.util.Hashtable$Enumerator.next(Hashtable.java:1408) at java.util.Hashtable.putAll(Hashtable.java:523) at org.apache.spark.util.Utils$.cloneProperties(Utils.scala:3474) at org.apache.spark.SparkContext.getCredentialResolvedProperties(SparkContext.scala:523) at org.apache.spark.SparkContext.runJobInternal(SparkContext.scala:3157) at org.apache.spark.rdd.RDD.$anonfun$collect$1(RDD.scala:1104) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:165) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:125) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) at org.apache.spark.rdd.RDD.withScope(RDD.scala:454) at org.apache.spark.rdd.RDD.collect(RDD.scala:1102) at org.apache.spark.mllib.evaluation.AreaUnderCurve$.of(AreaUnderCurve.scala:44) at org.apache.spark.mllib.evaluation.BinaryClassificationMetrics.areaUnderROC(BinaryClassificationMetrics.scala:127) at org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.evaluate(BinaryClassificationEvaluator.scala:101) at sun.reflect.GeneratedMethodAccessor323.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:397) at py4j.Gateway.invoke(Gateway.java:306) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:199) at py4j.ClientServerConnection.run(ClientServerConnection.java:119) at java.lang.Thread.run(Thread.java:750) ``` We should use the standard clone method. ### Does this PR introduce _any_ user-facing change? It fixes a very corner case bug as described above. ### How was this patch tested? It's difficult to test because the issue is from concurrent execution. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48978 from HyukjinKwon/SPARK-50430. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 +--- 1 file changed, 1 insertion(+), 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 5703128aacbb9..109db36d40695 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2982,9 +2982,7 @@ private[spark] object Utils if (props == null) { return props } - val resultProps = new Properties() - props.forEach((k, v) => resultProps.put(k, v)) - resultProps + props.clone().asInstanceOf[Properties] } /** From e03319fd9219da7162c12a15998d5718edc4c49e Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 27 Nov 2024 15:27:59 +0900 Subject: [PATCH 033/438] [SPARK-49676][SS][PYTHON] Add Support for Chaining of Operators in transformWithStateInPandas API ### What changes were proposed in this pull request? This PR adds support to define event time column in the output dataset of `TransformWithStateInPandas` operator. The new event time column will be used to evaluate watermark expressions in downstream operators. ### Why are the changes needed? This change is to couple with the scala implementation of chaining of operators. PR in Scala: https://github.com/apache/spark/pull/45376 ### Does this PR introduce _any_ user-facing change? Yes. User can now specify a event time column as: ``` df.groupBy("id") .transformWithStateInPandas( statefulProcessor=stateful_processor, outputStructType=output_schema, outputMode="Update", timeMode=timeMode, eventTimeColumnName="outputTimestamp" ) ``` ### How was this patch tested? Integration tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48124 from jingz-db/python-chaining-op. Lead-authored-by: jingz-db Co-authored-by: Jing Zhan <135738831+jingz-db@users.noreply.github.com> Signed-off-by: Jungtaek Lim --- python/pyspark/sql/pandas/group_ops.py | 2 + .../test_pandas_transform_with_state.py | 158 +++++++++++++++--- .../sql/catalyst/analysis/Analyzer.scala | 1 - .../UnsupportedOperationChecker.scala | 1 + .../sql/catalyst/optimizer/Optimizer.scala | 3 + .../spark/sql/KeyValueGroupedDataset.scala | 6 +- .../spark/sql/RelationalGroupedDataset.scala | 30 +++- .../spark/sql/execution/SparkStrategies.scala | 10 +- .../TransformWithStateInPandasExec.scala | 45 ++++- .../streaming/IncrementalExecution.scala | 17 ++ .../streaming/TransformWithStateExec.scala | 4 +- 11 files changed, 245 insertions(+), 32 deletions(-) diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 56efe0676c08f..d8f22e434374c 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -374,6 +374,7 @@ def transformWithStateInPandas( outputMode: str, timeMode: str, initialState: Optional["GroupedData"] = None, + eventTimeColumnName: str = "", ) -> DataFrame: """ Invokes methods defined in the stateful processor used in arbitrary state API v2. It @@ -662,6 +663,7 @@ def transformWithStateWithInitStateUDF( outputMode, timeMode, initial_state_java_obj, + eventTimeColumnName, ) return DataFrame(jdf, self.session) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 5143392498187..f385d7cd1abc0 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -27,13 +27,7 @@ from pyspark import SparkConf from pyspark.errors import PySparkRuntimeError from pyspark.sql.functions import split -from pyspark.sql.types import ( - StringType, - StructType, - StructField, - Row, - IntegerType, -) +from pyspark.sql.types import StringType, StructType, StructField, Row, IntegerType, TimestampType from pyspark.testing import assertDataFrameEqual from pyspark.testing.sqlutils import ( ReusedSQLTestCase, @@ -247,11 +241,15 @@ def check_results(batch_df, _): # test list state with ttl has the same behavior as list state when state doesn't expire. def test_transform_with_state_in_pandas_list_state_large_ttl(self): - def check_results(batch_df, _): - assert set(batch_df.sort("id").collect()) == { - Row(id="0", countAsString="2"), - Row(id="1", countAsString="2"), - } + def check_results(batch_df, batch_id): + if batch_id == 0: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } + else: + for q in self.spark.streams.active: + q.stop() self._test_transform_with_state_in_pandas_basic( ListStateLargeTTLProcessor(), check_results, True, "processingTime" @@ -268,11 +266,15 @@ def check_results(batch_df, _): # test map state with ttl has the same behavior as map state when state doesn't expire. def test_transform_with_state_in_pandas_map_state_large_ttl(self): - def check_results(batch_df, _): - assert set(batch_df.sort("id").collect()) == { - Row(id="0", countAsString="2"), - Row(id="1", countAsString="2"), - } + def check_results(batch_df, batch_id): + if batch_id == 0: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } + else: + for q in self.spark.streams.active: + q.stop() self._test_transform_with_state_in_pandas_basic( MapStateLargeTTLProcessor(), check_results, True, "processingTime" @@ -287,11 +289,14 @@ def check_results(batch_df, batch_id): Row(id="0", countAsString="2"), Row(id="1", countAsString="2"), } - else: + elif batch_id == 1: assert set(batch_df.sort("id").collect()) == { Row(id="0", countAsString="3"), Row(id="1", countAsString="2"), } + else: + for q in self.spark.streams.active: + q.stop() self._test_transform_with_state_in_pandas_basic( SimpleTTLStatefulProcessor(), check_results, False, "processingTime" @@ -348,6 +353,9 @@ def check_results(batch_df, batch_id): Row(id="ttl-map-state-count-1", count=3), ], ) + else: + for q in self.spark.streams.active: + q.stop() if batch_id == 0 or batch_id == 1: time.sleep(6) @@ -466,7 +474,7 @@ def check_results(batch_df, batch_id): ).first()["timeValues"] check_timestamp(batch_df) - else: + elif batch_id == 2: assert set(batch_df.sort("id").select("id", "countAsString").collect()) == { Row(id="0", countAsString="3"), Row(id="0", countAsString="-1"), @@ -480,6 +488,10 @@ def check_results(batch_df, batch_id): ).first()["timeValues"] assert current_batch_expired_timestamp > self.first_expired_timestamp + else: + for q in self.spark.streams.active: + q.stop() + self._test_transform_with_state_in_pandas_proc_timer( ProcTimeStatefulProcessor(), check_results ) @@ -552,12 +564,15 @@ def check_results(batch_df, batch_id): Row(id="a", timestamp="20"), Row(id="a-expired", timestamp="0"), } - else: + elif batch_id == 2: # verify that rows and expired timer produce the expected result assert set(batch_df.sort("id").collect()) == { Row(id="a", timestamp="15"), Row(id="a-expired", timestamp="10000"), } + else: + for q in self.spark.streams.active: + q.stop() self._test_transform_with_state_in_pandas_event_time( EventTimeStatefulProcessor(), check_results @@ -679,6 +694,9 @@ def check_results(batch_df, batch_id): Row(id1="0", id2="1", value=str(123 + 46)), Row(id1="1", id2="2", value=str(146 + 346)), } + else: + for q in self.spark.streams.active: + q.stop() self._test_transform_with_state_non_contiguous_grouping_cols( SimpleStatefulProcessorWithInitialState(), check_results @@ -692,6 +710,9 @@ def check_results(batch_df, batch_id): Row(id1="0", id2="1", value=str(789 + 123 + 46)), Row(id1="1", id2="2", value=str(146 + 346)), } + else: + for q in self.spark.streams.active: + q.stop() # grouping key of initial state is also not starting from the beginning of attributes data = [(789, "0", "1"), (987, "3", "2")] @@ -703,6 +724,88 @@ def check_results(batch_df, batch_id): SimpleStatefulProcessorWithInitialState(), check_results, initial_state ) + def _test_transform_with_state_in_pandas_chaining_ops( + self, stateful_processor, check_results, timeMode="None", grouping_cols=["outputTimestamp"] + ): + import pyspark.sql.functions as f + + input_path = tempfile.mkdtemp() + self._prepare_input_data(input_path + "/text-test3.txt", ["a", "b"], [10, 15]) + time.sleep(2) + self._prepare_input_data(input_path + "/text-test4.txt", ["a", "c"], [11, 25]) + time.sleep(2) + self._prepare_input_data(input_path + "/text-test1.txt", ["a"], [5]) + + df = self._build_test_df(input_path) + df = df.select( + "id", f.from_unixtime(f.col("temperature")).alias("eventTime").cast("timestamp") + ).withWatermark("eventTime", "5 seconds") + + for q in self.spark.streams.active: + q.stop() + self.assertTrue(df.isStreaming) + + output_schema = StructType( + [ + StructField("id", StringType(), True), + StructField("outputTimestamp", TimestampType(), True), + ] + ) + + q = ( + df.groupBy("id") + .transformWithStateInPandas( + statefulProcessor=stateful_processor, + outputStructType=output_schema, + outputMode="Append", + timeMode=timeMode, + eventTimeColumnName="outputTimestamp", + ) + .groupBy(grouping_cols) + .count() + .writeStream.queryName("chaining_ops_query") + .foreachBatch(check_results) + .outputMode("append") + .start() + ) + + self.assertEqual(q.name, "chaining_ops_query") + self.assertTrue(q.isActive) + q.processAllAvailable() + q.awaitTermination(10) + + def test_transform_with_state_in_pandas_chaining_ops(self): + def check_results(batch_df, batch_id): + import datetime + + if batch_id == 0: + assert batch_df.isEmpty() + elif batch_id == 1: + # eviction watermark = 15 - 5 = 10 (max event time from batch 0), + # late event watermark = 0 (eviction event time from batch 0) + assert set( + batch_df.sort("outputTimestamp").select("outputTimestamp", "count").collect() + ) == { + Row(outputTimestamp=datetime.datetime(1970, 1, 1, 0, 0, 10), count=1), + } + elif batch_id == 2: + # eviction watermark = 25 - 5 = 20, late event watermark = 10; + # row with watermark=5<10 is dropped so it does not show up in the results; + # row with eventTime<=20 are finalized and emitted + assert set( + batch_df.sort("outputTimestamp").select("outputTimestamp", "count").collect() + ) == { + Row(outputTimestamp=datetime.datetime(1970, 1, 1, 0, 0, 11), count=1), + Row(outputTimestamp=datetime.datetime(1970, 1, 1, 0, 0, 15), count=1), + } + + self._test_transform_with_state_in_pandas_chaining_ops( + StatefulProcessorChainingOps(), check_results, "eventTime" + ) + self._test_transform_with_state_in_pandas_chaining_ops( + StatefulProcessorChainingOps(), check_results, "eventTime", ["outputTimestamp", "id"] + ) + class SimpleStatefulProcessorWithInitialState(StatefulProcessor): # this dict is the same as input initial state dataframe @@ -888,6 +991,21 @@ def close(self) -> None: pass +class StatefulProcessorChainingOps(StatefulProcessor): + def init(self, handle: StatefulProcessorHandle) -> None: + pass + + def handleInputRows( + self, key, rows, timer_values, expired_timer_info + ) -> Iterator[pd.DataFrame]: + for pdf in rows: + timestamp_list = pdf["eventTime"].tolist() + yield pd.DataFrame({"id": key, "outputTimestamp": timestamp_list[0]}) + + def close(self) -> None: + pass + + # A stateful processor that inherit all behavior of SimpleStatefulProcessor except that it use # ttl state with a large timeout. class SimpleTTLStatefulProcessor(SimpleStatefulProcessor, unittest.TestCase): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index bed7bea61597f..e05f3533ae3c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -3653,7 +3653,6 @@ object CleanupAliases extends Rule[LogicalPlan] with AliasHelper { /** * Ignore event time watermark in batch query, which is only supported in Structured Streaming. - * TODO: add this rule into analyzer rule list. */ object EliminateEventTimeWatermark extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 4f33c26d5c3c3..5b7583c763c06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -103,6 +103,7 @@ object UnsupportedOperationChecker extends Logging { case d: Deduplicate if d.isStreaming && d.keys.exists(hasEventTimeCol) => true case d: DeduplicateWithinWatermark if d.isStreaming => true case t: TransformWithState if t.isStreaming => true + case t: TransformWithStateInPandas if t.isStreaming => true case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 29216523fefc5..0772c67ea27e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1031,6 +1031,9 @@ object ColumnPruning extends Rule[LogicalPlan] { // Can't prune the columns on LeafNode case p @ Project(_, _: LeafNode) => p + // Can't prune the columns on UpdateEventTimeWatermarkColumn + case p @ Project(_, _: UpdateEventTimeWatermarkColumn) => p + case NestedColumnAliasing(rewrittenPlan) => rewrittenPlan // for all other logical plans that inherits the output from it's children diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 392c3edab9895..6dcf01d3a9db2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.api.java.function._ -import org.apache.spark.sql.catalyst.analysis.{EliminateEventTimeWatermark, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{agnosticEncoderFor, ProductEncoder} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.Attribute @@ -289,11 +289,11 @@ class KeyValueGroupedDataset[K, V] private[sql]( transformWithState ) - Dataset[U](sparkSession, EliminateEventTimeWatermark( + Dataset[U](sparkSession, UpdateEventTimeWatermarkColumn( UnresolvedAttribute(eventTimeColumnName), None, - transformWithStateDataset.logicalPlan))) + transformWithStateDataset.logicalPlan)) } /** @inheritdoc */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 0974df55a6d84..6f0db42ec1f5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkRuntimeException import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -475,7 +475,8 @@ class RelationalGroupedDataset protected[sql]( outputStructType: StructType, outputModeStr: String, timeModeStr: String, - initialState: RelationalGroupedDataset): DataFrame = { + initialState: RelationalGroupedDataset, + eventTimeColumnName: String): DataFrame = { def exprToAttr(expr: Seq[Expression]): Seq[Attribute] = { expr.map { case ne: NamedExpression => ne @@ -529,7 +530,30 @@ class RelationalGroupedDataset protected[sql]( initialStateSchema = initialState.df.schema ) } - Dataset.ofRows(df.sparkSession, plan) + if (eventTimeColumnName.isEmpty) { + Dataset.ofRows(df.sparkSession, plan) + } else { + updateEventTimeColumnAfterTransformWithState(plan, eventTimeColumnName) + } + } + + /** + * Creates a new dataset with updated eventTimeColumn after the transformWithState + * logical node. + */ + private def updateEventTimeColumnAfterTransformWithState( + transformWithStateInPandas: LogicalPlan, + eventTimeColumnName: String): DataFrame = { + val transformWithStateDataset = Dataset.ofRows( + df.sparkSession, + transformWithStateInPandas + ) + + Dataset.ofRows(df.sparkSession, + UpdateEventTimeWatermarkColumn( + UnresolvedAttribute(eventTimeColumnName), + None, + transformWithStateDataset.logicalPlan)) } override def toString: String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 22082aca81a22..c621c151c0bd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.{SparkException, SparkUnsupportedOperationException} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{execution, AnalysisException, Strategy} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, InternalRow} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper, NormalizeFloatingNumbers} @@ -966,6 +966,14 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _: FlatMapGroupsInPandasWithState => // TODO(SPARK-40443): support applyInPandasWithState in batch query throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3176") + case t: TransformWithStateInPandas => + // TODO(SPARK-50428): support TransformWithStateInPandas in batch query + throw new ExtendedAnalysisException( + new AnalysisException( + "_LEGACY_ERROR_TEMP_3102", + Map( + "msg" -> "TransformWithStateInPandas is not supported with batch DataFrames/Datasets") + ), plan = t) case logical.CoGroup( f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, lOrder, rOrder, oAttr, left, right) => execution.CoGroupExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala index 7dd4d4647eeba..617c20c3a782a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, PythonUDF, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.ProcessingTime import org.apache.spark.sql.catalyst.plans.physical.Distribution import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} @@ -72,6 +73,8 @@ case class TransformWithStateInPandasExec( initialStateSchema: StructType) extends BinaryExecNode with StateStoreWriter with WatermarkSupport { + override def shortName: String = "transformWithStateInPandasExec" + private val pythonUDF = functionExpr.asInstanceOf[PythonUDF] private val pythonFunction = pythonUDF.func private val chainedFunc = @@ -126,6 +129,37 @@ case class TransformWithStateInPandasExec( List.empty } + override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = { + if (timeMode == ProcessingTime) { + // TODO SPARK-50180: check if we can return true only if actual timers are registered, + // or there is expired state + true + } else if (outputMode == OutputMode.Append || outputMode == OutputMode.Update) { + eventTimeWatermarkForEviction.isDefined && + newInputWatermark > eventTimeWatermarkForEviction.get + } else { + false + } + } + + /** + * Controls watermark propagation to downstream modes. If timeMode is + * ProcessingTime, the output rows cannot be interpreted in eventTime, hence + * this node will not propagate watermark in this timeMode. + * + * For timeMode EventTime, output watermark is same as input Watermark because + * transformWithState does not allow users to set the event time column to be + * earlier than the watermark. + */ + override def produceOutputWatermark(inputWatermarkMs: Long): Option[Long] = { + timeMode match { + case ProcessingTime => + None + case _ => + Some(inputWatermarkMs) + } + } + override def customStatefulOperatorMetrics: Seq[StatefulOperatorCustomMetric] = { Seq( // metrics around state variables @@ -214,8 +248,15 @@ case class TransformWithStateInPandasExec( val updatesStartTimeNs = currentTimeNs val (dedupAttributes, argOffsets) = resolveArgOffsets(child.output, groupingAttributes) - val data = - groupAndProject(dataIterator, groupingAttributes, child.output, dedupAttributes) + // If timeout is based on event time, then filter late data based on watermark + val filteredIter = watermarkPredicateForDataForLateEvents match { + case Some(predicate) => + applyRemovingRowsOlderThanWatermark(dataIterator, predicate) + case _ => + dataIterator + } + + val data = groupAndProject(filteredIter, groupingAttributes, child.output, dedupAttributes) val processorHandle = new StatefulProcessorHandleImpl(store, getStateInfo.queryRunId, groupingKeyExprEncoder, timeMode, isStreaming = true, batchTimestampMs, metrics) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 2a7e9818aedd9..719c4da14d729 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -439,6 +439,23 @@ class IncrementalExecution( eventTimeWatermarkForEviction = iwEviction) )) + // UpdateEventTimeColumnExec is used to tag the eventTime column, and validate + // emitted rows adhere to watermark in the output of transformWithStateInp. + // Hence, this node shares the same watermark value as TransformWithStateInPandasExec. + // This is the same as above in TransformWithStateExec. + // The only difference is TransformWithStateInPandasExec is analysed slightly different + // with no SerializeFromObjectExec wrapper. + case UpdateEventTimeColumnExec(eventTime, delay, None, t: TransformWithStateInPandasExec) + if t.stateInfo.isDefined => + val stateInfo = t.stateInfo.get + val iwLateEvents = inputWatermarkForLateEvents(stateInfo) + val iwEviction = inputWatermarkForEviction(stateInfo) + + UpdateEventTimeColumnExec(eventTime, delay, iwLateEvents, + t.copy( + eventTimeWatermarkForLateEvents = iwLateEvents, + eventTimeWatermarkForEviction = iwEviction) + ) case t: TransformWithStateExec if t.stateInfo.isDefined => t.copy( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala index f4705b89d5a87..9c31ff0a74437 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala @@ -85,8 +85,8 @@ case class TransformWithStateExec( override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = { if (timeMode == ProcessingTime) { - // TODO: check if we can return true only if actual timers are registered, or there is - // expired state + // TODO SPARK-50180: check if we can return true only if actual timers are registered, + // or there is expired state true } else if (outputMode == OutputMode.Append || outputMode == OutputMode.Update) { eventTimeWatermarkForEviction.isDefined && From 0138019b54978c3d023d5ad56e455a4936bbb7b8 Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Wed, 27 Nov 2024 17:08:51 +0900 Subject: [PATCH 034/438] [SPARK-50426][PYTHON] Avoid static Python data source lookup when using builtin or Java data sources ### What changes were proposed in this pull request? This PR updates the data source lookup logic to avoid triggering expensive Python data source lookups when the data source is a built-in or Java data source. ### Why are the changes needed? Python data source lookups can be expensive, and they are often unnecessary when users only use built-in or Java data sources. The primary benefit of the lookup is to display a better error message when a Java data source and a Python data source share the same name, which can be rare. In these cases, we can simply prefer loading Java data sources instead of throwing exceptions. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New UT. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48971 from allisonwang-db/spark-50426-avoid-static-pyds-lookup. Lead-authored-by: Allison Wang Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../user_guide/sql/python_data_source.rst | 5 +++++ .../execution/datasources/DataSource.scala | 7 +++---- .../datasources/DataSourceManager.scala | 1 + .../python/PythonDataSourceSuite.scala | 21 +++++++++++++++++++ 4 files changed, 30 insertions(+), 4 deletions(-) diff --git a/python/docs/source/user_guide/sql/python_data_source.rst b/python/docs/source/user_guide/sql/python_data_source.rst index 832987d19e5a4..22b2a0b5f3c7b 100644 --- a/python/docs/source/user_guide/sql/python_data_source.rst +++ b/python/docs/source/user_guide/sql/python_data_source.rst @@ -516,3 +516,8 @@ The following example demonstrates how to implement a basic Data Source using Ar df = spark.read.format("arrowbatch").load() df.show() + +Usage Notes +----------- + +- During Data Source resolution, built-in and Scala/Java Data Sources take precedence over Python Data Sources with the same name; to explicitly use a Python Data Source, make sure its name does not conflict with the other Data Sources. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index bc156cd82ed6a..58bbd91a8cc77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -682,11 +682,10 @@ object DataSource extends Logging { throw e } } - case _ :: Nil if isUserDefinedDataSource => - // There was DSv1 or DSv2 loaded, but the same name source was found - // in user defined data source. - throw QueryCompilationErrors.foundMultipleDataSources(provider) case head :: Nil => + // We do not check whether the provider is a Python data source + // (isUserDefinedDataSource) to avoid the lookup cost. Java data sources + // always take precedence over Python user-defined data sources. head.getClass case sources => // There are multiple registered aliases for the input. If there is single datasource 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 93fc6cf367cfc..711e096ebd1f8 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 @@ -101,6 +101,7 @@ object DataSourceManager extends Logging { private def initialStaticDataSourceBuilders: Map[String, UserDefinedPythonDataSource] = { if (shouldLoadPythonDataSources) this.synchronized { + logInfo("Loading static Python Data Sources.") if (dataSourceBuilders.isEmpty) { val maybeResult = try { Some(UserDefinedPythonDataSource.lookupAllDataSourcesInPython()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala index 1f2be12058eb7..73c05ff0e0b58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala @@ -94,6 +94,27 @@ abstract class PythonDataSourceSuiteBase extends QueryTest with SharedSparkSessi class PythonDataSourceSuite extends PythonDataSourceSuiteBase { import IntegratedUDFTestUtils._ + test("SPARK-50426: should not trigger static Python data source lookup") { + assume(shouldTestPandasUDFs) + val testAppender = new LogAppender("Python data source lookup") + // Using builtin and Java data sources should not trigger a static + // Python data source lookup + withLogAppender(testAppender) { + spark.read.format("org.apache.spark.sql.test").load() + spark.range(3).write.mode("overwrite").format("noop").save() + } + assert(!testAppender.loggingEvents + .exists(msg => msg.getMessage.getFormattedMessage.contains( + "Loading static Python Data Sources."))) + // Now trigger a Python data source lookup + withLogAppender(testAppender) { + spark.read.format(staticSourceName).load() + } + assert(testAppender.loggingEvents + .exists(msg => msg.getMessage.getFormattedMessage.contains( + "Loading static Python Data Sources."))) + } + test("SPARK-45917: automatic registration of Python Data Source") { assume(shouldTestPandasUDFs) val df = spark.read.format(staticSourceName).load() From 6edcf435661e179a7020e698aa5f23e2a528d808 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 27 Nov 2024 20:01:09 +0800 Subject: [PATCH 035/438] [SPARK-49966][SQL][FOLLOWUP] Codegen Support for `JsonToStructs`(`from_json`) - remove Invoke ### What changes were proposed in this pull request? The pr aims to restore the codegen implementation of `JsonToStructs`(`from_json`) in the way of `manually`, rather than in the way of `Invoke`. ### Why are the changes needed? Based on cloud-fan's double-check, https://github.com/apache/spark/pull/48509#discussion_r1858515472 I believe that restore to manual implementation will not result in regression. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Update existed UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48979 from panbingkun/SPARK-49966_REMOVE_INVOKE. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../json/JsonExpressionEvalUtils.scala | 1 + .../expressions/jsonExpressions.scala | 48 ++++++++++++------- .../expressions/JsonExpressionsSuite.scala | 2 +- .../optimizer/OptimizeJsonExprsSuite.scala | 2 +- .../function_from_json.explain | 2 +- .../function_from_json_orphaned.explain | 2 +- ...unction_from_json_with_json_schema.explain | 2 +- .../BaseScriptTransformationExec.scala | 2 +- 8 files changed, 37 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala index edc8012eb3da2..7a399aba4382c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala @@ -103,6 +103,7 @@ case class JsonToStructsEvaluator( } final def evaluate(json: UTF8String): Any = { + if (json == null) return null nullableSchema match { case _: VariantType => VariantExpressionEvalUtils.parseJson(json, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index ac6c233f7d2ea..0381d1e712aeb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -636,9 +636,8 @@ case class JsonToStructs( timeZoneId: Option[String] = None, variantAllowDuplicateKeys: Boolean = SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_DUPLICATE_KEYS)) extends UnaryExpression - with RuntimeReplaceable - with ExpectsInputTypes with TimeZoneAwareExpression + with ExpectsInputTypes with QueryErrorsBase { // The JSON input data might be missing certain fields. We force the nullability @@ -648,7 +647,9 @@ case class JsonToStructs( override def nullable: Boolean = true - override def nodePatternsInternal(): Seq[TreePattern] = Seq(JSON_TO_STRUCT, RUNTIME_REPLACEABLE) + final override def nodePatternsInternal(): Seq[TreePattern] = Seq(JSON_TO_STRUCT) + + override def nullIntolerant: Boolean = true // Used in `FunctionRegistry` def this(child: Expression, schema: Expression, options: Map[String, String]) = @@ -682,6 +683,32 @@ case class JsonToStructs( override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) + @transient + private val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) + + @transient + private lazy val evaluator = new JsonToStructsEvaluator( + options, nullableSchema, nameOfCorruptRecord, timeZoneId, variantAllowDuplicateKeys) + + override def nullSafeEval(json: Any): Any = evaluator.evaluate(json.asInstanceOf[UTF8String]) + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val refEvaluator = ctx.addReferenceObj("evaluator", evaluator) + val eval = child.genCode(ctx) + val resultType = CodeGenerator.boxedType(dataType) + val resultTerm = ctx.freshName("result") + ev.copy(code = + code""" + |${eval.code} + |$resultType $resultTerm = ($resultType) $refEvaluator.evaluate(${eval.value}); + |boolean ${ev.isNull} = $resultTerm == null; + |${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + |if (!${ev.isNull}) { + | ${ev.value} = $resultTerm; + |} + |""".stripMargin) + } + override def inputTypes: Seq[AbstractDataType] = StringTypeWithCollation :: Nil override def sql: String = schema match { @@ -691,21 +718,6 @@ case class JsonToStructs( override def prettyName: String = "from_json" - @transient - private val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) - - @transient - lazy val evaluator: JsonToStructsEvaluator = JsonToStructsEvaluator( - options, nullableSchema, nameOfCorruptRecord, timeZoneId, variantAllowDuplicateKeys) - - override def replacement: Expression = Invoke( - Literal.create(evaluator, ObjectType(classOf[JsonToStructsEvaluator])), - "evaluate", - dataType, - Seq(child), - Seq(child.dataType) - ) - override protected def withNewChildInternal(newChild: Expression): JsonToStructs = copy(child = newChild) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 3a58cb92cecf2..2c82ce712d957 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -420,7 +420,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("from_json escaping") { val schema = StructType(StructField("\"quote", IntegerType) :: Nil) GenerateUnsafeProjection.generate( - JsonToStructs(schema, Map.empty, Literal("\"quote"), UTC_OPT).replacement :: Nil) + JsonToStructs(schema, Map.empty, Literal("\"quote"), UTC_OPT) :: Nil) } test("from_json") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala index 7af2be2db01d1..eed06da609f8e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala @@ -292,7 +292,7 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { Seq("""{"a":1, "b":2, "c": 123, "d": "test"}""", null).foreach(v => { val row = create_row(v) - checkEvaluation(e1, replace(e2).eval(row), row) + checkEvaluation(e1, e2.eval(row), row) }) } diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json.explain index 9bc33b3b97d2c..8d1d122d156ff 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json.explain @@ -1,2 +1,2 @@ -Project [invoke(JsonToStructsEvaluator(Map(),StructType(StructField(id,LongType,true),StructField(a,IntegerType,true),StructField(b,DoubleType,true)),_corrupt_record,Some(America/Los_Angeles),false).evaluate(g#0)) AS from_json(g)#0] +Project [from_json(StructField(id,LongType,true), StructField(a,IntegerType,true), StructField(b,DoubleType,true), g#0, Some(America/Los_Angeles), false) AS from_json(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_orphaned.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_orphaned.explain index 9bc33b3b97d2c..8d1d122d156ff 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_orphaned.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_orphaned.explain @@ -1,2 +1,2 @@ -Project [invoke(JsonToStructsEvaluator(Map(),StructType(StructField(id,LongType,true),StructField(a,IntegerType,true),StructField(b,DoubleType,true)),_corrupt_record,Some(America/Los_Angeles),false).evaluate(g#0)) AS from_json(g)#0] +Project [from_json(StructField(id,LongType,true), StructField(a,IntegerType,true), StructField(b,DoubleType,true), g#0, Some(America/Los_Angeles), false) AS from_json(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_with_json_schema.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_with_json_schema.explain index 9bc33b3b97d2c..8d1d122d156ff 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_with_json_schema.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_json_with_json_schema.explain @@ -1,2 +1,2 @@ -Project [invoke(JsonToStructsEvaluator(Map(),StructType(StructField(id,LongType,true),StructField(a,IntegerType,true),StructField(b,DoubleType,true)),_corrupt_record,Some(America/Los_Angeles),false).evaluate(g#0)) AS from_json(g)#0] +Project [from_json(StructField(id,LongType,true), StructField(a,IntegerType,true), StructField(b,DoubleType,true), g#0, Some(America/Los_Angeles), false) AS from_json(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala index 64d2633c31079..60156bff1fb71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala @@ -239,7 +239,7 @@ trait BaseScriptTransformationExec extends UnaryExecNode { val complexTypeFactory = JsonToStructs(attr.dataType, ioschema.outputSerdeProps.toMap, Literal(null), Some(conf.sessionLocalTimeZone)) wrapperConvertException(data => - complexTypeFactory.evaluator.evaluate(UTF8String.fromString(data)), any => any) + complexTypeFactory.nullSafeEval(UTF8String.fromString(data)), any => any) case udt: UserDefinedType[_] => wrapperConvertException(data => udt.deserialize(data), converter) case dt => From 1a502d32ef5a69739e10b827be4c9063b2a20493 Mon Sep 17 00:00:00 2001 From: Ole Sasse Date: Wed, 27 Nov 2024 14:49:05 -0800 Subject: [PATCH 036/438] [SPARK-50285] Metrics for commits to StagedTable instances ### What changes were proposed in this pull request? Commands that commit using the StagedTable interface do not have any metrics at the moment because the interface does not support retrieving metrics after a commit. This PR * Adds a new interface StagedTableWithCommitMetrics that allows to retrieve metrics * Add a method StagingTableCatalog to indicate that the catalog supports metrics * Support metric retrieval in the commands that use the StagedTable interface ### Why are the changes needed? Many create table commands currently return no metrics at all ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New tests with a test catalog for the affected commands ### Was this patch authored or co-authored using generative AI tooling? No Closes #48830 from olaky/spark-50285-metrics-for-staged-table-commits. Authored-by: Ole Sasse Signed-off-by: Gengliang Wang --- .../sql/connector/catalog/StagedTable.java | 14 ++ .../catalog/StagingTableCatalog.java | 12 ++ .../catalog/StagingInMemoryTableCatalog.scala | 2 +- .../datasources/v2/DataSourceV2Utils.scala | 35 +++- .../datasources/v2/ReplaceTableExec.scala | 6 +- .../v2/WriteToDataSourceV2Exec.scala | 11 +- .../connector/DataSourceV2MetricsSuite.scala | 159 ++++++++++++++++++ 7 files changed, 232 insertions(+), 7 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetricsSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java index 60b250adb41ef..cbaea8cad8582 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java @@ -21,7 +21,9 @@ import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.Write; import org.apache.spark.sql.types.StructType; /** @@ -52,4 +54,16 @@ public interface StagedTable extends Table { * table's writers. */ void abortStagedChanges(); + + /** + * Retrieve driver metrics after a commit. This is analogous + * to {@link Write#reportDriverMetrics()}. Note that these metrics must be included in the + * supported custom metrics reported by `supportedCustomMetrics` of the + * {@link StagingTableCatalog} that returned the staged table. + * + * @return an Array of commit metric values. Throws if the table has not been committed yet. + */ + default CustomTaskMetric[] reportDriverMetrics() throws RuntimeException { + return new CustomTaskMetric[0]; + } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java index eead1ade40791..f457a4a3d7863 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java @@ -21,11 +21,13 @@ import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.metric.CustomMetric; import org.apache.spark.sql.connector.write.LogicalWriteInfo; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.Write; import org.apache.spark.sql.connector.write.WriterCommitMessage; import org.apache.spark.sql.errors.QueryCompilationErrors; import org.apache.spark.sql.types.StructType; @@ -200,4 +202,14 @@ default StagedTable stageCreateOrReplace( return stageCreateOrReplace( ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties); } + + /** + * @return An Array of commit metrics that are supported by the catalog. This is analogous to + * {@link Write#supportedCustomMetrics()}. The corresponding + * {@link StagedTable#reportDriverMetrics()} method must be called to + * retrieve the actual metric values after a commit. The methods are not in the same class + * because the supported metrics are required before the staged table object is created + * and only the staged table object can capture the write metrics during the commit. + */ + default CustomMetric[] supportedCustomMetrics() { return new CustomMetric[0]; } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala index f3c7bc98cec09..2a207901b83f5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala @@ -78,7 +78,7 @@ class StagingInMemoryTableCatalog extends InMemoryTableCatalog with StagingTable maybeSimulateFailedTableCreation(properties) } - private abstract class TestStagedTable( + protected abstract class TestStagedTable( ident: Identifier, delegateTable: InMemoryTable) extends StagedTable with SupportsWrite with SupportsRead { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala index 9ffa0d728ca28..9c19609dce79a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -23,16 +23,19 @@ import scala.jdk.CollectionConverters._ import com.fasterxml.jackson.databind.ObjectMapper +import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.catalyst.analysis.TimeTravelSpec import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SessionConfigSupport, SupportsCatalogOptions, SupportsRead, Table, TableProvider} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SessionConfigSupport, StagedTable, StagingTableCatalog, SupportsCatalogOptions, SupportsRead, Table, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -179,4 +182,34 @@ private[sql] object DataSourceV2Utils extends Logging { extraOptions + ("paths" -> objectMapper.writeValueAsString(paths.toArray)) } } + + /** + * If `table` is a StagedTable, commit the staged changes and report the commit metrics. + * Do nothing if the table is not a StagedTable. + */ + def commitStagedChanges( + sparkContext: SparkContext, table: Table, metrics: Map[String, SQLMetric]): Unit = { + table match { + case stagedTable: StagedTable => + stagedTable.commitStagedChanges() + + val driverMetrics = stagedTable.reportDriverMetrics() + if (driverMetrics.nonEmpty) { + for (taskMetric <- driverMetrics) { + metrics.get(taskMetric.name()).foreach(_.set(taskMetric.value())) + } + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + } + case _ => + } + } + + def commitMetrics( + sparkContext: SparkContext, tableCatalog: StagingTableCatalog): Map[String, SQLMetric] = { + tableCatalog.supportedCustomMetrics().map { + metric => metric.name() -> SQLMetrics.createV2CustomMetric(sparkContext, metric) + }.toMap + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index 104d8a706efb7..894a3a10d4193 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical.TableSpec import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, StagedTable, StagingTableCatalog, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.util.Utils case class ReplaceTableExec( @@ -65,6 +66,9 @@ case class AtomicReplaceTableExec( val tableProperties = CatalogV2Util.convertTableProperties(tableSpec) + override val metrics: Map[String, SQLMetric] = + DataSourceV2Utils.commitMetrics(sparkContext, catalog) + override protected def run(): Seq[InternalRow] = { if (catalog.tableExists(identifier)) { val table = catalog.loadTable(identifier) @@ -92,7 +96,7 @@ case class AtomicReplaceTableExec( private def commitOrAbortStagedChanges(staged: StagedTable): Unit = { Utils.tryWithSafeFinallyAndFailureCallbacks({ - staged.commitStagedChanges() + DataSourceV2Utils.commitStagedChanges(sparkContext, staged, metrics) })(catchBlock = { staged.abortStagedChanges() }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index b238b0ce9760c..bdcf7b8260a7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -110,6 +110,9 @@ case class AtomicCreateTableAsSelectExec( val properties = CatalogV2Util.convertTableProperties(tableSpec) + override val metrics: Map[String, SQLMetric] = + DataSourceV2Utils.commitMetrics(sparkContext, catalog) + override protected def run(): Seq[InternalRow] = { if (catalog.tableExists(ident)) { if (ifNotExists) { @@ -197,6 +200,9 @@ case class AtomicReplaceTableAsSelectExec( val properties = CatalogV2Util.convertTableProperties(tableSpec) + override val metrics: Map[String, SQLMetric] = + DataSourceV2Utils.commitMetrics(sparkContext, catalog) + override protected def run(): Seq[InternalRow] = { val columns = getV2Columns(query.schema, catalog.useNullableQuerySchema) if (catalog.tableExists(ident)) { @@ -630,10 +636,7 @@ private[v2] trait V2CreateTableAsSelectBaseExec extends LeafV2CommandExec { val qe = session.sessionState.executePlan(append) qe.assertCommandExecuted() - table match { - case st: StagedTable => st.commitStagedChanges() - case _ => - } + DataSourceV2Utils.commitStagedChanges(sparkContext, table, metrics) Nil })(catchBlock = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetricsSuite.scala new file mode 100644 index 0000000000000..fe28b85528632 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2MetricsSuite.scala @@ -0,0 +1,159 @@ +/* + * 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.connector + +import java.util + +import org.apache.spark.sql.QueryTest.withQueryExecutionsCaptured +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, InMemoryTable, InMemoryTableCatalog, StagedTable, StagingInMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.metric.{CustomMetric, CustomSumMetric, CustomTaskMetric} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2.{AtomicCreateTableAsSelectExec, AtomicReplaceTableAsSelectExec, AtomicReplaceTableExec, CreateTableAsSelectExec, ReplaceTableAsSelectExec, ReplaceTableExec} + +class StagingInMemoryTableCatalogWithMetrics extends StagingInMemoryTableCatalog { + + case class TestSupportedCommitMetric(name: String, description: String) extends CustomSumMetric + + override def supportedCustomMetrics(): Array[CustomMetric] = Array( + TestSupportedCommitMetric("numFiles", "number of written files"), + TestSupportedCommitMetric("numOutputRows", "number of output rows"), + TestSupportedCommitMetric("numOutputBytes", "written output")) + + private class TestStagedTableWithMetric( + ident: Identifier, + delegateTable: InMemoryTable + ) extends TestStagedTable(ident, delegateTable) with StagedTable { + + private var stagedChangesCommitted = false + + override def commitStagedChanges(): Unit = { + tables.put(ident, delegateTable) + stagedChangesCommitted = true + } + + override def reportDriverMetrics: Array[CustomTaskMetric] = { + assert(stagedChangesCommitted) + StagingInMemoryTableCatalogWithMetrics.testMetrics + } + } + + override def stageCreate( + ident: Identifier, + columns: Array[Column], + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + new TestStagedTableWithMetric( + ident, + new InMemoryTable(s"$name.${ident.quoted}", + CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties)) + } + + override def stageReplace( + ident: Identifier, + columns: Array[Column], + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = + stageCreate(ident, columns, partitions, properties) + + override def stageCreateOrReplace( + ident: Identifier, + columns: Array[Column], + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = + stageCreate(ident, columns, partitions, properties) +} + +object StagingInMemoryTableCatalogWithMetrics { + + case class TestCustomTaskMetric(name: String, value: Long) extends CustomTaskMetric + + val testMetrics: Array[CustomTaskMetric] = Array( + TestCustomTaskMetric("numFiles", 1337), + TestCustomTaskMetric("numOutputRows", 1338), + TestCustomTaskMetric("numOutputBytes", 1339)) +} + +class DataSourceV2MetricsSuite extends DatasourceV2SQLBase { + + private val testCatalog = "test_catalog" + private val atomicTestCatalog = "atomic_test_catalog" + private val nonExistingTable = "non_existing_table" + private val existingTable = "existing_table" + + private def captureStagedTableWrite(thunk: => Unit): SparkPlan = { + val physicalPlans = withQueryExecutionsCaptured(spark)(thunk).map(_.executedPlan) + val stagedTableWrites = physicalPlans.filter { + case _: AtomicCreateTableAsSelectExec | _: CreateTableAsSelectExec | + _: AtomicReplaceTableAsSelectExec | _: ReplaceTableAsSelectExec | + _: AtomicReplaceTableExec | _: ReplaceTableExec => true + case _ => false + } + assert(stagedTableWrites.size === 1) + stagedTableWrites.head + } + + private def commands: Seq[String => Unit] = Seq( + { catalogName => + sql(s"CREATE TABLE $catalogName.$nonExistingTable AS SELECT * FROM $existingTable") }, + { catalogName => + spark.table(existingTable).write.saveAsTable(s"$catalogName.$nonExistingTable") }, + { catalogName => + sql(s"CREATE OR REPLACE TABLE $catalogName.$nonExistingTable " + + s"AS SELECT * FROM $existingTable") }, + { catalogName => + sql(s"REPLACE TABLE $catalogName.$existingTable AS SELECT * FROM $existingTable") }, + { catalogName => + spark.table(existingTable) + .write.mode("overwrite").saveAsTable(s"$catalogName.$existingTable") }, + { catalogName => + sql(s"REPLACE TABLE $catalogName.$existingTable (id bigint, data string)") }) + + private def catalogCommitMetricsTest( + testName: String, catalogName: String)(testFunction: SparkPlan => Unit): Unit = { + commands.foreach { command => + test(s"$testName - $command") { + registerCatalog(testCatalog, classOf[InMemoryTableCatalog]) + registerCatalog(atomicTestCatalog, classOf[StagingInMemoryTableCatalogWithMetrics]) + withTable(existingTable, s"$catalogName.$existingTable") { + sql(s"CREATE TABLE $existingTable (id bigint, data string)") + sql(s"CREATE TABLE $catalogName.$existingTable (id bigint, data string)") + + testFunction(captureStagedTableWrite(command(catalogName))) + } + } + } + } + + catalogCommitMetricsTest( + "No metrics in the plan if the catalog does not support them", testCatalog) { sparkPlan => + val metrics = sparkPlan.metrics + + assert(metrics.isEmpty) + } + + catalogCommitMetricsTest( + "Plan metrics values are the values from the catalog", atomicTestCatalog) { sparkPlan => + val metrics = sparkPlan.metrics + + assert(metrics.size === StagingInMemoryTableCatalogWithMetrics.testMetrics.length) + StagingInMemoryTableCatalogWithMetrics.testMetrics.foreach(customTaskMetric => + assert(metrics(customTaskMetric.name()).value === customTaskMetric.value())) + } +} From 6697725f1715dee0fb152069d455bc2fa193a13d Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 28 Nov 2024 09:16:41 +0900 Subject: [PATCH 037/438] Revert "[MINOR][TESTS] Use SparkContext.setLogLevel in QuietTest" This reverts commit 9bab16a854d8c4e6decbb8812b332ae5e8825993. --- python/pyspark/testing/utils.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index e8c658ff05f87..c38cd928d584b 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -194,16 +194,14 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: class QuietTest: def __init__(self, sc): - self.sc = sc + self.log4j = sc._jvm.org.apache.log4j def __enter__(self): - self.old_level = ( - self.sc._jvm.org.apache.log4j.LogManager.getRootLogger().getLevel().toString() - ) - self.sc.setLogLevel("FATAL") + self.old_level = self.log4j.LogManager.getRootLogger().getLevel() + self.log4j.LogManager.getRootLogger().setLevel(self.log4j.Level.FATAL) def __exit__(self, exc_type, exc_val, exc_tb): - self.sc.setLogLevel(self.old_level) + self.log4j.LogManager.getRootLogger().setLevel(self.old_level) class PySparkTestCase(unittest.TestCase): From 59a7ad46588fa1d765a8ee388641e3b09b77a7d2 Mon Sep 17 00:00:00 2001 From: Jerry Peng Date: Thu, 28 Nov 2024 11:28:10 +0900 Subject: [PATCH 038/438] [SPARK-50437][SS] Reduce overhead of creating deserializers in TransformWithStateExec MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? An optimization that creates deserializers for keys and values at the beginning of a batch. ### Why are the changes needed? Currently, deserializers for key and values are created every time `handleInputRows` gets called. The function `ObjectOperator.deserializeRowToObject` is called twice do generate the deserializer for the key and value. The function will do the following things 1. Generate java code for the deserializer based on input expression and grouping attributes 2. Compile that java code to java class The problem is that step 1 will occur every time the function is called. The compiling the code, i.e. step 2, will only happen once since the generated class will be cached and can be reused. Step 1 can incur a heavy penalty in situations where handleInputRows will be called many times. For example, if there are many distinct keys. Note that handleInputRows is called once per distinct key. Here is a flamegraph: ![Uploading Screenshot 2024-11-26 at 11.55.12 PM.png…]() ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests should suffice ### Was this patch authored or co-authored using generative AI tooling? No Closes #48985 from jerrypeng/SPARK-50437. Authored-by: Jerry Peng Signed-off-by: Jungtaek Lim --- .../streaming/TransformWithStateExec.scala | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala index 9c31ff0a74437..5716242afc152 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala @@ -83,6 +83,17 @@ case class TransformWithStateExec( // dummy value schema, the real schema will get during state variable init time private val DUMMY_VALUE_ROW_SCHEMA = new StructType().add("value", BinaryType) + // We need to just initialize key and value deserializer once per partition. + // The deserializers need to be lazily created on the executor since they + // are not serializable. + // Ideas for for improvement can be found here: + // https://issues.apache.org/jira/browse/SPARK-50437 + private lazy val getKeyObj = + ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) + + private lazy val getValueObj = + ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) + override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = { if (timeMode == ProcessingTime) { // TODO SPARK-50180: check if we can return true only if actual timers are registered, @@ -230,11 +241,6 @@ case class TransformWithStateExec( private def handleInputRows(keyRow: UnsafeRow, valueRowIter: Iterator[InternalRow]): Iterator[InternalRow] = { - val getKeyObj = - ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) - - val getValueObj = - ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType) @@ -261,8 +267,6 @@ case class TransformWithStateExec( private def processInitialStateRows( keyRow: UnsafeRow, initStateIter: Iterator[InternalRow]): Unit = { - val getKeyObj = - ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) val getInitStateValueObj = ObjectOperator.deserializeRowToObject(initialStateDeserializer, initialStateDataAttrs) From 405644fc4ace98fdfeb8702ef230b42e042bd24d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 28 Nov 2024 11:10:39 +0800 Subject: [PATCH 039/438] [SPARK-50130][SQL][FOLLOWUP] Simplify the implementation of col.outer() ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/48664 to simplify the code. The new workflow is: 1. The `col.outer()` simplify wraps the expression with `LazyExpression` 2. `QueryExecution` does lazy analysis if its main query contains `LazyExpression`. Eager analysis is still performed if only subquery expressions contain `LazyExpression`. 3. The analyzer simply strips away `LazyExpression` at the beginning. After this simplification, we no longer need the special logic to strip `LazyOuterReference` in the DataFrame side. We no longer need the extra flag in the subquery expressions. It also makes the API easy to understand: `col.outer()` is just used to trigger lazy analysis for Spark Classic ### Why are the changes needed? cleanup ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48820 from cloud-fan/subquery. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Co-authored-by: Takuya UESHIN Co-authored-by: Takuya Ueshin Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 6 - python/pyspark/sql/column.py | 6 +- python/pyspark/sql/dataframe.py | 32 +-- python/pyspark/sql/tests/test_subquery.py | 186 +++++++++--------- .../scala/org/apache/spark/sql/Column.scala | 31 +-- .../spark/sql/internal/columnNodes.scala | 33 ++-- .../sql/catalyst/analysis/Analyzer.scala | 27 +-- .../sql/catalyst/analysis/CheckAnalysis.scala | 21 +- .../analysis/ColumnResolutionHelper.scala | 44 ++--- .../analysis/EliminateLazyExpression.scala | 34 ++++ .../sql/catalyst/analysis/unresolved.scala | 47 +---- .../sql/catalyst/expressions/Expression.scala | 16 +- .../sql/catalyst/expressions/subquery.scala | 15 +- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../sql/catalyst/optimizer/expressions.scala | 2 +- .../sql/catalyst/optimizer/subquery.scala | 15 +- .../sql/catalyst/trees/TreePatterns.scala | 6 +- .../scala/org/apache/spark/sql/Dataset.scala | 14 +- .../spark/sql/execution/QueryExecution.scala | 8 +- .../adaptive/PlanAdaptiveSubqueries.scala | 2 +- .../sql/internal/columnNodeSupport.scala | 16 +- .../spark/sql/DataFrameSubquerySuite.scala | 173 ++++++++-------- 22 files changed, 331 insertions(+), 405 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/EliminateLazyExpression.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 3c494704fd715..77437f6c56179 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4748,12 +4748,6 @@ ], "sqlState" : "42KD9" }, - "UNANALYZABLE_EXPRESSION" : { - "message" : [ - "The plan contains an unanalyzable expression that holds the analysis." - ], - "sqlState" : "03000" - }, "UNBOUND_SQL_PARAMETER" : { "message" : [ "Found the unbound parameter: . Please, fix `args` and provide a mapping of the parameter to either a SQL literal or collection constructor functions such as `map()`, `array()`, `struct()`." diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 06dd2860fe406..285d30fad3bc2 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -1524,7 +1524,11 @@ def over(self, window: "WindowSpec") -> "Column": @dispatch_col_method def outer(self) -> "Column": """ - Mark this column reference as an outer reference for subqueries. + Mark this column as an outer column if its expression refers to columns from an outer query. + + This is used to trigger lazy analysis of Spark Classic DataFrame, so that we can use it + to build subquery expressions. Spark Connect DataFrame is always lazily analyzed and + does not need to use this function. .. versionadded:: 4.0.0 diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 8a5b982bc7f23..085a1a629634a 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -6522,10 +6522,11 @@ def scalar(self) -> Column: in their department. >>> from pyspark.sql import functions as sf - >>> employees.where( + >>> employees.alias("e1").where( ... sf.col("salary") - ... > employees.where(sf.col("department_id") == sf.col("department_id").outer()) - ... .select(sf.avg("salary")).scalar() + ... > employees.alias("e2").where( + ... sf.col("e2.department_id") == sf.col("e1.department_id").outer() + ... ).select(sf.avg("salary")).scalar() ... ).select("name", "salary", "department_id").show() +-----+------+-------------+ | name|salary|department_id| @@ -6538,12 +6539,13 @@ def scalar(self) -> Column: department. >>> from pyspark.sql import functions as sf - >>> employees.select( + >>> employees.alias("e1").select( ... "name", "salary", "department_id", ... sf.format_number( ... sf.lit(100) * sf.col("salary") / - ... employees.where(sf.col("department_id") == sf.col("department_id").outer()) - ... .select(sf.sum("salary")).scalar().alias("avg_salary"), + ... employees.alias("e2").where( + ... sf.col("e2.department_id") == sf.col("e1.department_id").outer() + ... ).select(sf.sum("salary")).scalar().alias("avg_salary"), ... 1 ... ).alias("salary_proportion_in_department") ... ).show() @@ -6595,8 +6597,10 @@ def exists(self) -> Column: Example 1: Filter for customers who have placed at least one order. >>> from pyspark.sql import functions as sf - >>> customers.where( - ... orders.where(sf.col("customer_id") == sf.col("customer_id").outer()).exists() + >>> customers.alias("c").where( + ... orders.alias("o").where( + ... sf.col("o.customer_id") == sf.col("c.customer_id").outer() + ... ).exists() ... ).orderBy("customer_id").show() +-----------+-------------+-------+ |customer_id|customer_name|country| @@ -6609,8 +6613,10 @@ def exists(self) -> Column: Example 2: Filter for customers who have never placed an order. >>> from pyspark.sql import functions as sf - >>> customers.where( - ... ~orders.where(sf.col("customer_id") == sf.col("customer_id").outer()).exists() + >>> customers.alias("c").where( + ... ~orders.alias("o").where( + ... sf.col("o.customer_id") == sf.col("c.customer_id").outer() + ... ).exists() ... ).orderBy("customer_id").show() +-----------+-------------+---------+ |customer_id|customer_name| country| @@ -6621,9 +6627,9 @@ def exists(self) -> Column: Example 3: Find Orders from Customers in the USA. >>> from pyspark.sql import functions as sf - >>> orders.where( - ... customers.where( - ... (sf.col("customer_id") == sf.col("customer_id").outer()) + >>> orders.alias("o").where( + ... customers.alias("c").where( + ... (sf.col("c.customer_id") == sf.col("o.customer_id").outer()) ... & (sf.col("country") == "USA") ... ).exists() ... ).orderBy("order_id").show() diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index f58ff6364aed7..7cc0360c39421 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -47,18 +47,21 @@ def df2(self): ["c", "d"], ) - def test_unanalyzable_expression(self): - sub = self.spark.range(1).where(sf.col("id") == sf.col("id").outer()) + def test_noop_outer(self): + assertDataFrameEqual( + self.spark.range(1).select(sf.col("id").outer()), + self.spark.range(1).select(sf.col("id")), + ) with self.assertRaises(AnalysisException) as pe: - sub.schema + self.spark.range(1).select(sf.col("outer_col").outer()).collect() self.check_error( exception=pe.exception, - errorClass="UNANALYZABLE_EXPRESSION", - messageParameters={"expr": '"outer(id)"'}, + errorClass="UNRESOLVED_COLUMN.WITH_SUGGESTION", + messageParameters={"objectName": "`outer_col`", "proposal": "`id`"}, query_context_type=QueryContextType.DataFrame, - fragment="outer", + fragment="col", ) def test_simple_uncorrelated_scalar_subquery(self): @@ -189,7 +192,7 @@ def test_scalar_subquery_against_local_relations(self): "c1", ( self.spark.table("t2") - .where(sf.col("c2").outer() == sf.col("c2")) + .where(sf.col("t1.c2").outer() == sf.col("t2.c2")) .select(sf.max("c1")) .scalar() ), @@ -205,45 +208,72 @@ def test_correlated_scalar_subquery(self): self.df2.createOrReplaceTempView("r") with self.subTest("in where"): - assertDataFrameEqual( - self.spark.table("l").where( - sf.col("b") - < ( - self.spark.table("r") - .where(sf.col("a").outer() == sf.col("c")) - .select(sf.max("d")) - .scalar() + for cond in [ + sf.col("a").outer() == sf.col("c"), + (sf.col("a") == sf.col("c")).outer(), + sf.expr("a = c").outer(), + ]: + with self.subTest(cond=cond): + assertDataFrameEqual( + self.spark.table("l").where( + sf.col("b") + < self.spark.table("r").where(cond).select(sf.max("d")).scalar() + ), + self.spark.sql( + """select * from l where b < (select max(d) from r where a = c)""" + ), ) - ), - self.spark.sql( - """select * from l where b < (select max(d) from r where a = c)""" - ), - ) with self.subTest("in select"): + df1 = self.spark.table("l").alias("t1") + df2 = self.spark.table("l").alias("t2") + + for cond in [ + sf.col("t1.a") == sf.col("t2.a").outer(), + (sf.col("t1.a") == sf.col("t2.a")).outer(), + sf.expr("t1.a = t2.a").outer(), + ]: + with self.subTest(cond=cond): + assertDataFrameEqual( + df1.select( + "a", + df2.where(cond).select(sf.sum("b")).scalar().alias("sum_b"), + ), + self.spark.sql( + """ + select + a, (select sum(b) from l t2 where t2.a = t1.a) sum_b + from l t1 + """ + ), + ) + + with self.subTest("without .outer()"): assertDataFrameEqual( self.spark.table("l").select( "a", ( - self.spark.table("l") - .where(sf.col("a") == sf.col("a").outer()) - .select(sf.sum("b")) + self.spark.table("r") + .where(sf.col("b") == sf.col("a").outer()) + .select(sf.sum("d")) .scalar() - .alias("sum_b") + .alias("sum_d") ), ), self.spark.sql( - """select a, (select sum(b) from l l2 where l2.a = l1.a) sum_b from l l1""" + """select a, (select sum(d) from r where b = l.a) sum_d from l""" ), ) with self.subTest("in select (null safe)"): + df1 = self.spark.table("l").alias("t1") + df2 = self.spark.table("l").alias("t2") + assertDataFrameEqual( - self.spark.table("l").select( + df1.select( "a", ( - self.spark.table("l") - .where(sf.col("a").eqNullSafe(sf.col("a").outer())) + df2.where(sf.col("t2.a").eqNullSafe(sf.col("t1.a").outer())) .select(sf.sum("b")) .scalar() .alias("sum_b") @@ -278,15 +308,13 @@ def test_correlated_scalar_subquery(self): ) with self.subTest("non-aggregated"): + df1 = self.spark.table("l").alias("t1") + df2 = self.spark.table("l").alias("t2") + with self.assertRaises(SparkRuntimeException) as pe: - self.spark.table("l").select( + df1.select( "a", - ( - self.spark.table("l") - .where(sf.col("a") == sf.col("a").outer()) - .select("b") - .scalar() - ), + df2.where(sf.col("t1.a") == sf.col("t2.a").outer()).select("b").scalar(), ).collect() self.check_error( @@ -296,19 +324,21 @@ def test_correlated_scalar_subquery(self): ) with self.subTest("non-equal"): + df1 = self.spark.table("l").alias("t1") + df2 = self.spark.table("l").alias("t2") + assertDataFrameEqual( - self.spark.table("l").select( + df1.select( "a", ( - self.spark.table("l") - .where(sf.col("a") < sf.col("a").outer()) + df2.where(sf.col("t2.a") < sf.col("t1.a").outer()) .select(sf.sum("b")) .scalar() .alias("sum_b") ), ), self.spark.sql( - """select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1""" + """select a, (select sum(b) from l t2 where t2.a < t1.a) sum_b from l t1""" ), ) @@ -343,26 +373,30 @@ def test_exists_subquery(self): self.df2.createOrReplaceTempView("r") with self.subTest("EXISTS"): - assertDataFrameEqual( - self.spark.table("l").where( - self.spark.table("r").where(sf.col("a").outer() == sf.col("c")).exists() - ), - self.spark.sql( - """select * from l where exists (select * from r where l.a = r.c)""" - ), - ) + for cond in [ + sf.col("a").outer() == sf.col("c"), + (sf.col("a") == sf.col("c")).outer(), + sf.expr("a = c").outer(), + ]: + with self.subTest(cond=cond): + assertDataFrameEqual( + self.spark.table("l").where(self.spark.table("r").where(cond).exists()), + self.spark.sql( + """select * from l where exists (select * from r where l.a = r.c)""" + ), + ) - assertDataFrameEqual( - self.spark.table("l").where( - self.spark.table("r").where(sf.col("a").outer() == sf.col("c")).exists() - & (sf.col("a") <= sf.lit(2)) - ), - self.spark.sql( - """ + assertDataFrameEqual( + self.spark.table("l").where( + self.spark.table("r").where(cond).exists() + & (sf.col("a") <= sf.lit(2)) + ), + self.spark.sql( + """ select * from l where exists (select * from r where l.a = r.c) and l.a <= 2 """ - ), - ) + ), + ) with self.subTest("NOT EXISTS"): assertDataFrameEqual( @@ -450,46 +484,6 @@ def test_scalar_subquery_with_outer_reference_errors(self): fragment="col", ) - with self.subTest("extra `outer()`"): - with self.assertRaises(AnalysisException) as pe: - self.spark.table("l").select( - "a", - ( - self.spark.table("r") - .where(sf.col("c").outer() == sf.col("a").outer()) - .select(sf.sum("d")) - .scalar() - ), - ).collect() - - self.check_error( - exception=pe.exception, - errorClass="UNRESOLVED_COLUMN.WITH_SUGGESTION", - messageParameters={"objectName": "`c`", "proposal": "`a`, `b`"}, - query_context_type=QueryContextType.DataFrame, - fragment="outer", - ) - - with self.subTest("missing `outer()` for another outer"): - with self.assertRaises(AnalysisException) as pe: - self.spark.table("l").select( - "a", - ( - self.spark.table("r") - .where(sf.col("b") == sf.col("a").outer()) - .select(sf.sum("d")) - .scalar() - ), - ).collect() - - self.check_error( - exception=pe.exception, - errorClass="UNRESOLVED_COLUMN.WITH_SUGGESTION", - messageParameters={"objectName": "`b`", "proposal": "`c`, `d`"}, - query_context_type=QueryContextType.DataFrame, - fragment="col", - ) - class SubqueryTests(SubqueryTestsMixin, ReusedSQLTestCase): pass diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala index 8498ae04d9a2a..50ef61d4a7a19 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.parser.DataTypeParser import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions.{lit, map} -import org.apache.spark.sql.internal.{ColumnNode, LazyOuterReference, UnresolvedAttribute} +import org.apache.spark.sql.internal.ColumnNode import org.apache.spark.sql.types._ import org.apache.spark.util.ArrayImplicits._ @@ -1383,20 +1383,27 @@ class Column(val node: ColumnNode) extends Logging { def over(): Column = over(Window.spec) /** - * Marks this column reference as an outer reference for subqueries. + * Mark this column as an outer column if its expression refers to columns from an outer query. + * This is used to trigger lazy analysis of Spark Classic DataFrame, so that we can use it to + * build subquery expressions. Spark Connect DataFrame is always lazily analyzed and does not + * need to use this function. * - * @group subquery + * {{{ + * // Spark can't analyze this `df` now as it doesn't know how to resolve `t1.col`. + * val df = spark.table("t2").where($"t2.col" === $"t1.col".outer()) + * + * // Since this `df` is lazily analyzed, you won't see any error until you try to execute it. + * df.collect() // Fails with UNRESOLVED_COLUMN error. + * + * // Now Spark can resolve `t1.col` with the outer plan `spark.table("t1")`. + * spark.table("t1").where(df.exists()) + * }}} + * + * @group expr_ops * @since 4.0.0 */ - def outer(): Column = withOrigin { - node match { - case attr: UnresolvedAttribute if !attr.isMetadataColumn => - Column(LazyOuterReference(attr.nameParts, attr.planId)) - case _ => - throw new IllegalArgumentException( - "Only unresolved attributes can be used as outer references") - } - } + def outer(): Column = Column(internal.LazyExpression(node)) + } /** diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala index e3cc320a8b00f..f745c152170ee 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala @@ -167,24 +167,6 @@ private[sql] object UnresolvedAttribute { apply(unparsedIdentifier, None, false, CurrentOrigin.get) } -/** - * Reference to an attribute in the outer context, used for Subqueries. - * - * @param nameParts - * name of the attribute. - * @param planId - * id of the plan (Dataframe) that produces the attribute. - */ -private[sql] case class LazyOuterReference( - nameParts: Seq[String], - planId: Option[Long] = None, - override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { - override private[internal] def normalize(): LazyOuterReference = - copy(planId = None, origin = NO_ORIGIN) - override def sql: String = nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") -} - /** * Reference to all columns in a namespace (global, a Dataframe, or a nested struct). * @@ -593,3 +575,18 @@ private[sql] case class InvokeInlineUserDefinedFunction( private[sql] trait UserDefinedFunctionLike { def name: String = SparkClassUtils.getFormattedClassName(this) } + +/** + * A marker node to trigger Spark Classic DataFrame lazy analysis. + * + * @param child + * that needs to be lazily analyzed in Spark Classic DataFrame. + */ +private[sql] case class LazyExpression( + child: ColumnNode, + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { + override private[internal] def normalize(): ColumnNode = + copy(child = child.normalize(), origin = NO_ORIGIN) + override def sql: String = "lazy" + argumentsToSql(Seq(child)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e05f3533ae3c9..84b3ca2289f4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -279,7 +279,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor CTESubstitution, WindowsSubstitution, EliminateUnions, - SubstituteUnresolvedOrdinals), + SubstituteUnresolvedOrdinals, + EliminateLazyExpression), Batch("Disable Hints", Once, new ResolveHints.DisableHints), Batch("Hints", fixedPoint, @@ -2190,23 +2191,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor * can resolve outer references. * * Outer references of the subquery are updated as children of Subquery expression. - * - * If hasExplicitOuterRefs is true, the subquery should have an explicit outer reference, - * instead of common `UnresolvedAttribute`s. In this case, tries to resolve inner and outer - * references separately. */ private def resolveSubQuery( e: SubqueryExpression, - outer: LogicalPlan, - hasExplicitOuterRefs: Boolean = false)( + outer: LogicalPlan)( f: (LogicalPlan, Seq[Expression]) => SubqueryExpression): SubqueryExpression = { - val newSubqueryPlan = if (hasExplicitOuterRefs) { - executeSameContext(e.plan).transformAllExpressionsWithPruning( - _.containsPattern(UNRESOLVED_OUTER_REFERENCE)) { - case u: UnresolvedOuterReference => - resolveOuterReference(u.nameParts, outer).getOrElse(u) - } - } else AnalysisContext.withOuterPlan(outer) { + val newSubqueryPlan = AnalysisContext.withOuterPlan(outer) { executeSameContext(e.plan) } @@ -2231,11 +2221,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor */ private def resolveSubQueries(plan: LogicalPlan, outer: LogicalPlan): LogicalPlan = { plan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION), ruleId) { - case s @ ScalarSubquery(sub, _, exprId, _, _, _, _, hasExplicitOuterRefs) - if !sub.resolved => - resolveSubQuery(s, outer, hasExplicitOuterRefs)(ScalarSubquery(_, _, exprId)) - case e @ Exists(sub, _, exprId, _, _, hasExplicitOuterRefs) if !sub.resolved => - resolveSubQuery(e, outer, hasExplicitOuterRefs)(Exists(_, _, exprId)) + case s @ ScalarSubquery(sub, _, exprId, _, _, _, _) if !sub.resolved => + resolveSubQuery(s, outer)(ScalarSubquery(_, _, exprId)) + case e @ Exists(sub, _, exprId, _, _) if !sub.resolved => + resolveSubQuery(e, outer)(Exists(_, _, exprId)) case InSubquery(values, l @ ListQuery(_, _, exprId, _, _, _)) if values.forall(_.resolved) && !l.resolved => val expr = resolveSubQuery(l, outer)((plan, exprs) => { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 586a0312e1507..573619af1b5fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -457,11 +457,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB errorClass = "UNBOUND_SQL_PARAMETER", messageParameters = Map("name" -> p.name)) - case l: LazyAnalysisExpression => - l.failAnalysis( - errorClass = "UNANALYZABLE_EXPRESSION", - messageParameters = Map("expr" -> toSQLExpr(l))) - case _ => }) @@ -1067,20 +1062,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case _ => } - def checkUnresolvedOuterReference(p: LogicalPlan, expr: SubqueryExpression): Unit = { - expr.plan.foreachUp(_.expressions.foreach(_.foreachUp { - case o: UnresolvedOuterReference => - val cols = p.inputSet.toSeq.map(attr => toSQLId(attr.name)).mkString(", ") - o.failAnalysis( - errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", - messageParameters = Map("objectName" -> toSQLId(o.name), "proposal" -> cols)) - case _ => - })) - } - - // Check if there is unresolved outer attribute in the subquery plan. - checkUnresolvedOuterReference(plan, expr) - // Validate the subquery plan. checkAnalysis0(expr.plan) @@ -1088,7 +1069,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB checkOuterReference(plan, expr) expr match { - case ScalarSubquery(query, outerAttrs, _, _, _, _, _, _) => + case ScalarSubquery(query, outerAttrs, _, _, _, _, _) => // Scalar subquery must return one column as output. if (query.output.size != 1) { throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(query.output.size, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index e869cb281ce05..36fd4d02f8da1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -221,35 +221,35 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { val outerPlan = AnalysisContext.get.outerPlan if (outerPlan.isEmpty) return e - e.transformWithPruning(_.containsAnyPattern(UNRESOLVED_ATTRIBUTE, TEMP_RESOLVED_COLUMN)) { + def resolve(nameParts: Seq[String]): Option[Expression] = try { + outerPlan.get match { + // Subqueries in UnresolvedHaving can host grouping expressions and aggregate functions. + // We should resolve columns with `agg.output` and the rule `ResolveAggregateFunctions` will + // push them down to Aggregate later. This is similar to what we do in `resolveColumns`. + case u @ UnresolvedHaving(_, agg: Aggregate) => + agg.resolveChildren(nameParts, conf.resolver) + .orElse(u.resolveChildren(nameParts, conf.resolver)) + .map(wrapOuterReference) + case other => + other.resolveChildren(nameParts, conf.resolver).map(wrapOuterReference) + } + } catch { + case ae: AnalysisException => + logDebug(ae.getMessage) + None + } + + e.transformWithPruning( + _.containsAnyPattern(UNRESOLVED_ATTRIBUTE, TEMP_RESOLVED_COLUMN)) { case u: UnresolvedAttribute => - resolveOuterReference(u.nameParts, outerPlan.get).getOrElse(u) + resolve(u.nameParts).getOrElse(u) // Re-resolves `TempResolvedColumn` as outer references if it has tried to be resolved with // Aggregate but failed. case t: TempResolvedColumn if t.hasTried => - resolveOuterReference(t.nameParts, outerPlan.get).getOrElse(t) + resolve(t.nameParts).getOrElse(t) } } - protected def resolveOuterReference( - nameParts: Seq[String], outerPlan: LogicalPlan): Option[Expression] = try { - outerPlan match { - // Subqueries in UnresolvedHaving can host grouping expressions and aggregate functions. - // We should resolve columns with `agg.output` and the rule `ResolveAggregateFunctions` will - // push them down to Aggregate later. This is similar to what we do in `resolveColumns`. - case u @ UnresolvedHaving(_, agg: Aggregate) => - agg.resolveChildren(nameParts, conf.resolver) - .orElse(u.resolveChildren(nameParts, conf.resolver)) - .map(wrapOuterReference) - case other => - other.resolveChildren(nameParts, conf.resolver).map(wrapOuterReference) - } - } catch { - case ae: AnalysisException => - logDebug(ae.getMessage) - None - } - def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = { // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not. def maybeTempVariableName(nameParts: Seq[String]): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/EliminateLazyExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/EliminateLazyExpression.scala new file mode 100644 index 0000000000000..68f3f90e193b6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/EliminateLazyExpression.scala @@ -0,0 +1,34 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.LAZY_EXPRESSION + +/** + * `LazyExpression` is a marker node to trigger lazy analysis in DataFrames. It's useless when + * entering the analyzer and this rule removes it. + */ +object EliminateLazyExpression extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.resolveExpressionsUpWithPruning(_.containsPattern(LAZY_EXPRESSION)) { + case l: LazyExpression => l.child + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 40994f42e71d6..f366339d95c0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -1004,42 +1004,13 @@ case class UnresolvedTranspose( copy(child = newChild) } -case class UnresolvedOuterReference( - nameParts: Seq[String]) - extends LeafExpression with NamedExpression with Unevaluable { - - def name: String = - nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") - - override def exprId: ExprId = throw new UnresolvedException("exprId") - override def dataType: DataType = throw new UnresolvedException("dataType") - override def nullable: Boolean = throw new UnresolvedException("nullable") - override def qualifier: Seq[String] = throw new UnresolvedException("qualifier") - override lazy val resolved = false - - override def toAttribute: Attribute = throw new UnresolvedException("toAttribute") - override def newInstance(): UnresolvedOuterReference = this - - final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_OUTER_REFERENCE) -} - -case class LazyOuterReference( - nameParts: Seq[String]) - extends LeafExpression with NamedExpression with Unevaluable with LazyAnalysisExpression { - - def name: String = - nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") - - override def exprId: ExprId = throw new UnresolvedException("exprId") - override def dataType: DataType = throw new UnresolvedException("dataType") - override def nullable: Boolean = throw new UnresolvedException("nullable") - override def qualifier: Seq[String] = throw new UnresolvedException("qualifier") - - override def toAttribute: Attribute = throw new UnresolvedException("toAttribute") - override def newInstance(): NamedExpression = LazyOuterReference(nameParts) - - override def nodePatternsInternal(): Seq[TreePattern] = Seq(LAZY_OUTER_REFERENCE) - - override def prettyName: String = "outer" - override def sql: String = s"$prettyName($name)" +// A marker node to indicate that the logical plan containing this expression should be lazily +// analyzed in the DataFrame. This node will be removed at the beginning of analysis. +case class LazyExpression(child: Expression) extends UnaryExpression with Unevaluable { + override lazy val resolved: Boolean = false + override def dataType: DataType = child.dataType + override protected def withNewChildInternal(newChild: Expression): Expression = { + copy(child = newChild) + } + final override val nodePatterns: Seq[TreePattern] = Seq(LAZY_EXPRESSION) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index c454799852826..2090aab3b1f5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.trees.{BinaryLike, CurrentOrigin, LeafLike, QuaternaryLike, TernaryLike, TreeNode, UnaryLike} -import org.apache.spark.sql.catalyst.trees.TreePattern.{LAZY_ANALYSIS_EXPRESSION, RUNTIME_REPLACEABLE, TreePattern} +import org.apache.spark.sql.catalyst.trees.TreePattern.{RUNTIME_REPLACEABLE, TreePattern} import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.errors.{QueryErrorsBase, QueryExecutionErrors} @@ -410,20 +410,6 @@ trait Unevaluable extends Expression with FoldableUnevaluable { final override def foldable: Boolean = false } -/** - * An expression that cannot be analyzed. These expressions don't live analysis time or after - * and should not be evaluated during query planning and execution. - */ -trait LazyAnalysisExpression extends Expression { - final override lazy val resolved = false - - final override val nodePatterns: Seq[TreePattern] = - Seq(LAZY_ANALYSIS_EXPRESSION) ++ nodePatternsInternal() - - // Subclasses can override this function to provide more TreePatterns. - def nodePatternsInternal(): Seq[TreePattern] = Seq() -} - /** * An expression that gets replaced at runtime (currently by the optimizer) into a different * expression for evaluation. This is mainly used to provide compatibility with other databases. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index bd6f65b61468d..0c8253659dd56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -19,11 +19,9 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.analysis.{LazyOuterReference, UnresolvedOuterReference} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.TreePattern import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf @@ -374,13 +372,6 @@ object SubExprUtils extends PredicateHelper { val nonEquivalentGroupByExprs = groupByExprs -- correlatedEquivalentExprs nonEquivalentGroupByExprs } - - def removeLazyOuterReferences(logicalPlan: LogicalPlan): LogicalPlan = { - logicalPlan.transformAllExpressionsWithPruning( - _.containsPattern(TreePattern.LAZY_OUTER_REFERENCE)) { - case or: LazyOuterReference => UnresolvedOuterReference(or.nameParts) - } - } } /** @@ -407,8 +398,7 @@ case class ScalarSubquery( joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None, mayHaveCountBug: Option[Boolean] = None, - needSingleJoin: Option[Boolean] = None, - hasExplicitOuterRefs: Boolean = false) + needSingleJoin: Option[Boolean] = None) extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = { if (!plan.schema.fields.nonEmpty) { @@ -577,8 +567,7 @@ case class Exists( outerAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, - hint: Option[HintInfo] = None, - hasExplicitOuterRefs: Boolean = false) + hint: Option[HintInfo] = None) extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Predicate with Unevaluable { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 0772c67ea27e6..7ec467badce5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -346,7 +346,7 @@ abstract class Optimizer(catalogManager: CatalogManager) case d: DynamicPruningSubquery => d case s @ ScalarSubquery( PhysicalOperation(projections, predicates, a @ Aggregate(group, _, child, _)), - _, _, _, _, mayHaveCountBug, _, _) + _, _, _, _, mayHaveCountBug, _) if conf.getConf(SQLConf.DECORRELATE_SUBQUERY_PREVENT_CONSTANT_FOLDING_FOR_COUNT_BUG) && mayHaveCountBug.nonEmpty && mayHaveCountBug.get => // This is a subquery with an aggregate that may suffer from a COUNT bug. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 754fea85ec6d7..e867953bcf282 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -90,7 +90,7 @@ object ConstantFolding extends Rule[LogicalPlan] { } // Don't replace ScalarSubquery if its plan is an aggregate that may suffer from a COUNT bug. - case s @ ScalarSubquery(_, _, _, _, _, mayHaveCountBug, _, _) + case s @ ScalarSubquery(_, _, _, _, _, mayHaveCountBug, _) if conf.getConf(SQLConf.DECORRELATE_SUBQUERY_PREVENT_CONSTANT_FOLDING_FOR_COUNT_BUG) && mayHaveCountBug.nonEmpty && mayHaveCountBug.get => s diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 8c82769dbf4a3..5a4e9f37c3951 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -131,12 +131,12 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Filter the plan by applying left semi and left anti joins. withSubquery.foldLeft(newFilter) { - case (p, Exists(sub, _, _, conditions, subHint, _)) => + case (p, Exists(sub, _, _, conditions, subHint)) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) val join = buildJoin(outerPlan, rewriteDomainJoinsIfPresent(outerPlan, sub, joinCond), LeftSemi, joinCond, subHint) Project(p.output, join) - case (p, Not(Exists(sub, _, _, conditions, subHint, _))) => + case (p, Not(Exists(sub, _, _, conditions, subHint))) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) val join = buildJoin(outerPlan, rewriteDomainJoinsIfPresent(outerPlan, sub, joinCond), LeftAnti, joinCond, subHint) @@ -319,7 +319,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { val introducedAttrs = ArrayBuffer.empty[Attribute] val newExprs = exprs.map { e => e.transformDownWithPruning(_.containsAnyPattern(EXISTS_SUBQUERY, IN_SUBQUERY)) { - case Exists(sub, _, _, conditions, subHint, _) => + case Exists(sub, _, _, conditions, subHint) => val exists = AttributeReference("exists", BooleanType, nullable = false)() val existenceJoin = ExistenceJoin(exists) val newCondition = conditions.reduceLeftOption(And) @@ -507,7 +507,7 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper plan.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { case ScalarSubquery(sub, children, exprId, conditions, hint, - mayHaveCountBugOld, needSingleJoinOld, _) + mayHaveCountBugOld, needSingleJoinOld) if children.nonEmpty => def mayHaveCountBugAgg(a: Aggregate): Boolean = { @@ -560,7 +560,7 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper } ScalarSubquery(newPlan, children, exprId, getJoinCondition(newCond, conditions), hint, Some(mayHaveCountBug), Some(needSingleJoin)) - case Exists(sub, children, exprId, conditions, hint, _) if children.nonEmpty => + case Exists(sub, children, exprId, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = if (SQLConf.get.decorrelateInnerQueryEnabledForExistsIn) { decorrelate(sub, plan, handleCountBug = true) } else { @@ -818,7 +818,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe val subqueryAttrMapping = ArrayBuffer[(Attribute, Attribute)]() val newChild = subqueries.foldLeft(child) { case (currentChild, ScalarSubquery(sub, _, _, conditions, subHint, mayHaveCountBug, - needSingleJoin, _)) => + needSingleJoin)) => val query = DecorrelateInnerQuery.rewriteDomainJoins(currentChild, sub, conditions) val origOutput = query.output.head // The subquery appears on the right side of the join, hence add its hint to the right @@ -1064,8 +1064,7 @@ object OptimizeOneRowRelationSubquery extends Rule[LogicalPlan] { case p: LogicalPlan => p.transformExpressionsUpWithPruning( _.containsPattern(SCALAR_SUBQUERY)) { - case s @ ScalarSubquery( - OneRowSubquery(p @ Project(_, _: OneRowRelation)), _, _, _, _, _, _, _) + case s @ ScalarSubquery(OneRowSubquery(p @ Project(_, _: OneRowRelation)), _, _, _, _, _, _) if !hasCorrelatedSubquery(s.plan) && s.joinCond.isEmpty => assert(p.projectList.size == 1) stripOuterReferences(p.projectList).head diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 24b787054fb13..7435f4c527034 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -63,6 +63,7 @@ object TreePattern extends Enumeration { val LAMBDA_VARIABLE: Value = Value val LATERAL_COLUMN_ALIAS_REFERENCE: Value = Value val LATERAL_SUBQUERY: Value = Value + val LAZY_EXPRESSION: Value = Value val LIKE_FAMLIY: Value = Value val LIST_SUBQUERY: Value = Value val LITERAL: Value = Value @@ -154,7 +155,6 @@ object TreePattern extends Enumeration { val UNRESOLVED_HINT: Value = Value val UNRESOLVED_WINDOW_EXPRESSION: Value = Value val UNRESOLVED_IDENTIFIER_WITH_CTE: Value = Value - val UNRESOLVED_OUTER_REFERENCE: Value = Value // Unresolved Plan patterns (Alphabetically ordered) val UNRESOLVED_FUNC: Value = Value @@ -169,8 +169,4 @@ object TreePattern extends Enumeration { // Execution Plan patterns (alphabetically ordered) val EXCHANGE: Value = Value - - // Lazy analysis expression patterns (alphabetically ordered) - val LAZY_ANALYSIS_EXPRESSION: Value = Value - val LAZY_OUTER_REFERENCE: Value = Value } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 4766a74308a1f..a74d93b44db98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -280,9 +280,9 @@ class Dataset[T] private[sql]( // The resolved `ExpressionEncoder` which can be used to turn rows to objects of type T, after // collecting rows to the driver side. - private lazy val resolvedEnc = { - exprEnc.resolveAndBind(logicalPlan.output, sparkSession.sessionState.analyzer) - } + private lazy val resolvedEnc = exprEnc.resolveAndBind( + queryExecution.commandExecuted.output, sparkSession.sessionState.analyzer) + private implicit def classTag: ClassTag[T] = encoder.clsTag @@ -996,16 +996,12 @@ class Dataset[T] private[sql]( /** @inheritdoc */ def scalar(): Column = { - Column(ExpressionColumnNode( - ScalarSubqueryExpr(SubExprUtils.removeLazyOuterReferences(logicalPlan), - hasExplicitOuterRefs = true))) + Column(ExpressionColumnNode(ScalarSubqueryExpr(logicalPlan))) } /** @inheritdoc */ def exists(): Column = { - Column(ExpressionColumnNode( - Exists(SubExprUtils.removeLazyOuterReferences(logicalPlan), - hasExplicitOuterRefs = true))) + Column(ExpressionColumnNode(Exists(logicalPlan))) } /** @inheritdoc */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 490184c93620a..5695ea57e7fbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -31,12 +31,11 @@ import org.apache.spark.internal.LogKeys.EXTENDED_EXPLAIN_GENERATOR import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, ExtendedExplainGenerator, Row, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.catalyst.analysis.{LazyExpression, UnsupportedOperationChecker} import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{AppendData, Command, CommandResult, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect, ReturnAnswer, Union} import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} -import org.apache.spark.sql.catalyst.trees.TreePattern.LAZY_ANALYSIS_EXPRESSION import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} @@ -69,7 +68,10 @@ class QueryExecution( // TODO: Move the planner an optimizer into here from SessionState. protected def planner = sparkSession.sessionState.planner - lazy val isLazyAnalysis: Boolean = logical.containsAnyPattern(LAZY_ANALYSIS_EXPRESSION) + lazy val isLazyAnalysis: Boolean = { + // Only check the main query as subquery expression can be resolved now with the main query. + logical.exists(_.expressions.exists(_.exists(_.isInstanceOf[LazyExpression]))) + } def assertAnalyzed(): Unit = { try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala index 35a815d83922d..5f2638655c37c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala @@ -30,7 +30,7 @@ case class PlanAdaptiveSubqueries( def apply(plan: SparkPlan): SparkPlan = { plan.transformAllExpressionsWithPruning( _.containsAnyPattern(SCALAR_SUBQUERY, IN_SUBQUERY, DYNAMIC_PRUNING_SUBQUERY)) { - case expressions.ScalarSubquery(_, _, exprId, _, _, _, _, _) => + case expressions.ScalarSubquery(_, _, exprId, _, _, _, _) => val subquery = SubqueryExec.createForScalarSubquery( s"subquery#${exprId.id}", subqueryMap(exprId.id)) execution.ScalarSubquery(subquery, exprId) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala index 64eacba1c6bf3..00e9a01f33c1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala @@ -88,9 +88,6 @@ private[sql] trait ColumnNodeToExpressionConverter extends (ColumnNode => Expres isDistinct = isDistinct, isInternal = isInternal) - case LazyOuterReference(nameParts, planId, _) => - convertLazyOuterReference(nameParts, planId) - case Alias(child, Seq(name), None, _) => expressions.Alias(apply(child), name)( nonInheritableMetadataKeys = Seq(Dataset.DATASET_ID_KEY, Dataset.COL_POS_KEY)) @@ -193,6 +190,9 @@ private[sql] trait ColumnNodeToExpressionConverter extends (ColumnNode => Expres case _ => transformed } + case l: LazyExpression => + analysis.LazyExpression(apply(l.child)) + case node => throw SparkException.internalError("Unsupported ColumnNode: " + node) } @@ -248,16 +248,6 @@ private[sql] trait ColumnNodeToExpressionConverter extends (ColumnNode => Expres } attribute } - - private def convertLazyOuterReference( - nameParts: Seq[String], - planId: Option[Long]): analysis.LazyOuterReference = { - val lazyOuterReference = analysis.LazyOuterReference(nameParts) - if (planId.isDefined) { - lazyOuterReference.setTagValue(LogicalPlan.PLAN_ID_TAG, planId.get) - } - lazyOuterReference - } } private[sql] object ColumnNodeToExpressionConverter extends ColumnNodeToExpressionConverter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index d656c36ce842a..2420ad34d9bab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -53,23 +53,15 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { r.createOrReplaceTempView("r") } - test("unanalyzable expression") { - val sub = spark.range(1).select($"id" === $"id".outer()) - - checkError( - intercept[AnalysisException](sub.schema), - condition = "UNANALYZABLE_EXPRESSION", - parameters = Map("expr" -> "\"outer(id)\""), - queryContext = - Array(ExpectedContext(fragment = "outer", callSitePattern = getCurrentClassCallSitePattern)) - ) - + test("noop outer()") { + checkAnswer(spark.range(1).select($"id".outer()), Row(0)) checkError( - intercept[AnalysisException](sub.encoder), - condition = "UNANALYZABLE_EXPRESSION", - parameters = Map("expr" -> "\"outer(id)\""), - queryContext = - Array(ExpectedContext(fragment = "outer", callSitePattern = getCurrentClassCallSitePattern)) + intercept[AnalysisException](spark.range(1).select($"outer_col".outer()).collect()), + "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`outer_col`", "proposal" -> "`id`"), + context = ExpectedContext( + fragment = "$", + callSitePattern = getCurrentClassCallSitePattern) ) } @@ -148,6 +140,64 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } } + test("correlated scalar subquery in SELECT with outer() function") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") + // We can use the `.outer()` function to wrap either the outer column, or the entire condition, + // or the SQL string of the condition. + Seq( + $"t1.a" === $"t2.a".outer(), + ($"t1.a" === $"t2.a").outer(), + expr("t1.a = t2.a").outer()).foreach { cond => + checkAnswer( + df1.select( + $"a", + df2.where(cond).select(sum($"b")).scalar().as("sum_b") + ), + sql("select a, (select sum(b) from l t1 where t1.a = t2.a) sum_b from l t2") + ) + } + } + + test("correlated scalar subquery in WHERE with outer() function") { + // We can use the `.outer()` function to wrap either the outer column, or the entire condition, + // or the SQL string of the condition. + Seq( + $"a".outer() === $"c", + ($"a" === $"c").outer(), + expr("a = c").outer()).foreach { cond => + checkAnswer( + spark.table("l").where( + $"b" < spark.table("r").where(cond).select(max($"d")).scalar() + ), + sql("select * from l where b < (select max(d) from r where a = c)") + ) + } + } + + test("EXISTS predicate subquery with outer() function") { + // We can use the `.outer()` function to wrap either the outer column, or the entire condition, + // or the SQL string of the condition. + Seq( + $"a".outer() === $"c", + ($"a" === $"c").outer(), + expr("a = c").outer()).foreach { cond => + checkAnswer( + spark.table("l").where( + spark.table("r").where(cond).exists() + ), + sql("select * from l where exists (select * from r where l.a = r.c)") + ) + + checkAnswer( + spark.table("l").where( + spark.table("r").where(cond).exists() && $"a" <= lit(2) + ), + sql("select * from l where exists (select * from r where l.a = r.c) and l.a <= 2") + ) + } + } + test("SPARK-15677: Queries against local relations with scalar subquery in Select list") { withTempView("t1", "t2") { Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") @@ -192,22 +242,6 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } } - test("EXISTS predicate subquery") { - checkAnswer( - spark.table("l").where( - spark.table("r").where($"a".outer() === $"c").exists() - ), - sql("select * from l where exists (select * from r where l.a = r.c)") - ) - - checkAnswer( - spark.table("l").where( - spark.table("r").where($"a".outer() === $"c").exists() && $"a" <= lit(2) - ), - sql("select * from l where exists (select * from r where l.a = r.c) and l.a <= 2") - ) - } - test("NOT EXISTS predicate subquery") { checkAnswer( spark.table("l").where( @@ -244,32 +278,15 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { ) } - test("correlated scalar subquery in where") { - checkAnswer( - spark.table("l").where( - $"b" < spark.table("r").where($"a".outer() === $"c").select(max($"d")).scalar() - ), - sql("select * from l where b < (select max(d) from r where a = c)") - ) - } - - test("correlated scalar subquery in select") { + test("correlated scalar subquery in select (null safe equal)") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") checkAnswer( - spark.table("l").select( + df1.select( $"a", - spark.table("l").where($"a" === $"a".outer()).select(sum($"b")).scalar().as("sum_b") + df2.where($"t2.a" <=> $"t1.a".outer()).select(sum($"b")).scalar().as("sum_b") ), - sql("select a, (select sum(b) from l l2 where l2.a = l1.a) sum_b from l l1") - ) - } - - test("correlated scalar subquery in select (null safe)") { - checkAnswer( - spark.table("l").select( - $"a", - spark.table("l").where($"a" <=> $"a".outer()).select(sum($"b")).scalar().as("sum_b") - ), - sql("select a, (select sum(b) from l l2 where l2.a <=> l1.a) sum_b from l l1") + sql("select a, (select sum(b) from l t2 where t2.a <=> t1.a) sum_b from l t1") ) } @@ -300,10 +317,12 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } test("non-aggregated correlated scalar subquery") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") val exception1 = intercept[SparkRuntimeException] { - spark.table("l").select( + df1.select( $"a", - spark.table("l").where($"a" === $"a".outer()).select($"b").scalar().as("sum_b") + df2.where($"t1.a" === $"t2.a".outer()).select($"b").scalar().as("sum_b") ).collect() } checkError( @@ -313,12 +332,14 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } test("non-equal correlated scalar subquery") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") checkAnswer( - spark.table("l").select( + df1.select( $"a", - spark.table("l").where($"a" < $"a".outer()).select(sum($"b")).scalar().as("sum_b") + df2.where($"t2.a" < $"t1.a".outer()).select(sum($"b")).scalar().as("sum_b") ), - sql("select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1") + sql("select a, (select sum(b) from l t2 where t2.a < t1.a) sum_b from l t1") ) } @@ -346,7 +367,7 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { spark.table("l").select( $"a", spark.table("r").where($"c" === $"a").select(sum($"d")).scalar() - ).collect() + ) } checkError( exception1, @@ -355,35 +376,5 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { queryContext = Array(ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern)) ) - - // Extra `outer()` - val exception2 = intercept[AnalysisException] { - spark.table("l").select( - $"a", - spark.table("r").where($"c".outer() === $"a".outer()).select(sum($"d")).scalar() - ).collect() - } - checkError( - exception2, - condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", - parameters = Map("objectName" -> "`c`", "proposal" -> "`a`, `b`"), - queryContext = - Array(ExpectedContext(fragment = "outer", callSitePattern = getCurrentClassCallSitePattern)) - ) - - // Missing `outer()` for another outer - val exception3 = intercept[AnalysisException] { - spark.table("l").select( - $"a", - spark.table("r").where($"b" === $"a".outer()).select(sum($"d")).scalar() - ).collect() - } - checkError( - exception3, - condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", - parameters = Map("objectName" -> "`b`", "proposal" -> "`c`, `d`"), - queryContext = - Array(ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern)) - ) } } From ce57a3d131545bd18661cc839d894f59a47047b8 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 28 Nov 2024 11:13:05 +0800 Subject: [PATCH 040/438] [SPARK-50067][SPARK-50066][SPARK-49954][SQL][FOLLOWUP] Make returnNullable as false ### What changes were proposed in this pull request? The pr is following up [schema_of_json](https://github.com/apache/spark/pull/48473), [schema_of_xml](https://github.com/apache/spark/pull/48594) and [schema_of_csv](https://github.com/apache/spark/pull/48595), to make returnNullable as false. ### Why are the changes needed? As `cloud-fan`'s comment https://github.com/apache/spark/pull/48594/files#r1860534460, we should follow the original logic, otherwise it's a regression. https://github.com/apache/spark/blob/1a502d32ef5a69739e10b827be4c9063b2a20493/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala#L846 https://github.com/apache/spark/blob/1a502d32ef5a69739e10b827be4c9063b2a20493/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala#L166 https://github.com/apache/spark/blob/1a502d32ef5a69739e10b827be4c9063b2a20493/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala#L141 ### 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 #48987 from panbingkun/SPARK-50066_FOLLOWUP. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/csvExpressions.scala | 3 ++- .../spark/sql/catalyst/expressions/jsonExpressions.scala | 4 ++-- .../spark/sql/catalyst/expressions/xmlExpressions.scala | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index 02e5488835c91..bf3d8eee9ed5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -173,7 +173,8 @@ case class SchemaOfCsv( "evaluate", dataType, Seq(child), - Seq(child.dataType)) + Seq(child.dataType), + returnNullable = false) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 0381d1e712aeb..68cce1c2a138b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -886,8 +886,8 @@ case class SchemaOfJson( Literal(jsonOptions, jsonOptionsObjectType), Literal(jsonInferSchema, jsonInferSchemaObjectType), child), - Seq(jsonFactoryObjectType, jsonOptionsObjectType, jsonInferSchemaObjectType, child.dataType) - ) + Seq(jsonFactoryObjectType, jsonOptionsObjectType, jsonInferSchemaObjectType, child.dataType), + returnNullable = false) override def prettyName: String = "schema_of_json" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala index 6f004cbce4262..66f7f25e4abe8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala @@ -208,8 +208,8 @@ case class SchemaOfXml( dataType, "schemaOfXml", Seq(Literal(xmlInferSchema, xmlInferSchemaObjectType), child), - Seq(xmlInferSchemaObjectType, child.dataType) - ) + Seq(xmlInferSchemaObjectType, child.dataType), + returnNullable = false) } /** From 68be1da5ba8e54a67eb61822cdf7f8eb47926319 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 28 Nov 2024 11:20:52 +0800 Subject: [PATCH 041/438] [SPARK-49566][SQL] Add SQL pipe syntax for the SET operator ### What changes were proposed in this pull request? This PR adds SQL pipe syntax support for SET operator. This operator removes one or more existing column from the input table and replaces each one with a new computed column whose value is equal to evaluating the specified expression. This is equivalent to `SELECT * EXCEPT (name), AS name` in the SQL compiler. It is provided as a convenience feature and some functionality overlap exists with lateral column aliases. For example: ``` -- Setting with an expression. values (0, 'pqr', 2), (3, 'tuv', 5) as tab(a, b, c) |> set c = a + length(b); 0, 'pqr', 3 3, 'tuv', 6 ``` ### Why are the changes needed? The SQL pipe operator syntax will let users compose queries in a more flexible fashion. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds a few unit test cases, but mostly relies on golden file test coverage. I did this to make sure the answers are correct as this feature is implemented and also so we can look at the analyzer output plans to ensure they look right as well. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48940 from dtenedor/pipe-syntax-set. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 10 + .../sql/catalyst/analysis/unresolved.scala | 34 ++- .../catalyst/expressions/pipeOperators.scala | 1 + .../sql/catalyst/parser/AstBuilder.scala | 49 ++++- .../analyzer-results/pipe-operators.sql.out | 203 ++++++++++++++++++ .../sql-tests/inputs/pipe-operators.sql | 79 +++++++ .../sql-tests/results/pipe-operators.sql.out | 195 +++++++++++++++++ .../sql/execution/SparkSqlParserSuite.scala | 1 + 8 files changed, 561 insertions(+), 11 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index cdee8c906054d..2e3235d6f932c 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -1504,6 +1504,7 @@ version operatorPipeRightSide : selectClause windowClause? | EXTEND extendList=namedExpressionSeq + | SET operatorPipeSetAssignmentSeq // Note that the WINDOW clause is not allowed in the WHERE pipe operator, but we add it here in // the grammar simply for purposes of catching this invalid syntax and throwing a specific // dedicated error message. @@ -1520,6 +1521,15 @@ operatorPipeRightSide | AGGREGATE namedExpressionSeq? aggregationClause? ; +operatorPipeSetAssignmentSeq + : ident+=errorCapturingIdentifier + (DOT errorCapturingIdentifier)* // This is invalid syntax; we just capture it here. + EQ expression + (COMMA ident+=errorCapturingIdentifier + (DOT errorCapturingIdentifier)* // This is invalid syntax; we just capture it here. + EQ expression)* + ; + // When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index f366339d95c0c..7fc8aff72b81d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -462,13 +462,13 @@ abstract class Star extends LeafExpression with NamedExpression { * "SELECT record.* from (SELECT struct(a,b,c) as record ...) * * @param target an optional name that should be the target of the expansion. If omitted all - * targets' columns are produced. This can either be a table name or struct name. This - * is a list of identifiers that is the path of the expansion. + * targets' columns are produced. This can either be a table name or struct name. This + * is a list of identifiers that is the path of the expansion. * * This class provides the shared behavior between the classes for SELECT * ([[UnresolvedStar]]) - * and SELECT * EXCEPT ([[UnresolvedStarExcept]]). [[UnresolvedStar]] is just a case class of this, - * while [[UnresolvedStarExcept]] adds some additional logic to the expand method. - */ + * and SELECT * EXCEPT ([[UnresolvedStarExceptOrReplace]]). [[UnresolvedStar]] is just a case class + * of this, while [[UnresolvedStarExceptOrReplace]] adds some additional logic to the expand method. + */ abstract class UnresolvedStarBase(target: Option[Seq[String]]) extends Star with Unevaluable { /** * Returns true if the nameParts is a subset of the last elements of qualifier of the attribute. @@ -571,8 +571,15 @@ abstract class UnresolvedStarBase(target: Option[Seq[String]]) extends Star with * * @param excepts a list of names that should be excluded from the expansion. * + * @param replacements an optional list of expressions that should be used to replace the + * expressions removed by EXCEPT. If present, the length of this list must + * be the same as the length of the EXCEPT list. This supports replacing + * expressions instead of excluding them from the original SELECT list. */ -case class UnresolvedStarExcept(target: Option[Seq[String]], excepts: Seq[Seq[String]]) +case class UnresolvedStarExceptOrReplace( + target: Option[Seq[String]], + excepts: Seq[Seq[String]], + replacements: Option[Seq[NamedExpression]]) extends UnresolvedStarBase(target) { /** @@ -652,7 +659,14 @@ case class UnresolvedStarExcept(target: Option[Seq[String]], excepts: Seq[Seq[St // group the except pairs by the column they refer to. NOTE: no groupMap until scala 2.13 val groupedExcepts: AttributeMap[Seq[Seq[String]]] = AttributeMap(excepts.groupBy(_._1.toAttribute).transform((_, v) => v.map(_._2))) - + // If the 'replacements' list is populated to indicate we should replace excepted columns + // with new expressions, we must have the same number of replacements as excepts. Keep an + // index to track the current replacement. + replacements.foreach { r => + assert(excepts.size == r.size, + "The number of replacements must be the same as the number of excepts") + } + var replacementIndex = 0 // map input columns while searching for the except entry corresponding to the current column columns.map(col => col -> groupedExcepts.get(col.toAttribute)).collect { // pass through columns that don't match anything in groupedExcepts @@ -679,11 +693,15 @@ case class UnresolvedStarExcept(target: Option[Seq[String]], excepts: Seq[Seq[St filterColumns(extractedFields.toImmutableArraySeq, newExcepts)), col.name)() // if there are multiple nestedExcepts but one is empty we must have overlapping except // columns. throw an error. - case (col, Some(nestedExcepts)) if nestedExcepts.size > 1 => + case (_, Some(nestedExcepts)) if nestedExcepts.size > 1 => throw new AnalysisException( errorClass = "EXCEPT_OVERLAPPING_COLUMNS", messageParameters = Map( "columns" -> this.excepts.map(_.mkString(".")).mkString(", "))) + // found a match and the 'replacements' list is populated - replace the column + case (_, Some(_)) if replacements.nonEmpty => + replacementIndex += 1 + replacements.get(replacementIndex - 1) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala index 1b5ee54729136..fe8f0f264e85f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala @@ -67,6 +67,7 @@ object PipeOperators { val offsetClause = "OFFSET" val orderByClause = "ORDER BY" val selectClause = "SELECT" + val setClause = "SET" val sortByClause = "SORT BY" val sortByDistributeByClause = "SORT BY ... DISTRIBUTE BY ..." val windowClause = "WINDOW" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 08a8cf6bab87a..6a9a97d0f5c8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2275,9 +2275,10 @@ class AstBuilder extends DataTypeAstBuilder def visitStarExcept(ctx: StarContext, target: Option[Seq[String]]): Expression = withOrigin(ctx) { val exceptCols = ctx.exceptClause .exceptCols.multipartIdentifier.asScala.map(typedVisit[Seq[String]]) - UnresolvedStarExcept( + UnresolvedStarExceptOrReplace( target, - exceptCols.toSeq) + exceptCols.toSeq, + replacements = None) } /** @@ -5952,6 +5953,8 @@ class AstBuilder extends DataTypeAstBuilder }.get val projectList: Seq[NamedExpression] = Seq(UnresolvedStar(None)) ++ extendExpressions Project(projectList, left) + }.getOrElse(Option(ctx.SET).map { _ => + visitOperatorPipeSet(ctx, left) }.getOrElse(Option(ctx.whereClause).map { c => if (ctx.windowClause() != null) { throw QueryParsingErrors.windowClauseInPipeOperatorWhereClauseNotAllowedError(ctx) @@ -5978,7 +5981,47 @@ class AstBuilder extends DataTypeAstBuilder withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true) }.getOrElse( visitOperatorPipeAggregate(ctx, left) - ))))))))) + )))))))))) + } + + private def visitOperatorPipeSet( + ctx: OperatorPipeRightSideContext, left: LogicalPlan): LogicalPlan = { + val (setIdentifiers: Seq[String], setTargets: Seq[Expression]) = + visitOperatorPipeSetAssignmentSeq(ctx.operatorPipeSetAssignmentSeq()) + var plan = left + setIdentifiers.zip(setTargets).foreach { + case (_, _: Alias) => + operationNotAllowed( + "SQL pipe syntax |> SET operator with an alias assigned with [AS] aliasName", ctx) + case (ident, target) => + // Add an UnresolvedStarExceptOrReplace to exclude the SET expression name from the relation + // and add the new SET expression to the projection list. + // Use a PipeSelect expression to make sure it does not contain any aggregate functions. + val replacement = + Alias(PipeExpression(target, isAggregate = false, PipeOperators.setClause), ident)() + val projectList: Seq[NamedExpression] = + Seq(UnresolvedStarExceptOrReplace( + target = None, excepts = Seq(Seq(ident)), replacements = Some(Seq(replacement)))) + // Add a projection to implement the SET operator using the UnresolvedStarExceptOrReplace + // expression. We do this once per SET assignment to allow for multiple SET assignments with + // optional lateral references to previous ones. + plan = Project(projectList, plan) + } + plan + } + + override def visitOperatorPipeSetAssignmentSeq( + ctx: OperatorPipeSetAssignmentSeqContext): (Seq[String], Seq[Expression]) = { + withOrigin(ctx) { + if (!ctx.DOT.isEmpty) { + operationNotAllowed( + s"SQL pipe syntax |> SET operator with multi-part assignment key " + + s"(only single-part keys are allowed)", ctx) + } + val setIdentifiers: Seq[String] = ctx.errorCapturingIdentifier().asScala.map(_.getText).toSeq + val setTargets: Seq[Expression] = ctx.expression().asScala.map(typedVisit[Expression]).toSeq + (setIdentifiers, setTargets) + } } private def visitOperatorPipeAggregate( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index bc5b642e3da09..a4cda92fab2e3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -731,6 +731,209 @@ org.apache.spark.sql.AnalysisException } +-- !query +table t +|> set x = 1 +-- !query analysis +Project [pipeexpression(1, false, SET) AS x#x, y#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> set y = x +-- !query analysis +Project [x#x, pipeexpression(x#x, false, SET) AS y#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y) +-- !query analysis +Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x] ++- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = x + 1 +-- !query analysis +Project [x#x, y#x, z#x, pipeexpression((x#x + 1), false, SET) AS zz#x] ++- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x, zz#x] + +- Project [x#x, y#x, z#x, pipeexpression(2, false, EXTEND) AS zz#x] + +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table other +|> extend 3 as c +|> set a = b, b = c +-- !query analysis +Project [a#x, pipeexpression(c#x, false, SET) AS b#x, c#x] ++- Project [pipeexpression(b#x, false, SET) AS a#x, b#x, c#x] + +- Project [a#x, b#x, pipeexpression(3, false, EXTEND) AS c#x] + +- SubqueryAlias spark_catalog.default.other + +- Relation spark_catalog.default.other[a#x,b#x] json + + +-- !query +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = z + 1 +-- !query analysis +Project [x#x, y#x, z#x, pipeexpression((z#x + 1), false, SET) AS zz#x] ++- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x, zz#x] + +- Project [x#x, y#x, z#x, pipeexpression(2, false, EXTEND) AS zz#x] + +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y) +|> set z = z + 1 +-- !query analysis +Project [x#x, y#x, pipeexpression((z#x + 1), false, SET) AS z#x] ++- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x] + +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y), z = z + 1 +-- !query analysis +Project [x#x, y#x, pipeexpression((z#x + 1), false, SET) AS z#x] ++- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x] + +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +select col from st +|> extend 1 as z +|> set z = col.i1 +-- !query analysis +Project [col#x, pipeexpression(col#x.i1, false, SET) AS z#x] ++- Project [col#x, pipeexpression(1, false, EXTEND) AS z#x] + +- Project [col#x] + +- SubqueryAlias spark_catalog.default.st + +- Relation spark_catalog.default.st[x#x,col#x] parquet + + +-- !query +table t +|> set y = (select a from other where x = a limit 1) +-- !query analysis +Project [x#x, pipeexpression(scalar-subquery#x [x#x], false, SET) AS y#x] +: +- GlobalLimit 1 +: +- LocalLimit 1 +: +- Project [a#x] +: +- Filter (outer(x#x) = a#x) +: +- SubqueryAlias spark_catalog.default.other +: +- Relation spark_catalog.default.other[a#x,b#x] json ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as `x.y.z` +|> set `x.y.z` = x + length(y) +-- !query analysis +Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS x.y.z#x] ++- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS x.y.z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as z +|> set z = first_value(x) over (partition by y) +-- !query analysis +Project [x#x, y#x, z#x] ++- Project [x#x, y#x, _we0#x, pipeexpression(_we0#x, false, SET) AS z#x] + +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#x], [y#x] + +- Project [x#x, y#x] + +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> set z = 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`z`", + "proposal" : "`x`, `y`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 20, + "fragment" : "table t\n|> set z = 1" + } ] +} + + +-- !query +table t +|> set x = 1 as z +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'as'", + "hint" : "" + } +} + + +-- !query +select col from st +|> set col.i1 = 42 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "SQL pipe syntax |> SET operator with multi-part assignment key (only single-part keys are allowed)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 37, + "fragment" : "col.i1 = 42" + } ] +} + + -- !query table t |> where true diff --git a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql index b9224db129ea4..6d0e490649d6f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql @@ -241,6 +241,85 @@ table t table t |> extend *; +-- SET operators: positive tests. +--------------------------------- + +-- Setting with a constant. +-- The indicated column is not the last column in the table, and the SET operator will replace it +-- with the new value in its existing position. +table t +|> set x = 1; + +-- Setting with an attribute. +table t +|> set y = x; + +-- Setting with an expression. +table t +|> extend 1 as z +|> set z = x + length(y); + +-- Setting two times. +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = x + 1; + +table other +|> extend 3 as c +|> set a = b, b = c; + +-- Setting two times with a lateral reference. +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = z + 1; + +-- Setting two times in sequence. +table t +|> extend 1 as z +|> set z = x + length(y) +|> set z = z + 1; + +-- SET assignments with duplicate keys. This is supported, and we can update the column as we go. +table t +|> extend 1 as z +|> set z = x + length(y), z = z + 1; + +-- Setting with a struct field. +select col from st +|> extend 1 as z +|> set z = col.i1; + +-- Setting with a subquery. +table t +|> set y = (select a from other where x = a limit 1); + +-- Setting with a backquoted column name with a dot inside. +table t +|> extend 1 as `x.y.z` +|> set `x.y.z` = x + length(y); + +-- Window functions are allowed in the pipe operator SET list. +table t +|> extend 1 as z +|> set z = first_value(x) over (partition by y); + +-- SET operators: negative tests. +--------------------------------- + +-- SET with a column name that does not exist in the input relation. +table t +|> set z = 1; + +-- SET with an alias. +table t +|> set x = 1 as z; + +-- Setting nested fields in structs is not supported. +select col from st +|> set col.i1 = 42; + -- WHERE operators: positive tests. ----------------------------------- diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index 53aabce4d5abc..85a411f60fe22 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -678,6 +678,201 @@ org.apache.spark.sql.AnalysisException } +-- !query +table t +|> set x = 1 +-- !query schema +struct +-- !query output +1 abc +1 def + + +-- !query +table t +|> set y = x +-- !query schema +struct +-- !query output +0 0 +1 1 + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y) +-- !query schema +struct +-- !query output +0 abc 3 +1 def 4 + + +-- !query +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = x + 1 +-- !query schema +struct +-- !query output +0 abc 3 1 +1 def 4 2 + + +-- !query +table other +|> extend 3 as c +|> set a = b, b = c +-- !query schema +struct +-- !query output +1 3 3 +2 3 3 +4 3 3 + + +-- !query +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = z + 1 +-- !query schema +struct +-- !query output +0 abc 3 4 +1 def 4 5 + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y) +|> set z = z + 1 +-- !query schema +struct +-- !query output +0 abc 4 +1 def 5 + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y), z = z + 1 +-- !query schema +struct +-- !query output +0 abc 4 +1 def 5 + + +-- !query +select col from st +|> extend 1 as z +|> set z = col.i1 +-- !query schema +struct,z:int> +-- !query output +{"i1":2,"i2":3} 2 + + +-- !query +table t +|> set y = (select a from other where x = a limit 1) +-- !query schema +struct +-- !query output +0 NULL +1 1 + + +-- !query +table t +|> extend 1 as `x.y.z` +|> set `x.y.z` = x + length(y) +-- !query schema +struct +-- !query output +0 abc 3 +1 def 4 + + +-- !query +table t +|> extend 1 as z +|> set z = first_value(x) over (partition by y) +-- !query schema +struct +-- !query output +0 abc 0 +1 def 1 + + +-- !query +table t +|> set z = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`z`", + "proposal" : "`x`, `y`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 20, + "fragment" : "table t\n|> set z = 1" + } ] +} + + +-- !query +table t +|> set x = 1 as z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'as'", + "hint" : "" + } +} + + +-- !query +select col from st +|> set col.i1 = 42 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "SQL pipe syntax |> SET operator with multi-part assignment key (only single-part keys are allowed)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 37, + "fragment" : "col.i1 = 42" + } ] +} + + -- !query table t |> where true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 03d6eb1a50209..36a003883a771 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -900,6 +900,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { checkPipeSelect("VALUES (0), (1) tab(col) |> SELECT col * 2 AS result") checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y") checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y, X + 2 Z") + checkPipeSelect("TABLE t |> EXTEND 1 AS z, 2 AS Z |> SET z = 1, Z = 2") // Basic WHERE operators. def checkPipeWhere(query: String): Unit = check(query, Seq(FILTER)) checkPipeWhere("TABLE t |> WHERE X = 1") From 0c31f5a807e7aa01cd46424d52441f514e491943 Mon Sep 17 00:00:00 2001 From: Eric Marnadi Date: Thu, 28 Nov 2024 12:48:24 +0900 Subject: [PATCH 042/438] [SPARK-50443][SS] Fixing Maven build errors introduced by Guava cache in RocksDBStateStoreProvider ### What changes were proposed in this pull request? There are maven errors introduced by the guava dependency in `sql/core`, as we use the Guava cache to store the Avro encoders, outlined in this comment: https://github.com/apache/spark/pull/48401#issuecomment-2504353098 Introduced a new constructor for the NonFateSharingCache and used this with the RocksDBStateStoreProvider. ### Why are the changes needed? To resolve maven build errors, so that the Avro change here: https://github.com/apache/spark/pull/48401 does not get reverted. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing unit tests are sufficient and maven build works on devbox ``` [INFO] Tests run: 47, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 17.64 s -- in test.org.apache.spark.sql.JavaDatasetSuite [INFO] [INFO] Results: [INFO] [INFO] Tests run: 47, Failures: 0, Errors: 0, Skipped: 0 [INFO] [INFO] [INFO] --- surefire:3.2.5:test (test) spark-sql_2.13 --- [INFO] Skipping execution of surefire because it has already been run for this configuration [INFO] [INFO] --- scalatest:2.2.0:test (test) spark-sql_2.13 --- [INFO] ScalaTest report directory: /home/eric.marnadi/spark/sql/core/target/surefire-reports WARNING: Using incubator modules: jdk.incubator.vector Discovery starting. Discovery completed in 2 seconds, 737 milliseconds. Run starting. Expected test count is: 0 DiscoverySuite: Run completed in 2 seconds, 765 milliseconds. Total number of tests run: 0 Suites: completed 1, aborted 0 Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0 No tests were executed. [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 03:15 min [INFO] Finished at: 2024-11-28T01:10:36Z [INFO] ------------------------------------------------------------------------ ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48996 from ericm-db/chm. Authored-by: Eric Marnadi Signed-off-by: Jungtaek Lim --- .../apache/spark/util/NonFateSharingCache.scala | 16 +++++++++++++++- .../state/RocksDBStateStoreProvider.scala | 17 ++++++++--------- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala b/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala index 21184d70b386a..7d01facc1e421 100644 --- a/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala +++ b/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.util.concurrent.Callable +import java.util.concurrent.{Callable, TimeUnit} import com.google.common.cache.{Cache, CacheBuilder, CacheLoader, LoadingCache} @@ -68,6 +68,20 @@ private[spark] object NonFateSharingCache { override def load(k: K): V = loadingFunc.apply(k) })) } + + def apply[K, V]( + maximumSize: Long, + expireAfterAccessTime: Long, + expireAfterAccessTimeUnit: TimeUnit): NonFateSharingCache[K, V] = { + val builder = CacheBuilder.newBuilder().asInstanceOf[CacheBuilder[K, V]] + if (maximumSize > 0L) { + builder.maximumSize(maximumSize) + } + if(expireAfterAccessTime > 0) { + builder.expireAfterAccess(expireAfterAccessTime, expireAfterAccessTimeUnit) + } + new NonFateSharingCache(builder.build[K, V]()) + } } private[spark] class NonFateSharingCache[K, V](protected val cache: Cache[K, V]) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index e5a4175aeec1a..c9c987fa1620d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -23,7 +23,6 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.util.control.NonFatal -import com.google.common.cache.CacheBuilder import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -613,15 +612,15 @@ object RocksDBStateStoreProvider { val VIRTUAL_COL_FAMILY_PREFIX_BYTES = 2 private val MAX_AVRO_ENCODERS_IN_CACHE = 1000 - // Add the cache at companion object level so it persists across provider instances - private val avroEncoderMap: NonFateSharingCache[String, AvroEncoder] = { - val guavaCache = CacheBuilder.newBuilder() - .maximumSize(MAX_AVRO_ENCODERS_IN_CACHE) // Adjust size based on your needs - .expireAfterAccess(1, TimeUnit.HOURS) // Optional: Add expiration if needed - .build[String, AvroEncoder]() + private val AVRO_ENCODER_LIFETIME_HOURS = 1L - new NonFateSharingCache(guavaCache) - } + // Add the cache at companion object level so it persists across provider instances + private val avroEncoderMap: NonFateSharingCache[String, AvroEncoder] = + NonFateSharingCache( + maximumSize = MAX_AVRO_ENCODERS_IN_CACHE, + expireAfterAccessTime = AVRO_ENCODER_LIFETIME_HOURS, + expireAfterAccessTimeUnit = TimeUnit.HOURS + ) def getAvroEnc( stateStoreEncoding: String, From 2a61dd48a45b13983b3e267311e808b5d5e63906 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 28 Nov 2024 11:54:12 +0800 Subject: [PATCH 043/438] [SPARK-50445][PYTHON][DOCS] Refine docString of generator Functions ### What changes were proposed in this pull request? Refine docString of generator Functions ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? new doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48998 from zhengruifeng/py_doc_11. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/builtin.py | 542 +++++++++++++----------- 1 file changed, 289 insertions(+), 253 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index ac4f2fc9881b0..11011c2f30252 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -18710,7 +18710,7 @@ def explode(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name Target column to work on. Returns @@ -18723,6 +18723,8 @@ def explode(col: "ColumnOrName") -> Column: :meth:`pyspark.sql.functions.posexplode` :meth:`pyspark.sql.functions.explode_outer` :meth:`pyspark.sql.functions.posexplode_outer` + :meth:`pyspark.sql.functions.inline` + :meth:`pyspark.sql.functions.inline_outer` Notes ----- @@ -18732,119 +18734,79 @@ def explode(col: "ColumnOrName") -> Column: -------- Example 1: Exploding an array column - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(id=1, values=[1, 2, 3])]) - >>> df.select(sf.explode(df.values).alias("value")).show() - +-----+ - |value| - +-----+ - | 1| - | 2| - | 3| - +-----+ + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,ARRAY(1,2,3,NULL)), (2,ARRAY()), (3,NULL) AS t(i,a)') + >>> df.show() + +---+---------------+ + | i| a| + +---+---------------+ + | 1|[1, 2, 3, NULL]| + | 2| []| + | 3| NULL| + +---+---------------+ + + >>> df.select('*', sf.explode('a')).show() + +---+---------------+----+ + | i| a| col| + +---+---------------+----+ + | 1|[1, 2, 3, NULL]| 1| + | 1|[1, 2, 3, NULL]| 2| + | 1|[1, 2, 3, NULL]| 3| + | 1|[1, 2, 3, NULL]|NULL| + +---+---------------+----+ Example 2: Exploding a map column - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(id=1, values={"a": "b", "c": "d"})]) - >>> df.select(sf.explode(df.values).alias("key", "value")).show() - +---+-----+ - |key|value| - +---+-----+ - | a| b| - | c| d| - +---+-----+ - - Example 3: Exploding an array column with multiple rows - - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame( - ... [Row(id=1, values=[1, 2]), Row(id=2, values=[3, 4])]) - >>> df.select("id", sf.explode(df.values).alias("value")).show() - +---+-----+ - | id|value| - +---+-----+ - | 1| 1| - | 1| 2| - | 2| 3| - | 2| 4| - +---+-----+ - - Example 4: Exploding a map column with multiple rows - - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([ - ... Row(id=1, values={"a": "b", "c": "d"}), - ... Row(id=2, values={"e": "f", "g": "h"}) - ... ]) - >>> df.select("id", sf.explode(df.values).alias("key", "value")).show() - +---+---+-----+ - | id|key|value| - +---+---+-----+ - | 1| a| b| - | 1| c| d| - | 2| e| f| - | 2| g| h| - +---+---+-----+ - - Example 5: Exploding multiple array columns + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,MAP(1,2,3,4,5,NULL)), (2,MAP()), (3,NULL) AS t(i,m)') + >>> df.show(truncate=False) + +---+---------------------------+ + |i |m | + +---+---------------------------+ + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}| + |2 |{} | + |3 |NULL | + +---+---------------------------+ - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(a=1, list1=[1, 2], list2=[3, 4])]) - >>> df.select(sf.explode(df.list1).alias("list1"), "list2") \\ - ... .select("list1", sf.explode(df.list2).alias("list2")).show() - +-----+-----+ - |list1|list2| - +-----+-----+ - | 1| 3| - | 1| 4| - | 2| 3| - | 2| 4| - +-----+-----+ + >>> df.select('*', sf.explode('m')).show(truncate=False) + +---+---------------------------+---+-----+ + |i |m |key|value| + +---+---------------------------+---+-----+ + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|1 |2 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|3 |4 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|5 |NULL | + +---+---------------------------+---+-----+ - Example 6: Exploding an array of struct column + Example 3: Exploding multiple array columns >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame( - ... [(1, [(1, 2), (3, 4)])], - ... "id: int, structlist: array>") - >>> df = df.select(sf.explode(df.structlist).alias("struct")) - >>> df.select("struct.*").show() + >>> df = spark.sql('SELECT ARRAY(1,2) AS a1, ARRAY(3,4,5) AS a2') + >>> df.select( + ... '*', sf.explode('a1').alias('v1') + ... ).select('*', sf.explode('a2').alias('v2')).show() + +------+---------+---+---+ + | a1| a2| v1| v2| + +------+---------+---+---+ + |[1, 2]|[3, 4, 5]| 1| 3| + |[1, 2]|[3, 4, 5]| 1| 4| + |[1, 2]|[3, 4, 5]| 1| 5| + |[1, 2]|[3, 4, 5]| 2| 3| + |[1, 2]|[3, 4, 5]| 2| 4| + |[1, 2]|[3, 4, 5]| 2| 5| + +------+---------+---+---+ + + Example 4: Exploding an array of struct column + + >>> import pyspark.sql.functions as sf + >>> df = spark.sql('SELECT ARRAY(NAMED_STRUCT("a",1,"b",2), NAMED_STRUCT("a",3,"b",4)) AS a') + >>> df.select(sf.explode('a').alias("s")).select("s.*").show() +---+---+ | a| b| +---+---+ | 1| 2| | 3| 4| +---+---+ - - Example 7: Exploding an empty array column - - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([(1, [])], "id: int, values: array") - >>> df.select(sf.explode(df.values).alias("value")).show() - +-----+ - |value| - +-----+ - +-----+ - - Example 8: Exploding an empty map column - - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([(1, {})], "id: int, values: map") - >>> df.select(sf.explode(df.values).alias("key", "value")).show() - +---+-----+ - |key|value| - +---+-----+ - +---+-----+ - """ + """ # noqa: E501 return _invoke_function_over_columns("explode", col) @@ -18862,7 +18824,7 @@ def posexplode(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -18870,20 +18832,61 @@ def posexplode(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` one row per array item or map key value including positions as a separate column. + See Also + -------- + :meth:`pyspark.sql.functions.explode` + :meth:`pyspark.sql.functions.explode_outer` + :meth:`pyspark.sql.functions.posexplode_outer` + :meth:`pyspark.sql.functions.inline` + :meth:`pyspark.sql.functions.inline_outer` + Examples -------- - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})]) - >>> df.select(posexplode(df.intlist)).collect() - [Row(pos=0, col=1), Row(pos=1, col=2), Row(pos=2, col=3)] + Example 1: Exploding an array column - >>> df.select(posexplode(df.mapfield)).show() - +---+---+-----+ - |pos|key|value| - +---+---+-----+ - | 0| a| b| - +---+---+-----+ - """ + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,ARRAY(1,2,3,NULL)), (2,ARRAY()), (3,NULL) AS t(i,a)') + >>> df.show() + +---+---------------+ + | i| a| + +---+---------------+ + | 1|[1, 2, 3, NULL]| + | 2| []| + | 3| NULL| + +---+---------------+ + + >>> df.select('*', sf.posexplode('a')).show() + +---+---------------+---+----+ + | i| a|pos| col| + +---+---------------+---+----+ + | 1|[1, 2, 3, NULL]| 0| 1| + | 1|[1, 2, 3, NULL]| 1| 2| + | 1|[1, 2, 3, NULL]| 2| 3| + | 1|[1, 2, 3, NULL]| 3|NULL| + +---+---------------+---+----+ + + Example 2: Exploding a map column + + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,MAP(1,2,3,4,5,NULL)), (2,MAP()), (3,NULL) AS t(i,m)') + >>> df.show(truncate=False) + +---+---------------------------+ + |i |m | + +---+---------------------------+ + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}| + |2 |{} | + |3 |NULL | + +---+---------------------------+ + + >>> df.select('*', sf.posexplode('m')).show(truncate=False) + +---+---------------------------+---+---+-----+ + |i |m |pos|key|value| + +---+---------------------------+---+---+-----+ + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|0 |1 |2 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|1 |3 |4 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|2 |5 |NULL | + +---+---------------------------+---+---+-----+ + """ # noqa: E501 return _invoke_function_over_columns("posexplode", col) @@ -18899,7 +18902,7 @@ def inline(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name Input column of values to explode. Returns @@ -18910,6 +18913,9 @@ def inline(col: "ColumnOrName") -> Column: See Also -------- :meth:`pyspark.sql.functions.explode` + :meth:`pyspark.sql.functions.explode_outer` + :meth:`pyspark.sql.functions.posexplode` + :meth:`pyspark.sql.functions.posexplode_outer` :meth:`pyspark.sql.functions.inline_outer` Examples @@ -18917,102 +18923,89 @@ def inline(col: "ColumnOrName") -> Column: Example 1: Using inline with a single struct array column >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(structlist=[Row(a=1, b=2), Row(a=3, b=4)])]) - >>> df.select(sf.inline(df.structlist)).show() - +---+---+ - | a| b| - +---+---+ - | 1| 2| - | 3| 4| - +---+---+ + >>> df = spark.sql('SELECT ARRAY(NAMED_STRUCT("a",1,"b",2), NAMED_STRUCT("a",3,"b",4)) AS a') + >>> df.select('*', sf.inline(df.a)).show() + +----------------+---+---+ + | a| a| b| + +----------------+---+---+ + |[{1, 2}, {3, 4}]| 1| 2| + |[{1, 2}, {3, 4}]| 3| 4| + +----------------+---+---+ Example 2: Using inline with a column name >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(structlist=[Row(a=1, b=2), Row(a=3, b=4)])]) - >>> df.select(sf.inline("structlist")).show() - +---+---+ - | a| b| - +---+---+ - | 1| 2| - | 3| 4| - +---+---+ + >>> df = spark.sql('SELECT ARRAY(NAMED_STRUCT("a",1,"b",2), NAMED_STRUCT("a",3,"b",4)) AS a') + >>> df.select('*', sf.inline('a')).show() + +----------------+---+---+ + | a| a| b| + +----------------+---+---+ + |[{1, 2}, {3, 4}]| 1| 2| + |[{1, 2}, {3, 4}]| 3| 4| + +----------------+---+---+ Example 3: Using inline with an alias >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(structlist=[Row(a=1, b=2), Row(a=3, b=4)])]) - >>> df.select(sf.inline("structlist").alias("c1", "c2")).show() - +---+---+ - | c1| c2| - +---+---+ - | 1| 2| - | 3| 4| - +---+---+ + >>> df = spark.sql('SELECT ARRAY(NAMED_STRUCT("a",1,"b",2), NAMED_STRUCT("a",3,"b",4)) AS a') + >>> df.select('*', sf.inline('a').alias("c1", "c2")).show() + +----------------+---+---+ + | a| c1| c2| + +----------------+---+---+ + |[{1, 2}, {3, 4}]| 1| 2| + |[{1, 2}, {3, 4}]| 3| 4| + +----------------+---+---+ Example 4: Using inline with multiple struct array columns >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([ - ... Row(structlist1=[Row(a=1, b=2), Row(a=3, b=4)], - ... structlist2=[Row(c=5, d=6), Row(c=7, d=8)]) - ... ]) - >>> df.select(sf.inline("structlist1"), "structlist2") \\ - ... .select("a", "b", sf.inline("structlist2")).show() - +---+---+---+---+ - | a| b| c| d| - +---+---+---+---+ - | 1| 2| 5| 6| - | 1| 2| 7| 8| - | 3| 4| 5| 6| - | 3| 4| 7| 8| - +---+---+---+---+ + >>> df = spark.sql('SELECT ARRAY(NAMED_STRUCT("a",1,"b",2), NAMED_STRUCT("a",3,"b",4)) AS a1, ARRAY(NAMED_STRUCT("c",5,"d",6), NAMED_STRUCT("c",7,"d",8)) AS a2') + >>> df.select( + ... '*', sf.inline('a1') + ... ).select('*', sf.inline('a2')).show() + +----------------+----------------+---+---+---+---+ + | a1| a2| a| b| c| d| + +----------------+----------------+---+---+---+---+ + |[{1, 2}, {3, 4}]|[{5, 6}, {7, 8}]| 1| 2| 5| 6| + |[{1, 2}, {3, 4}]|[{5, 6}, {7, 8}]| 1| 2| 7| 8| + |[{1, 2}, {3, 4}]|[{5, 6}, {7, 8}]| 3| 4| 5| 6| + |[{1, 2}, {3, 4}]|[{5, 6}, {7, 8}]| 3| 4| 7| 8| + +----------------+----------------+---+---+---+---+ Example 5: Using inline with a nested struct array column >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([ - ... Row(structlist=Row(a=1, b=2, nested=[Row(c=3, d=4), Row(c=5, d=6)])) - ... ]) - >>> df.select(sf.inline("structlist.nested")).show() - +---+---+ - | c| d| - +---+---+ - | 3| 4| - | 5| 6| - +---+---+ + >>> df = spark.sql('SELECT NAMED_STRUCT("a",1,"b",2,"c",ARRAY(NAMED_STRUCT("c",3,"d",4), NAMED_STRUCT("c",5,"d",6))) AS s') + >>> df.select('*', sf.inline('s.c')).show(truncate=False) + +------------------------+---+---+ + |s |c |d | + +------------------------+---+---+ + |{1, 2, [{3, 4}, {5, 6}]}|3 |4 | + |{1, 2, [{3, 4}, {5, 6}]}|5 |6 | + +------------------------+---+---+ - Example 6: Using inline with an empty struct array column + Example 6: Using inline with a column containing: array continaing null, empty array and null - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame( - ... [Row(structlist=[])], "structlist: array>") - >>> df.select(sf.inline(df.structlist)).show() - +---+---+ - | a| b| - +---+---+ - +---+---+ - - Example 7: Using inline with a struct array column containing null values + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,ARRAY(NAMED_STRUCT("a",1,"b",2), NULL, NAMED_STRUCT("a",3,"b",4))), (2,ARRAY()), (3,NULL) AS t(i,s)') + >>> df.show(truncate=False) + +---+----------------------+ + |i |s | + +---+----------------------+ + |1 |[{1, 2}, NULL, {3, 4}]| + |2 |[] | + |3 |NULL | + +---+----------------------+ - >>> import pyspark.sql.functions as sf - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([Row(structlist=[Row(a=1, b=2), None, Row(a=3, b=4)])]) - >>> df.select(sf.inline(df.structlist)).show() - +----+----+ - | a| b| - +----+----+ - | 1| 2| - |NULL|NULL| - | 3| 4| - +----+----+ - """ + >>> df.select('*', sf.inline('s')).show(truncate=False) + +---+----------------------+----+----+ + |i |s |a |b | + +---+----------------------+----+----+ + |1 |[{1, 2}, NULL, {3, 4}]|1 |2 | + |1 |[{1, 2}, NULL, {3, 4}]|NULL|NULL| + |1 |[{1, 2}, NULL, {3, 4}]|3 |4 | + +---+----------------------+----+----+ + """ # noqa: E501 return _invoke_function_over_columns("inline", col) @@ -19031,7 +19024,7 @@ def explode_outer(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -19039,31 +19032,47 @@ def explode_outer(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` one row per array item or map key value. + See Also + -------- + :meth:`pyspark.sql.functions.explode` + :meth:`pyspark.sql.functions.posexplode` + :meth:`pyspark.sql.functions.posexplode_outer` + :meth:`pyspark.sql.functions.inline` + :meth:`pyspark.sql.functions.inline_outer` + Examples -------- - >>> df = spark.createDataFrame( - ... [(1, ["foo", "bar"], {"x": 1.0}), (2, [], {}), (3, None, None)], - ... ("id", "an_array", "a_map") - ... ) - >>> df.select("id", "an_array", explode_outer("a_map")).show() - +---+----------+----+-----+ - | id| an_array| key|value| - +---+----------+----+-----+ - | 1|[foo, bar]| x| 1.0| - | 2| []|NULL| NULL| - | 3| NULL|NULL| NULL| - +---+----------+----+-----+ - - >>> df.select("id", "a_map", explode_outer("an_array")).show() - +---+----------+----+ - | id| a_map| col| - +---+----------+----+ - | 1|{x -> 1.0}| foo| - | 1|{x -> 1.0}| bar| - | 2| {}|NULL| - | 3| NULL|NULL| - +---+----------+----+ - """ + Example 1: Using an array column + + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,ARRAY(1,2,3,NULL)), (2,ARRAY()), (3,NULL) AS t(i,a)') + >>> df.select('*', sf.explode_outer('a')).show() + +---+---------------+----+ + | i| a| col| + +---+---------------+----+ + | 1|[1, 2, 3, NULL]| 1| + | 1|[1, 2, 3, NULL]| 2| + | 1|[1, 2, 3, NULL]| 3| + | 1|[1, 2, 3, NULL]|NULL| + | 2| []|NULL| + | 3| NULL|NULL| + +---+---------------+----+ + + Example 2: Using a map column + + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,MAP(1,2,3,4,5,NULL)), (2,MAP()), (3,NULL) AS t(i,m)') + >>> df.select('*', sf.explode_outer('m')).show(truncate=False) + +---+---------------------------+----+-----+ + |i |m |key |value| + +---+---------------------------+----+-----+ + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|1 |2 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|3 |4 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|5 |NULL | + |2 |{} |NULL|NULL | + |3 |NULL |NULL|NULL | + +---+---------------------------+----+-----+ + """ # noqa: E501 return _invoke_function_over_columns("explode_outer", col) @@ -19082,7 +19091,7 @@ def posexplode_outer(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -19090,30 +19099,47 @@ def posexplode_outer(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` one row per array item or map key value including positions as a separate column. + See Also + -------- + :meth:`pyspark.sql.functions.explode` + :meth:`pyspark.sql.functions.explode_outer` + :meth:`pyspark.sql.functions.posexplode` + :meth:`pyspark.sql.functions.inline` + :meth:`pyspark.sql.functions.inline_outer` + Examples -------- - >>> df = spark.createDataFrame( - ... [(1, ["foo", "bar"], {"x": 1.0}), (2, [], {}), (3, None, None)], - ... ("id", "an_array", "a_map") - ... ) - >>> df.select("id", "an_array", posexplode_outer("a_map")).show() - +---+----------+----+----+-----+ - | id| an_array| pos| key|value| - +---+----------+----+----+-----+ - | 1|[foo, bar]| 0| x| 1.0| - | 2| []|NULL|NULL| NULL| - | 3| NULL|NULL|NULL| NULL| - +---+----------+----+----+-----+ - >>> df.select("id", "a_map", posexplode_outer("an_array")).show() - +---+----------+----+----+ - | id| a_map| pos| col| - +---+----------+----+----+ - | 1|{x -> 1.0}| 0| foo| - | 1|{x -> 1.0}| 1| bar| - | 2| {}|NULL|NULL| - | 3| NULL|NULL|NULL| - +---+----------+----+----+ - """ + Example 1: Using an array column + + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,ARRAY(1,2,3,NULL)), (2,ARRAY()), (3,NULL) AS t(i,a)') + >>> df.select('*', sf.posexplode_outer('a')).show() + +---+---------------+----+----+ + | i| a| pos| col| + +---+---------------+----+----+ + | 1|[1, 2, 3, NULL]| 0| 1| + | 1|[1, 2, 3, NULL]| 1| 2| + | 1|[1, 2, 3, NULL]| 2| 3| + | 1|[1, 2, 3, NULL]| 3|NULL| + | 2| []|NULL|NULL| + | 3| NULL|NULL|NULL| + +---+---------------+----+----+ + + Example 2: Using a map column + + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,MAP(1,2,3,4,5,NULL)), (2,MAP()), (3,NULL) AS t(i,m)') + >>> df.select('*', sf.posexplode_outer('m')).show(truncate=False) + +---+---------------------------+----+----+-----+ + |i |m |pos |key |value| + +---+---------------------------+----+----+-----+ + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|0 |1 |2 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|1 |3 |4 | + |1 |{1 -> 2, 3 -> 4, 5 -> NULL}|2 |5 |NULL | + |2 |{} |NULL|NULL|NULL | + |3 |NULL |NULL|NULL|NULL | + +---+---------------------------+----+----+-----+ + """ # noqa: E501 return _invoke_function_over_columns("posexplode_outer", col) @@ -19127,7 +19153,7 @@ def inline_outer(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name input column of values to explode. Returns @@ -19137,7 +19163,10 @@ def inline_outer(col: "ColumnOrName") -> Column: See Also -------- + :meth:`pyspark.sql.functions.explode` :meth:`pyspark.sql.functions.explode_outer` + :meth:`pyspark.sql.functions.posexplode` + :meth:`pyspark.sql.functions.posexplode_outer` :meth:`pyspark.sql.functions.inline` Notes @@ -19146,20 +19175,27 @@ def inline_outer(col: "ColumnOrName") -> Column: Examples -------- - >>> from pyspark.sql import Row - >>> df = spark.createDataFrame([ - ... Row(id=1, structlist=[Row(a=1, b=2), Row(a=3, b=4)]), - ... Row(id=2, structlist=[]) - ... ]) - >>> df.select('id', inline_outer(df.structlist)).show() - +---+----+----+ - | id| a| b| - +---+----+----+ - | 1| 1| 2| - | 1| 3| 4| - | 2|NULL|NULL| - +---+----+----+ - """ + >>> from pyspark.sql import functions as sf + >>> df = spark.sql('SELECT * FROM VALUES (1,ARRAY(NAMED_STRUCT("a",1,"b",2), NULL, NAMED_STRUCT("a",3,"b",4))), (2,ARRAY()), (3,NULL) AS t(i,s)') + >>> df.printSchema() + root + |-- i: integer (nullable = false) + |-- s: array (nullable = true) + | |-- element: struct (containsNull = true) + | | |-- a: integer (nullable = false) + | | |-- b: integer (nullable = false) + + >>> df.select('*', sf.inline_outer('s')).show(truncate=False) + +---+----------------------+----+----+ + |i |s |a |b | + +---+----------------------+----+----+ + |1 |[{1, 2}, NULL, {3, 4}]|1 |2 | + |1 |[{1, 2}, NULL, {3, 4}]|NULL|NULL| + |1 |[{1, 2}, NULL, {3, 4}]|3 |4 | + |2 |[] |NULL|NULL| + |3 |NULL |NULL|NULL| + +---+----------------------+----+----+ + """ # noqa: E501 return _invoke_function_over_columns("inline_outer", col) From bb994d14966e2dc68eaef597099278eebd0f0913 Mon Sep 17 00:00:00 2001 From: mihailoale-db Date: Thu, 28 Nov 2024 12:55:15 +0800 Subject: [PATCH 044/438] [SPARK-50440][SQL] Refactor AttributeSeq.resolveCandidates ### What changes were proposed in this pull request? Refactor of `AttributeSeq.resolveCandidates` method ### Why are the changes needed? Refactoring is needed for the Single-pass Analyzer project (please check [link](https://issues.apache.org/jira/browse/SPARK-49834)) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests + CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #48988 from mihailoale-db/mihailoale-db/refactorattributeseqresolve. Authored-by: mihailoale-db Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/package.scala | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 86d3cee6a0600..114a43c34c040 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -346,7 +346,16 @@ package object expressions { */ def resolve(nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = { val (candidates, nestedFields) = getCandidatesForResolution(nameParts, resolver) - resolveCandidates(nameParts, resolver, candidates, nestedFields) + val resolvedCandidates = resolveCandidates(nameParts, resolver, candidates, nestedFields) + resolvedCandidates match { + case Seq() => None + case Seq(a) => Some(a) + case _ => + throw QueryCompilationErrors.ambiguousReferenceError( + UnresolvedAttribute(nameParts).name, + resolvedCandidates.map(_.toAttribute) + ) + } } def getCandidatesForResolution( @@ -371,7 +380,7 @@ package object expressions { nameParts: Seq[String], resolver: Resolver, candidates: Seq[Attribute], - nestedFields: Seq[String]): Option[NamedExpression] = { + nestedFields: Seq[String]): Seq[NamedExpression] = { def name = UnresolvedAttribute(nameParts).name // We may have resolved the attributes from metadata columns. The resolved attributes will be // put in a logical plan node and becomes normal attributes. They can still keep the special @@ -389,19 +398,19 @@ package object expressions { val fieldExprs = nestedFields.foldLeft(a: Expression) { (e, name) => ExtractValue(e, Literal(name), resolver) } - Some(Alias(fieldExprs, nestedFields.last)()) + Seq(Alias(fieldExprs, nestedFields.last)()) case Seq(a) => // One match, no nested fields, use it. - Some(a) + Seq(a) case Seq() => // No matches. - None + Seq() case ambiguousReferences => // More than one match. - throw QueryCompilationErrors.ambiguousReferenceError(name, ambiguousReferences) + ambiguousReferences } } } From e6252d6c6143f134201a17bb25978a136186cbfa Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 28 Nov 2024 15:34:37 +0900 Subject: [PATCH 045/438] [SPARK-50194][SS][PYTHON] Integration of New Timer API and Initial State API with Timer ### What changes were proposed in this pull request? As Scala side, we modify the timer API with a separate `handleExpiredTimer` function inside `StatefulProcessor`, this PR make a change to the timer API to couple with API on Scala side. Also adds a timer parameter to pass into `handleInitialState` function to support use cases for registering timers in the first batch for initial state rows. ### Why are the changes needed? This change is to couple with Scala side of APIs: https://github.com/apache/spark/pull/48553 ### Does this PR introduce _any_ user-facing change? Yes. We add a new user defined function to explicitly handle expired timeres: ``` def handleExpiredTimer( self, key: Any, timer_values: TimerValues, expired_timer_info: ExpiredTimerInfo ``` We also add a new timer parameter to enable users to register timers for keys exist in the initial state: ``` def handleInitialState( self, key: Any, initialState: "PandasDataFrameLike", timer_values: TimerValues) -> None ``` ### How was this patch tested? Add a new test in `test_pandas_transform_with_state` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48838 from jingz-db/python-new-timer. Lead-authored-by: jingz-db Co-authored-by: Jing Zhan <135738831+jingz-db@users.noreply.github.com> Co-authored-by: Jungtaek Lim Signed-off-by: Jungtaek Lim --- python/pyspark/sql/pandas/group_ops.py | 107 +++++---- python/pyspark/sql/pandas/serializers.py | 13 +- .../sql/streaming/stateful_processor.py | 49 ++-- .../stateful_processor_api_client.py | 94 +++++--- .../sql/streaming/stateful_processor_util.py | 27 +++ .../test_pandas_transform_with_state.py | 227 +++++++++++------- python/pyspark/worker.py | 68 +++--- ...ransformWithStateInPandasStateServer.scala | 2 + 8 files changed, 363 insertions(+), 224 deletions(-) create mode 100644 python/pyspark/sql/streaming/stateful_processor_util.py diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index d8f22e434374c..688ad4b05732e 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -35,6 +35,7 @@ TimerValues, ) from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle +from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode from pyspark.sql.types import StructType, _parse_datatype_string if TYPE_CHECKING: @@ -503,58 +504,59 @@ def transformWithStateInPandas( if isinstance(outputStructType, str): outputStructType = cast(StructType, _parse_datatype_string(outputStructType)) - def handle_data_with_timers( + def handle_data_rows( statefulProcessorApiClient: StatefulProcessorApiClient, key: Any, - inputRows: Iterator["PandasDataFrameLike"], + inputRows: Optional[Iterator["PandasDataFrameLike"]] = None, ) -> Iterator["PandasDataFrameLike"]: statefulProcessorApiClient.set_implicit_key(key) - if timeMode != "none": - batch_timestamp = statefulProcessorApiClient.get_batch_timestamp() - watermark_timestamp = statefulProcessorApiClient.get_watermark_timestamp() + + batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( + timeMode + ) + + # process with data rows + if inputRows is not None: + data_iter = statefulProcessor.handleInputRows( + key, inputRows, TimerValues(batch_timestamp, watermark_timestamp) + ) + return data_iter else: - batch_timestamp = -1 - watermark_timestamp = -1 - # process with invalid expiry timer info and emit data rows - data_iter = statefulProcessor.handleInputRows( - key, - inputRows, - TimerValues(batch_timestamp, watermark_timestamp), - ExpiredTimerInfo(False), + return iter([]) + + def handle_expired_timers( + statefulProcessorApiClient: StatefulProcessorApiClient, + ) -> Iterator["PandasDataFrameLike"]: + batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( + timeMode ) - statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.DATA_PROCESSED) - if timeMode == "processingtime": + if timeMode.lower() == "processingtime": expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( batch_timestamp ) - elif timeMode == "eventtime": + elif timeMode.lower() == "eventtime": expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( watermark_timestamp ) else: expiry_list_iter = iter([[]]) - result_iter_list = [data_iter] - # process with valid expiry time info and with empty input rows, - # only timer related rows will be emitted + # process with expiry timers, only timer related rows will be emitted for expiry_list in expiry_list_iter: for key_obj, expiry_timestamp in expiry_list: - result_iter_list.append( - statefulProcessor.handleInputRows( - key_obj, - iter([]), - TimerValues(batch_timestamp, watermark_timestamp), - ExpiredTimerInfo(True, expiry_timestamp), - ) - ) - # TODO(SPARK-49603) set the handle state in the lazily initialized iterator - - result = itertools.chain(*result_iter_list) - return result + statefulProcessorApiClient.set_implicit_key(key_obj) + for pd in statefulProcessor.handleExpiredTimer( + key=key_obj, + timer_values=TimerValues(batch_timestamp, watermark_timestamp), + expired_timer_info=ExpiredTimerInfo(expiry_timestamp), + ): + yield pd + statefulProcessorApiClient.delete_timer(expiry_timestamp) def transformWithStateUDF( statefulProcessorApiClient: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, key: Any, inputRows: Iterator["PandasDataFrameLike"], ) -> Iterator["PandasDataFrameLike"]: @@ -566,19 +568,28 @@ def transformWithStateUDF( StatefulProcessorHandleState.INITIALIZED ) - # Key is None when we have processed all the input data from the worker and ready to - # proceed with the cleanup steps. - if key is None: + if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: + statefulProcessorApiClient.set_handle_state( + StatefulProcessorHandleState.DATA_PROCESSED + ) + result = handle_expired_timers(statefulProcessorApiClient) + return result + elif mode == TransformWithStateInPandasFuncMode.COMPLETE: + statefulProcessorApiClient.set_handle_state( + StatefulProcessorHandleState.TIMER_PROCESSED + ) statefulProcessorApiClient.remove_implicit_key() statefulProcessor.close() statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.CLOSED) return iter([]) - - result = handle_data_with_timers(statefulProcessorApiClient, key, inputRows) - return result + else: + # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA + result = handle_data_rows(statefulProcessorApiClient, key, inputRows) + return result def transformWithStateWithInitStateUDF( statefulProcessorApiClient: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, key: Any, inputRows: Iterator["PandasDataFrameLike"], initialStates: Optional[Iterator["PandasDataFrameLike"]] = None, @@ -603,20 +614,30 @@ def transformWithStateWithInitStateUDF( StatefulProcessorHandleState.INITIALIZED ) - # Key is None when we have processed all the input data from the worker and ready to - # proceed with the cleanup steps. - if key is None: + if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: + statefulProcessorApiClient.set_handle_state( + StatefulProcessorHandleState.DATA_PROCESSED + ) + result = handle_expired_timers(statefulProcessorApiClient) + return result + elif mode == TransformWithStateInPandasFuncMode.COMPLETE: statefulProcessorApiClient.remove_implicit_key() statefulProcessor.close() statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.CLOSED) return iter([]) + else: + # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA + batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( + timeMode + ) # only process initial state if first batch and initial state is not None if initialStates is not None: for cur_initial_state in initialStates: statefulProcessorApiClient.set_implicit_key(key) - # TODO(SPARK-50194) integration with new timer API with initial state - statefulProcessor.handleInitialState(key, cur_initial_state) + statefulProcessor.handleInitialState( + key, cur_initial_state, TimerValues(batch_timestamp, watermark_timestamp) + ) # if we don't have input rows for the given key but only have initial state # for the grouping key, the inputRows iterator could be empty @@ -629,7 +650,7 @@ def transformWithStateWithInitStateUDF( inputRows = itertools.chain([first], inputRows) if not input_rows_empty: - result = handle_data_with_timers(statefulProcessorApiClient, key, inputRows) + result = handle_data_rows(statefulProcessorApiClient, key, inputRows) else: result = iter([]) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 5bf07b87400fe..536bf7307065c 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -36,6 +36,7 @@ _create_converter_from_pandas, _create_converter_to_pandas, ) +from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode from pyspark.sql.types import ( DataType, StringType, @@ -1197,7 +1198,11 @@ def generate_data_batches(batches): data_batches = generate_data_batches(_batches) for k, g in groupby(data_batches, key=lambda x: x[0]): - yield (k, g) + yield (TransformWithStateInPandasFuncMode.PROCESS_DATA, k, g) + + yield (TransformWithStateInPandasFuncMode.PROCESS_TIMER, None, None) + + yield (TransformWithStateInPandasFuncMode.COMPLETE, None, None) def dump_stream(self, iterator, stream): """ @@ -1281,4 +1286,8 @@ def flatten_columns(cur_batch, col_name): data_batches = generate_data_batches(_batches) for k, g in groupby(data_batches, key=lambda x: x[0]): - yield (k, g) + yield (TransformWithStateInPandasFuncMode.PROCESS_DATA, k, g) + + yield (TransformWithStateInPandasFuncMode.PROCESS_TIMER, None, None) + + yield (TransformWithStateInPandasFuncMode.COMPLETE, None, None) diff --git a/python/pyspark/sql/streaming/stateful_processor.py b/python/pyspark/sql/streaming/stateful_processor.py index 20078c215bace..9caa9304d6a87 100644 --- a/python/pyspark/sql/streaming/stateful_processor.py +++ b/python/pyspark/sql/streaming/stateful_processor.py @@ -105,21 +105,13 @@ def get_current_watermark_in_ms(self) -> int: class ExpiredTimerInfo: """ - Class used for arbitrary stateful operations with transformWithState to access expired timer - info. When is_valid is false, the expiry timestamp is invalid. + Class used to provide access to expired timer's expiry time. .. versionadded:: 4.0.0 """ - def __init__(self, is_valid: bool, expiry_time_in_ms: int = -1) -> None: - self._is_valid = is_valid + def __init__(self, expiry_time_in_ms: int = -1) -> None: self._expiry_time_in_ms = expiry_time_in_ms - def is_valid(self) -> bool: - """ - Whether the expiry info is valid. - """ - return self._is_valid - def get_expiry_time_in_ms(self) -> int: """ Get the timestamp for expired timer, return timestamp in millisecond. @@ -398,7 +390,6 @@ def handleInputRows( key: Any, rows: Iterator["PandasDataFrameLike"], timer_values: TimerValues, - expired_timer_info: ExpiredTimerInfo, ) -> Iterator["PandasDataFrameLike"]: """ Function that will allow users to interact with input data rows along with the grouping key. @@ -420,11 +411,29 @@ def handleInputRows( timer_values: TimerValues Timer value for the current batch that process the input rows. Users can get the processing or event time timestamp from TimerValues. - expired_timer_info: ExpiredTimerInfo - Timestamp of expired timers on the grouping key. """ ... + def handleExpiredTimer( + self, key: Any, timer_values: TimerValues, expired_timer_info: ExpiredTimerInfo + ) -> Iterator["PandasDataFrameLike"]: + """ + Optional to implement. Will act return an empty iterator if not defined. + Function that will be invoked when a timer is fired for a given key. Users can choose to + evict state, register new timers and optionally provide output rows. + + Parameters + ---------- + key : Any + grouping key. + timer_values: TimerValues + Timer value for the current batch that process the input rows. + Users can get the processing or event time timestamp from TimerValues. + expired_timer_info: ExpiredTimerInfo + Instance of ExpiredTimerInfo that provides access to expired timer. + """ + return iter([]) + @abstractmethod def close(self) -> None: """ @@ -433,9 +442,21 @@ def close(self) -> None: """ ... - def handleInitialState(self, key: Any, initialState: "PandasDataFrameLike") -> None: + def handleInitialState( + self, key: Any, initialState: "PandasDataFrameLike", timer_values: TimerValues + ) -> None: """ Optional to implement. Will act as no-op if not defined or no initial state input. Function that will be invoked only in the first batch for users to process initial states. + + Parameters + ---------- + key : Any + grouping key. + initialState: :class:`pandas.DataFrame` + One dataframe in the initial state associated with the key. + timer_values: TimerValues + Timer value for the current batch that process the input rows. + Users can get the processing or event time timestamp from TimerValues. """ pass diff --git a/python/pyspark/sql/streaming/stateful_processor_api_client.py b/python/pyspark/sql/streaming/stateful_processor_api_client.py index 353f75e267962..53704188081c3 100644 --- a/python/pyspark/sql/streaming/stateful_processor_api_client.py +++ b/python/pyspark/sql/streaming/stateful_processor_api_client.py @@ -62,6 +62,10 @@ def __init__(self, state_server_port: int, key_schema: StructType) -> None: # Dictionaries to store the mapping between iterator id and a tuple of pandas DataFrame # and the index of the last row that was read. self.list_timer_iterator_cursors: Dict[str, Tuple["PandasDataFrameLike", int]] = {} + # statefulProcessorApiClient is initialized per batch per partition, + # so we will have new timestamps for a new batch + self._batch_timestamp = -1 + self._watermark_timestamp = -1 def set_handle_state(self, state: StatefulProcessorHandleState) -> None: import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage @@ -266,47 +270,15 @@ def get_expiry_timers_iterator( # TODO(SPARK-49233): Classify user facing errors. raise PySparkRuntimeError(f"Error getting expiry timers: " f"{response_message[1]}") - def get_batch_timestamp(self) -> int: - import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage - - get_processing_time_call = stateMessage.GetProcessingTime() - timer_value_call = stateMessage.TimerValueRequest( - getProcessingTimer=get_processing_time_call - ) - timer_request = stateMessage.TimerRequest(timerValueRequest=timer_value_call) - message = stateMessage.StateRequest(timerRequest=timer_request) - - self._send_proto_message(message.SerializeToString()) - response_message = self._receive_proto_message_with_long_value() - status = response_message[0] - if status != 0: - # TODO(SPARK-49233): Classify user facing errors. - raise PySparkRuntimeError( - f"Error getting processing timestamp: " f"{response_message[1]}" - ) + def get_timestamps(self, time_mode: str) -> Tuple[int, int]: + if time_mode.lower() == "none": + return -1, -1 else: - timestamp = response_message[2] - return timestamp - - def get_watermark_timestamp(self) -> int: - import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage - - get_watermark_call = stateMessage.GetWatermark() - timer_value_call = stateMessage.TimerValueRequest(getWatermark=get_watermark_call) - timer_request = stateMessage.TimerRequest(timerValueRequest=timer_value_call) - message = stateMessage.StateRequest(timerRequest=timer_request) - - self._send_proto_message(message.SerializeToString()) - response_message = self._receive_proto_message_with_long_value() - status = response_message[0] - if status != 0: - # TODO(SPARK-49233): Classify user facing errors. - raise PySparkRuntimeError( - f"Error getting eventtime timestamp: " f"{response_message[1]}" - ) - else: - timestamp = response_message[2] - return timestamp + if self._batch_timestamp == -1: + self._batch_timestamp = self._get_batch_timestamp() + if self._watermark_timestamp == -1: + self._watermark_timestamp = self._get_watermark_timestamp() + return self._batch_timestamp, self._watermark_timestamp def get_map_state( self, @@ -353,6 +325,48 @@ def delete_if_exists(self, state_name: str) -> None: # TODO(SPARK-49233): Classify user facing errors. raise PySparkRuntimeError(f"Error deleting state: " f"{response_message[1]}") + def _get_batch_timestamp(self) -> int: + import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage + + get_processing_time_call = stateMessage.GetProcessingTime() + timer_value_call = stateMessage.TimerValueRequest( + getProcessingTimer=get_processing_time_call + ) + timer_request = stateMessage.TimerRequest(timerValueRequest=timer_value_call) + message = stateMessage.StateRequest(timerRequest=timer_request) + + self._send_proto_message(message.SerializeToString()) + response_message = self._receive_proto_message_with_long_value() + status = response_message[0] + if status != 0: + # TODO(SPARK-49233): Classify user facing errors. + raise PySparkRuntimeError( + f"Error getting processing timestamp: " f"{response_message[1]}" + ) + else: + timestamp = response_message[2] + return timestamp + + def _get_watermark_timestamp(self) -> int: + import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage + + get_watermark_call = stateMessage.GetWatermark() + timer_value_call = stateMessage.TimerValueRequest(getWatermark=get_watermark_call) + timer_request = stateMessage.TimerRequest(timerValueRequest=timer_value_call) + message = stateMessage.StateRequest(timerRequest=timer_request) + + self._send_proto_message(message.SerializeToString()) + response_message = self._receive_proto_message_with_long_value() + status = response_message[0] + if status != 0: + # TODO(SPARK-49233): Classify user facing errors. + raise PySparkRuntimeError( + f"Error getting eventtime timestamp: " f"{response_message[1]}" + ) + else: + timestamp = response_message[2] + return timestamp + def _send_proto_message(self, message: bytes) -> None: # Writing zero here to indicate message version. This allows us to evolve the message # format or even changing the message protocol in the future. diff --git a/python/pyspark/sql/streaming/stateful_processor_util.py b/python/pyspark/sql/streaming/stateful_processor_util.py new file mode 100644 index 0000000000000..6130a9581bc24 --- /dev/null +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -0,0 +1,27 @@ +# +# 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. +# + +from enum import Enum + +# This file places the utilities for transformWithStateInPandas; we have a separate file to avoid +# putting internal classes to the stateful_processor.py file which contains public APIs. + + +class TransformWithStateInPandasFuncMode(Enum): + PROCESS_DATA = 1 + PROCESS_TIMER = 2 + COMPLETE = 3 diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index f385d7cd1abc0..60f2c9348db3f 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -55,6 +55,7 @@ def conf(cls): "org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider", ) cfg.set("spark.sql.execution.arrow.transformWithStateInPandas.maxRecordsPerBatch", "2") + cfg.set("spark.sql.session.timeZone", "UTC") return cfg def _prepare_input_data(self, input_path, col1, col2): @@ -558,14 +559,25 @@ def prepare_batch3(input_path): def test_transform_with_state_in_pandas_event_time(self): def check_results(batch_df, batch_id): if batch_id == 0: - assert set(batch_df.sort("id").collect()) == {Row(id="a", timestamp="20")} - elif batch_id == 1: + # watermark for late event = 0 + # watermark for eviction = 0 + # timer is registered with expiration time = 0, hence expired at the same batch assert set(batch_df.sort("id").collect()) == { Row(id="a", timestamp="20"), Row(id="a-expired", timestamp="0"), } + elif batch_id == 1: + # watermark for late event = 0 + # watermark for eviction = 10 (20 - 10) + # timer is registered with expiration time = 10, hence expired at the same batch + assert set(batch_df.sort("id").collect()) == { + Row(id="a", timestamp="4"), + Row(id="a-expired", timestamp="10000"), + } elif batch_id == 2: - # verify that rows and expired timer produce the expected result + # watermark for late event = 10 + # watermark for eviction = 10 (unchanged as 4 < 10) + # timer is registered with expiration time = 10, hence expired at the same batch assert set(batch_df.sort("id").collect()) == { Row(id="a", timestamp="15"), Row(id="a-expired", timestamp="10000"), @@ -578,7 +590,9 @@ def check_results(batch_df, batch_id): EventTimeStatefulProcessor(), check_results ) - def _test_transform_with_state_init_state_in_pandas(self, stateful_processor, check_results): + def _test_transform_with_state_init_state_in_pandas( + self, stateful_processor, check_results, time_mode="None" + ): input_path = tempfile.mkdtemp() self._prepare_test_resource1(input_path) time.sleep(2) @@ -606,7 +620,7 @@ def _test_transform_with_state_init_state_in_pandas(self, stateful_processor, ch statefulProcessor=stateful_processor, outputStructType=output_schema, outputMode="Update", - timeMode="None", + timeMode=time_mode, initialState=initial_state, ) .writeStream.queryName("this_query") @@ -806,6 +820,45 @@ def check_results(batch_df, batch_id): StatefulProcessorChainingOps(), check_results, "eventTime", ["outputTimestamp", "id"] ) + def test_transform_with_state_init_state_with_timers(self): + def check_results(batch_df, batch_id): + if batch_id == 0: + # timers are registered and handled in the first batch for + # rows in initial state; For key=0 and key=3 which contains + # expired timers, both should be handled by handleExpiredTimers + # regardless of whether key exists in the data rows or not + expired_df = batch_df.filter(batch_df["id"].contains("expired")) + data_df = batch_df.filter(~batch_df["id"].contains("expired")) + assert set(expired_df.sort("id").select("id").collect()) == { + Row(id="0-expired"), + Row(id="3-expired"), + } + assert set(data_df.sort("id").collect()) == { + Row(id="0", value=str(789 + 123 + 46)), + Row(id="1", value=str(146 + 346)), + } + elif batch_id == 1: + # handleInitialState is only processed in the first batch, + # no more timer is registered so no more expired timers + assert set(batch_df.sort("id").collect()) == { + Row(id="0", value=str(789 + 123 + 46 + 67)), + Row(id="3", value=str(987 + 12)), + } + else: + for q in self.spark.streams.active: + q.stop() + + self._test_transform_with_state_init_state_in_pandas( + StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" + ) + + # run the same test suites again but with single shuffle partition + def test_transform_with_state_with_timers_single_partition(self): + with self.sql_conf({"spark.sql.shuffle.partitions": "1"}): + self.test_transform_with_state_init_state_with_timers() + self.test_transform_with_state_in_pandas_event_time() + self.test_transform_with_state_in_pandas_proc_timer() + class SimpleStatefulProcessorWithInitialState(StatefulProcessor): # this dict is the same as input initial state dataframe @@ -814,10 +867,9 @@ class SimpleStatefulProcessorWithInitialState(StatefulProcessor): def init(self, handle: StatefulProcessorHandle) -> None: state_schema = StructType([StructField("value", IntegerType(), True)]) self.value_state = handle.getValueState("value_state", state_schema) + self.handle = handle - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: exists = self.value_state.exists() if exists: value_row = self.value_state.get() @@ -840,7 +892,7 @@ def handleInputRows( else: yield pd.DataFrame({"id": key, "value": str(accumulated_value)}) - def handleInitialState(self, key, initialState) -> None: + def handleInitialState(self, key, initialState, timer_values) -> None: init_val = initialState.at[0, "initVal"] self.value_state.update((init_val,)) if len(key) == 1: @@ -850,6 +902,19 @@ def close(self) -> None: pass +class StatefulProcessorWithInitialStateTimers(SimpleStatefulProcessorWithInitialState): + def handleExpiredTimer(self, key, timer_values, expired_timer_info) -> Iterator[pd.DataFrame]: + self.handle.deleteTimer(expired_timer_info.get_expiry_time_in_ms()) + str_key = f"{str(key[0])}-expired" + yield pd.DataFrame( + {"id": (str_key,), "value": str(expired_timer_info.get_expiry_time_in_ms())} + ) + + def handleInitialState(self, key, initialState, timer_values) -> None: + super().handleInitialState(key, initialState, timer_values) + self.handle.registerTimer(timer_values.get_current_processing_time_in_ms() - 1) + + # A stateful processor that output the max event time it has seen. Register timer for # current watermark. Clear max state if timer expires. class EventTimeStatefulProcessor(StatefulProcessor): @@ -858,33 +923,30 @@ def init(self, handle: StatefulProcessorHandle) -> None: self.handle = handle self.max_state = handle.getValueState("max_state", state_schema) - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: - if expired_timer_info.is_valid(): - self.max_state.clear() - self.handle.deleteTimer(expired_timer_info.get_expiry_time_in_ms()) - str_key = f"{str(key[0])}-expired" - yield pd.DataFrame( - {"id": (str_key,), "timestamp": str(expired_timer_info.get_expiry_time_in_ms())} - ) + def handleExpiredTimer(self, key, timer_values, expired_timer_info) -> Iterator[pd.DataFrame]: + self.max_state.clear() + self.handle.deleteTimer(expired_timer_info.get_expiry_time_in_ms()) + str_key = f"{str(key[0])}-expired" + yield pd.DataFrame( + {"id": (str_key,), "timestamp": str(expired_timer_info.get_expiry_time_in_ms())} + ) - else: - timestamp_list = [] - for pdf in rows: - # int64 will represent timestamp in nanosecond, restore to second - timestamp_list.extend((pdf["eventTime"].astype("int64") // 10**9).tolist()) + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: + timestamp_list = [] + for pdf in rows: + # int64 will represent timestamp in nanosecond, restore to second + timestamp_list.extend((pdf["eventTime"].astype("int64") // 10**9).tolist()) - if self.max_state.exists(): - cur_max = int(self.max_state.get()[0]) - else: - cur_max = 0 - max_event_time = str(max(cur_max, max(timestamp_list))) + if self.max_state.exists(): + cur_max = int(self.max_state.get()[0]) + else: + cur_max = 0 + max_event_time = str(max(cur_max, max(timestamp_list))) - self.max_state.update((max_event_time,)) - self.handle.registerTimer(timer_values.get_current_watermark_in_ms()) + self.max_state.update((max_event_time,)) + self.handle.registerTimer(timer_values.get_current_watermark_in_ms()) - yield pd.DataFrame({"id": key, "timestamp": max_event_time}) + yield pd.DataFrame({"id": key, "timestamp": max_event_time}) def close(self) -> None: pass @@ -898,54 +960,49 @@ def init(self, handle: StatefulProcessorHandle) -> None: self.handle = handle self.count_state = handle.getValueState("count_state", state_schema) - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: - if expired_timer_info.is_valid(): - # reset count state each time the timer is expired - timer_list_1 = [e for e in self.handle.listTimers()] - timer_list_2 = [] - idx = 0 - for e in self.handle.listTimers(): - timer_list_2.append(e) - # check multiple iterator on the same grouping key works - assert timer_list_2[idx] == timer_list_1[idx] - idx += 1 - - if len(timer_list_1) > 0: - # before deleting the expiring timers, there are 2 timers - - # one timer we just registered, and one that is going to be deleted - assert len(timer_list_1) == 2 - self.count_state.clear() - self.handle.deleteTimer(expired_timer_info.get_expiry_time_in_ms()) - yield pd.DataFrame( - { - "id": key, - "countAsString": str("-1"), - "timeValues": str(expired_timer_info.get_expiry_time_in_ms()), - } - ) + def handleExpiredTimer(self, key, timer_values, expired_timer_info) -> Iterator[pd.DataFrame]: + # reset count state each time the timer is expired + timer_list_1 = [e for e in self.handle.listTimers()] + timer_list_2 = [] + idx = 0 + for e in self.handle.listTimers(): + timer_list_2.append(e) + # check multiple iterator on the same grouping key works + assert timer_list_2[idx] == timer_list_1[idx] + idx += 1 + + if len(timer_list_1) > 0: + assert len(timer_list_1) == 2 + self.count_state.clear() + self.handle.deleteTimer(expired_timer_info.get_expiry_time_in_ms()) + yield pd.DataFrame( + { + "id": key, + "countAsString": str("-1"), + "timeValues": str(expired_timer_info.get_expiry_time_in_ms()), + } + ) + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: + if not self.count_state.exists(): + count = 0 else: - if not self.count_state.exists(): - count = 0 - else: - count = int(self.count_state.get()[0]) + count = int(self.count_state.get()[0]) - if key == ("0",): - self.handle.registerTimer(timer_values.get_current_processing_time_in_ms()) + if key == ("0",): + self.handle.registerTimer(timer_values.get_current_processing_time_in_ms() + 1) - rows_count = 0 - for pdf in rows: - pdf_count = len(pdf) - rows_count += pdf_count + rows_count = 0 + for pdf in rows: + pdf_count = len(pdf) + rows_count += pdf_count - count = count + rows_count + count = count + rows_count - self.count_state.update((str(count),)) - timestamp = str(timer_values.get_current_processing_time_in_ms()) + self.count_state.update((str(count),)) + timestamp = str(timer_values.get_current_processing_time_in_ms()) - yield pd.DataFrame({"id": key, "countAsString": str(count), "timeValues": timestamp}) + yield pd.DataFrame({"id": key, "countAsString": str(count), "timeValues": timestamp}) def close(self) -> None: pass @@ -961,9 +1018,7 @@ def init(self, handle: StatefulProcessorHandle) -> None: self.temp_state = handle.getValueState("tempState", state_schema) handle.deleteIfExists("tempState") - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: with self.assertRaisesRegex(PySparkRuntimeError, "Error checking value state exists"): self.temp_state.exists() new_violations = 0 @@ -995,9 +1050,7 @@ class StatefulProcessorChainingOps(StatefulProcessor): def init(self, handle: StatefulProcessorHandle) -> None: pass - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: for pdf in rows: timestamp_list = pdf["eventTime"].tolist() yield pd.DataFrame({"id": key, "outputTimestamp": timestamp_list[0]}) @@ -1027,9 +1080,7 @@ def init(self, handle: StatefulProcessorHandle) -> None: "ttl-map-state", user_key_schema, state_schema, 10000 ) - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: count = 0 ttl_count = 0 ttl_list_state_count = 0 @@ -1079,9 +1130,7 @@ def init(self, handle: StatefulProcessorHandle) -> None: state_schema = StructType([StructField("value", IntegerType(), True)]) self.num_violations_state = handle.getValueState("numViolations", state_schema) - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: count = 0 exists = self.num_violations_state.exists() assert not exists @@ -1105,9 +1154,7 @@ def init(self, handle: StatefulProcessorHandle) -> None: self.list_state1 = handle.getListState("listState1", state_schema) self.list_state2 = handle.getListState("listState2", state_schema) - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: count = 0 for pdf in rows: list_state_rows = [(120,), (20,)] @@ -1162,9 +1209,7 @@ def init(self, handle: StatefulProcessorHandle): value_schema = StructType([StructField("count", IntegerType(), True)]) self.map_state = handle.getMapState("mapState", key_schema, value_schema) - def handleInputRows( - self, key, rows, timer_values, expired_timer_info - ) -> Iterator[pd.DataFrame]: + def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: count = 0 key1 = ("key1",) key2 = ("key2",) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 04f95e9f52648..1ebc04520ecad 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -34,6 +34,7 @@ _deserialize_accumulator, ) from pyspark.sql.streaming.stateful_processor_api_client import StatefulProcessorApiClient +from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode from pyspark.taskcontext import BarrierTaskContext, TaskContext from pyspark.resource import ResourceInformation from pyspark.util import PythonEvalType, local_connect_and_auth @@ -493,36 +494,36 @@ def wrapped(key_series, value_series): def wrap_grouped_transform_with_state_pandas_udf(f, return_type, runner_conf): - def wrapped(stateful_processor_api_client, key, value_series_gen): + def wrapped(stateful_processor_api_client, mode, key, value_series_gen): import pandas as pd values = (pd.concat(x, axis=1) for x in value_series_gen) - result_iter = f(stateful_processor_api_client, key, values) + result_iter = f(stateful_processor_api_client, mode, key, values) # TODO(SPARK-49100): add verification that elements in result_iter are # indeed of type pd.DataFrame and confirm to assigned cols return result_iter - return lambda p, k, v: [(wrapped(p, k, v), to_arrow_type(return_type))] + return lambda p, m, k, v: [(wrapped(p, m, k, v), to_arrow_type(return_type))] def wrap_grouped_transform_with_state_pandas_init_state_udf(f, return_type, runner_conf): - def wrapped(stateful_processor_api_client, key, value_series_gen): + def wrapped(stateful_processor_api_client, mode, key, value_series_gen): import pandas as pd state_values_gen, init_states_gen = itertools.tee(value_series_gen, 2) state_values = (df for x, _ in state_values_gen if not (df := pd.concat(x, axis=1)).empty) init_states = (df for _, x in init_states_gen if not (df := pd.concat(x, axis=1)).empty) - result_iter = f(stateful_processor_api_client, key, state_values, init_states) + result_iter = f(stateful_processor_api_client, mode, key, state_values, init_states) # TODO(SPARK-49100): add verification that elements in result_iter are # indeed of type pd.DataFrame and confirm to assigned cols return result_iter - return lambda p, k, v: [(wrapped(p, k, v), to_arrow_type(return_type))] + return lambda p, m, k, v: [(wrapped(p, m, k, v), to_arrow_type(return_type))] def wrap_grouped_map_pandas_udf_with_state(f, return_type): @@ -1697,18 +1698,22 @@ def mapper(a): ser.key_offsets = parsed_offsets[0][0] stateful_processor_api_client = StatefulProcessorApiClient(state_server_port, key_schema) - # Create function like this: - # mapper a: f([a[0]], [a[0], a[1]]) def mapper(a): - key = a[0] + mode = a[0] - def values_gen(): - for x in a[1]: - retVal = [x[1][o] for o in parsed_offsets[0][1]] - yield retVal + if mode == TransformWithStateInPandasFuncMode.PROCESS_DATA: + key = a[1] - # This must be generator comprehension - do not materialize. - return f(stateful_processor_api_client, key, values_gen()) + def values_gen(): + for x in a[2]: + retVal = [x[1][o] for o in parsed_offsets[0][1]] + yield retVal + + # This must be generator comprehension - do not materialize. + return f(stateful_processor_api_client, mode, key, values_gen()) + else: + # mode == PROCESS_TIMER or mode == COMPLETE + return f(stateful_processor_api_client, mode, None, iter([])) elif eval_type == PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_INIT_STATE_UDF: # We assume there is only one UDF here because grouped map doesn't @@ -1731,16 +1736,22 @@ def values_gen(): stateful_processor_api_client = StatefulProcessorApiClient(state_server_port, key_schema) def mapper(a): - key = a[0] + mode = a[0] - def values_gen(): - for x in a[1]: - retVal = [x[1][o] for o in parsed_offsets[0][1]] - initVal = [x[2][o] for o in parsed_offsets[1][1]] - yield retVal, initVal + if mode == TransformWithStateInPandasFuncMode.PROCESS_DATA: + key = a[1] - # This must be generator comprehension - do not materialize. - return f(stateful_processor_api_client, key, values_gen()) + def values_gen(): + for x in a[2]: + retVal = [x[1][o] for o in parsed_offsets[0][1]] + initVal = [x[2][o] for o in parsed_offsets[1][1]] + yield retVal, initVal + + # This must be generator comprehension - do not materialize. + return f(stateful_processor_api_client, mode, key, values_gen()) + else: + # mode == PROCESS_TIMER or mode == COMPLETE + return f(stateful_processor_api_client, mode, None, iter([])) elif eval_type == PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF: import pyarrow as pa @@ -1958,17 +1969,6 @@ def process(): try: serializer.dump_stream(out_iter, outfile) finally: - # Sending a signal to TransformWithState UDF to perform proper cleanup steps. - if ( - eval_type == PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF - or eval_type == PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_INIT_STATE_UDF - ): - # Sending key as None to indicate that process() has finished. - end_iter = func(split_index, iter([(None, None)])) - # Need to materialize the iterator to trigger the cleanup steps, nothing needs - # to be done here. - for _ in end_iter: - pass if hasattr(out_iter, "close"): out_iter.close() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala index 0373c8607ff2c..2957f4b387580 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala @@ -120,6 +120,8 @@ class TransformWithStateInPandasStateServer( } /** Timer related class variables */ + // An iterator to store all expired timer info. This is meant to be consumed only once per + // partition. This should be called after finishing handling all input rows. private var expiryTimestampIter: Option[Iterator[(Any, Long)]] = if (expiryTimerIterForTest != null) { Option(expiryTimerIterForTest) From b9bff4b145e5710e69a45601906a862ae4c88652 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 28 Nov 2024 17:11:52 +0800 Subject: [PATCH 046/438] [SPARK-50295][INFRA] Add a script to build docs with image ### What changes were proposed in this pull request? The pr aims to add a script to build docs with image. The overall idea is as follows: - prepare compiled Spark packages for various subsequent documents (on host) - build image from cache - run image as container Mount local files to the container (this way, there is no need to copy the spark file to the container, and `the compiled spark package` is already prepared in the `local spark folder`, so there is `no need` to compile it again in the container, otherwise it will re-download many dependency jars, which is very time-cost) - generate `error docs`, `scala doc`, `python doc` and `sql doc` in container. - generate `r docs` in host. Why does `r` document need to be compiled outside the container ? Because when compiling inside the container, the permission of the directory `/__w/spark/spark/R/pkg/docs` automatically generated by `RScript` is `dr-xr--r-x`, and when writing to subsequent files, will throw an error as: `! [EACCES] Failed to copy '/usr/local/lib/R/site-library/pkgdown/BS5/assets/katex-auto.js' to '/__w/spark/spark/R/pkg/docs/katex-auto.js': permission denied` ### Why are the changes needed? For developers of pyspark, some python libraries are conflicts between the environment for generating docs and the development environment. In order to help developers verify more easily. ### Does this PR introduce _any_ user-facing change? No, only for spark developers. ### How was this patch tested? - Pass GA. - Manually test (The verification process can be found in the comments). ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48860 from panbingkun/SPARK-50295. Lead-authored-by: panbingkun Co-authored-by: panbingkun Signed-off-by: panbingkun --- dev/spark-test-image-util/docs/build-docs | 71 +++++++++++++++++++ .../docs/run-in-container | 37 ++++++++++ docs/_plugins/build_api_docs.rb | 19 ++++- 3 files changed, 124 insertions(+), 3 deletions(-) create mode 100644 dev/spark-test-image-util/docs/build-docs create mode 100644 dev/spark-test-image-util/docs/run-in-container diff --git a/dev/spark-test-image-util/docs/build-docs b/dev/spark-test-image-util/docs/build-docs new file mode 100644 index 0000000000000..6ff9c7cd9455c --- /dev/null +++ b/dev/spark-test-image-util/docs/build-docs @@ -0,0 +1,71 @@ +#!/usr/bin/env bash + +# +# 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. +# + +if ! [ -x "$(command -v docker)" ]; then + echo "Error: Docker is not installed." >&2 + exit 1 +fi + +DOCKER_CACHE_IMG="ghcr.io/apache/spark/apache-spark-github-action-image-docs-cache:master" +REPO_OWNER="apache/spark" +REPOSITORY="apache-spark-ci-image-docs" +IMG_TAG=$(date +%s) +IMG_NAME="${REPOSITORY}:${IMG_TAG}" +IMG_URL="$REPO_OWNER/$IMG_NAME" +DOCKER_MOUNT_SPARK_HOME="/__w/spark/spark" +BUILD_DOCS_SCRIPT_PATH="${DOCKER_MOUNT_SPARK_HOME}/dev/spark-test-image-util/docs/run-in-container" + +FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" +SPARK_HOME="$(cd "`dirname "${BASH_SOURCE[0]}"`"/../../..; pwd)" + +# 1.Compile spark outside the container to prepare for generating documents inside the container. +build/sbt -Phive -Pkinesis-asl clean unidoc package + +# 2.Build container image. +docker buildx build \ + --cache-from type=registry,ref="${DOCKER_CACHE_IMG}" \ + --tag "${IMG_URL}" "${FWDIR}" \ + --file "${SPARK_HOME}/dev/spark-test-image/docs/Dockerfile" + +# 3.Build docs on container: `error docs`, `scala doc`, `python doc`, `sql doc`. +docker run \ + --mount type=bind,source="${SPARK_HOME}",target="${DOCKER_MOUNT_SPARK_HOME}" \ + --interactive --tty "${IMG_URL}" \ + /bin/bash -c "sh ${BUILD_DOCS_SCRIPT_PATH}" + +# 4.Build docs on host: `r doc`. +# +# Why does `r` document need to be compiled outside the container? +# Because when compiling inside the container, the permission of the directory +# `/__w/spark/spark/R/pkg/docs` automatically generated by `RScript` is `dr-xr--r-x`, +# and when writing to subsequent files, will throw an error as: +# `! [EACCES] Failed to copy '/usr/local/lib/R/site-library/pkgdown/BS5/assets/katex-auto.js' +# to '/__w/spark/spark/R/pkg/docs/katex-auto.js': permission denied` +export SKIP_ERRORDOC=1 +export SKIP_SCALADOC=1 +export SKIP_PYTHONDOC=1 +export SKIP_SQLDOC=1 +cd docs +bundle exec jekyll build + +# 5.Remove container image. +IMG_ID=$(docker images | grep "${IMG_TAG}" | awk '{print $3}') +docker image rm --force "${IMG_ID}" + +echo "Build doc done." diff --git a/dev/spark-test-image-util/docs/run-in-container b/dev/spark-test-image-util/docs/run-in-container new file mode 100644 index 0000000000000..1d43c602f7c72 --- /dev/null +++ b/dev/spark-test-image-util/docs/run-in-container @@ -0,0 +1,37 @@ +# +# 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. +# + +# 1.Set env variable. +export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-arm64 +export PATH=$JAVA_HOME/bin:$PATH +export SPARK_DOCS_IS_BUILT_ON_HOST=1 +# We expect to compile the R document on the host. +export SKIP_RDOC=1 + +# 2.Install bundler. +gem install bundler -v 2.4.22 +cd /__w/spark/spark/docs +bundle install + +# 3.Build docs, includes: `error docs`, `scala doc`, `python doc`, `sql doc`, excludes: `r doc`. +# We need this link to make sure `python3` points to `python3.9` which contains the prerequisite packages. +ln -s "$(which python3.9)" "/usr/local/bin/python3" + +# Build docs first with SKIP_API to ensure they are buildable without requiring any +# language docs to be built beforehand. +cd /__w/spark/spark/docs +bundle exec jekyll build diff --git a/docs/_plugins/build_api_docs.rb b/docs/_plugins/build_api_docs.rb index 79aad9695a3c7..e2ddcca6cdde5 100644 --- a/docs/_plugins/build_api_docs.rb +++ b/docs/_plugins/build_api_docs.rb @@ -34,6 +34,11 @@ def print_header(text) end def build_spark_if_necessary + # If spark has already been compiled on the host, skip here. + if ENV['SPARK_DOCS_IS_BUILT_ON_HOST'] == '1' + return + end + if $spark_package_is_built return end @@ -116,6 +121,16 @@ def copy_and_update_java_docs(source, dest, scala_source) File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end +def build_spark_scala_and_java_docs_if_necessary + # If spark's docs has already been compiled on the host, skip here. + if ENV['SPARK_DOCS_IS_BUILT_ON_HOST'] == '1' + return + end + + command = "build/sbt -Pkinesis-asl unidoc" + puts "Running '#{command}'..." + system(command) || raise("Unidoc generation failed") +end def build_scala_and_java_docs build_spark_if_necessary @@ -123,9 +138,7 @@ def build_scala_and_java_docs print_header "Building Scala and Java API docs." cd(SPARK_PROJECT_ROOT) - command = "build/sbt -Pkinesis-asl unidoc" - puts "Running '#{command}'..." - system(command) || raise("Unidoc generation failed") + build_spark_scala_and_java_docs_if_necessary puts "Moving back into docs dir." cd("docs") From 13c1da7aa91d80e4eca25842eef81229a13acffb Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Thu, 28 Nov 2024 18:20:59 +0900 Subject: [PATCH 047/438] [SPARK-50311][PYTHON] Support `(add|remove|get|clear)Tag(s)` APIs for PySpark ### What changes were proposed in this pull request? This PR proposes to support `(add|remove|get|clear)Tag(s)` for PySpark ### Why are the changes needed? For improving the compatibility between Spark Connect and Spark Classic. These tag-related APIs are currently only supported from Spark Connect. ### Does this PR introduce _any_ user-facing change? New APIs are added - addTag - removeTag - getTags - clearTags ### How was this patch tested? Added UTs ### Was this patch authored or co-authored using generative AI tooling? No Closes #48843 from itholic/SPARK-50311. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- .../reference/pyspark.sql/spark_session.rst | 8 +- python/pyspark/sql/session.py | 40 ++-- .../connect/test_parity_job_cancellation.py | 68 +++++++ .../pyspark/sql/tests/connect/test_session.py | 183 +----------------- .../sql/tests/test_job_cancellation.py | 183 ++++++++++++++++++ python/pyspark/sql/tests/test_session.py | 4 - python/pyspark/util.py | 41 +++- 7 files changed, 319 insertions(+), 208 deletions(-) create mode 100644 python/pyspark/sql/tests/connect/test_parity_job_cancellation.py create mode 100644 python/pyspark/sql/tests/test_job_cancellation.py diff --git a/python/docs/source/reference/pyspark.sql/spark_session.rst b/python/docs/source/reference/pyspark.sql/spark_session.rst index 859332fa5e428..1677d3e8e0209 100644 --- a/python/docs/source/reference/pyspark.sql/spark_session.rst +++ b/python/docs/source/reference/pyspark.sql/spark_session.rst @@ -44,13 +44,17 @@ See also :class:`SparkSession`. .. autosummary:: :toctree: api/ + SparkSession.addTag SparkSession.catalog + SparkSession.clearTags SparkSession.conf SparkSession.createDataFrame SparkSession.dataSource SparkSession.getActiveSession + SparkSession.getTags SparkSession.newSession SparkSession.profile + SparkSession.removeTag SparkSession.range SparkSession.read SparkSession.readStream @@ -79,15 +83,11 @@ Spark Connect Only SparkSession.addArtifact SparkSession.addArtifacts - SparkSession.addTag SparkSession.clearProgressHandlers - SparkSession.clearTags SparkSession.client SparkSession.copyFromLocalToFs - SparkSession.getTags SparkSession.interruptAll SparkSession.interruptOperation SparkSession.interruptTag SparkSession.registerProgressHandler SparkSession.removeProgressHandler - SparkSession.removeTag diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index e35a1cc82d795..d19e01eecc89c 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -2283,15 +2283,15 @@ def addTag(self, tag: str) -> None: .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Parameters ---------- tag : str The tag to be added. Cannot contain ',' (comma) character or be an empty string. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.addTag"}, - ) + self._jsparkSession.addTag(tag) @remote_only def removeTag(self, tag: str) -> None: @@ -2301,15 +2301,15 @@ def removeTag(self, tag: str) -> None: .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Parameters ---------- tag : list of str The tag to be removed. Cannot contain ',' (comma) character or be an empty string. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.removeTag"}, - ) + self._jsparkSession.removeTag(tag) @remote_only def getTags(self) -> Set[str]: @@ -2319,15 +2319,23 @@ def getTags(self) -> Set[str]: .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Returns ------- set of str Set of tags of interrupted operations. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.getTags"}, - ) + java_set = self._jsparkSession.getTags() + python_set = set() + + # Use iterator to manually iterate through Java Set + java_iterator = java_set.iterator() + while java_iterator.hasNext(): + python_set.add(str(java_iterator.next())) + + return python_set @remote_only def clearTags(self) -> None: @@ -2335,11 +2343,11 @@ def clearTags(self) -> None: Clear the current thread's operation tags. .. versionadded:: 3.5.0 + + .. versionchanged:: 4.0.0 + Supports Spark Classic. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.clearTags"}, - ) + self._jsparkSession.clearTags() def _test() -> None: diff --git a/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py new file mode 100644 index 0000000000000..c7f50495af15b --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py @@ -0,0 +1,68 @@ +# +# 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. +# + +import threading + +from pyspark import inheritable_thread_target +from pyspark.sql.tests.test_job_cancellation import JobCancellationTestsMixin +from pyspark.testing.connectutils import ReusedConnectTestCase + + +class JobCancellationParityTests(JobCancellationTestsMixin, ReusedConnectTestCase): + def test_inheritable_tags_with_deco(self): + @inheritable_thread_target(self.spark) + def func(target): + return target() + + self.check_inheritable_tags( + create_thread=lambda target, session: threading.Thread(target=func, args=(target,)) + ) + + def test_interrupt_tag(self): + thread_ids = range(4) + self.check_job_cancellation( + lambda job_group: self.spark.addTag(job_group), + lambda job_group: self.spark.interruptTag(job_group), + thread_ids, + [i for i in thread_ids if i % 2 == 0], + [i for i in thread_ids if i % 2 != 0], + ) + self.spark.clearTags() + + def test_interrupt_all(self): + thread_ids = range(4) + self.check_job_cancellation( + lambda job_group: None, + lambda job_group: self.spark.interruptAll(), + thread_ids, + thread_ids, + [], + ) + self.spark.clearTags() + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.connect.test_parity_serde import * # noqa: F401 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/connect/test_session.py b/python/pyspark/sql/tests/connect/test_session.py index 6f0e4aaad3f89..e327c868895f4 100644 --- a/python/pyspark/sql/tests/connect/test_session.py +++ b/python/pyspark/sql/tests/connect/test_session.py @@ -14,18 +14,12 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import threading -import time + import unittest from typing import Optional -from pyspark import InheritableThread, inheritable_thread_target from pyspark.sql.connect.client import DefaultChannelBuilder from pyspark.sql.connect.session import SparkSession as RemoteSparkSession -from pyspark.testing.connectutils import should_test_connect - -if should_test_connect: - from pyspark.testing.connectutils import ReusedConnectTestCase class CustomChannelBuilder(DefaultChannelBuilder): @@ -104,178 +98,3 @@ def test_default_session_expires_when_client_closes(self): s3 = RemoteSparkSession.builder.remote("sc://other").getOrCreate() self.assertIsNot(s1, s3) - - -class JobCancellationTests(ReusedConnectTestCase): - def test_tags(self): - self.spark.clearTags() - self.spark.addTag("a") - self.assertEqual(self.spark.getTags(), {"a"}) - self.spark.addTag("b") - self.spark.removeTag("a") - self.assertEqual(self.spark.getTags(), {"b"}) - self.spark.addTag("c") - self.spark.clearTags() - self.assertEqual(self.spark.getTags(), set()) - self.spark.clearTags() - - def test_tags_multithread(self): - output1 = None - output2 = None - - def tag1(): - nonlocal output1 - - self.spark.addTag("tag1") - output1 = self.spark.getTags() - - def tag2(): - nonlocal output2 - - self.spark.addTag("tag2") - output2 = self.spark.getTags() - - t1 = threading.Thread(target=tag1) - t1.start() - t1.join() - t2 = threading.Thread(target=tag2) - t2.start() - t2.join() - - self.assertIsNotNone(output1) - self.assertEquals(output1, {"tag1"}) - self.assertIsNotNone(output2) - self.assertEquals(output2, {"tag2"}) - - def test_interrupt_tag(self): - thread_ids = range(4) - self.check_job_cancellation( - lambda job_group: self.spark.addTag(job_group), - lambda job_group: self.spark.interruptTag(job_group), - thread_ids, - [i for i in thread_ids if i % 2 == 0], - [i for i in thread_ids if i % 2 != 0], - ) - self.spark.clearTags() - - def test_interrupt_all(self): - thread_ids = range(4) - self.check_job_cancellation( - lambda job_group: None, - lambda job_group: self.spark.interruptAll(), - thread_ids, - thread_ids, - [], - ) - self.spark.clearTags() - - def check_job_cancellation( - self, setter, canceller, thread_ids, thread_ids_to_cancel, thread_ids_to_run - ): - job_id_a = "job_ids_to_cancel" - job_id_b = "job_ids_to_run" - threads = [] - - # A list which records whether job is cancelled. - # The index of the array is the thread index which job run in. - is_job_cancelled = [False for _ in thread_ids] - - def run_job(job_id, index): - """ - Executes a job with the group ``job_group``. Each job waits for 3 seconds - and then exits. - """ - try: - setter(job_id) - - def func(itr): - for pdf in itr: - time.sleep(pdf._1.iloc[0]) - yield pdf - - self.spark.createDataFrame([[20]]).repartition(1).mapInPandas( - func, schema="_1 LONG" - ).collect() - is_job_cancelled[index] = False - except Exception: - # Assume that exception means job cancellation. - is_job_cancelled[index] = True - - # Test if job succeeded when not cancelled. - run_job(job_id_a, 0) - self.assertFalse(is_job_cancelled[0]) - self.spark.clearTags() - - # Run jobs - for i in thread_ids_to_cancel: - t = threading.Thread(target=run_job, args=(job_id_a, i)) - t.start() - threads.append(t) - - for i in thread_ids_to_run: - t = threading.Thread(target=run_job, args=(job_id_b, i)) - t.start() - threads.append(t) - - # Wait to make sure all jobs are executed. - time.sleep(10) - # And then, cancel one job group. - canceller(job_id_a) - - # Wait until all threads launching jobs are finished. - for t in threads: - t.join() - - for i in thread_ids_to_cancel: - self.assertTrue( - is_job_cancelled[i], "Thread {i}: Job in group A was not cancelled.".format(i=i) - ) - - for i in thread_ids_to_run: - self.assertFalse( - is_job_cancelled[i], "Thread {i}: Job in group B did not succeeded.".format(i=i) - ) - - def test_inheritable_tags(self): - self.check_inheritable_tags( - create_thread=lambda target, session: InheritableThread(target, session=session) - ) - self.check_inheritable_tags( - create_thread=lambda target, session: threading.Thread( - target=inheritable_thread_target(session)(target) - ) - ) - - # Test decorator usage - @inheritable_thread_target(self.spark) - def func(target): - return target() - - self.check_inheritable_tags( - create_thread=lambda target, session: threading.Thread(target=func, args=(target,)) - ) - - def check_inheritable_tags(self, create_thread): - spark = self.spark - spark.addTag("a") - first = set() - second = set() - - def get_inner_local_prop(): - spark.addTag("c") - second.update(spark.getTags()) - - def get_outer_local_prop(): - spark.addTag("b") - first.update(spark.getTags()) - t2 = create_thread(target=get_inner_local_prop, session=spark) - t2.start() - t2.join() - - t1 = create_thread(target=get_outer_local_prop, session=spark) - t1.start() - t1.join() - - self.assertEqual(spark.getTags(), {"a"}) - self.assertEqual(first, {"a", "b"}) - self.assertEqual(second, {"a", "b", "c"}) diff --git a/python/pyspark/sql/tests/test_job_cancellation.py b/python/pyspark/sql/tests/test_job_cancellation.py new file mode 100644 index 0000000000000..6fc404b0a0b3a --- /dev/null +++ b/python/pyspark/sql/tests/test_job_cancellation.py @@ -0,0 +1,183 @@ +# +# 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. +# + +import unittest +import threading +import time + +from pyspark import InheritableThread, inheritable_thread_target +from pyspark.testing.sqlutils import ReusedSQLTestCase + + +class JobCancellationTestsMixin: + def test_tags(self): + self.spark.clearTags() + self.spark.addTag("a") + self.assertEqual(self.spark.getTags(), {"a"}) + self.spark.addTag("b") + self.spark.removeTag("a") + self.assertEqual(self.spark.getTags(), {"b"}) + self.spark.addTag("c") + self.spark.clearTags() + self.assertEqual(self.spark.getTags(), set()) + self.spark.clearTags() + + def test_tags_multithread(self): + output1 = None + output2 = None + + def tag1(): + nonlocal output1 + + self.spark.addTag("tag1") + output1 = self.spark.getTags() + + def tag2(): + nonlocal output2 + + self.spark.addTag("tag2") + output2 = self.spark.getTags() + + t1 = threading.Thread(target=tag1) + t1.start() + t1.join() + t2 = threading.Thread(target=tag2) + t2.start() + t2.join() + + self.assertIsNotNone(output1) + self.assertEquals(output1, {"tag1"}) + self.assertIsNotNone(output2) + self.assertEquals(output2, {"tag2"}) + + def check_job_cancellation( + self, setter, canceller, thread_ids, thread_ids_to_cancel, thread_ids_to_run + ): + job_id_a = "job_ids_to_cancel" + job_id_b = "job_ids_to_run" + threads = [] + + # A list which records whether job is cancelled. + # The index of the array is the thread index which job run in. + is_job_cancelled = [False for _ in thread_ids] + + def run_job(job_id, index): + """ + Executes a job with the group ``job_group``. Each job waits for 3 seconds + and then exits. + """ + try: + setter(job_id) + + def func(itr): + for pdf in itr: + time.sleep(pdf._1.iloc[0]) + yield pdf + + self.spark.createDataFrame([[20]]).repartition(1).mapInPandas( + func, schema="_1 LONG" + ).collect() + is_job_cancelled[index] = False + except Exception: + # Assume that exception means job cancellation. + is_job_cancelled[index] = True + + # Test if job succeeded when not cancelled. + run_job(job_id_a, 0) + self.assertFalse(is_job_cancelled[0]) + self.spark.clearTags() + + # Run jobs + for i in thread_ids_to_cancel: + t = threading.Thread(target=run_job, args=(job_id_a, i)) + t.start() + threads.append(t) + + for i in thread_ids_to_run: + t = threading.Thread(target=run_job, args=(job_id_b, i)) + t.start() + threads.append(t) + + # Wait to make sure all jobs are executed. + time.sleep(10) + # And then, cancel one job group. + canceller(job_id_a) + + # Wait until all threads launching jobs are finished. + for t in threads: + t.join() + + for i in thread_ids_to_cancel: + self.assertTrue( + is_job_cancelled[i], "Thread {i}: Job in group A was not cancelled.".format(i=i) + ) + + for i in thread_ids_to_run: + self.assertFalse( + is_job_cancelled[i], "Thread {i}: Job in group B did not succeeded.".format(i=i) + ) + + def test_inheritable_tags(self): + self.check_inheritable_tags( + create_thread=lambda target, session: InheritableThread(target, session=session) + ) + self.check_inheritable_tags( + create_thread=lambda target, session: threading.Thread( + target=inheritable_thread_target(session)(target) + ) + ) + + def check_inheritable_tags(self, create_thread): + spark = self.spark + spark.addTag("a") + first = set() + second = set() + + def get_inner_local_prop(): + spark.addTag("c") + second.update(spark.getTags()) + + def get_outer_local_prop(): + spark.addTag("b") + first.update(spark.getTags()) + t2 = create_thread(target=get_inner_local_prop, session=spark) + t2.start() + t2.join() + + t1 = create_thread(target=get_outer_local_prop, session=spark) + t1.start() + t1.join() + + self.assertEqual(spark.getTags(), {"a"}) + self.assertEqual(first, {"a", "b"}) + self.assertEqual(second, {"a", "b", "c"}) + + +class JobCancellationTests(JobCancellationTestsMixin, ReusedSQLTestCase): + pass + + +if __name__ == "__main__": + from pyspark.sql.tests.test_session import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/test_session.py b/python/pyspark/sql/tests/test_session.py index de94b0d1882ff..3fbc0be943e45 100644 --- a/python/pyspark/sql/tests/test_session.py +++ b/python/pyspark/sql/tests/test_session.py @@ -229,10 +229,6 @@ def test_unsupported_api(self): (lambda: session.copyFromLocalToFs("", ""), "copyFromLocalToFs"), (lambda: session.interruptTag(""), "interruptTag"), (lambda: session.interruptOperation(""), "interruptOperation"), - (lambda: session.addTag(""), "addTag"), - (lambda: session.removeTag(""), "removeTag"), - (session.getTags, "getTags"), - (session.clearTags, "clearTags"), ] for func, name in unsupported: diff --git a/python/pyspark/util.py b/python/pyspark/util.py index 3b38b8b72c615..3e9a68ccfe2e5 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -27,6 +27,7 @@ import traceback import typing import socket +import warnings from types import TracebackType from typing import Any, Callable, IO, Iterator, List, Optional, TextIO, Tuple, Union @@ -366,7 +367,8 @@ def inheritable_thread_target(f: Optional[Union[Callable, "SparkSession"]] = Non >>> Thread(target=inheritable_thread_target(target_func)).start() # doctest: +SKIP - If you're using Spark Connect, you should explicitly provide Spark session as follows: + If you're using Spark Connect or if you want to inherit the tags properly, + you should explicitly provide Spark session as follows: >>> @inheritable_thread_target(session) # doctest: +SKIP ... def target_func(): @@ -406,13 +408,41 @@ def inner(*args: Any, **kwargs: Any) -> Any: return outer - # Non Spark Connect + # Non Spark Connect with SparkSession or Callable + from pyspark.sql import SparkSession from pyspark import SparkContext from py4j.clientserver import ClientServer if isinstance(SparkContext._gateway, ClientServer): # Here's when the pinned-thread mode (PYSPARK_PIN_THREAD) is on. + if isinstance(f, SparkSession): + session = f + assert session is not None + tags = set(session.getTags()) + # Local properties are copied when wrapping the function. + assert SparkContext._active_spark_context is not None + properties = SparkContext._active_spark_context._jsc.sc().getLocalProperties().clone() + + def outer(ff: Callable) -> Callable: + @functools.wraps(ff) + def wrapped(*args: Any, **kwargs: Any) -> Any: + # Apply properties and tags in the child thread. + assert SparkContext._active_spark_context is not None + SparkContext._active_spark_context._jsc.sc().setLocalProperties(properties) + for tag in tags: + session.addTag(tag) # type: ignore[union-attr] + return ff(*args, **kwargs) + + return wrapped + + return outer + + warnings.warn( + "Spark session is not provided. Tags will not be inherited.", + UserWarning, + ) + # NOTICE the internal difference vs `InheritableThread`. `InheritableThread` # copies local properties when the thread starts but `inheritable_thread_target` # copies when the function is wrapped. @@ -506,11 +536,15 @@ def copy_local_properties(*a: Any, **k: Any) -> Any: from pyspark import SparkContext from py4j.clientserver import ClientServer + self._session = session # type: ignore[assignment] if isinstance(SparkContext._gateway, ClientServer): # Here's when the pinned-thread mode (PYSPARK_PIN_THREAD) is on. def copy_local_properties(*a: Any, **k: Any) -> Any: # self._props is set before starting the thread to match the behavior with JVM. assert hasattr(self, "_props") + if hasattr(self, "_tags"): + for tag in self._tags: # type: ignore[has-type] + self._session.addTag(tag) assert SparkContext._active_spark_context is not None SparkContext._active_spark_context._jsc.sc().setLocalProperties(self._props) return target(*a, **k) @@ -546,6 +580,9 @@ def start(self) -> None: self._props = ( SparkContext._active_spark_context._jsc.sc().getLocalProperties().clone() ) + if self._session is not None: + self._tags = self._session.getTags() + return super(InheritableThread, self).start() From 4eed18455332164e63e58bbd8e828ebb9639143c Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Thu, 28 Nov 2024 17:30:26 +0800 Subject: [PATCH 048/438] [SPARK-50439][BUILD] Upgrade `Arrow` to 18.1.0 ### What changes were proposed in this pull request? This pr aims to upgrade apache arrow from 18.0.0 to 18.1.0 ### Why are the changes needed? To bring the latest improvements and bug fixes: - https://arrow.apache.org/release/18.1.0.html ### 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 #48975 from LuciferYang/arrow-18.1.0. Authored-by: yangjie01 Signed-off-by: panbingkun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 10 +++++----- pom.xml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index aef06b8a20f53..db325fbf546aa 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -16,11 +16,11 @@ antlr4-runtime/4.13.1//antlr4-runtime-4.13.1.jar aopalliance-repackaged/3.0.6//aopalliance-repackaged-3.0.6.jar arpack/3.0.3//arpack-3.0.3.jar arpack_combined_all/0.1//arpack_combined_all-0.1.jar -arrow-format/18.0.0//arrow-format-18.0.0.jar -arrow-memory-core/18.0.0//arrow-memory-core-18.0.0.jar -arrow-memory-netty-buffer-patch/18.0.0//arrow-memory-netty-buffer-patch-18.0.0.jar -arrow-memory-netty/18.0.0//arrow-memory-netty-18.0.0.jar -arrow-vector/18.0.0//arrow-vector-18.0.0.jar +arrow-format/18.1.0//arrow-format-18.1.0.jar +arrow-memory-core/18.1.0//arrow-memory-core-18.1.0.jar +arrow-memory-netty-buffer-patch/18.1.0//arrow-memory-netty-buffer-patch-18.1.0.jar +arrow-memory-netty/18.1.0//arrow-memory-netty-18.1.0.jar +arrow-vector/18.1.0//arrow-vector-18.1.0.jar audience-annotations/0.12.0//audience-annotations-0.12.0.jar avro-ipc/1.12.0//avro-ipc-1.12.0.jar avro-mapred/1.12.0//avro-mapred-1.12.0.jar diff --git a/pom.xml b/pom.xml index 65c644cc0690a..de8da1ea64a88 100644 --- a/pom.xml +++ b/pom.xml @@ -229,7 +229,7 @@ ./python/pyspark/sql/pandas/utils.py, ./python/packaging/classic/setup.py and ./python/packaging/connect/setup.py too. --> - 18.0.0 + 18.1.0 3.0.0 0.12.6 From 2c2c0e05512731e73fda98715551c021f12e24e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Du=C5=A1an=20Ti=C5=A1ma?= Date: Thu, 28 Nov 2024 20:47:12 +0800 Subject: [PATCH 049/438] [SPARK-48356][SQL] Support for FOR statement MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? In this PR, support for FOR statement in SQL scripting is introduced. Examples: ``` FOR row AS SELECT * FROM t DO SELECT row.intCol; END FOR; ``` ``` FOR SELECT * FROM t DO SELECT intCol; END FOR; ``` Implementation notes: As local variables for SQL scripting are currently a work in progress, session variables are used to simulate them. When FOR begins executing, session variables are declared for each column in the result set, and optionally for the for variable if it is present ("row" in the example above). On each iteration, these variables are overwritten with the values from the row currently being iterated. The variables are dropped upon loop completion. This means that if a session variable which matches the name of a column in the result set already exists, the for statement will drop that variable after completion. If that variable would be referenced after the for statement, the script would fail as the variable would not exist. This limitation is already present in the current iteration of SQL scripting, and will be fixed once local variables are introduced. Also, with local variables the implementation of for statement will be much simpler. Grammar/parser changes: `forStatement` grammar rule `visitForStatement` rule visitor `ForStatement` logical operator ### Why are the changes needed? FOR statement is an part of SQL scripting control flow logic. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New tests are introduced to all of the three scripting test suites: `SqlScriptingParserSuite`, `SqlScriptingExecutionNodeSuite` and `SqlScriptingInterpreterSuite`. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48794 from dusantism-db/scripting-for-loop. Authored-by: Dušan Tišma Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 5 + .../sql/catalyst/parser/AstBuilder.scala | 46 +- .../logical/SqlScriptingLogicalPlans.scala | 28 + .../parser/SqlScriptingParserSuite.scala | 268 ++++- .../scripting/SqlScriptingExecutionNode.scala | 229 +++- .../scripting/SqlScriptingInterpreter.scala | 13 +- .../SqlScriptingExecutionNodeSuite.scala | 389 +++++- .../SqlScriptingInterpreterSuite.scala | 1054 +++++++++++++++++ 8 files changed, 2006 insertions(+), 26 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 2e3235d6f932c..4b7b4634b74b2 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -70,6 +70,7 @@ compoundStatement | leaveStatement | iterateStatement | loopStatement + | forStatement ; setStatementWithOptionalVarKeyword @@ -111,6 +112,10 @@ loopStatement : beginLabel? LOOP compoundBody END LOOP endLabel? ; +forStatement + : beginLabel? FOR (multipartIdentifier AS)? query DO compoundBody END FOR endLabel? + ; + singleStatement : (statement|setResetStatement) SEMICOLON* EOF ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 6a9a97d0f5c8c..d558689a5c196 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -226,6 +226,8 @@ class AstBuilder extends DataTypeAstBuilder visitSearchedCaseStatementImpl(searchedCaseContext, labelCtx) case simpleCaseContext: SimpleCaseStatementContext => visitSimpleCaseStatementImpl(simpleCaseContext, labelCtx) + case forStatementContext: ForStatementContext => + visitForStatementImpl(forStatementContext, labelCtx) case stmt => visit(stmt).asInstanceOf[CompoundPlanStatement] } } else { @@ -347,28 +349,48 @@ class AstBuilder extends DataTypeAstBuilder RepeatStatement(condition, body, Some(labelText)) } + private def visitForStatementImpl( + ctx: ForStatementContext, + labelCtx: SqlScriptingLabelContext): ForStatement = { + val labelText = labelCtx.enterLabeledScope(Option(ctx.beginLabel()), Option(ctx.endLabel())) + + val queryCtx = ctx.query() + val query = withOrigin(queryCtx) { + SingleStatement(visitQuery(queryCtx)) + } + val varName = Option(ctx.multipartIdentifier()).map(_.getText) + val body = visitCompoundBodyImpl(ctx.compoundBody(), None, allowVarDeclare = false, labelCtx) + labelCtx.exitLabeledScope(Option(ctx.beginLabel())) + + ForStatement(query, varName, body, Some(labelText)) + } + private def leaveOrIterateContextHasLabel( ctx: RuleContext, label: String, isIterate: Boolean): Boolean = { ctx match { case c: BeginEndCompoundBlockContext - if Option(c.beginLabel()).isDefined && - c.beginLabel().multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) => - if (isIterate) { + if Option(c.beginLabel()).exists { b => + b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + } => if (isIterate) { throw SqlScriptingErrors.invalidIterateLabelUsageForCompound(CurrentOrigin.get, label) } true case c: WhileStatementContext - if Option(c.beginLabel()).isDefined && - c.beginLabel().multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) - => true + if Option(c.beginLabel()).exists { b => + b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + } => true case c: RepeatStatementContext - if Option(c.beginLabel()).isDefined && - c.beginLabel().multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) - => true + if Option(c.beginLabel()).exists { b => + b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + } => true case c: LoopStatementContext - if Option(c.beginLabel()).isDefined && - c.beginLabel().multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) - => true + if Option(c.beginLabel()).exists { b => + b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + } => true + case c: ForStatementContext + if Option(c.beginLabel()).exists { b => + b.multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) + } => true case _ => false } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala index e6018e5e57b9c..4faf1f5d26672 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala @@ -267,3 +267,31 @@ case class LoopStatement( LoopStatement(newChildren(0).asInstanceOf[CompoundBody], label) } } + +/** + * Logical operator for FOR statement. + * @param query Query which is executed once, then it's result set is iterated on, row by row. + * @param variableName Name of variable which is used to access the current row during iteration. + * @param body Compound body is a collection of statements that are executed for each row in + * the result set of the query. + * @param label An optional label for the loop which is unique amongst all labels for statements + * within which the FOR statement is contained. + * If an end label is specified it must match the beginning label. + * The label can be used to LEAVE or ITERATE the loop. + */ +case class ForStatement( + query: SingleStatement, + variableName: Option[String], + body: CompoundBody, + label: Option[String]) extends CompoundPlanStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq(query, body) + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = newChildren match { + case IndexedSeq(query: SingleStatement, body: CompoundBody) => + ForStatement(query, variableName, body, label) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index 3bb84f603dc67..ab647f83b42a4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{Alias, EqualTo, Expression, In, Literal, ScalarSubquery} import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.catalyst.plans.logical.{CaseStatement, CompoundBody, CreateVariable, IfElseStatement, IterateStatement, LeaveStatement, LoopStatement, Project, RepeatStatement, SingleStatement, WhileStatement} +import org.apache.spark.sql.catalyst.plans.logical.{CaseStatement, CompoundBody, CreateVariable, ForStatement, IfElseStatement, IterateStatement, LeaveStatement, LoopStatement, Project, RepeatStatement, SingleStatement, WhileStatement} import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.exceptions.SqlScriptingException import org.apache.spark.sql.internal.SQLConf @@ -1176,7 +1176,6 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { head.asInstanceOf[SingleStatement].getText == "SELECT 42") assert(whileStmt.label.contains("lbl")) - } test("searched case statement") { @@ -1823,6 +1822,25 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { parameters = Map("label" -> toSQLId("l_loop"))) } + test("unique label names: nested for loops") { + val sqlScriptText = + """BEGIN + |f_loop: FOR x AS SELECT 1 DO + | f_loop: FOR y AS SELECT 2 DO + | SELECT 1; + | END FOR; + |END FOR; + |END + """.stripMargin + val exception = intercept[SqlScriptingException] { + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + } + checkError( + exception = exception, + condition = "LABEL_ALREADY_EXISTS", + parameters = Map("label" -> toSQLId("f_loop"))) + } + test("unique label names: begin-end block on the same level") { val sqlScriptText = """BEGIN @@ -1858,10 +1876,13 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SELECT 4; |UNTIL 1=1 |END REPEAT; + |lbl: FOR x AS SELECT 1 DO + | SELECT 5; + |END FOR; |END """.stripMargin val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] - assert(tree.collection.length == 4) + assert(tree.collection.length == 5) assert(tree.collection.head.isInstanceOf[CompoundBody]) assert(tree.collection.head.asInstanceOf[CompoundBody].label.get == "lbl") assert(tree.collection(1).isInstanceOf[WhileStatement]) @@ -1870,6 +1891,8 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(tree.collection(2).asInstanceOf[LoopStatement].label.get == "lbl") assert(tree.collection(3).isInstanceOf[RepeatStatement]) assert(tree.collection(3).asInstanceOf[RepeatStatement].label.get == "lbl") + assert(tree.collection(4).isInstanceOf[ForStatement]) + assert(tree.collection(4).asInstanceOf[ForStatement].label.get == "lbl") } test("unique label names: nested labeled scope statements") { @@ -1879,7 +1902,9 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | lbl_1: WHILE 1=1 DO | lbl_2: LOOP | lbl_3: REPEAT - | SELECT 4; + | lbl_4: FOR x AS SELECT 1 DO + | SELECT 4; + | END FOR; | UNTIL 1=1 | END REPEAT; | END LOOP; @@ -1905,6 +1930,241 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { // Repeat statement val repeatStatement = loopStatement.body.collection.head.asInstanceOf[RepeatStatement] assert(repeatStatement.label.get == "lbl_3") + // For statement + val forStatement = repeatStatement.body.collection.head.asInstanceOf[ForStatement] + assert(forStatement.label.get == "lbl_4") + } + + test("for statement") { + val sqlScriptText = + """ + |BEGIN + | lbl: FOR x AS SELECT 5 DO + | SELECT 1; + | END FOR; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT 5") + assert(forStmt.variableName.contains("x")) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 1) + assert(forStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(forStmt.body.collection.head.asInstanceOf[SingleStatement].getText == "SELECT 1") + + assert(forStmt.label.contains("lbl")) + } + + test("for statement - no label") { + val sqlScriptText = + """ + |BEGIN + | FOR x AS SELECT 5 DO + | SELECT 1; + | END FOR; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT 5") + assert(forStmt.variableName.contains("x")) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 1) + assert(forStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(forStmt.body.collection.head.asInstanceOf[SingleStatement].getText == "SELECT 1") + + // when not explicitly set, label is random UUID + assert(forStmt.label.isDefined) + } + + test("for statement - with complex subquery") { + val sqlScriptText = + """ + |BEGIN + | lbl: FOR x AS SELECT c1, c2 FROM t WHERE c2 = 5 GROUP BY c1 ORDER BY c1 DO + | SELECT x.c1; + | SELECT x.c2; + | END FOR; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT c1, c2 FROM t WHERE c2 = 5 GROUP BY c1 ORDER BY c1") + assert(forStmt.variableName.contains("x")) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 2) + assert(forStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(forStmt.body.collection.head.asInstanceOf[SingleStatement].getText == "SELECT x.c1") + assert(forStmt.body.collection(1).isInstanceOf[SingleStatement]) + assert(forStmt.body.collection(1).asInstanceOf[SingleStatement].getText == "SELECT x.c2") + + assert(forStmt.label.contains("lbl")) + } + + test("for statement - nested") { + val sqlScriptText = + """ + |BEGIN + | lbl1: FOR i AS SELECT 1 DO + | lbl2: FOR j AS SELECT 2 DO + | SELECT i + j; + | END FOR lbl2; + | END FOR lbl1; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT 1") + assert(forStmt.variableName.contains("i")) + assert(forStmt.label.contains("lbl1")) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 1) + assert(forStmt.body.collection.head.isInstanceOf[ForStatement]) + val nestedForStmt = forStmt.body.collection.head.asInstanceOf[ForStatement] + + assert(nestedForStmt.query.isInstanceOf[SingleStatement]) + assert(nestedForStmt.query.getText == "SELECT 2") + assert(nestedForStmt.variableName.contains("j")) + assert(nestedForStmt.label.contains("lbl2")) + + assert(nestedForStmt.body.isInstanceOf[CompoundBody]) + assert(nestedForStmt.body.collection.length == 1) + assert(nestedForStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(nestedForStmt.body.collection. + head.asInstanceOf[SingleStatement].getText == "SELECT i + j") + } + + test("for statement - no variable") { + val sqlScriptText = + """ + |BEGIN + | lbl: FOR SELECT 5 DO + | SELECT 1; + | END FOR; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT 5") + assert(forStmt.variableName.isEmpty) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 1) + assert(forStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(forStmt.body.collection.head.asInstanceOf[SingleStatement].getText == "SELECT 1") + + assert(forStmt.label.contains("lbl")) + } + + test("for statement - no variable - no label") { + val sqlScriptText = + """ + |BEGIN + | FOR SELECT 5 DO + | SELECT 1; + | END FOR; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT 5") + assert(forStmt.variableName.isEmpty) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 1) + assert(forStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(forStmt.body.collection.head.asInstanceOf[SingleStatement].getText == "SELECT 1") + + // when not explicitly set, label is random UUID + assert(forStmt.label.isDefined) + } + + test("for statement - no variable - with complex subquery") { + val sqlScriptText = + """ + |BEGIN + | lbl: FOR SELECT c1, c2 FROM t WHERE c2 = 5 GROUP BY c1 ORDER BY c1 DO + | SELECT 1; + | SELECT 2; + | END FOR; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT c1, c2 FROM t WHERE c2 = 5 GROUP BY c1 ORDER BY c1") + assert(forStmt.variableName.isEmpty) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 2) + assert(forStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(forStmt.body.collection.head.asInstanceOf[SingleStatement].getText == "SELECT 1") + assert(forStmt.body.collection(1).isInstanceOf[SingleStatement]) + assert(forStmt.body.collection(1).asInstanceOf[SingleStatement].getText == "SELECT 2") + + assert(forStmt.label.contains("lbl")) + } + + test("for statement - no variable - nested") { + val sqlScriptText = + """ + |BEGIN + | lbl1: FOR SELECT 1 DO + | lbl2: FOR SELECT 2 DO + | SELECT 3; + | END FOR lbl2; + | END FOR lbl1; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[ForStatement]) + + val forStmt = tree.collection.head.asInstanceOf[ForStatement] + assert(forStmt.query.isInstanceOf[SingleStatement]) + assert(forStmt.query.getText == "SELECT 1") + assert(forStmt.variableName.isEmpty) + assert(forStmt.label.contains("lbl1")) + + assert(forStmt.body.isInstanceOf[CompoundBody]) + assert(forStmt.body.collection.length == 1) + assert(forStmt.body.collection.head.isInstanceOf[ForStatement]) + val nestedForStmt = forStmt.body.collection.head.asInstanceOf[ForStatement] + + assert(nestedForStmt.query.isInstanceOf[SingleStatement]) + assert(nestedForStmt.query.getText == "SELECT 2") + assert(nestedForStmt.variableName.isEmpty) + assert(nestedForStmt.label.contains("lbl2")) + + assert(nestedForStmt.body.isInstanceOf[CompoundBody]) + assert(nestedForStmt.body.collection.length == 1) + assert(nestedForStmt.body.collection.head.isInstanceOf[SingleStatement]) + assert(nestedForStmt.body.collection. + head.asInstanceOf[SingleStatement].getText == "SELECT 3") } // Helper methods diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index 94284ec514f55..e3559e8f18ae2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.scripting +import java.util + import org.apache.spark.SparkException import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} -import org.apache.spark.sql.catalyst.analysis.NameParameterizedQuery -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.analysis.{NameParameterizedQuery, UnresolvedAttribute, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.expressions.{Alias, CreateArray, CreateMap, CreateNamedStruct, Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, DefaultValueExpression, DropVariable, LogicalPlan, OneRowRelation, Project, SetVariable} import org.apache.spark.sql.catalyst.trees.{Origin, WithOrigin} import org.apache.spark.sql.errors.SqlScriptingErrors import org.apache.spark.sql.types.BooleanType @@ -662,3 +664,222 @@ class LoopStatementExec( body.reset() } } + +/** + * Executable node for ForStatement. + * @param query Executable node for the query. + * @param variableName Name of variable used for accessing current row during iteration. + * @param body Executable node for the body. + * @param label Label set to ForStatement by user or None otherwise. + * @param session Spark session that SQL script is executed within. + */ +class ForStatementExec( + query: SingleStatementExec, + variableName: Option[String], + body: CompoundBodyExec, + val label: Option[String], + session: SparkSession) extends NonLeafStatementExec { + + private object ForState extends Enumeration { + val VariableAssignment, Body, VariableCleanup = Value + } + private var state = ForState.VariableAssignment + private var areVariablesDeclared = false + + // map of all variables created internally by the for statement + // (variableName -> variableExpression) + private var variablesMap: Map[String, Expression] = Map() + + // compound body used for dropping variables while in ForState.VariableAssignment + private var dropVariablesExec: CompoundBodyExec = null + + private var queryResult: util.Iterator[Row] = _ + private var isResultCacheValid = false + private def cachedQueryResult(): util.Iterator[Row] = { + if (!isResultCacheValid) { + queryResult = query.buildDataFrame(session).toLocalIterator() + query.isExecuted = true + isResultCacheValid = true + } + queryResult + } + + /** + * For can be interrupted by LeaveStatementExec + */ + private var interrupted: Boolean = false + + private lazy val treeIterator: Iterator[CompoundStatementExec] = + new Iterator[CompoundStatementExec] { + + override def hasNext: Boolean = !interrupted && (state match { + case ForState.VariableAssignment => cachedQueryResult().hasNext + case ForState.Body => true + case ForState.VariableCleanup => dropVariablesExec.getTreeIterator.hasNext + }) + + override def next(): CompoundStatementExec = state match { + + case ForState.VariableAssignment => + variablesMap = createVariablesMapFromRow(cachedQueryResult().next()) + + if (!areVariablesDeclared) { + // create and execute declare var statements + variablesMap.keys.toSeq + .map(colName => createDeclareVarExec(colName, variablesMap(colName))) + .foreach(declareVarExec => declareVarExec.buildDataFrame(session).collect()) + areVariablesDeclared = true + } + + // create and execute set var statements + variablesMap.keys.toSeq + .map(colName => createSetVarExec(colName, variablesMap(colName))) + .foreach(setVarExec => setVarExec.buildDataFrame(session).collect()) + + state = ForState.Body + body.reset() + next() + + case ForState.Body => + val retStmt = body.getTreeIterator.next() + + // Handle LEAVE or ITERATE statement if it has been encountered. + retStmt match { + case leaveStatementExec: LeaveStatementExec if !leaveStatementExec.hasBeenMatched => + if (label.contains(leaveStatementExec.label)) { + leaveStatementExec.hasBeenMatched = true + } + interrupted = true + // If this for statement encounters LEAVE, it will either not be executed + // again, or it will be reset before being executed. + // In either case, variables will not + // be dropped normally, from ForState.VariableCleanup, so we drop them here. + dropVars() + return retStmt + case iterStatementExec: IterateStatementExec if !iterStatementExec.hasBeenMatched => + if (label.contains(iterStatementExec.label)) { + iterStatementExec.hasBeenMatched = true + } else { + // if an outer loop is being iterated, this for statement will either not be + // executed again, or it will be reset before being executed. + // In either case, variables will not + // be dropped normally, from ForState.VariableCleanup, so we drop them here. + dropVars() + } + switchStateFromBody() + return retStmt + case _ => + } + + if (!body.getTreeIterator.hasNext) { + switchStateFromBody() + } + retStmt + + case ForState.VariableCleanup => + dropVariablesExec.getTreeIterator.next() + } + } + + /** + * Recursively creates a Catalyst expression from Scala value.
+ * See https://spark.apache.org/docs/latest/sql-ref-datatypes.html for Spark -> Scala mappings + */ + private def createExpressionFromValue(value: Any): Expression = value match { + case m: Map[_, _] => + // arguments of CreateMap are in the format: (key1, val1, key2, val2, ...) + val mapArgs = m.keys.toSeq.flatMap { key => + Seq(createExpressionFromValue(key), createExpressionFromValue(m(key))) + } + CreateMap(mapArgs, useStringTypeWhenEmpty = false) + + // structs and rows match this case + case s: Row => + // arguments of CreateNamedStruct are in the format: (name1, val1, name2, val2, ...) + val namedStructArgs = s.schema.names.toSeq.flatMap { colName => + val valueExpression = createExpressionFromValue(s.getAs(colName)) + Seq(Literal(colName), valueExpression) + } + CreateNamedStruct(namedStructArgs) + + // arrays match this case + case a: collection.Seq[_] => + val arrayArgs = a.toSeq.map(createExpressionFromValue(_)) + CreateArray(arrayArgs, useStringTypeWhenEmpty = false) + + case _ => Literal(value) + } + + private def createVariablesMapFromRow(row: Row): Map[String, Expression] = { + var variablesMap = row.schema.names.toSeq.map { colName => + colName -> createExpressionFromValue(row.getAs(colName)) + }.toMap + + if (variableName.isDefined) { + val namedStructArgs = variablesMap.keys.toSeq.flatMap { colName => + Seq(Literal(colName), variablesMap(colName)) + } + val forVariable = CreateNamedStruct(namedStructArgs) + variablesMap = variablesMap + (variableName.get -> forVariable) + } + variablesMap + } + + /** + * Create and immediately execute dropVariable exec nodes for all variables in variablesMap. + */ + private def dropVars(): Unit = { + variablesMap.keys.toSeq + .map(colName => createDropVarExec(colName)) + .foreach(dropVarExec => dropVarExec.buildDataFrame(session).collect()) + areVariablesDeclared = false + } + + private def switchStateFromBody(): Unit = { + state = if (cachedQueryResult().hasNext) ForState.VariableAssignment + else { + // create compound body for dropping nodes after execution is complete + dropVariablesExec = new CompoundBodyExec( + variablesMap.keys.toSeq.map(colName => createDropVarExec(colName)) + ) + ForState.VariableCleanup + } + } + + private def createDeclareVarExec(varName: String, variable: Expression): SingleStatementExec = { + val defaultExpression = DefaultValueExpression(Literal(null, variable.dataType), "null") + val declareVariable = CreateVariable( + UnresolvedIdentifier(Seq(varName)), + defaultExpression, + replace = true + ) + new SingleStatementExec(declareVariable, Origin(), Map.empty, isInternal = true) + } + + private def createSetVarExec(varName: String, variable: Expression): SingleStatementExec = { + val projectNamedStruct = Project( + Seq(Alias(variable, varName)()), + OneRowRelation() + ) + val setIdentifierToCurrentRow = + SetVariable(Seq(UnresolvedAttribute(varName)), projectNamedStruct) + new SingleStatementExec(setIdentifierToCurrentRow, Origin(), Map.empty, isInternal = true) + } + + private def createDropVarExec(varName: String): SingleStatementExec = { + val dropVar = DropVariable(UnresolvedIdentifier(Seq(varName)), ifExists = true) + new SingleStatementExec(dropVar, Origin(), Map.empty, isInternal = true) + } + + override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + + override def reset(): Unit = { + state = ForState.VariableAssignment + isResultCacheValid = false + variablesMap = Map() + areVariablesDeclared = false + dropVariablesExec = null + interrupted = false + body.reset() + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala index 387ae36b881f4..a3dc3d4599314 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.scripting import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.{CaseStatement, CompoundBody, CompoundPlanStatement, CreateVariable, DropVariable, IfElseStatement, IterateStatement, LeaveStatement, LogicalPlan, LoopStatement, RepeatStatement, SingleStatement, WhileStatement} +import org.apache.spark.sql.catalyst.plans.logical.{CaseStatement, CompoundBody, CompoundPlanStatement, CreateVariable, DropVariable, ForStatement, IfElseStatement, IterateStatement, LeaveStatement, LogicalPlan, LoopStatement, RepeatStatement, SingleStatement, WhileStatement} import org.apache.spark.sql.catalyst.trees.Origin /** @@ -145,6 +145,17 @@ case class SqlScriptingInterpreter(session: SparkSession) { .asInstanceOf[CompoundBodyExec] new LoopStatementExec(bodyExec, label) + case ForStatement(query, variableNameOpt, body, label) => + val queryExec = + new SingleStatementExec( + query.parsedPlan, + query.origin, + args, + isInternal = false) + val bodyExec = + transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec] + new ForStatementExec(queryExec, variableNameOpt, bodyExec, label, session) + case leaveStatement: LeaveStatement => new LeaveStatementExec(leaveStatement.label) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala index 4874ea3d2795f..a997b5beadd34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.scripting import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, OneRowRelation, Project} +import org.apache.spark.sql.catalyst.plans.logical.{DropVariable, LeafNode, OneRowRelation, Project} import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} /** * Unit tests for execution nodes from SqlScriptingExecutionNode.scala. @@ -82,9 +83,9 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } case class TestRepeat( - condition: TestLoopCondition, - body: CompoundBodyExec, - label: Option[String] = None) + condition: TestLoopCondition, + body: CompoundBodyExec, + label: Option[String] = None) extends RepeatStatementExec(condition, body, label, spark) { private val evaluator = new LoopBooleanConditionEvaluator(condition) @@ -94,6 +95,23 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi statement: LeafStatementExec): Boolean = evaluator.evaluateLoopBooleanCondition() } + case class MockQuery(numberOfRows: Int, columnName: String, description: String) + extends SingleStatementExec( + DummyLogicalPlan(), + Origin(startIndex = Some(0), stopIndex = Some(description.length)), + Map.empty, + isInternal = false) { + override def buildDataFrame(session: SparkSession): DataFrame = { + val data = Seq.range(0, numberOfRows).map(Row(_)) + val schema = List(StructField(columnName, IntegerType)) + + spark.createDataFrame( + spark.sparkContext.parallelize(data), + StructType(schema) + ) + } + } + private def extractStatementValue(statement: CompoundStatementExec): String = statement match { case TestLeafStatement(testVal) => testVal @@ -102,6 +120,9 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi case loopStmt: LoopStatementExec => loopStmt.label.get case leaveStmt: LeaveStatementExec => leaveStmt.label case iterateStmt: IterateStatementExec => iterateStmt.label + case forStmt: ForStatementExec => forStmt.label.get + case dropStmt: SingleStatementExec if dropStmt.parsedPlan.isInstanceOf[DropVariable] + => "DropVariable" case _ => fail("Unexpected statement type") } @@ -688,4 +709,362 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi val statements = iter.map(extractStatementValue).toSeq assert(statements === Seq("body1", "lbl")) } + + test("for statement - enters body once") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(1, "intCol", "query1"), + variableName = Some("x"), + label = Some("for1"), + session = spark, + body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "body", + "DropVariable", // drop for query var intCol + "DropVariable" // drop for loop var x + )) + } + + test("for statement - enters body with multiple statements multiple times") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = Some("x"), + label = Some("for1"), + session = spark, + body = new CompoundBodyExec( + Seq(TestLeafStatement("statement1"), TestLeafStatement("statement2")) + ) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "statement1", + "statement2", + "statement1", + "statement2", + "DropVariable", // drop for query var intCol + "DropVariable" // drop for loop var x + )) + } + + test("for statement - empty result") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(0, "intCol", "query1"), + variableName = Some("x"), + label = Some("for1"), + session = spark, + body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq.empty[String]) + } + + test("for statement - nested") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = Some("x"), + label = Some("for1"), + session = spark, + body = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol1", "query2"), + variableName = Some("y"), + label = Some("for2"), + session = spark, + body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + ) + )) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "body", + "body", + "DropVariable", // drop for query var intCol1 + "DropVariable", // drop for loop var y + "body", + "body", + "DropVariable", // drop for query var intCol1 + "DropVariable", // drop for loop var y + "DropVariable", // drop for query var intCol + "DropVariable" // drop for loop var x + )) + } + + test("for statement no variable - enters body once") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(1, "intCol", "query1"), + variableName = None, + label = Some("for1"), + session = spark, + body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "body", + "DropVariable" // drop for query var intCol + )) + } + + test("for statement no variable - enters body with multiple statements multiple times") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = None, + label = Some("for1"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("statement1"), + TestLeafStatement("statement2"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "statement1", "statement2", "statement1", "statement2", + "DropVariable" // drop for query var intCol + )) + } + + test("for statement no variable - empty result") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(0, "intCol", "query1"), + variableName = None, + label = Some("for1"), + session = spark, + body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq.empty[String]) + } + + test("for statement no variable - nested") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = None, + label = Some("for1"), + session = spark, + body = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol1", "query2"), + variableName = None, + label = Some("for2"), + session = spark, + body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + ) + )) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "body", "body", + "DropVariable", // drop for query var intCol1 + "body", "body", + "DropVariable", // drop for query var intCol1 + "DropVariable" // drop for query var intCol + )) + } + + test("for statement - iterate") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = Some("x"), + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("statement1"), + new IterateStatementExec("lbl1"), + TestLeafStatement("statement2"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "statement1", + "lbl1", + "statement1", + "lbl1", + "DropVariable", // drop for query var intCol + "DropVariable" // drop for loop var x + )) + } + + test("for statement - leave") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = Some("x"), + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("statement1"), + new LeaveStatementExec("lbl1"), + TestLeafStatement("statement2"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("statement1", "lbl1")) + } + + test("for statement - nested - iterate outer loop") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = Some("x"), + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("outer_body"), + new ForStatementExec( + query = MockQuery(2, "intCol1", "query2"), + variableName = Some("y"), + label = Some("lbl2"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("body1"), + new IterateStatementExec("lbl1"), + TestLeafStatement("body2"))) + ) + )) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "outer_body", + "body1", + "lbl1", + "outer_body", + "body1", + "lbl1", + "DropVariable", // drop for query var intCol + "DropVariable" // drop for loop var x + )) + } + + test("for statement - nested - leave outer loop") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = Some("x"), + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query2"), + variableName = Some("y"), + label = Some("lbl2"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("body1"), + new LeaveStatementExec("lbl1"), + TestLeafStatement("body2"))) + ) + )) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("body1", "lbl1")) + } + + test("for statement no variable - iterate") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = None, + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("statement1"), + new IterateStatementExec("lbl1"), + TestLeafStatement("statement2"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "statement1", "lbl1", "statement1", "lbl1", + "DropVariable" // drop for query var intCol + )) + } + + test("for statement no variable - leave") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = None, + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("statement1"), + new LeaveStatementExec("lbl1"), + TestLeafStatement("statement2"))) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("statement1", "lbl1")) + } + + test("for statement no variable - nested - iterate outer loop") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = None, + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("outer_body"), + new ForStatementExec( + query = MockQuery(2, "intCol1", "query2"), + variableName = None, + label = Some("lbl2"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("body1"), + new IterateStatementExec("lbl1"), + TestLeafStatement("body2"))) + ) + )) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq( + "outer_body", "body1", "lbl1", "outer_body", "body1", "lbl1", + "DropVariable" // drop for query var intCol + )) + } + + test("for statement no variable - nested - leave outer loop") { + val iter = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol", "query1"), + variableName = None, + label = Some("lbl1"), + session = spark, + body = new CompoundBodyExec(Seq( + new ForStatementExec( + query = MockQuery(2, "intCol1", "query2"), + variableName = None, + label = Some("lbl2"), + session = spark, + body = new CompoundBodyExec(Seq( + TestLeafStatement("body1"), + new LeaveStatementExec("lbl1"), + TestLeafStatement("body2"))) + ) + )) + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("body1", "lbl1")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 177ffc24d180a..71556c5502225 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -1550,4 +1550,1058 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { ) verifySqlScriptResult(sqlScriptText, expected) } + + test("for statement - enters body once") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING, doubleCol DOUBLE) using parquet; + | INSERT INTO t VALUES (1, 'first', 1.0); + | FOR row AS SELECT * FROM t DO + | SELECT row.intCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(1)), // select row.intCol + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - enters body with multiple statements multiple times") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING, doubleCol DOUBLE) using parquet; + | INSERT INTO t VALUES (1, 'first', 1.0); + | INSERT INTO t VALUES (2, 'second', 2.0); + | FOR row AS SELECT * FROM t ORDER BY intCol DO + | SELECT row.intCol; + | SELECT intCol; + | SELECT row.stringCol; + | SELECT stringCol; + | SELECT row.doubleCol; + | SELECT doubleCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(1)), // select row.intCol + Seq(Row(1)), // select intCol + Seq(Row("first")), // select row.stringCol + Seq(Row("first")), // select stringCol + Seq(Row(1.0)), // select row.doubleCol + Seq(Row(1.0)), // select doubleCol + Seq(Row(2)), // select row.intCol + Seq(Row(2)), // select intCol + Seq(Row("second")), // select row.stringCol + Seq(Row("second")), // select stringCol + Seq(Row(2.0)), // select row.doubleCol + Seq(Row(2.0)), // select doubleCol + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - sum of column from table") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | DECLARE sumOfCols = 0; + | CREATE TABLE t (intCol INT) using parquet; + | INSERT INTO t VALUES (1), (2), (3), (4); + | FOR row AS SELECT * FROM t DO + | SET sumOfCols = sumOfCols + row.intCol; + | END FOR; + | SELECT sumOfCols; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // declare sumOfCols + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq(Row(10)), // select sumOfCols + Seq.empty[Row] // drop sumOfCols + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - map, struct, array") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (int_column INT, map_column MAP, + | struct_column STRUCT, array_column ARRAY); + | INSERT INTO t VALUES + | (1, MAP('a', 1), STRUCT('John', 25), ARRAY('apricot', 'quince')), + | (2, MAP('b', 2), STRUCT('Jane', 30), ARRAY('plum', 'pear')); + | FOR row AS SELECT * FROM t ORDER BY int_column DO + | SELECT row.map_column; + | SELECT map_column; + | SELECT row.struct_column; + | SELECT struct_column; + | SELECT row.array_column; + | SELECT array_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Map("a" -> 1))), // select row.map_column + Seq(Row(Map("a" -> 1))), // select map_column + Seq(Row(Row("John", 25))), // select row.struct_column + Seq(Row(Row("John", 25))), // select struct_column + Seq(Row(Array("apricot", "quince"))), // select row.array_column + Seq(Row(Array("apricot", "quince"))), // select array_column + Seq(Row(Map("b" -> 2))), // select row.map_column + Seq(Row(Map("b" -> 2))), // select map_column + Seq(Row(Row("Jane", 30))), // select row.struct_column + Seq(Row(Row("Jane", 30))), // select struct_column + Seq(Row(Array("plum", "pear"))), // select row.array_column + Seq(Row(Array("plum", "pear"))), // select array_column + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested struct") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t + | (int_column INT, + | struct_column STRUCT>>); + | INSERT INTO t VALUES + | (1, STRUCT(1, STRUCT(STRUCT("one")))), + | (2, STRUCT(2, STRUCT(STRUCT("two")))); + | FOR row AS SELECT * FROM t ORDER BY int_column DO + | SELECT row.struct_column; + | SELECT struct_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Row(1, Row(Row("one"))))), // select row.struct_column + Seq(Row(Row(1, Row(Row("one"))))), // select struct_column + Seq(Row(Row(2, Row(Row("two"))))), // select row.struct_column + Seq(Row(Row(2, Row(Row("two"))))), // select struct_column + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested map") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (int_column INT, map_column MAP>>); + | INSERT INTO t VALUES + | (1, MAP('a', MAP(1, MAP(false, 10)))), + | (2, MAP('b', MAP(2, MAP(true, 20)))); + | FOR row AS SELECT * FROM t ORDER BY int_column DO + | SELECT row.map_column; + | SELECT map_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Map("a" -> Map(1 -> Map(false -> 10))))), // select row.map_column + Seq(Row(Map("a" -> Map(1 -> Map(false -> 10))))), // select map_column + Seq(Row(Map("b" -> Map(2 -> Map(true -> 20))))), // select row.map_column + Seq(Row(Map("b" -> Map(2 -> Map(true -> 20))))), // select map_column + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested array") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t + | (int_column INT, array_column ARRAY>>); + | INSERT INTO t VALUES + | (1, ARRAY(ARRAY(ARRAY(1, 2), ARRAY(3, 4)), ARRAY(ARRAY(5, 6)))), + | (2, ARRAY(ARRAY(ARRAY(7, 8), ARRAY(9, 10)), ARRAY(ARRAY(11, 12)))); + | FOR row AS SELECT * FROM t ORDER BY int_column DO + | SELECT row.array_column; + | SELECT array_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Seq(Seq(Seq(1, 2), Seq(3, 4)), Seq(Seq(5, 6))))), // row.array_column + Seq(Row(Seq(Seq(Seq(1, 2), Seq(3, 4)), Seq(Seq(5, 6))))), // array_column + Seq(Row(Array(Seq(Seq(7, 8), Seq(9, 10)), Seq(Seq(11, 12))))), // row.array_column + Seq(Row(Array(Seq(Seq(7, 8), Seq(9, 10)), Seq(Seq(11, 12))))), // array_column + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement empty result") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | FOR row AS SELECT * FROM t ORDER BY intCol DO + | SELECT row.intCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row] // create table + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement iterate") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING) using parquet; + | INSERT INTO t VALUES (1, 'first'), (2, 'second'), (3, 'third'), (4, 'fourth'); + | + | lbl: FOR x AS SELECT * FROM t ORDER BY intCol DO + | IF x.intCol = 2 THEN + | ITERATE lbl; + | END IF; + | SELECT stringCol; + | SELECT x.stringCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row("first")), // select stringCol + Seq(Row("first")), // select x.stringCol + Seq(Row("third")), // select stringCol + Seq(Row("third")), // select x.stringCol + Seq(Row("fourth")), // select stringCol + Seq(Row("fourth")), // select x.stringCol + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement leave") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING) using parquet; + | INSERT INTO t VALUES (1, 'first'), (2, 'second'), (3, 'third'), (4, 'fourth'); + | + | lbl: FOR x AS SELECT * FROM t ORDER BY intCol DO + | IF x.intCol = 3 THEN + | LEAVE lbl; + | END IF; + | SELECT stringCol; + | SELECT x.stringCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row("first")), // select stringCol + Seq(Row("first")), // select x.stringCol + Seq(Row("second")), // select stringCol + Seq(Row("second")) // select x.stringCol + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested - in while") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | DECLARE cnt = 0; + | CREATE TABLE t (intCol INT) using parquet; + | INSERT INTO t VALUES (0); + | WHILE cnt < 2 DO + | SET cnt = cnt + 1; + | FOR x AS SELECT * FROM t ORDER BY intCol DO + | SELECT x.intCol; + | END FOR; + | INSERT INTO t VALUES (cnt); + | END WHILE; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // declare cnt + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq(Row(1)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop cnt + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested - in other for") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | FOR x as SELECT * FROM t ORDER BY intCol DO + | FOR y AS SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT x.intCol; + | SELECT intCol; + | SELECT y.intCol2; + | SELECT intCol2; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(0)), // select x.intCol + Seq(Row(0)), // select intCol + Seq(Row(3)), // select y.intCol2 + Seq(Row(3)), // select intCol2 + Seq(Row(0)), // select x.intCol + Seq(Row(0)), // select intCol + Seq(Row(2)), // select y.intCol2 + Seq(Row(2)), // select intCol2 + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq(Row(1)), // select x.intCol + Seq(Row(1)), // select intCol + Seq(Row(3)), // select y.intCol2 + Seq(Row(3)), // select intCol2 + Seq(Row(1)), // select x.intCol + Seq(Row(1)), // select intCol + Seq(Row(2)), // select y.intCol2 + Seq(Row(2)), // select intCol2 + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop outer var + Seq.empty[Row] // drop outer var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + // ignored until loops are fixed to support empty bodies + ignore("for statement - nested - empty result set") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | REPEAT + | FOR x AS SELECT * FROM t ORDER BY intCol DO + | SELECT x.intCol; + | END FOR; + | UNTIL 1 = 1 + | END REPEAT; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // declare cnt + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq(Row(1)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop cnt + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested - iterate outer loop") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | lbl1: FOR x as SELECT * FROM t ORDER BY intCol DO + | lbl2: FOR y AS SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT y.intCol2; + | SELECT intCol2; + | ITERATE lbl1; + | SELECT 1; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(3)), // select y.intCol2 + Seq(Row(3)), // select intCol2 + Seq(Row(3)), // select y.intCol2 + Seq(Row(3)), // select intCol2 + Seq.empty[Row], // drop outer var + Seq.empty[Row] // drop outer var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested - leave outer loop") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | lbl1: FOR x as SELECT * FROM t ORDER BY intCol DO + | lbl2: FOR y AS SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT y.intCol2; + | SELECT intCol2; + | LEAVE lbl1; + | SELECT 1; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(3)), // select y.intCol2 + Seq(Row(3)) // select intCol2 + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested - leave inner loop") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | lbl1: FOR x as SELECT * FROM t ORDER BY intCol DO + | lbl2: FOR y AS SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT y.intCol2; + | SELECT intCol2; + | LEAVE lbl2; + | SELECT 1; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(3)), // select y.intCol2 + Seq(Row(3)), // select intCol2 + Seq(Row(3)), // select y.intCol2 + Seq(Row(3)), // select intCol2 + Seq.empty[Row], // drop outer var + Seq.empty[Row] // drop outer var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - enters body once") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING, doubleCol DOUBLE) using parquet; + | INSERT INTO t VALUES (1, 'first', 1.0); + | FOR SELECT * FROM t DO + | SELECT intCol; + | SELECT stringCol; + | SELECT doubleCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(1)), // select intCol + Seq(Row("first")), // select stringCol + Seq(Row(1.0)), // select doubleCol + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - enters body with multiple statements multiple times") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING, doubleCol DOUBLE) using parquet; + | INSERT INTO t VALUES (1, 'first', 1.0); + | INSERT INTO t VALUES (2, 'second', 2.0); + | FOR SELECT * FROM t ORDER BY intCol DO + | SELECT intCol; + | SELECT stringCol; + | SELECT doubleCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(1)), // select intCol + Seq(Row("first")), // select stringCol + Seq(Row(1.0)), // select doubleCol + Seq(Row(2)), // select intCol + Seq(Row("second")), // select stringCol + Seq(Row(2.0)), // select doubleCol + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - sum of column from table") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | DECLARE sumOfCols = 0; + | CREATE TABLE t (intCol INT) using parquet; + | INSERT INTO t VALUES (1), (2), (3), (4); + | FOR SELECT * FROM t DO + | SET sumOfCols = sumOfCols + intCol; + | END FOR; + | SELECT sumOfCols; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // declare sumOfCols + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // set sumOfCols + Seq.empty[Row], // drop local var + Seq(Row(10)), // select sumOfCols + Seq.empty[Row] // drop sumOfCols + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - map, struct, array") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (int_column INT, map_column MAP, + | struct_column STRUCT, array_column ARRAY); + | INSERT INTO t VALUES + | (1, MAP('a', 1), STRUCT('John', 25), ARRAY('apricot', 'quince')), + | (2, MAP('b', 2), STRUCT('Jane', 30), ARRAY('plum', 'pear')); + | FOR SELECT * FROM t ORDER BY int_column DO + | SELECT map_column; + | SELECT struct_column; + | SELECT array_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Map("a" -> 1))), // select map_column + Seq(Row(Row("John", 25))), // select struct_column + Seq(Row(Array("apricot", "quince"))), // select array_column + Seq(Row(Map("b" -> 2))), // select map_column + Seq(Row(Row("Jane", 30))), // select struct_column + Seq(Row(Array("plum", "pear"))), // select array_column + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested struct") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (int_column INT, + | struct_column STRUCT>>); + | INSERT INTO t VALUES + | (1, STRUCT(1, STRUCT(STRUCT("one")))), + | (2, STRUCT(2, STRUCT(STRUCT("two")))); + | FOR SELECT * FROM t ORDER BY int_column DO + | SELECT struct_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Row(1, Row(Row("one"))))), // select struct_column + Seq(Row(Row(2, Row(Row("two"))))), // select struct_column + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested map") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (int_column INT, map_column MAP>>); + | INSERT INTO t VALUES + | (1, MAP('a', MAP(1, MAP(false, 10)))), + | (2, MAP('b', MAP(2, MAP(true, 20)))); + | FOR SELECT * FROM t ORDER BY int_column DO + | SELECT map_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Map("a" -> Map(1 -> Map(false -> 10))))), // select map_column + Seq(Row(Map("b" -> Map(2 -> Map(true -> 20))))), // select map_column + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested array") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t + | (int_column INT, array_column ARRAY>>); + | INSERT INTO t VALUES + | (1, ARRAY(ARRAY(ARRAY(1, 2), ARRAY(3, 4)), ARRAY(ARRAY(5, 6)))), + | (2, ARRAY(ARRAY(ARRAY(7, 8), ARRAY(9, 10)), ARRAY(ARRAY(11, 12)))); + | FOR SELECT * FROM t ORDER BY int_column DO + | SELECT array_column; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row(Seq(Seq(Seq(1, 2), Seq(3, 4)), Seq(Seq(5, 6))))), // array_column + Seq(Row(Array(Seq(Seq(7, 8), Seq(9, 10)), Seq(Seq(11, 12))))), // array_column + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - empty result") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | FOR SELECT * FROM t ORDER BY intCol DO + | SELECT intCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row] // create table + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - iterate") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING) using parquet; + | INSERT INTO t VALUES (1, 'first'), (2, 'second'), (3, 'third'), (4, 'fourth'); + | + | lbl: FOR SELECT * FROM t ORDER BY intCol DO + | IF intCol = 2 THEN + | ITERATE lbl; + | END IF; + | SELECT stringCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row("first")), // select stringCol + Seq(Row("third")), // select stringCol + Seq(Row("fourth")), // select stringCol + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - leave") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING) using parquet; + | INSERT INTO t VALUES (1, 'first'), (2, 'second'), (3, 'third'), (4, 'fourth'); + | + | lbl: FOR SELECT * FROM t ORDER BY intCol DO + | IF intCol = 3 THEN + | LEAVE lbl; + | END IF; + | SELECT stringCol; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq(Row("first")), // select stringCol + Seq(Row("second")) // select stringCol + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested - in while") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | DECLARE cnt = 0; + | CREATE TABLE t (intCol INT) using parquet; + | INSERT INTO t VALUES (0); + | WHILE cnt < 2 DO + | SET cnt = cnt + 1; + | FOR SELECT * FROM t ORDER BY intCol DO + | SELECT intCol; + | END FOR; + | INSERT INTO t VALUES (cnt); + | END WHILE; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // declare cnt + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq(Row(1)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop cnt + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested - in other for") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | FOR SELECT * FROM t ORDER BY intCol DO + | FOR SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT intCol; + | SELECT intCol2; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(0)), // select intCol + Seq(Row(3)), // select intCol2 + Seq(Row(0)), // select intCol + Seq(Row(2)), // select intCol2 + Seq.empty[Row], // drop local var + Seq(Row(1)), // select intCol + Seq(Row(3)), // select intCol2 + Seq(Row(1)), // select intCol + Seq(Row(2)), // select intCol2 + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop outer var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + // ignored until loops are fixed to support empty bodies + ignore("for statement - no variable - nested - empty result set") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | REPEAT + | FOR SELECT * FROM t ORDER BY intCol DO + | SELECT intCol; + | END FOR; + | UNTIL 1 = 1 + | END REPEAT; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // declare cnt + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // set cnt + Seq(Row(0)), // select intCol + Seq(Row(1)), // select intCol + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop cnt + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested - iterate outer loop") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | lbl1: FOR SELECT * FROM t ORDER BY intCol DO + | lbl2: FOR SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT intCol2; + | ITERATE lbl1; + | SELECT 1; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(3)), // select intCol2 + Seq(Row(3)), // select intCol2 + Seq.empty[Row] // drop outer var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested - leave outer loop") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | lbl1: FOR SELECT * FROM t ORDER BY intCol DO + | lbl2: FOR SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT intCol2; + | LEAVE lbl1; + | SELECT 1; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(3)) // select intCol2 + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - no variable - nested - leave inner loop") { + withTable("t", "t2") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT) using parquet; + | CREATE TABLE t2 (intCol2 INT) using parquet; + | INSERT INTO t VALUES (0), (1); + | INSERT INTO t2 VALUES (2), (3); + | lbl1: FOR SELECT * FROM t ORDER BY intCol DO + | lbl2: FOR SELECT * FROM t2 ORDER BY intCol2 DESC DO + | SELECT intCol2; + | LEAVE lbl2; + | SELECT 1; + | END FOR; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // insert + Seq(Row(3)), // select intCol2 + Seq(Row(3)), // select intCol2 + Seq.empty[Row] // drop outer var + ) + verifySqlScriptResult(sqlScript, expected) + } + } } From aaf859053c7a6200527e0ae1010a7bbc10727ccf Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 29 Nov 2024 09:15:37 +0800 Subject: [PATCH 050/438] [SPARK-50448][SQL][TESTS] Extract postgres image common conf as PostgresDatabaseOnDocker ### What changes were proposed in this pull request? The pr aims to extract `postgres` image `common` conf as `PostgresDatabaseOnDocker`. ### Why are the changes needed? Reduce the cost of the next `postgres` image upgrade. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass existed UT & GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49004 from panbingkun/SPARK-50448. Authored-by: panbingkun Signed-off-by: panbingkun --- .../sql/jdbc/PostgresDatabaseOnDocker.scala | 36 +++++++++++++++++++ .../sql/jdbc/PostgresIntegrationSuite.scala | 11 +----- .../jdbc/PostgresKrbIntegrationSuite.scala | 9 +---- .../querytest/GeneratedSubquerySuite.scala | 11 +----- ....scala => PostgresSQLQueryTestSuite.scala} | 14 ++------ .../jdbc/v2/PostgresIntegrationSuite.scala | 13 ++----- .../sql/jdbc/v2/PostgresNamespaceSuite.scala | 13 ++----- 7 files changed, 45 insertions(+), 62 deletions(-) create mode 100644 connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresDatabaseOnDocker.scala rename connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/{PostgreSQLQueryTestSuite.scala => PostgresSQLQueryTestSuite.scala} (83%) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresDatabaseOnDocker.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresDatabaseOnDocker.scala new file mode 100644 index 0000000000000..db2495ad3c698 --- /dev/null +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresDatabaseOnDocker.scala @@ -0,0 +1,36 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging + +class PostgresDatabaseOnDocker extends DatabaseOnDocker with Logging { + lazy override val imageName: String = + sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") + private val postgres_user = "postgres" + private val postgres_password = "rootpass" + override val env: Map[String, String] = Map( + "POSTGRES_PASSWORD" -> postgres_password + ) + override val usesIpc = false + override val jdbcPort: Int = 5432 + + override def getJdbcUrl(ip: String, port: Int): String = { + s"jdbc:postgresql://$ip:$port/postgres?user=$postgres_user&password=$postgres_password" + } +} diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index c3a705a1c7e28..5c985da226b06 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -41,16 +41,7 @@ import org.apache.spark.tags.DockerTest */ @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") - override val env = Map( - "POSTGRES_PASSWORD" -> "rootpass" - ) - override val usesIpc = false - override val jdbcPort = 5432 - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass" - } + override val db = new PostgresDatabaseOnDocker override def dataPreparation(conn: Connection): Unit = { conn.prepareStatement("CREATE DATABASE foo").executeUpdate() diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index 6bb6bbc36a2d4..b3cfe8bd77e2b 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -37,14 +37,7 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val userName = s"postgres/$dockerIp" override protected val keytabFileName = "postgres.keytab" - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") - override val env = Map( - "POSTGRES_PASSWORD" -> "rootpass" - ) - override val usesIpc = false - override val jdbcPort = 5432 - + override val db = new PostgresDatabaseOnDocker { override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:postgresql://$ip:$port/postgres?user=$principal&gsslib=gssapi" diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala index c51ae4c9ed71a..3a1d5e18b7e5a 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala @@ -38,16 +38,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class GeneratedSubquerySuite extends DockerJDBCIntegrationSuite with QueryGeneratorHelper { - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") - override val env = Map( - "POSTGRES_PASSWORD" -> "rootpass" - ) - override val usesIpc = false - override val jdbcPort = 5432 - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass" - } + override val db = new PostgresDatabaseOnDocker private val FIRST_COLUMN = "a" private val SECOND_COLUMN = "b" diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgresSQLQueryTestSuite.scala similarity index 83% rename from connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala rename to connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgresSQLQueryTestSuite.scala index 84a04cafb6453..28320a9e0a949 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgresSQLQueryTestSuite.scala @@ -38,23 +38,13 @@ import org.apache.spark.tags.DockerTest * }}} */ @DockerTest -class PostgreSQLQueryTestSuite extends CrossDbmsQueryTestSuite { +class PostgresSQLQueryTestSuite extends CrossDbmsQueryTestSuite { val DATABASE_NAME = CrossDbmsQueryTestSuite.POSTGRES // Scope to only subquery directory for now. protected val customInputFilePath: String = new File(inputFilePath, "subquery").getAbsolutePath - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") - override val env = Map( - "POSTGRES_PASSWORD" -> "rootpass" - ) - override val usesIpc = false - override val jdbcPort = 5432 - - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass" - } + override val db = new PostgresDatabaseOnDocker override def dataPreparation(conn: Connection): Unit = { conn.prepareStatement( diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index ce79618a011cb..ef52aebd723a6 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.{SparkConf, SparkSQLException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog -import org.apache.spark.sql.jdbc.DatabaseOnDocker +import org.apache.spark.sql.jdbc.PostgresDatabaseOnDocker import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest @@ -37,16 +37,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest { override val catalogName: String = "postgresql" - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") - override val env = Map( - "POSTGRES_PASSWORD" -> "rootpass" - ) - override val usesIpc = false - override val jdbcPort = 5432 - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass" - } + override val db = new PostgresDatabaseOnDocker override def sparkConf: SparkConf = super.sparkConf .set("spark.sql.catalog.postgresql", classOf[JDBCTableCatalog].getName) .set("spark.sql.catalog.postgresql.url", db.getJdbcUrl(dockerIp, externalPort)) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala index 649267788ad60..f84bdb46850f2 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala @@ -21,7 +21,7 @@ import java.sql.Connection import scala.jdk.CollectionConverters._ -import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite} +import org.apache.spark.sql.jdbc.{DockerJDBCIntegrationSuite, PostgresDatabaseOnDocker} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.tags.DockerTest @@ -34,16 +34,7 @@ import org.apache.spark.tags.DockerTest */ @DockerTest class PostgresNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespaceTest { - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine") - override val env = Map( - "POSTGRES_PASSWORD" -> "rootpass" - ) - override val usesIpc = false - override val jdbcPort = 5432 - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass" - } + override val db = new PostgresDatabaseOnDocker val map = new CaseInsensitiveStringMap( Map("url" -> db.getJdbcUrl(dockerIp, externalPort), From b45045e56f81deab1aec829dfee310991f17a8bd Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Fri, 29 Nov 2024 10:45:33 +0800 Subject: [PATCH 051/438] [SPARK-49992][SQL] Default collation resolution for DDL and DML queries ### What changes were proposed in this pull request? This PR proposes not using session-level collation in DDL commands (create/alter view/table, add/replace columns). Also, resolution of default collation should happen in the analyzer and not in the parser. However, due to how we are checking for default string type (using reference equals with `StringType` object) we cannot just replace this object with `StringType("UTF8_BINARY")` because they compare as equal so the tree node framework will just return the old plan. Because of this we have to perform this resolution twice, once by changing the `StringType` object into a `TemporaryStringType` and then back to `StringType("UTF8_BINARY")` which is not considered a default string type anymore. Another thing is that the dependent rules `ResolveInlineTables` and `CollationTypeCoercion` are updated so that they don't execute if there are still unresolved string types in the plan. ### Why are the changes needed? The default collation for DDL commands should be tied to the object being created or altered (e.g., table, view, schema) rather than the session-level setting. Since object-level collations are not yet supported, we will assume the UTF8_BINARY collation by default for now. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48962 from stefankandic/fixSessionCollationOrder. Lead-authored-by: Stefan Kandic Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../catalyst/parser/DataTypeAstBuilder.scala | 2 +- .../internal/types/AbstractStringType.scala | 3 +- .../apache/spark/sql/types/ArrayType.scala | 9 + .../org/apache/spark/sql/types/DataType.scala | 7 + .../org/apache/spark/sql/types/MapType.scala | 12 + .../apache/spark/sql/types/StringType.scala | 11 +- .../apache/spark/sql/types/StructType.scala | 12 + .../sql/catalyst/analysis/Analyzer.scala | 1 + .../analysis/CollationTypeCoercion.scala | 14 +- .../analysis/ResolveDefaultStringTypes.scala | 188 +++++++ .../analysis/ResolveInlineTables.scala | 6 +- .../analysis/TypeCoercionHelper.scala | 4 +- .../spark/sql/catalyst/expressions/misc.scala | 4 +- .../expressions/stringExpressions.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 40 +- .../catalyst/plans/logical/v2Commands.scala | 6 + .../sql/catalyst/rules/RuleExecutor.scala | 31 +- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../spark/sql/types/DataTypeSuite.scala | 2 +- .../analysis/ResolveSessionCatalog.scala | 5 + .../spark/sql/execution/datasources/ddl.scala | 4 +- .../sql/CollationSQLExpressionsSuite.scala | 15 +- .../spark/sql/CollationSQLRegexpSuite.scala | 20 +- .../sql/CollationStringExpressionsSuite.scala | 4 +- .../org/apache/spark/sql/CollationSuite.scala | 23 - .../collation/DefaultCollationTestSuite.scala | 490 ++++++++++++++++++ 26 files changed, 855 insertions(+), 62 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultStringTypes.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala index 71e8517a4164e..c2cb4a7154076 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala @@ -76,7 +76,7 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { case (TIMESTAMP_LTZ, Nil) => TimestampType case (STRING, Nil) => typeCtx.children.asScala.toSeq match { - case Seq(_) => SqlApiConf.get.defaultStringType + case Seq(_) => StringType case Seq(_, ctx: CollateClauseContext) => val collationName = visitCollateClause(ctx) val collationId = CollationFactory.collationNameToId(collationName) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractStringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractStringType.scala index 49d8bf9e001ab..6dcb8a876b7a2 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractStringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractStringType.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.internal.types -import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types.{AbstractDataType, DataType, StringType} /** @@ -26,7 +25,7 @@ import org.apache.spark.sql.types.{AbstractDataType, DataType, StringType} abstract class AbstractStringType(supportsTrimCollation: Boolean = false) extends AbstractDataType with Serializable { - override private[sql] def defaultConcreteType: DataType = SqlApiConf.get.defaultStringType + override private[sql] def defaultConcreteType: DataType = StringType override private[sql] def simpleString: String = "string" override private[sql] def acceptsType(other: DataType): Boolean = other match { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index fc32248b4baf3..53dfc5e9b2828 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -110,4 +110,13 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT override private[spark] def existsRecursively(f: (DataType) => Boolean): Boolean = { f(this) || elementType.existsRecursively(f) } + + override private[spark] def transformRecursively( + f: PartialFunction[DataType, DataType]): DataType = { + if (f.isDefinedAt(this)) { + f(this) + } else { + ArrayType(elementType.transformRecursively(f), containsNull) + } + } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala index 036de22b4189a..12cfed5b58685 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -105,6 +105,13 @@ abstract class DataType extends AbstractDataType { */ private[spark] def existsRecursively(f: (DataType) => Boolean): Boolean = f(this) + /** + * Recursively applies the provided partial function `f` to transform this DataType tree. + */ + private[spark] def transformRecursively(f: PartialFunction[DataType, DataType]): DataType = { + if (f.isDefinedAt(this)) f(this) else this + } + final override private[sql] def defaultConcreteType: DataType = this override private[sql] def acceptsType(other: DataType): Boolean = sameType(other) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala index 1dfb9aaf9e29b..de656c13ca4bf 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -89,6 +89,18 @@ case class MapType(keyType: DataType, valueType: DataType, valueContainsNull: Bo override private[spark] def existsRecursively(f: (DataType) => Boolean): Boolean = { f(this) || keyType.existsRecursively(f) || valueType.existsRecursively(f) } + + override private[spark] def transformRecursively( + f: PartialFunction[DataType, DataType]): DataType = { + if (f.isDefinedAt(this)) { + f(this) + } else { + MapType( + keyType.transformRecursively(f), + valueType.transformRecursively(f), + valueContainsNull) + } + } } /** diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index 1eb645e37c4aa..b2cf502f8bdc1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.CollationFactory * The id of collation for this StringType. */ @Stable -class StringType private (val collationId: Int) extends AtomicType with Serializable { +class StringType private[sql] (val collationId: Int) extends AtomicType with Serializable { /** * Support for Binary Equality implies that strings are considered equal only if they are byte @@ -75,7 +75,14 @@ class StringType private (val collationId: Int) extends AtomicType with Serializ */ override def typeName: String = if (isUTF8BinaryCollation) "string" - else s"string collate ${CollationFactory.fetchCollation(collationId).collationName}" + else s"string collate $collationName" + + override def toString: String = + if (isUTF8BinaryCollation) "StringType" + else s"StringType($collationName)" + + private[sql] def collationName: String = + CollationFactory.fetchCollation(collationId).collationName // Due to backwards compatibility and compatibility with other readers // all string types are serialized in json as regular strings and diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala index 07f6b50bd4a7a..cc95d8ee94b02 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -502,6 +502,18 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override private[spark] def existsRecursively(f: (DataType) => Boolean): Boolean = { f(this) || fields.exists(field => field.dataType.existsRecursively(f)) } + + override private[spark] def transformRecursively( + f: PartialFunction[DataType, DataType]): DataType = { + if (f.isDefinedAt(this)) { + return f(this) + } + + val newFields = fields.map { field => + field.copy(dataType = field.dataType.transformRecursively(f)) + } + StructType(newFields) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 84b3ca2289f4c..8e1b9da927c9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -320,6 +320,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolveAliases :: ResolveSubquery :: ResolveSubqueryColumnAliases :: + ResolveDefaultStringTypes :: ResolveWindowOrder :: ResolveWindowFrame :: ResolveNaturalAndUsingJoin :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index 532e5e0d0a066..cca1d21df3a7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType, haveS import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType} +import org.apache.spark.sql.types.{ArrayType, DataType, StringType} import org.apache.spark.sql.util.SchemaUtils /** @@ -93,13 +93,6 @@ object CollationTypeCoercion { val Seq(newStr, newPad) = collateToSingleType(Seq(str, pad)) stringPadExpr.withNewChildren(Seq(newStr, len, newPad)) - case raiseError: RaiseError => - val newErrorParams = raiseError.errorParms.dataType match { - case MapType(StringType, StringType, _) => raiseError.errorParms - case _ => Cast(raiseError.errorParms, MapType(StringType, StringType)) - } - raiseError.withNewChildren(Seq(raiseError.errorClass, newErrorParams)) - case framelessOffsetWindow @ (_: Lag | _: Lead) => val Seq(input, offset, default) = framelessOffsetWindow.children val Seq(newInput, newDefault) = collateToSingleType(Seq(input, default)) @@ -219,6 +212,11 @@ object CollationTypeCoercion { */ private def findLeastCommonStringType(expressions: Seq[Expression]): Option[StringType] = { if (!expressions.exists(e => SchemaUtils.hasNonUTF8BinaryCollation(e.dataType))) { + // if there are no collated types we don't need to do anything + return None + } else if (ResolveDefaultStringTypes.needsResolution(expressions)) { + // if any of the strings types are still not resolved + // we need to wait for them to be resolved first return None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultStringTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultStringTypes.scala new file mode 100644 index 0000000000000..75958ff3e1177 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultStringTypes.scala @@ -0,0 +1,188 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumn, AlterViewAs, ColumnDefinition, CreateView, LogicalPlan, QualifiedColType, ReplaceColumns, V1CreateTablePlan, V2CreateTablePlan} +import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} +import org.apache.spark.sql.types.{DataType, StringType} + +/** + * Resolves default string types in queries and commands. For queries, the default string type is + * determined by the session's default string type. For DDL, the default string type is the + * default type of the object (table -> schema -> catalog). However, this is not implemented yet. + * So, we will just use UTF8_BINARY for now. + */ +object ResolveDefaultStringTypes extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val newPlan = apply0(plan) + if (plan.ne(newPlan)) { + // Due to how tree transformations work and StringType object being equal to + // StringType("UTF8_BINARY"), we need to transform the plan twice + // to ensure the correct results for occurrences of default string type. + val finalPlan = apply0(newPlan) + RuleExecutor.forceAdditionalIteration(finalPlan) + finalPlan + } else { + newPlan + } + } + + private def apply0(plan: LogicalPlan): LogicalPlan = { + if (isDDLCommand(plan)) { + transformDDL(plan) + } else { + transformPlan(plan, sessionDefaultStringType) + } + } + + /** + * Returns whether any of the given `plan` needs to have its + * default string type resolved. + */ + def needsResolution(plan: LogicalPlan): Boolean = { + if (!isDDLCommand(plan) && isDefaultSessionCollationUsed) { + return false + } + + plan.exists(node => needsResolution(node.expressions)) + } + + /** + * Returns whether any of the given `expressions` needs to have its + * default string type resolved. + */ + def needsResolution(expressions: Seq[Expression]): Boolean = { + expressions.exists(needsResolution) + } + + /** + * Returns whether the given `expression` needs to have its + * default string type resolved. + */ + def needsResolution(expression: Expression): Boolean = { + expression.exists(e => transformExpression.isDefinedAt(e)) + } + + private def isDefaultSessionCollationUsed: Boolean = conf.defaultStringType == StringType + + /** + * Returns the default string type that should be used in a given DDL command (for now always + * UTF8_BINARY). + */ + private def stringTypeForDDLCommand(table: LogicalPlan): StringType = + StringType("UTF8_BINARY") + + /** Returns the session default string type */ + private def sessionDefaultStringType: StringType = + StringType(conf.defaultStringType.collationId) + + private def isDDLCommand(plan: LogicalPlan): Boolean = plan exists { + case _: AddColumns | _: ReplaceColumns | _: AlterColumn => true + case _ => isCreateOrAlterPlan(plan) + } + + private def isCreateOrAlterPlan(plan: LogicalPlan): Boolean = plan match { + case _: V1CreateTablePlan | _: V2CreateTablePlan | _: CreateView | _: AlterViewAs => true + case _ => false + } + + private def transformDDL(plan: LogicalPlan): LogicalPlan = { + val newType = stringTypeForDDLCommand(plan) + + plan resolveOperators { + case p if isCreateOrAlterPlan(p) => + transformPlan(p, newType) + + case addCols: AddColumns => + addCols.copy(columnsToAdd = replaceColumnTypes(addCols.columnsToAdd, newType)) + + case replaceCols: ReplaceColumns => + replaceCols.copy(columnsToAdd = replaceColumnTypes(replaceCols.columnsToAdd, newType)) + + case alter: AlterColumn + if alter.dataType.isDefined && hasDefaultStringType(alter.dataType.get) => + alter.copy(dataType = Some(replaceDefaultStringType(alter.dataType.get, newType))) + } + } + + /** + * Transforms the given plan, by transforming all expressions in its operators to use the given + * new type instead of the default string type. + */ + private def transformPlan(plan: LogicalPlan, newType: StringType): LogicalPlan = { + plan resolveExpressionsUp { expression => + transformExpression + .andThen(_.apply(newType)) + .applyOrElse(expression, identity[Expression]) + } + } + + /** + * Transforms the given expression, by changing all default string types to the given new type. + */ + private def transformExpression: PartialFunction[Expression, StringType => Expression] = { + case columnDef: ColumnDefinition if hasDefaultStringType(columnDef.dataType) => + newType => columnDef.copy(dataType = replaceDefaultStringType(columnDef.dataType, newType)) + + case cast: Cast if hasDefaultStringType(cast.dataType) => + newType => cast.copy(dataType = replaceDefaultStringType(cast.dataType, newType)) + + case Literal(value, dt) if hasDefaultStringType(dt) => + newType => Literal(value, replaceDefaultStringType(dt, newType)) + } + + private def hasDefaultStringType(dataType: DataType): Boolean = + dataType.existsRecursively(isDefaultStringType) + + private def isDefaultStringType(dataType: DataType): Boolean = { + dataType match { + case st: StringType => + // should only return true for StringType object and not StringType("UTF8_BINARY") + st.eq(StringType) || st.isInstanceOf[TemporaryStringType] + case _ => false + } + } + + private def replaceDefaultStringType(dataType: DataType, newType: StringType): DataType = { + dataType.transformRecursively { + case currentType: StringType if isDefaultStringType(currentType) => + if (currentType == newType) { + TemporaryStringType() + } else { + newType + } + } + } + + private def replaceColumnTypes( + colTypes: Seq[QualifiedColType], + newType: StringType): Seq[QualifiedColType] = { + colTypes.map { + case colWithDefault if hasDefaultStringType(colWithDefault.dataType) => + val replaced = replaceDefaultStringType(colWithDefault.dataType, newType) + colWithDefault.copy(dataType = replaced) + + case col => col + } + } +} + +case class TemporaryStringType() extends StringType(1) { + override def toString: String = s"TemporaryStringType($collationId)" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala index 62f3997491c07..b9e9e49a39647 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala @@ -29,8 +29,12 @@ import org.apache.spark.sql.catalyst.trees.AlwaysProcess object ResolveInlineTables extends Rule[LogicalPlan] with EvalHelper { override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsWithPruning(AlwaysProcess.fn, ruleId) { - case table: UnresolvedInlineTable if table.expressionsResolved => + case table: UnresolvedInlineTable if canResolveTable(table) => EvaluateUnresolvedInlineTable.evaluateUnresolvedInlineTable(table) } } + + private def canResolveTable(table: UnresolvedInlineTable): Boolean = { + table.expressionsResolved && !ResolveDefaultStringTypes.needsResolution(table) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala index 5b4d76a2a73ed..3fc4b71c986ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala @@ -415,7 +415,7 @@ abstract class TypeCoercionHelper { if conf.concatBinaryAsString || !children.map(_.dataType).forall(_ == BinaryType) => val newChildren = c.children.map { e => - implicitCast(e, SQLConf.get.defaultStringType).getOrElse(e) + implicitCast(e, StringType).getOrElse(e) } c.copy(children = newChildren) case other => other @@ -465,7 +465,7 @@ abstract class TypeCoercionHelper { if (conf.eltOutputAsString || !children.tail.map(_.dataType).forall(_ == BinaryType)) { children.tail.map { e => - implicitCast(e, SQLConf.get.defaultStringType).getOrElse(e) + implicitCast(e, StringType).getOrElse(e) } } else { children.tail diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 5f1b3dc0a01ac..622a0e0aa5bb7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.util.{MapData, RandomUUIDGenerator} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.errors.QueryExecutionErrors.raiseError import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.types.StringTypeWithCollation +import org.apache.spark.sql.internal.types.{AbstractMapType, StringTypeWithCollation} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -85,7 +85,7 @@ case class RaiseError(errorClass: Expression, errorParms: Expression, dataType: override def foldable: Boolean = false override def nullable: Boolean = true override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, MapType(StringType, StringType)) + Seq(StringTypeWithCollation, AbstractMapType(StringTypeWithCollation, StringTypeWithCollation)) override def left: Expression = errorClass override def right: Expression = errorParms diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index c97920619ba4d..2ea53350fea36 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -1863,7 +1863,7 @@ trait PadExpressionBuilderBase extends ExpressionBuilder { BinaryPad(funcName, expressions(0), expressions(1), Literal(Array[Byte](0))) } else { createStringPad(expressions(0), - expressions(1), Literal.create(" ", SQLConf.get.defaultStringType)) + expressions(1), Literal(" ")) } } else if (numArgs == 3) { if (expressions(0).dataType == BinaryType && expressions(2).dataType == BinaryType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d558689a5c196..3d74e9d314d57 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2150,7 +2150,7 @@ class AstBuilder extends DataTypeAstBuilder } val unresolvedTable = UnresolvedInlineTable(aliases, rows.toSeq) - val table = if (conf.getConf(SQLConf.EAGER_EVAL_OF_UNRESOLVED_INLINE_TABLE_ENABLED)) { + val table = if (canEagerlyEvaluateInlineTable(ctx, unresolvedTable)) { EvaluateUnresolvedInlineTable.evaluate(unresolvedTable) } else { unresolvedTable @@ -2158,6 +2158,42 @@ class AstBuilder extends DataTypeAstBuilder table.optionalMap(ctx.tableAlias.strictIdentifier)(aliasPlan) } + /** + * Determines if the inline table can be eagerly evaluated. + */ + private def canEagerlyEvaluateInlineTable( + ctx: InlineTableContext, + table: UnresolvedInlineTable): Boolean = { + if (!conf.getConf(SQLConf.EAGER_EVAL_OF_UNRESOLVED_INLINE_TABLE_ENABLED)) { + return false + } else if (!ResolveDefaultStringTypes.needsResolution(table.expressions)) { + // if there are no strings to be resolved we can always evaluate eagerly + return true + } + + val isSessionCollationSet = conf.defaultStringType != StringType + + // if either of these are true we need to resolve + // the string types first + !isSessionCollationSet && !contextInsideCreate(ctx) + } + + private def contextInsideCreate(ctx: ParserRuleContext): Boolean = { + var currentContext: RuleContext = ctx + + while (currentContext != null) { + if (currentContext.isInstanceOf[CreateTableContext] || + currentContext.isInstanceOf[ReplaceTableContext] || + currentContext.isInstanceOf[CreateViewContext]) { + return true + } + + currentContext = currentContext.parent + } + + false + } + /** * Create an alias (SubqueryAlias) for a join relation. This is practically the same as * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different @@ -3369,7 +3405,7 @@ class AstBuilder extends DataTypeAstBuilder * Create a String literal expression. */ override def visitStringLiteral(ctx: StringLiteralContext): Literal = withOrigin(ctx) { - Literal.create(createString(ctx), conf.defaultStringType) + Literal.create(createString(ctx), StringType) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index b465e0e11612f..857522728eaff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -459,6 +459,12 @@ trait V2CreateTableAsSelectPlan newQuery: LogicalPlan): V2CreateTableAsSelectPlan } +/** + * A trait used for logical plan nodes that create V1 table definitions, + * and so that rules from the catalyst module can identify them. + */ +trait V1CreateTablePlan extends LogicalPlan + /** A trait used for logical plan nodes that create or replace V2 table definitions. */ trait V2CreateTablePlan extends LogicalPlan { def name: LogicalPlan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 76d36fab2096a..bdbf698db2e01 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -22,7 +22,8 @@ import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.QueryPlanningTracker -import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.rules.RuleExecutor.getForceIterationValue +import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.errors.QueryExecutionErrors @@ -30,6 +31,27 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils object RuleExecutor { + + /** + * A tag used to explicitly request an additional iteration of the current batch during + * rule execution, even if the query plan remains unchanged. Increment the tag's value + * to enforce another iteration. + */ + private val FORCE_ADDITIONAL_ITERATION = TreeNodeTag[Int]("forceAdditionalIteration") + + /** + * Increments the value of the FORCE_ADDITIONAL_ITERATION tag on the given plan to + * explicitly force another iteration of the current batch during rule execution. + */ + def forceAdditionalIteration(plan: TreeNode[_]): Unit = { + val oldValue = getForceIterationValue(plan) + plan.setTagValue(FORCE_ADDITIONAL_ITERATION, oldValue + 1) + } + + private def getForceIterationValue(plan: TreeNode[_]): Int = { + plan.getTagValue(FORCE_ADDITIONAL_ITERATION).getOrElse(0) + } + protected val queryExecutionMeter = QueryExecutionMetering() /** Dump statistics about time spent running specific rules. */ @@ -303,7 +325,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { continue = false } - if (curPlan.fastEquals(lastPlan)) { + if (isFixedPointReached(lastPlan, curPlan)) { logTrace( s"Fixed point reached for batch ${batch.name} after ${iteration - 1} iterations.") continue = false @@ -317,4 +339,9 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { curPlan } + + private def isFixedPointReached(oldPlan: TreeType, newPlan: TreeType): Boolean = { + oldPlan.fastEquals(newPlan) && + getForceIterationValue(newPlan) <= getForceIterationValue(oldPlan) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 378eca09097f5..e8031580c1165 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -5580,7 +5580,7 @@ class SQLConf extends Serializable with Logging with SqlApiConf { if (getConf(DEFAULT_COLLATION).toUpperCase(Locale.ROOT) == "UTF8_BINARY") { StringType } else { - StringType(CollationFactory.collationNameToId(getConf(DEFAULT_COLLATION))) + StringType(getConf(DEFAULT_COLLATION)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index f6d8f2a66e202..7250b6e2b90e6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -715,7 +715,7 @@ class DataTypeSuite extends SparkFunSuite { checkEqualsIgnoreCompatibleCollation(StringType, StringType("UTF8_LCASE"), expected = true) checkEqualsIgnoreCompatibleCollation( - StringType("UTF8_BINARY"), StringType("UTF8_LCASE"), expected = true) + StringType("UTF8_LCASE"), StringType("UTF8_BINARY"), expected = true) // Complex types. checkEqualsIgnoreCompatibleCollation( ArrayType(StringType), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 92c74f7bede18..5f1ab089cf3e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -54,6 +54,11 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case _ if ResolveDefaultStringTypes.needsResolution(plan) => + // if there are still unresolved string types in the plan + // we should not try to resolve it + plan + case AddColumns(ResolvedV1TableIdentifier(ident), cols) => cols.foreach { c => if (c.name.length > 1) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index d9367d92d462e..eb9d5813cff7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V1CreateTablePlan} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command.{DDLUtils, LeafRunnableCommand} @@ -43,7 +43,7 @@ import org.apache.spark.sql.types._ case class CreateTable( tableDesc: CatalogTable, mode: SaveMode, - query: Option[LogicalPlan]) extends LogicalPlan { + query: Option[LogicalPlan]) extends LogicalPlan with V1CreateTablePlan { assert(tableDesc.provider.isDefined, "The table to be created must have a provider.") if (query.isEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala index 6feb4587b816f..cf494fcd87451 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala @@ -376,6 +376,10 @@ class CollationSQLExpressionsSuite StructField("B", DoubleType, nullable = true) )), CsvToStructsTestCase("\"Spark\"", "UNICODE", "'a STRING'", "", + Row("Spark"), Seq( + StructField("a", StringType, nullable = true) + )), + CsvToStructsTestCase("\"Spark\"", "UTF8_BINARY", "'a STRING COLLATE UNICODE'", "", Row("Spark"), Seq( StructField("a", StringType("UNICODE"), nullable = true) )), @@ -1291,6 +1295,10 @@ class CollationSQLExpressionsSuite StructField("B", DoubleType, nullable = true) )), XmlToStructsTestCase("

Spark

", "UNICODE", "'s STRING'", "", + Row("Spark"), Seq( + StructField("s", StringType, nullable = true) + )), + XmlToStructsTestCase("

Spark

", "UTF8_BINARY", "'s STRING COLLATE UNICODE'", "", Row("Spark"), Seq( StructField("s", StringType("UNICODE"), nullable = true) )), @@ -1515,8 +1523,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( VariantGetTestCase("{\"a\": 1}", "$.a", "int", "UTF8_BINARY", 1, IntegerType), VariantGetTestCase("{\"a\": 1}", "$.b", "int", "UTF8_LCASE", null, IntegerType), - VariantGetTestCase("[1, \"2\"]", "$[1]", "string", "UNICODE", "2", StringType("UNICODE")), + VariantGetTestCase("[1, \"2\"]", "$[1]", "string", "UNICODE", "2", + StringType), + VariantGetTestCase("[1, \"2\"]", "$[1]", "string collate unicode", "UTF8_BINARY", "2", + StringType("UNICODE")), VariantGetTestCase("[1, \"2\"]", "$[2]", "string", "UNICODE_CI", null, + StringType), + VariantGetTestCase("[1, \"2\"]", "$[2]", "string collate unicode_CI", "UTF8_BINARY", null, StringType("UNICODE_CI")) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala index 5bb8511d0d935..7cafb999ffcf0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala @@ -111,15 +111,17 @@ class CollationSQLRegexpSuite } val tableNameLcase = "T_LCASE" withTable(tableNameLcase) { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UTF8_LCASE") { - sql(s"CREATE TABLE IF NOT EXISTS $tableNameLcase(c STRING) using PARQUET") - sql(s"INSERT INTO $tableNameLcase(c) VALUES('ABC')") - checkAnswer(sql(s"select c like 'ab%' FROM $tableNameLcase"), Row(true)) - checkAnswer(sql(s"select c like '%bc' FROM $tableNameLcase"), Row(true)) - checkAnswer(sql(s"select c like 'a%c' FROM $tableNameLcase"), Row(true)) - checkAnswer(sql(s"select c like '%b%' FROM $tableNameLcase"), Row(true)) - checkAnswer(sql(s"select c like 'abc' FROM $tableNameLcase"), Row(true)) - } + sql(s""" + |CREATE TABLE IF NOT EXISTS $tableNameLcase( + | c STRING COLLATE UTF8_LCASE + |) using PARQUET + |""".stripMargin) + sql(s"INSERT INTO $tableNameLcase(c) VALUES('ABC')") + checkAnswer(sql(s"select c like 'ab%' FROM $tableNameLcase"), Row(true)) + checkAnswer(sql(s"select c like '%bc' FROM $tableNameLcase"), Row(true)) + checkAnswer(sql(s"select c like 'a%c' FROM $tableNameLcase"), Row(true)) + checkAnswer(sql(s"select c like '%b%' FROM $tableNameLcase"), Row(true)) + checkAnswer(sql(s"select c like 'abc' FROM $tableNameLcase"), Row(true)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala index 2a0b84c075079..626bd0b239366 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala @@ -198,8 +198,8 @@ class CollationStringExpressionsSuite checkError( exception = intercept[AnalysisException] { val expr = StringSplitSQL( - Collate(Literal.create("1a2"), "UTF8_BINARY"), - Collate(Literal.create("a"), "UTF8_LCASE")) + Collate(Literal.create("1a2", StringType("UTF8_BINARY")), "UTF8_BINARY"), + Collate(Literal.create("a", StringType("UTF8_BINARY")), "UTF8_LCASE")) CollationTypeCasts.transform(expr) }, condition = "COLLATION_MISMATCH.EXPLICIT", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index 1707820053837..f0f81e713457b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -1096,29 +1096,6 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } - test("SPARK-47431: Default collation set to UNICODE, column type test") { - withTable("t") { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - sql(s"CREATE TABLE t(c1 STRING) USING PARQUET") - sql(s"INSERT INTO t VALUES ('a')") - checkAnswer(sql(s"SELECT collation(c1) FROM t"), Seq(Row("UNICODE"))) - } - } - } - - test("SPARK-47431: Create table with UTF8_BINARY, make sure collation persists on read") { - withTable("t") { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UTF8_BINARY") { - sql("CREATE TABLE t(c1 STRING) USING PARQUET") - sql("INSERT INTO t VALUES ('a')") - checkAnswer(sql("SELECT collation(c1) FROM t"), Seq(Row("UTF8_BINARY"))) - } - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkAnswer(sql("SELECT collation(c1) FROM t"), Seq(Row("UTF8_BINARY"))) - } - } - } - test("Create dataframe with non utf8 binary collation") { val schema = StructType(Seq(StructField("Name", StringType("UNICODE_CI")))) val data = Seq(Row("Alice"), Row("Bob"), Row("bob")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala new file mode 100644 index 0000000000000..0de638d4e9bf9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala @@ -0,0 +1,490 @@ +/* + * 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.collation + +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.connector.DatasourceV2SQLBase +import org.apache.spark.sql.internal.SqlApiConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StringType + +abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSession { + + def dataSource: String = "parquet" + def testTable: String = "test_tbl" + def testView: String = "test_view" + + def withSessionCollationAndTable(collation: String, testTables: String*)(f: => Unit): Unit = { + withTable(testTables: _*) { + withSessionCollation(collation) { + f + } + } + } + + def withSessionCollationAndView(collation: String, viewNames: String*)(f: => Unit): Unit = { + withView(viewNames: _*) { + withSessionCollation(collation) { + f + } + } + } + + def withSessionCollation(collation: String)(f: => Unit): Unit = { + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) { + f + } + } + + def assertTableColumnCollation( + table: String, + column: String, + expectedCollation: String): Unit = { + val colType = spark.table(table).schema(column).dataType + assert(colType === StringType(expectedCollation)) + } + + def assertThrowsImplicitMismatch(f: => DataFrame): Unit = { + val exception = intercept[AnalysisException] { + f + } + assert(exception.getCondition === "COLLATION_MISMATCH.IMPLICIT") + } + + // region DDL tests + + test("create/alter table") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + // create table with implicit collation + sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") + assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") + + // alter table add column with implicit collation + sql(s"ALTER TABLE $testTable ADD COLUMN c2 STRING") + assertTableColumnCollation(testTable, "c2", "UTF8_BINARY") + + sql(s"ALTER TABLE $testTable ALTER COLUMN c2 TYPE STRING COLLATE UNICODE") + assertTableColumnCollation(testTable, "c2", "UNICODE") + + sql(s"ALTER TABLE $testTable ALTER COLUMN c2 TYPE STRING") + assertTableColumnCollation(testTable, "c2", "UTF8_BINARY") + } + } + + test("create table with explicit collation") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UTF8_LCASE) USING $dataSource") + assertTableColumnCollation(testTable, "c1", "UTF8_LCASE") + } + + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UNICODE) USING $dataSource") + assertTableColumnCollation(testTable, "c1", "UNICODE") + } + } + + test("create table as select") { + // literals in select do not pick up session collation + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable USING $dataSource AS SELECT + | 'a' AS c1, + | 'a' || 'a' AS c2, + | SUBSTRING('a', 1, 1) AS c3, + | SUBSTRING(SUBSTRING('ab', 1, 1), 1, 1) AS c4, + | 'a' = 'A' AS truthy + |""".stripMargin) + assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") + assertTableColumnCollation(testTable, "c2", "UTF8_BINARY") + assertTableColumnCollation(testTable, "c3", "UTF8_BINARY") + assertTableColumnCollation(testTable, "c4", "UTF8_BINARY") + + checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable WHERE truthy"), Seq(Row(0))) + } + + // literals in inline table do not pick up session collation + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable USING $dataSource AS + |SELECT c1, c1 = 'A' as c2 FROM VALUES ('a'), ('A') AS vals(c1) + |""".stripMargin) + assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") + checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable WHERE c2"), Seq(Row(1))) + } + + // cast in select does not pick up session collation + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable USING $dataSource AS SELECT cast('a' AS STRING) AS c1") + assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") + } + } + + test("ctas with complex types") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable USING $dataSource AS + |SELECT + | struct('a') AS c1, + | map('a', 'b') AS c2, + | array('a') AS c3 + |""".stripMargin) + + checkAnswer(sql(s"SELECT COLLATION(c1.col1) FROM $testTable"), Seq(Row("UTF8_BINARY"))) + checkAnswer(sql(s"SELECT COLLATION(c2['a']) FROM $testTable"), Seq(Row("UTF8_BINARY"))) + checkAnswer(sql(s"SELECT COLLATION(c3[0]) FROM $testTable"), Seq(Row("UTF8_BINARY"))) + } + } + + test("ctas with union") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable USING $dataSource AS + |SELECT 'a' = 'A' AS c1 + |UNION + |SELECT 'b' = 'B' AS c1 + |""".stripMargin) + + checkAnswer(sql(s"SELECT * FROM $testTable"), Seq(Row(false))) + } + + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable USING $dataSource AS + |SELECT 'a' = 'A' AS c1 + |UNION ALL + |SELECT 'b' = 'B' AS c1 + |""".stripMargin) + + checkAnswer(sql(s"SELECT * FROM $testTable"), Seq(Row(false), Row(false))) + } + } + + test("add column") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UTF8_LCASE) USING $dataSource") + assertTableColumnCollation(testTable, "c1", "UTF8_LCASE") + + sql(s"ALTER TABLE $testTable ADD COLUMN c2 STRING") + assertTableColumnCollation(testTable, "c2", "UTF8_BINARY") + + sql(s"ALTER TABLE $testTable ADD COLUMN c3 STRING COLLATE UNICODE") + assertTableColumnCollation(testTable, "c3", "UNICODE") + } + } + + test("inline table in CTAS") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable + |USING $dataSource + |AS SELECT * + |FROM (VALUES ('a', 'a' = 'A')) + |AS inline_table(c1, c2); + |""".stripMargin) + + assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") + checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable WHERE c2"), Seq(Row(0))) + } + } + + test("subsequent analyzer iterations correctly resolve default string types") { + // since concat coercion happens after resolving default types this test + // makes sure that we are correctly resolving the default string types + // in subsequent analyzer iterations + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable + |USING $dataSource AS + |SELECT CONCAT(X'68656C6C6F', 'world') AS c1 + |""".stripMargin) + + checkAnswer(sql(s"SELECT c1 FROM $testTable"), Seq(Row("helloworld"))) + } + + // ELT is similar + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s""" + |CREATE TABLE $testTable + |USING $dataSource AS + |SELECT ELT(1, X'68656C6C6F', 'world') AS c1 + |""".stripMargin) + + checkAnswer(sql(s"SELECT c1 FROM $testTable"), Seq(Row("hello"))) + } + } + + // endregion + + // region DML tests + + test("literals with default collation") { + val sessionCollation = "UTF8_LCASE" + withSessionCollation(sessionCollation) { + + // literal without collation + checkAnswer(sql("SELECT COLLATION('a')"), Seq(Row(sessionCollation))) + + checkAnswer(sql("SELECT COLLATION(map('a', 'b')['a'])"), Seq(Row(sessionCollation))) + + checkAnswer(sql("SELECT COLLATION(array('a')[0])"), Seq(Row(sessionCollation))) + + checkAnswer(sql("SELECT COLLATION(struct('a' as c)['c'])"), Seq(Row(sessionCollation))) + } + } + + test("literals with explicit collation") { + withSessionCollation("UTF8_LCASE") { + checkAnswer(sql("SELECT COLLATION('a' collate unicode)"), Seq(Row("UNICODE"))) + + checkAnswer( + sql("SELECT COLLATION(map('a', 'b' collate unicode)['a'])"), + Seq(Row("UNICODE"))) + + checkAnswer(sql("SELECT COLLATION(array('a' collate unicode)[0])"), Seq(Row("UNICODE"))) + + checkAnswer( + sql("SELECT COLLATION(struct('a' collate unicode as c)['c'])"), + Seq(Row("UNICODE"))) + } + } + + test("cast is aware of session collation") { + val sessionCollation = "UTF8_LCASE" + withSessionCollation(sessionCollation) { + checkAnswer(sql("SELECT COLLATION(cast('a' as STRING))"), Seq(Row(sessionCollation))) + + checkAnswer( + sql("SELECT COLLATION(cast(map('a', 'b') as MAP)['a'])"), + Seq(Row(sessionCollation))) + + checkAnswer( + sql("SELECT COLLATION(map_keys(cast(map('a', 'b') as MAP))[0])"), + Seq(Row(sessionCollation))) + + checkAnswer( + sql("SELECT COLLATION(cast(array('a') as ARRAY)[0])"), + Seq(Row(sessionCollation))) + + checkAnswer( + sql("SELECT COLLATION(cast(struct('a' as c) as STRUCT)['c'])"), + Seq(Row(sessionCollation))) + } + } + + test("expressions in where are aware of session collation") { + withSessionCollation("UTF8_LCASE") { + // expression in where is aware of session collation + checkAnswer(sql("SELECT 1 WHERE 'a' = 'A'"), Seq(Row(1))) + + checkAnswer(sql("SELECT 1 WHERE 'a' = cast('A' as STRING)"), Seq(Row(1))) + } + } + + test("having group by is aware of session collation") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") + sql(s"INSERT INTO $testTable VALUES ('a'), ('A')") + + // having clause uses session (default) collation + checkAnswer( + sql(s"SELECT COUNT(*) FROM $testTable GROUP BY c1 HAVING 'a' = 'A'"), + Seq(Row(1), Row(1))) + + // having clause uses column (implicit) collation + checkAnswer( + sql(s"SELECT COUNT(*) FROM $testTable GROUP BY c1 HAVING c1 = 'A'"), + Seq(Row(1))) + } + } + + test("min/max are aware of session collation") { + // scalastyle:off nonascii + withSessionCollationAndTable("UNICODE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") + sql(s"INSERT INTO $testTable VALUES ('1'), ('½')") + + checkAnswer(sql(s"SELECT MIN(c1) FROM $testTable"), Seq(Row("1"))) + + checkAnswer(sql(s"SELECT MAX(c1) FROM $testTable"), Seq(Row("½"))) + } + // scalastyle:on nonascii + } + + test("union operation with subqueries") { + withSessionCollation("UTF8_LCASE") { + checkAnswer( + sql(s""" + |SELECT 'a' = 'A' + |UNION + |SELECT 'b' = 'B' + |""".stripMargin), + Seq(Row(true))) + + checkAnswer( + sql(s""" + |SELECT 'a' = 'A' + |UNION ALL + |SELECT 'b' = 'B' + |""".stripMargin), + Seq(Row(true), Row(true))) + } + } + + test("inline table in SELECT") { + withSessionCollation("UTF8_LCASE") { + val df = s""" + |SELECT * + |FROM (VALUES ('a', 'a' = 'A')) + |""".stripMargin + + checkAnswer(sql(df), Seq(Row("a", true))) + } + } + + test("inline table in insert") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING, c2 BOOLEAN) USING $dataSource") + + sql(s"INSERT INTO $testTable VALUES ('a', 'a' = 'A')") + checkAnswer(sql(s"SELECT * FROM $testTable"), Seq(Row("a", true))) + } + } + + test("literals in insert inherit session level collation") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 BOOLEAN) USING $dataSource") + + sql(s"INSERT INTO $testTable VALUES ('a' = 'A')") + sql(s"INSERT INTO $testTable VALUES (array_contains(array('a'), 'A'))") + sql(s"INSERT INTO $testTable VALUES (CONCAT(X'68656C6C6F', 'world') = 'HELLOWORLD')") + + checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable WHERE c1"), Seq(Row(3))) + } + } + + // endregion +} + +class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { + + test("create/alter view created from a table") { + val sessionCollation = "UTF8_LCASE" + withSessionCollationAndTable(sessionCollation, testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING, c2 STRING COLLATE UNICODE_CI) USING $dataSource") + sql(s"INSERT INTO $testTable VALUES ('a', 'a'), ('A', 'A')") + + withView(testView) { + sql(s"CREATE VIEW $testView AS SELECT * FROM $testTable") + + assertTableColumnCollation(testView, "c1", "UTF8_BINARY") + assertTableColumnCollation(testView, "c2", "UNICODE_CI") + checkAnswer( + sql(s"SELECT DISTINCT COLLATION(c1), COLLATION('a') FROM $testView"), + Row("UTF8_BINARY", sessionCollation)) + + // filter should use session collation + checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE 'a' = 'A'"), Row(2)) + + // filter should use column collation + checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Row(1)) + + checkAnswer( + sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = substring('A', 0, 1)"), + Row(1)) + + // literal with explicit collation wins + checkAnswer( + sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A' collate UNICODE_CI"), + Row(2)) + + // two implicit collations -> errors out + assertThrowsImplicitMismatch(sql(s"SELECT c1 = c2 FROM $testView")) + + sql(s"ALTER VIEW $testView AS SELECT c1 COLLATE UNICODE_CI AS c1, c2 FROM $testTable") + assertTableColumnCollation(testView, "c1", "UNICODE_CI") + assertTableColumnCollation(testView, "c2", "UNICODE_CI") + checkAnswer( + sql(s"SELECT DISTINCT COLLATION(c1), COLLATION('a') FROM $testView"), + Row("UNICODE_CI", sessionCollation)) + + // after alter both rows should be returned + checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Row(2)) + } + } + } + + test("join view with table") { + val viewTableName = "view_table" + val joinTableName = "join_table" + val sessionCollation = "sr" + + withSessionCollationAndTable(sessionCollation, viewTableName, joinTableName) { + sql(s"CREATE TABLE $viewTableName (c1 STRING COLLATE UNICODE_CI) USING $dataSource") + sql(s"CREATE TABLE $joinTableName (c1 STRING COLLATE UTF8_LCASE) USING $dataSource") + sql(s"INSERT INTO $viewTableName VALUES ('a')") + sql(s"INSERT INTO $joinTableName VALUES ('A')") + + withView(testView) { + sql(s"CREATE VIEW $testView AS SELECT * FROM $viewTableName") + + assertThrowsImplicitMismatch( + sql(s"SELECT * FROM $testView JOIN $joinTableName ON $testView.c1 = $joinTableName.c1")) + + checkAnswer( + sql(s""" + |SELECT COLLATION($testView.c1), COLLATION($joinTableName.c1) + |FROM $testView JOIN $joinTableName + |ON $testView.c1 = $joinTableName.c1 COLLATE UNICODE_CI + |""".stripMargin), + Row("UNICODE_CI", "UTF8_LCASE")) + } + } + } +} + +class DefaultCollationTestSuiteV2 extends DefaultCollationTestSuite with DatasourceV2SQLBase { + override def testTable: String = s"testcat.${super.testTable}" + override def testView: String = s"testcat.${super.testView}" + + // delete only works on v2 + test("delete behavior") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") + sql(s"INSERT INTO $testTable VALUES ('a'), ('A')") + + sql(s"DELETE FROM $testTable WHERE 'a' = 'A'") + checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable"), Seq(Row(0))) + } + } + + test("inline table in RTAS") { + withSessionCollationAndTable("UTF8_LCASE", testTable) { + sql(s"CREATE TABLE $testTable (c1 STRING, c2 BOOLEAN) USING $dataSource") + sql(s""" + |REPLACE TABLE $testTable + |USING $dataSource + |AS SELECT * + |FROM (VALUES ('a', 'a' = 'A')) + |AS inline_table(c1, c2); + |""".stripMargin) + + assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") + checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable WHERE c2"), Seq(Row(0))) + } + } +} From 3791de906f820e75a0a91a85d400f71968f9f17e Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 29 Nov 2024 10:47:42 +0800 Subject: [PATCH 052/438] [SPARK-50444][BUILD] Upgrade dropwizard metrics to 4.2.29 ### What changes were proposed in this pull request? This pr aims to upgrade `dropwizard metrics` from `4.2.28` to `4.2.29`. ### Why are the changes needed? v4.2.128 VS v.4.2.29 https://github.com/dropwizard/metrics/compare/v4.2.28...v4.2.29 https://github.com/dropwizard/metrics/pull/4540 ### 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 #48997 from panbingkun/SPARK-50444. Authored-by: panbingkun Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 10 +++++----- pom.xml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index db325fbf546aa..9f51a70745f27 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -196,11 +196,11 @@ log4j-layout-template-json/2.24.2//log4j-layout-template-json-2.24.2.jar log4j-slf4j2-impl/2.24.2//log4j-slf4j2-impl-2.24.2.jar logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.8.0//lz4-java-1.8.0.jar -metrics-core/4.2.28//metrics-core-4.2.28.jar -metrics-graphite/4.2.28//metrics-graphite-4.2.28.jar -metrics-jmx/4.2.28//metrics-jmx-4.2.28.jar -metrics-json/4.2.28//metrics-json-4.2.28.jar -metrics-jvm/4.2.28//metrics-jvm-4.2.28.jar +metrics-core/4.2.29//metrics-core-4.2.29.jar +metrics-graphite/4.2.29//metrics-graphite-4.2.29.jar +metrics-jmx/4.2.29//metrics-jmx-4.2.29.jar +metrics-json/4.2.29//metrics-json-4.2.29.jar +metrics-jvm/4.2.29//metrics-jvm-4.2.29.jar minlog/1.3.0//minlog-1.3.0.jar netty-all/4.1.114.Final//netty-all-4.1.114.Final.jar netty-buffer/4.1.114.Final//netty-buffer-4.1.114.Final.jar diff --git a/pom.xml b/pom.xml index de8da1ea64a88..db74659a5f69c 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ If you change codahale.metrics.version, you also need to change the link to metrics.dropwizard.io in docs/monitoring.md. --> - 4.2.28 + 4.2.29 1.12.0 1.12.0 From 0c16e93dce06639b49fe6641aedcd1e9fc4df96a Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 29 Nov 2024 13:39:53 +0900 Subject: [PATCH 053/438] [SPARK-50446][PYTHON] Concurrent level in Arrow-optimized Python UDF ### What changes were proposed in this pull request? This PR proposes to add a configuration for async execution for Python UDF with Arrow optimization. One usecase is, for exmaple, to have RESTful API requests within Python UDF, and it slows down by I/O. By this configuration, those I/O requests can happen in parallel. ### Why are the changes needed? In order to speed up UDF executions. For example, the code below: ```python spark.conf.set("spark.sql.execution.pythonUDF.arrow.concurrency.level", 10) spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "true") from pyspark.sql.functions import udf, col import time udf def my_rest_func(x): import requests requests.get("https://httpbin.org/get") start_time = time.time() _ = spark.range(100).coalesce(1).select(my_rest_func(col("id"))).collect() print(time.time() - start_time) ``` can be 10x faster. ### Does this PR introduce _any_ user-facing change? Yes, it adds a new configuration called `spark.sql.execution.pythonUDF.arrow.concurrency.level`. ### How was this patch tested? Tested as shown above, and unittests were also added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49000 from HyukjinKwon/async-exec. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../sql/tests/test_arrow_python_udf.py | 14 ++++++++++++ python/pyspark/worker.py | 22 ++++++++++++++----- .../apache/spark/sql/internal/SQLConf.scala | 13 +++++++++++ .../execution/python/ArrowPythonRunner.scala | 5 ++++- 4 files changed, 48 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/tests/test_arrow_python_udf.py b/python/pyspark/sql/tests/test_arrow_python_udf.py index 095414334848b..a3fd8c01992ad 100644 --- a/python/pyspark/sql/tests/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/test_arrow_python_udf.py @@ -238,6 +238,20 @@ def tearDownClass(cls): super(PythonUDFArrowTests, cls).tearDownClass() +class AsyncPythonUDFArrowTests(PythonUDFArrowTests): + @classmethod + def setUpClass(cls): + super(AsyncPythonUDFArrowTests, cls).setUpClass() + cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.concurrency.level", "4") + + @classmethod + def tearDownClass(cls): + try: + cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.concurrency.level") + finally: + super(AsyncPythonUDFArrowTests, cls).tearDownClass() + + if __name__ == "__main__": from pyspark.sql.tests.test_arrow_python_udf import * # noqa: F401 diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 1ebc04520ecad..a11465e7a3239 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -154,7 +154,7 @@ def verify_result_length(result, length): ) -def wrap_arrow_batch_udf(f, args_offsets, kwargs_offsets, return_type): +def wrap_arrow_batch_udf(f, args_offsets, kwargs_offsets, return_type, runner_conf): import pandas as pd func, args_kwargs_offsets = wrap_kwargs_support(f, args_offsets, kwargs_offsets) @@ -172,9 +172,21 @@ def wrap_arrow_batch_udf(f, args_offsets, kwargs_offsets, return_type): elif type(return_type) == BinaryType: result_func = lambda r: bytes(r) if r is not None else r # noqa: E731 - @fail_on_stopiteration - def evaluate(*args: pd.Series) -> pd.Series: - return pd.Series([result_func(func(*row)) for row in zip(*args)]) + if "spark.sql.execution.pythonUDF.arrow.concurrency.level" in runner_conf: + from concurrent.futures import ThreadPoolExecutor + + c = int(runner_conf["spark.sql.execution.pythonUDF.arrow.concurrency.level"]) + + @fail_on_stopiteration + def evaluate(*args: pd.Series) -> pd.Series: + with ThreadPoolExecutor(max_workers=c) as pool: + return pd.Series(list(pool.map(lambda row: result_func(func(*row)), zip(*args)))) + + else: + + @fail_on_stopiteration + def evaluate(*args: pd.Series) -> pd.Series: + return pd.Series([result_func(func(*row)) for row in zip(*args)]) def verify_result_length(result, length): if len(result) != length: @@ -855,7 +867,7 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index, profil if eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF: return wrap_scalar_pandas_udf(func, args_offsets, kwargs_offsets, return_type) elif eval_type == PythonEvalType.SQL_ARROW_BATCHED_UDF: - return wrap_arrow_batch_udf(func, args_offsets, kwargs_offsets, return_type) + return wrap_arrow_batch_udf(func, args_offsets, kwargs_offsets, return_type, runner_conf) elif eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF: return args_offsets, wrap_pandas_batch_iter_udf(func, return_type) elif eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e8031580c1165..2a05508a17544 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3321,6 +3321,17 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PYTHON_UDF_ARROW_CONCURRENCY_LEVEL = + buildConf("spark.sql.execution.pythonUDF.arrow.concurrency.level") + .doc("The level of concurrency to execute Arrow-optimized Python UDF. " + + "This can be useful if Python UDFs use I/O intensively.") + .version("4.0.0") + .intConf + .checkValue(_ > 1, + "The value of spark.sql.execution.pythonUDF.arrow.concurrency.level" + + " must be more than one.") + .createOptional + val PYTHON_TABLE_UDF_ARROW_ENABLED = buildConf("spark.sql.execution.pythonUDTF.arrow.enabled") .doc("Enable Arrow optimization for Python UDTFs.") @@ -5997,6 +6008,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def pythonUDFWorkerFaulthandlerEnabled: Boolean = getConf(PYTHON_UDF_WORKER_FAULTHANLDER_ENABLED) + def pythonUDFArrowConcurrencyLevel: Option[Int] = getConf(PYTHON_UDF_ARROW_CONCURRENCY_LEVEL) + def pysparkPlotMaxRows: Int = getConf(PYSPARK_PLOT_MAX_ROWS) def arrowSparkREnabled: Boolean = getConf(ARROW_SPARKR_EXECUTION_ENABLED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index a555d660ea1ac..72e9c5210194a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -116,6 +116,9 @@ object ArrowPythonRunner { conf.pandasGroupedMapAssignColumnsByName.toString) val arrowSafeTypeCheck = Seq(SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION.key -> conf.arrowSafeTypeConversion.toString) - Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck: _*) + val arrowAyncParallelism = conf.pythonUDFArrowConcurrencyLevel.map(v => + Seq(SQLConf.PYTHON_UDF_ARROW_CONCURRENCY_LEVEL.key -> v.toString) + ).getOrElse(Seq.empty) + Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck ++ arrowAyncParallelism: _*) } } From 376bd4aec8dee3e239e1b4bb76cf81ea9864e1c6 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 29 Nov 2024 14:41:08 +0800 Subject: [PATCH 054/438] [SPARK-50333][SQL][FOLLOWUP] Codegen Support for `CsvToStructs`(`from_csv`) - remove Invoke ### What changes were proposed in this pull request? The pr aims to implement the codegen of `CsvToStructs`(`from_csv`) in the way of `manually`, rather than in the way of `Invoke`. ### Why are the changes needed? Based on cloud-fan's double-check, https://github.com/apache/spark/pull/48509#discussion_r1858515472 I believe that restore to manual implementation will not result in regression. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Update existed UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48983 from panbingkun/SPARK-50333_FOLLOWUP. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../csv/CsvExpressionEvalUtils.scala | 3 +- .../catalyst/expressions/csvExpressions.scala | 50 ++++++++++++------- .../explain-results/function_from_csv.explain | 2 +- 3 files changed, 35 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala index a91e4ab13001b..fd298b33450b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.{DataType, NullType, StructType} import org.apache.spark.unsafe.types.UTF8String /** - * The expression `CsvToStructs` will utilize the `Invoke` to call it, support codegen. + * The expression `CsvToStructs` will utilize it to support codegen. */ case class CsvToStructsEvaluator( options: Map[String, String], @@ -86,6 +86,7 @@ case class CsvToStructsEvaluator( } final def evaluate(csv: UTF8String): InternalRow = { + if (csv == null) return null converter(parser.parse(csv.toString)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index bf3d8eee9ed5a..21b6295a59f02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -23,10 +23,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} import org.apache.spark.sql.catalyst.csv._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper import org.apache.spark.sql.catalyst.expressions.csv.{CsvToStructsEvaluator, SchemaOfCsvEvaluator} import org.apache.spark.sql.catalyst.expressions.objects.Invoke -import org.apache.spark.sql.catalyst.trees.TreePattern.{RUNTIME_REPLACEABLE, TreePattern} import org.apache.spark.sql.catalyst.util.TypeUtils._ import org.apache.spark.sql.errors.QueryErrorsBase import org.apache.spark.sql.internal.SQLConf @@ -57,17 +57,12 @@ case class CsvToStructs( timeZoneId: Option[String] = None, requiredSchema: Option[StructType] = None) extends UnaryExpression - with RuntimeReplaceable - with ExpectsInputTypes - with TimeZoneAwareExpression { + with TimeZoneAwareExpression + with ExpectsInputTypes { override def nullable: Boolean = child.nullable - override def nodePatternsInternal(): Seq[TreePattern] = Seq(RUNTIME_REPLACEABLE) - - // The CSV input data might be missing certain fields. We force the nullability - // of the user-provided schema to avoid data corruptions. - private val nullableSchema: StructType = schema.asNullable + override def nullIntolerant: Boolean = true // Used in `FunctionRegistry` def this(child: Expression, schema: Expression, options: Map[String, String]) = @@ -86,8 +81,6 @@ case class CsvToStructs( child = child, timeZoneId = None) - private val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) - override def dataType: DataType = requiredSchema.getOrElse(schema).asNullable override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = { @@ -98,16 +91,37 @@ case class CsvToStructs( override def prettyName: String = "from_csv" + // The CSV input data might be missing certain fields. We force the nullability + // of the user-provided schema to avoid data corruptions. + private val nullableSchema: StructType = schema.asNullable + + @transient + private val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) + @transient private lazy val evaluator: CsvToStructsEvaluator = CsvToStructsEvaluator( options, nullableSchema, nameOfCorruptRecord, timeZoneId, requiredSchema) - override def replacement: Expression = Invoke( - Literal.create(evaluator, ObjectType(classOf[CsvToStructsEvaluator])), - "evaluate", - dataType, - Seq(child), - Seq(child.dataType)) + override def nullSafeEval(input: Any): Any = { + evaluator.evaluate(input.asInstanceOf[UTF8String]) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val refEvaluator = ctx.addReferenceObj("evaluator", evaluator) + val eval = child.genCode(ctx) + val resultType = CodeGenerator.boxedType(dataType) + val resultTerm = ctx.freshName("result") + ev.copy(code = + code""" + |${eval.code} + |$resultType $resultTerm = ($resultType) $refEvaluator.evaluate(${eval.value}); + |boolean ${ev.isNull} = $resultTerm == null; + |${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + |if (!${ev.isNull}) { + | ${ev.value} = $resultTerm; + |} + |""".stripMargin) + } override protected def withNewChildInternal(newChild: Expression): CsvToStructs = copy(child = newChild) diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain index ef87c18948b23..89e03c8188232 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain @@ -1,2 +1,2 @@ -Project [invoke(CsvToStructsEvaluator(Map(mode -> FAILFAST),StructType(StructField(id,LongType,true),StructField(a,IntegerType,true),StructField(b,DoubleType,true)),_corrupt_record,Some(America/Los_Angeles),None).evaluate(g#0)) AS from_csv(g)#0] +Project [from_csv(StructField(id,LongType,true), StructField(a,IntegerType,true), StructField(b,DoubleType,true), (mode,FAILFAST), g#0, Some(America/Los_Angeles), None) AS from_csv(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] From aad357583c8ca6d648b37b9feb955656174f1bdf Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 29 Nov 2024 16:37:10 +0800 Subject: [PATCH 055/438] [SPARK-50453][BUILD] Upgrade Netty to 4.1.115 ### What changes were proposed in this pull request? This pr aims to upgrade netty from 4.1.114 to 4.1.115. ### Why are the changes needed? This version bring some bug-fix and also include a fix for CVE-2024-47535: - https://github.com/netty/netty/commit/fbf7a704a82e7449b48bd0bbb679f5661c6d61a3 The full release notes as follows: - https://netty.io/news/2024/11/12/4-1-115-Final.html ### 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 windows Closes #49008 from LuciferYang/netty-4.1.115. Authored-by: yangjie01 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 9f51a70745f27..a35e4eae8da12 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -202,31 +202,31 @@ metrics-jmx/4.2.29//metrics-jmx-4.2.29.jar metrics-json/4.2.29//metrics-json-4.2.29.jar metrics-jvm/4.2.29//metrics-jvm-4.2.29.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.114.Final//netty-all-4.1.114.Final.jar -netty-buffer/4.1.114.Final//netty-buffer-4.1.114.Final.jar -netty-codec-http/4.1.114.Final//netty-codec-http-4.1.114.Final.jar -netty-codec-http2/4.1.114.Final//netty-codec-http2-4.1.114.Final.jar -netty-codec-socks/4.1.114.Final//netty-codec-socks-4.1.114.Final.jar -netty-codec/4.1.114.Final//netty-codec-4.1.114.Final.jar -netty-common/4.1.114.Final//netty-common-4.1.114.Final.jar -netty-handler-proxy/4.1.114.Final//netty-handler-proxy-4.1.114.Final.jar -netty-handler/4.1.114.Final//netty-handler-4.1.114.Final.jar -netty-resolver/4.1.114.Final//netty-resolver-4.1.114.Final.jar +netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar +netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar +netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar +netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar +netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar +netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar +netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar +netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar +netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar +netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar netty-tcnative-boringssl-static/2.0.69.Final/linux-aarch_64/netty-tcnative-boringssl-static-2.0.69.Final-linux-aarch_64.jar netty-tcnative-boringssl-static/2.0.69.Final/linux-x86_64/netty-tcnative-boringssl-static-2.0.69.Final-linux-x86_64.jar netty-tcnative-boringssl-static/2.0.69.Final/osx-aarch_64/netty-tcnative-boringssl-static-2.0.69.Final-osx-aarch_64.jar netty-tcnative-boringssl-static/2.0.69.Final/osx-x86_64/netty-tcnative-boringssl-static-2.0.69.Final-osx-x86_64.jar netty-tcnative-boringssl-static/2.0.69.Final/windows-x86_64/netty-tcnative-boringssl-static-2.0.69.Final-windows-x86_64.jar netty-tcnative-classes/2.0.69.Final//netty-tcnative-classes-2.0.69.Final.jar -netty-transport-classes-epoll/4.1.114.Final//netty-transport-classes-epoll-4.1.114.Final.jar -netty-transport-classes-kqueue/4.1.114.Final//netty-transport-classes-kqueue-4.1.114.Final.jar -netty-transport-native-epoll/4.1.114.Final/linux-aarch_64/netty-transport-native-epoll-4.1.114.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.114.Final/linux-riscv64/netty-transport-native-epoll-4.1.114.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.114.Final/linux-x86_64/netty-transport-native-epoll-4.1.114.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.114.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.114.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.114.Final/osx-x86_64/netty-transport-native-kqueue-4.1.114.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.114.Final//netty-transport-native-unix-common-4.1.114.Final.jar -netty-transport/4.1.114.Final//netty-transport-4.1.114.Final.jar +netty-transport-classes-epoll/4.1.115.Final//netty-transport-classes-epoll-4.1.115.Final.jar +netty-transport-classes-kqueue/4.1.115.Final//netty-transport-classes-kqueue-4.1.115.Final.jar +netty-transport-native-epoll/4.1.115.Final/linux-aarch_64/netty-transport-native-epoll-4.1.115.Final-linux-aarch_64.jar +netty-transport-native-epoll/4.1.115.Final/linux-riscv64/netty-transport-native-epoll-4.1.115.Final-linux-riscv64.jar +netty-transport-native-epoll/4.1.115.Final/linux-x86_64/netty-transport-native-epoll-4.1.115.Final-linux-x86_64.jar +netty-transport-native-kqueue/4.1.115.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.115.Final-osx-aarch_64.jar +netty-transport-native-kqueue/4.1.115.Final/osx-x86_64/netty-transport-native-kqueue-4.1.115.Final-osx-x86_64.jar +netty-transport-native-unix-common/4.1.115.Final//netty-transport-native-unix-common-4.1.115.Final.jar +netty-transport/4.1.115.Final//netty-transport-4.1.115.Final.jar objenesis/3.3//objenesis-3.3.jar okhttp/3.12.12//okhttp-3.12.12.jar okio/1.17.6//okio-1.17.6.jar diff --git a/pom.xml b/pom.xml index db74659a5f69c..85c29aacab8be 100644 --- a/pom.xml +++ b/pom.xml @@ -214,7 +214,7 @@ 1.79 1.15.0 6.1.1 - 4.1.114.Final + 4.1.115.Final 2.0.69.Final 76.1 5.11.3 From cd687ff7d95fbb96ed149e9e019970e9a4e76c09 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 29 Nov 2024 17:58:42 +0900 Subject: [PATCH 056/438] [SPARK-50310][PYTHON] Add a flag to disable DataFrameQueryContext for PySpark ### What changes were proposed in this pull request? We disabled the DataFrameQueryContext from https://github.com/apache/spark/pull/48827, and we also need adding a flag for PySpark for the same performant reason. ### Why are the changes needed? To avoid the performance slowdown for the case when the DataFrameQueryContext too much stacked ### Does this PR introduce _any_ user-facing change? No API changes, but the DataFrameQueryContext would no longer displayed when the flag is disabled ### How was this patch tested? Manually tested: 1. **FLAG ON (almost 25sec)** ```python >>> spark.conf.get("spark.python.sql.dataFrameDebugging.enabled") 'true' >>> import time >>> import pyspark.sql.functions as F >>> >>> c = F.col("name") >>> start = time.time() >>> for i in range(10000): ... _ = c.alias("a") ... >>> print(time.time() - start) 24.78217577934265 ``` 2. **FLAG OFF (only 1 sec)** ```python >>> spark.conf.get("spark.python.sql.dataFrameDebugging.enabled") 'false' >>> import time >>> import pyspark.sql.functions as F >>> >>> c = F.col("name") >>> start = time.time() >>> for i in range(10000): ... _ = c.alias("a") ... >>> print(time.time() - start) 1.0222370624542236 ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48964 from itholic/SPARK-50310. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- python/pyspark/errors/utils.py | 45 +++++++++++++------ .../spark/sql/internal/StaticSQLConf.scala | 10 +++++ 2 files changed, 42 insertions(+), 13 deletions(-) diff --git a/python/pyspark/errors/utils.py b/python/pyspark/errors/utils.py index cbe5739204ac1..416a2323b170e 100644 --- a/python/pyspark/errors/utils.py +++ b/python/pyspark/errors/utils.py @@ -31,21 +31,42 @@ Type, Optional, Union, - TYPE_CHECKING, overload, cast, ) import pyspark from pyspark.errors.error_classes import ERROR_CLASSES_MAP -if TYPE_CHECKING: - from pyspark.sql import SparkSession - T = TypeVar("T") FuncT = TypeVar("FuncT", bound=Callable[..., Any]) _current_origin = threading.local() +# Providing DataFrame debugging options to reduce performance slowdown. +# Default is True. +_enable_debugging_cache = None + + +def is_debugging_enabled() -> bool: + global _enable_debugging_cache + + if _enable_debugging_cache is None: + from pyspark.sql import SparkSession + + spark = SparkSession.getActiveSession() + if spark is not None: + _enable_debugging_cache = ( + spark.conf.get( + "spark.python.sql.dataFrameDebugging.enabled", + "true", # type: ignore[union-attr] + ).lower() + == "true" + ) + else: + _enable_debugging_cache = False + + return _enable_debugging_cache + def current_origin() -> threading.local: global _current_origin @@ -164,17 +185,12 @@ def get_message_template(self, errorClass: str) -> str: return message_template -def _capture_call_site(spark_session: "SparkSession", depth: int) -> str: +def _capture_call_site(depth: int) -> str: """ Capture the call site information including file name, line number, and function name. This function updates the thread-local storage from JVM side (PySparkCurrentOrigin) with the current call site information when a PySpark API function is called. - Parameters - ---------- - spark_session : SparkSession - Current active Spark session. - Notes ----- The call site information is used to enhance error messages with the exact location @@ -245,7 +261,7 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: # Getting the configuration requires RPC call. Uses the default value for now. depth = 1 - set_current_origin(func.__name__, _capture_call_site(spark, depth)) + set_current_origin(func.__name__, _capture_call_site(depth)) try: return func(*args, **kwargs) @@ -262,7 +278,7 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: ) ) # Update call site when the function is called - jvm_pyspark_origin.set(func.__name__, _capture_call_site(spark, depth)) + jvm_pyspark_origin.set(func.__name__, _capture_call_site(depth)) try: return func(*args, **kwargs) @@ -297,7 +313,10 @@ def with_origin_to_class( return lambda cls: with_origin_to_class(cls, ignores) else: cls = cls_or_ignores - if os.environ.get("PYSPARK_PIN_THREAD", "true").lower() == "true": + if ( + os.environ.get("PYSPARK_PIN_THREAD", "true").lower() == "true" + and is_debugging_enabled() + ): skipping = set( ["__init__", "__new__", "__iter__", "__nonzero__", "__repr__", "__bool__"] + (ignores or []) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index cd17a63e5d433..407baba8280c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -295,4 +295,14 @@ object StaticSQLConf { .version("3.1.0") .stringConf .createWithDefault("") + + val DATA_FRAME_DEBUGGING_ENABLED = + buildStaticConf("spark.python.sql.dataFrameDebugging.enabled") + .internal() + .doc( + "Enable the DataFrame debugging. This feature is enabled by default, but has a " + + "non-trivial performance overhead because of the stack trace collection.") + .version("4.0.0") + .booleanConf + .createWithDefault(true) } From 547661002fb8f772de13b048db50dffdc28da676 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 29 Nov 2024 18:00:16 +0900 Subject: [PATCH 057/438] [SPARK-50311][PYTHON][FOLLOWUP] Remove @remote_only from supported APIs ### What changes were proposed in this pull request? This PR followups https://github.com/apache/spark/pull/48843 to remove remote_only decorator from supported API and update the test ### Why are the changes needed? To check Connect parity properly ### Does this PR introduce _any_ user-facing change? No, it's test-only ### How was this patch tested? Updated the existing test ### Was this patch authored or co-authored using generative AI tooling? No Closes #49012 from itholic/SPARK-50311-followup. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- python/pyspark/sql/session.py | 4 ---- python/pyspark/sql/tests/test_connect_compatibility.py | 4 ---- 2 files changed, 8 deletions(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index d19e01eecc89c..1ec85e52bbcd3 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -2268,7 +2268,6 @@ def interruptOperation(self, op_id: str) -> List[str]: messageParameters={"feature": "SparkSession.interruptOperation"}, ) - @remote_only def addTag(self, tag: str) -> None: """ Add a tag to be assigned to all the operations started by this thread in this session. @@ -2293,7 +2292,6 @@ def addTag(self, tag: str) -> None: """ self._jsparkSession.addTag(tag) - @remote_only def removeTag(self, tag: str) -> None: """ Remove a tag previously added to be assigned to all the operations started by this thread in @@ -2311,7 +2309,6 @@ def removeTag(self, tag: str) -> None: """ self._jsparkSession.removeTag(tag) - @remote_only def getTags(self) -> Set[str]: """ Get the tags that are currently set to be assigned to all the operations started by this @@ -2337,7 +2334,6 @@ def getTags(self) -> Set[str]: return python_set - @remote_only def clearTags(self) -> None: """ Clear the current thread's operation tags. diff --git a/python/pyspark/sql/tests/test_connect_compatibility.py b/python/pyspark/sql/tests/test_connect_compatibility.py index 3d74e796cd7a0..ef83dc3834d0c 100644 --- a/python/pyspark/sql/tests/test_connect_compatibility.py +++ b/python/pyspark/sql/tests/test_connect_compatibility.py @@ -264,18 +264,14 @@ def test_spark_session_compatibility(self): expected_missing_connect_methods = { "addArtifact", "addArtifacts", - "addTag", "clearProgressHandlers", - "clearTags", "copyFromLocalToFs", - "getTags", "interruptAll", "interruptOperation", "interruptTag", "newSession", "registerProgressHandler", "removeProgressHandler", - "removeTag", } expected_missing_classic_methods = set() self.check_compatibility( From d5b534d18c060e960f049d2756df39d1d046a48c Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 29 Nov 2024 18:56:40 +0800 Subject: [PATCH 058/438] [SPARK-50295][DOCS][FOLLOWUP] Document `build-docs` in `docs/README.md` ### What changes were proposed in this pull request? The pr aims to document `dev/spark-test-image-util/docs/build-docs` in `docs/README.md`. ### Why are the changes needed? Add documentation for using script `dev/spark-test-image-util/docs/build-docs`. ### Does this PR introduce _any_ user-facing change? Yes, Spark developers will be able to know the existence of script `dev/spark-test-image-util/docs/build-docs` through documentation `docs/README.md`. ### How was this patch tested? Manually check. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49013 from panbingkun/SPARK-50295_FOLLOWUP. Lead-authored-by: panbingkun Co-authored-by: panbingkun Signed-off-by: panbingkun --- dev/spark-test-image-util/docs/build-docs | 0 docs/README.md | 13 +++++++++++++ 2 files changed, 13 insertions(+) mode change 100644 => 100755 dev/spark-test-image-util/docs/build-docs diff --git a/dev/spark-test-image-util/docs/build-docs b/dev/spark-test-image-util/docs/build-docs old mode 100644 new mode 100755 diff --git a/docs/README.md b/docs/README.md index 363f1c2076363..1235efe91812b 100644 --- a/docs/README.md +++ b/docs/README.md @@ -126,3 +126,16 @@ To control what API docs get built, you can set any combination of the following * `SKIP_RDOC=1`: Skip the R API docs. * `SKIP_SQLDOC=1`: Skip the SQL API docs. +## Build docs with docker image (Optional) + +As a Spark developer, you can generate all documents locally as follows: + +Note: Before running it, you need to have `docker` installed. + +```sh +$ dev/spark-test-image-util/docs/build-docs +``` + +It will generate all documents on the `container` and `host`. +Especially when there are conflicts between the libraries required by Python development environment +and the libraries required by generating Python docs environment, this is a good choice. From 4b97e11b53c22791f186f514694d0bddab92ab64 Mon Sep 17 00:00:00 2001 From: Mikhail Nikoliukin Date: Fri, 29 Nov 2024 21:32:20 +0800 Subject: [PATCH 059/438] [SPARK-42746][SQL] Implement LISTAGG function ### What changes were proposed in this pull request? Implement new aggregation function `listagg([ALL | DISTINCT] expr[, sep]) [WITHIN GROUP (ORDER BY key [ASC | DESC] [,...])]` ### Why are the changes needed? Listagg is a popular function implemented by many other vendors. For now, users have to use workarounds like [this](https://kb.databricks.com/sql/recreate-listagg-functionality-with-spark-sql). PR will close the gap. ### Does this PR introduce _any_ user-facing change? Yes, the new `listagg` function. BigQuery and PostgreSQL have the same function but with `string_agg` name so I added it as an alias. ### How was this patch tested? With new unit tests ### Was this patch authored or co-authored using generative AI tooling? Generated-by: GitHub Copilot Closes #48748 from mikhailnik-db/SPARK-42746-add-listagg. Lead-authored-by: Mikhail Nikoliukin Co-authored-by: Jia Fan Signed-off-by: Wenchen Fan --- .../apache/spark/unsafe/types/ByteArray.java | 36 +- .../spark/unsafe/array/ByteArraySuite.java | 55 +++ .../resources/error/error-conditions.json | 51 +- python/pyspark/sql/tests/test_functions.py | 8 +- .../org/apache/spark/sql/functions.scala | 71 +++ .../sql/catalyst/analysis/Analyzer.scala | 3 + .../sql/catalyst/analysis/CheckAnalysis.scala | 15 +- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../analysis/FunctionResolution.scala | 17 +- .../catalyst/expressions/aggregate/Mode.scala | 4 +- .../SupportsOrderingWithinGroup.scala | 21 +- .../expressions/aggregate/collect.scala | 298 +++++++++++- .../expressions/aggregate/percentiles.scala | 12 +- .../sql/errors/QueryCompilationErrors.scala | 36 +- .../spark/sql/execution/SparkStrategies.scala | 7 +- .../sql-functions/sql-expression-schema.md | 2 + .../listagg-collations.sql.out | 86 ++++ .../analyzer-results/listagg.sql.out | 435 ++++++++++++++++++ .../sql-tests/analyzer-results/mode.sql.out | 8 +- .../analyzer-results/percentiles.sql.out | 10 +- .../sql-tests/inputs/listagg-collations.sql | 12 + .../resources/sql-tests/inputs/listagg.sql | 38 ++ .../results/listagg-collations.sql.out | 82 ++++ .../sql-tests/results/listagg.sql.out | 368 +++++++++++++++ .../resources/sql-tests/results/mode.sql.out | 8 +- .../sql-tests/results/percentiles.sql.out | 10 +- .../spark/sql/DataFrameAggregateSuite.scala | 35 ++ .../spark/sql/DataFrameFunctionsSuite.scala | 4 +- .../ThriftServerQueryTestSuite.scala | 1 + 29 files changed, 1655 insertions(+), 80 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/listagg-collations.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/listagg.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/listagg.sql.out diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java index aae47aa963201..f12408fb49313 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java @@ -135,27 +135,57 @@ public static byte[] subStringSQL(byte[] bytes, int pos, int len) { return Arrays.copyOfRange(bytes, start, end); } + /** + * Concatenate multiple byte arrays into one. + * If one of the inputs is null then null will be returned. + * + * @param inputs byte arrays to concatenate + * @return the concatenated byte array or null if one of the arguments is null + */ public static byte[] concat(byte[]... inputs) { + return concatWS(EMPTY_BYTE, inputs); + } + + /** + * Concatenate multiple byte arrays with a given delimiter. + * If the delimiter or one of the inputs is null then null will be returned. + * + * @param delimiter byte array to be placed between each input + * @param inputs byte arrays to concatenate + * @return the concatenated byte array or null if one of the arguments is null + */ + public static byte[] concatWS(byte[] delimiter, byte[]... inputs) { + if (delimiter == null) { + return null; + } // Compute the total length of the result long totalLength = 0; for (byte[] input : inputs) { if (input != null) { - totalLength += input.length; + totalLength += input.length + delimiter.length; } else { return null; } } - + if (totalLength > 0) totalLength -= delimiter.length; // Allocate a new byte array, and copy the inputs one by one into it final byte[] result = new byte[Ints.checkedCast(totalLength)]; int offset = 0; - for (byte[] input : inputs) { + for (int i = 0; i < inputs.length; i++) { + byte[] input = inputs[i]; int len = input.length; Platform.copyMemory( input, Platform.BYTE_ARRAY_OFFSET, result, Platform.BYTE_ARRAY_OFFSET + offset, len); offset += len; + if (delimiter.length > 0 && i < inputs.length - 1) { + Platform.copyMemory( + delimiter, Platform.BYTE_ARRAY_OFFSET, + result, Platform.BYTE_ARRAY_OFFSET + offset, + delimiter.length); + offset += delimiter.length; + } } return result; } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/ByteArraySuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/ByteArraySuite.java index aff619175ff7b..5e221b4e359d4 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/ByteArraySuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/ByteArraySuite.java @@ -67,4 +67,59 @@ public void testCompareBinary() { byte[] y4 = new byte[]{(byte) 100, (byte) 200}; Assertions.assertEquals(0, ByteArray.compareBinary(x4, y4)); } + + @Test + public void testConcat() { + byte[] x1 = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + byte[] y1 = new byte[]{(byte) 4, (byte) 5, (byte) 6}; + byte[] result1 = ByteArray.concat(x1, y1); + byte[] expected1 = new byte[]{(byte) 1, (byte) 2, (byte) 3, (byte) 4, (byte) 5, (byte) 6}; + Assertions.assertArrayEquals(expected1, result1); + + byte[] x2 = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + byte[] y2 = new byte[0]; + byte[] result2 = ByteArray.concat(x2, y2); + byte[] expected2 = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + Assertions.assertArrayEquals(expected2, result2); + + byte[] x3 = new byte[0]; + byte[] y3 = new byte[]{(byte) 4, (byte) 5, (byte) 6}; + byte[] result3 = ByteArray.concat(x3, y3); + byte[] expected3 = new byte[]{(byte) 4, (byte) 5, (byte) 6}; + Assertions.assertArrayEquals(expected3, result3); + + byte[] x4 = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + byte[] y4 = null; + byte[] result4 = ByteArray.concat(x4, y4); + Assertions.assertArrayEquals(null, result4); + } + + @Test + public void testConcatWS() { + byte[] separator = new byte[]{(byte) 42}; + + byte[] x1 = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + byte[] y1 = new byte[]{(byte) 4, (byte) 5, (byte) 6}; + byte[] result1 = ByteArray.concatWS(separator, x1, y1); + byte[] expected1 = new byte[]{(byte) 1, (byte) 2, (byte) 3, (byte) 42, + (byte) 4, (byte) 5, (byte) 6}; + Assertions.assertArrayEquals(expected1, result1); + + byte[] x2 = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + byte[] y2 = new byte[0]; + byte[] result2 = ByteArray.concatWS(separator, x2, y2); + byte[] expected2 = new byte[]{(byte) 1, (byte) 2, (byte) 3, (byte) 42}; + Assertions.assertArrayEquals(expected2, result2); + + byte[] x3 = new byte[0]; + byte[] y3 = new byte[]{(byte) 4, (byte) 5, (byte) 6}; + byte[] result3 = ByteArray.concatWS(separator, x3, y3); + byte[] expected3 = new byte[]{(byte) 42, (byte) 4, (byte) 5, (byte) 6}; + Assertions.assertArrayEquals(expected3, result3); + + byte[] x4 = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + byte[] y4 = null; + byte[] result4 = ByteArray.concatWS(separator, x4, y4); + Assertions.assertArrayEquals(null, result4); + } } diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 77437f6c56179..024caf86cf94b 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2627,29 +2627,6 @@ ], "sqlState" : "22006" }, - "INVALID_INVERSE_DISTRIBUTION_FUNCTION" : { - "message" : [ - "Invalid inverse distribution function ." - ], - "subClass" : { - "DISTINCT_UNSUPPORTED" : { - "message" : [ - "Cannot use DISTINCT with WITHIN GROUP." - ] - }, - "WITHIN_GROUP_MISSING" : { - "message" : [ - "WITHIN GROUP is required for inverse distribution function." - ] - }, - "WRONG_NUM_ORDERINGS" : { - "message" : [ - "Requires orderings in WITHIN GROUP but got ." - ] - } - }, - "sqlState" : "42K0K" - }, "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME" : { "message" : [ " is not a valid identifier of Java and cannot be used as field name", @@ -3364,6 +3341,34 @@ ], "sqlState" : "42601" }, + "INVALID_WITHIN_GROUP_EXPRESSION" : { + "message" : [ + "Invalid function with WITHIN GROUP." + ], + "subClass" : { + "DISTINCT_UNSUPPORTED" : { + "message" : [ + "The function does not support DISTINCT with WITHIN GROUP." + ] + }, + "MISMATCH_WITH_DISTINCT_INPUT" : { + "message" : [ + "The function is invoked with DISTINCT and WITHIN GROUP but expressions and do not match. The WITHIN GROUP ordering expression must be picked from the function inputs." + ] + }, + "WITHIN_GROUP_MISSING" : { + "message" : [ + "WITHIN GROUP is required for the function." + ] + }, + "WRONG_NUM_ORDERINGS" : { + "message" : [ + "The function requires orderings in WITHIN GROUP but got ." + ] + } + }, + "sqlState" : "42K0K" + }, "INVALID_WRITER_COMMIT_MESSAGE" : { "message" : [ "The data source writer has generated an invalid number of commit messages. Expected exactly one writer commit message from each task, but received ." diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index cf8f685ea4499..6c7ce80072923 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -83,7 +83,13 @@ def test_function_parity(self): missing_in_py = jvm_fn_set.difference(py_fn_set) # Functions that we expect to be missing in python until they are added to pyspark - expected_missing_in_py = set() + expected_missing_in_py = { + # TODO(SPARK-50220): listagg functions will soon be added and removed from this list + "listagg_distinct", + "listagg", + "string_agg", + "string_agg_distinct", + } self.assertEqual( expected_missing_in_py, missing_in_py, "Missing functions in pyspark not as expected" diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala index 2a04212ee2585..9f509fa843a2b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala @@ -1147,6 +1147,77 @@ object functions { */ def sum_distinct(e: Column): Column = Column.fn("sum", isDistinct = true, e) + /** + * Aggregate function: returns the concatenation of non-null input values. + * + * @group agg_funcs + * @since 4.0.0 + */ + def listagg(e: Column): Column = Column.fn("listagg", e) + + /** + * Aggregate function: returns the concatenation of non-null input values, separated by the + * delimiter. + * + * @group agg_funcs + * @since 4.0.0 + */ + def listagg(e: Column, delimiter: Column): Column = Column.fn("listagg", e, delimiter) + + /** + * Aggregate function: returns the concatenation of distinct non-null input values. + * + * @group agg_funcs + * @since 4.0.0 + */ + def listagg_distinct(e: Column): Column = Column.fn("listagg", isDistinct = true, e) + + /** + * Aggregate function: returns the concatenation of distinct non-null input values, separated by + * the delimiter. + * + * @group agg_funcs + * @since 4.0.0 + */ + def listagg_distinct(e: Column, delimiter: Column): Column = + Column.fn("listagg", isDistinct = true, e, delimiter) + + /** + * Aggregate function: returns the concatenation of non-null input values. Alias for `listagg`. + * + * @group agg_funcs + * @since 4.0.0 + */ + def string_agg(e: Column): Column = Column.fn("string_agg", e) + + /** + * Aggregate function: returns the concatenation of non-null input values, separated by the + * delimiter. Alias for `listagg`. + * + * @group agg_funcs + * @since 4.0.0 + */ + def string_agg(e: Column, delimiter: Column): Column = Column.fn("string_agg", e, delimiter) + + /** + * Aggregate function: returns the concatenation of distinct non-null input values. Alias for + * `listagg`. + * + * @group agg_funcs + * @since 4.0.0 + */ + def string_agg_distinct(e: Column): Column = Column.fn("string_agg", isDistinct = true, e) + + /** + * Aggregate function: returns the concatenation of distinct non-null input values, separated by + * the delimiter. Alias for `listagg`. + * + * @group agg_funcs + * @since 4.0.0 + */ + def string_agg_distinct(e: Column, delimiter: Column): Column = + Column.fn("string_agg", isDistinct = true, e, delimiter) + /** * Aggregate function: alias for `var_samp`. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8e1b9da927c9f..3af3565220bdb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2772,6 +2772,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ne case e: Expression if e.foldable => e // No need to create an attribute reference if it will be evaluated as a Literal. + case e: SortOrder => + // For SortOder just recursively extract the from child expression. + e.copy(child = extractExpr(e.child)) case e: NamedArgumentExpression => // For NamedArgumentExpression, we extract the value and replace it with // an AttributeReference (with an internal column name, e.g. "_w0"). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 573619af1b5fc..1c76fd7d00f76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Median, PercentileCont, PercentileDisc} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, ListAgg, Median, PercentileCont, PercentileDisc} import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery, InlineCTE} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -423,10 +423,23 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB "funcName" -> toSQLExpr(wf), "windowExpr" -> toSQLExpr(w))) + case agg @ AggregateExpression(listAgg: ListAgg, _, _, _, _) + if agg.isDistinct && listAgg.needSaveOrderValue => + throw QueryCompilationErrors.functionAndOrderExpressionMismatchError( + listAgg.prettyName, listAgg.child, listAgg.orderExpressions) + case w: WindowExpression => // Only allow window functions with an aggregate expression or an offset window // function or a Pandas window UDF. w.windowFunction match { + case agg @ AggregateExpression(fun: ListAgg, _, _, _, _) + // listagg(...) WITHIN GROUP (ORDER BY ...) OVER (ORDER BY ...) is unsupported + if fun.orderingFilled && (w.windowSpec.orderSpec.nonEmpty || + w.windowSpec.frameSpecification != + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing)) => + agg.failAnalysis( + errorClass = "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + messageParameters = Map("aggFunc" -> toSQLExpr(agg.aggregateFunction))) case agg @ AggregateExpression( _: PercentileCont | _: PercentileDisc | _: Median, _, _, _, _) if w.windowSpec.orderSpec.nonEmpty || w.windowSpec.frameSpecification != diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 5103f8048856a..d9e9f49ce065e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -506,6 +506,8 @@ object FunctionRegistry { expression[CollectList]("collect_list"), expression[CollectList]("array_agg", true, Some("3.3.0")), expression[CollectSet]("collect_set"), + expression[ListAgg]("listagg"), + expression[ListAgg]("string_agg", setAlias = true), expressionBuilder("count_min_sketch", CountMinSketchAggExpressionBuilder), expression[BoolAnd]("every", true), expression[BoolAnd]("bool_and"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala index 5a27a72190325..800126e0030e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala @@ -128,18 +128,15 @@ class FunctionResolution( numArgs: Int, u: UnresolvedFunction): Expression = { func match { - case owg: SupportsOrderingWithinGroup if u.isDistinct => - throw QueryCompilationErrors.distinctInverseDistributionFunctionUnsupportedError( - owg.prettyName - ) + case owg: SupportsOrderingWithinGroup if !owg.isDistinctSupported && u.isDistinct => + throw QueryCompilationErrors.distinctWithOrderingFunctionUnsupportedError(owg.prettyName) case owg: SupportsOrderingWithinGroup - if !owg.orderingFilled && u.orderingWithinGroup.isEmpty => - throw QueryCompilationErrors.inverseDistributionFunctionMissingWithinGroupError( - owg.prettyName - ) + if owg.isOrderingMandatory && !owg.orderingFilled && u.orderingWithinGroup.isEmpty => + throw QueryCompilationErrors.functionMissingWithinGroupError(owg.prettyName) case owg: SupportsOrderingWithinGroup if owg.orderingFilled && u.orderingWithinGroup.nonEmpty => - throw QueryCompilationErrors.wrongNumOrderingsForInverseDistributionFunctionError( + // e.g mode(expr1) within group (order by expr2) is not supported + throw QueryCompilationErrors.wrongNumOrderingsForFunctionError( owg.prettyName, 0, u.orderingWithinGroup.length @@ -198,7 +195,7 @@ class FunctionResolution( case agg: AggregateFunction => // Note: PythonUDAF does not support these advanced clauses. if (agg.isInstanceOf[PythonUDAF]) checkUnsupportedAggregateClause(agg, u) - // After parse, the inverse distribution functions not set the ordering within group yet. + // After parse, the functions not set the ordering within group yet. val newAgg = agg match { case owg: SupportsOrderingWithinGroup if !owg.orderingFilled && u.orderingWithinGroup.nonEmpty => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Mode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Mode.scala index 97add0b8e45bc..f3eeaa96b3d46 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Mode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Mode.scala @@ -183,6 +183,8 @@ case class Mode( } override def orderingFilled: Boolean = child != UnresolvedWithinGroup + override def isOrderingMandatory: Boolean = true + override def isDistinctSupported: Boolean = false assert(orderingFilled || (!orderingFilled && reverseOpt.isEmpty)) @@ -190,7 +192,7 @@ case class Mode( child match { case UnresolvedWithinGroup => if (orderingWithinGroup.length != 1) { - throw QueryCompilationErrors.wrongNumOrderingsForInverseDistributionFunctionError( + throw QueryCompilationErrors.wrongNumOrderingsForFunctionError( nodeName, 1, orderingWithinGroup.length) } orderingWithinGroup.head match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/SupportsOrderingWithinGroup.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/SupportsOrderingWithinGroup.scala index 9c0502a2c1fcf..453251ac61cde 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/SupportsOrderingWithinGroup.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/SupportsOrderingWithinGroup.scala @@ -20,9 +20,26 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.expressions.SortOrder /** - * The trait used to set the [[SortOrder]] after inverse distribution functions parsed. + * The trait used to set the [[SortOrder]] for supporting functions. */ trait SupportsOrderingWithinGroup { self: AggregateFunction => - def orderingFilled: Boolean = false def withOrderingWithinGroup(orderingWithinGroup: Seq[SortOrder]): AggregateFunction + + /** Indicator that ordering was set. */ + def orderingFilled: Boolean + + /** + * Tells Analyzer that WITHIN GROUP (ORDER BY ...) is mandatory for function. + * + * @see [[QueryCompilationErrors.functionMissingWithinGroupError]] + */ + def isOrderingMandatory: Boolean + + /** + * Tells Analyzer that DISTINCT is supported. + * The DISTINCT can conflict with order so some functions can ban it. + * + * @see [[QueryCompilationErrors.functionMissingWithinGroupError]] + */ + def isDistinctSupported: Boolean } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 3aaf353043a9a..7789c23b50a48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -18,16 +18,22 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import scala.collection.mutable -import scala.collection.mutable.Growable +import scala.collection.mutable.{ArrayBuffer, Growable} +import scala.util.{Left, Right} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.catalyst.types.PhysicalDataType import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, TypeUtils, UnsafeRowUtils} +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} +import org.apache.spark.sql.errors.DataTypeErrors.{toSQLId, toSQLType} +import org.apache.spark.sql.internal.types.StringTypeWithCollation import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.{ByteArray, UTF8String} import org.apache.spark.util.BoundedPriorityQueue /** @@ -36,8 +42,7 @@ import org.apache.spark.util.BoundedPriorityQueue * We have to store all the collected elements in memory, and so notice that too many elements * can cause GC paused and eventually OutOfMemory Errors. */ -abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImperativeAggregate[T] - with UnaryLike[Expression] { +abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImperativeAggregate[T] { val child: Expression @@ -102,7 +107,8 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper case class CollectList( child: Expression, mutableAggBufferOffset: Int = 0, - inputAggBufferOffset: Int = 0) extends Collect[mutable.ArrayBuffer[Any]] { + inputAggBufferOffset: Int = 0) extends Collect[mutable.ArrayBuffer[Any]] + with UnaryLike[Expression] { def this(child: Expression) = this(child, 0, 0) @@ -149,7 +155,7 @@ case class CollectSet( child: Expression, mutableAggBufferOffset: Int = 0, inputAggBufferOffset: Int = 0) - extends Collect[mutable.HashSet[Any]] with QueryErrorsBase { + extends Collect[mutable.HashSet[Any]] with QueryErrorsBase with UnaryLike[Expression] { def this(child: Expression) = this(child, 0, 0) @@ -215,7 +221,8 @@ case class CollectTopK( num: Int, reverse: Boolean = false, mutableAggBufferOffset: Int = 0, - inputAggBufferOffset: Int = 0) extends Collect[BoundedPriorityQueue[Any]] { + inputAggBufferOffset: Int = 0) extends Collect[BoundedPriorityQueue[Any]] + with UnaryLike[Expression] { assert(num > 0) def this(child: Expression, num: Int) = this(child, num, false, 0, 0) @@ -265,3 +272,280 @@ private[aggregate] object CollectTopK { case _ => throw QueryCompilationErrors.invalidNumParameter(e) } } + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(expr[, delimiter])[ WITHIN GROUP (ORDER BY key [ASC | DESC] [,...])] - Returns + the concatenation of non-null input values, separated by the delimiter ordered by key. + If all values are null, null is returned. + """, + arguments = """ + Arguments: + * expr - a string or binary expression to be concatenated. + * delimiter - an optional string or binary foldable expression used to separate the input values. + If null, the concatenation will be performed without a delimiter. Default is null. + * key - an optional expression for ordering the input values. Multiple keys can be specified. + If none are specified, the order of the rows in the result is non-deterministic. + """, + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES ('a'), ('b'), ('c') AS tab(col); + abc + > SELECT _FUNC_(col) WITHIN GROUP (ORDER BY col DESC) FROM VALUES ('a'), ('b'), ('c') AS tab(col); + cba + > SELECT _FUNC_(col) FROM VALUES ('a'), (NULL), ('b') AS tab(col); + ab + > SELECT _FUNC_(col) FROM VALUES ('a'), ('a') AS tab(col); + aa + > SELECT _FUNC_(DISTINCT col) FROM VALUES ('a'), ('a'), ('b') AS tab(col); + ab + > SELECT _FUNC_(col, ', ') FROM VALUES ('a'), ('b'), ('c') AS tab(col); + a, b, c + > SELECT _FUNC_(col) FROM VALUES (NULL), (NULL) AS tab(col); + NULL + """, + note = """ + * If the order is not specified, the function is non-deterministic because + the order of the rows may be non-deterministic after a shuffle. + * If DISTINCT is specified, then expr and key must be the same expression. + """, + group = "agg_funcs", + since = "4.0.0" +) +// scalastyle:on line.size.limit +case class ListAgg( + child: Expression, + delimiter: Expression = Literal(null), + orderExpressions: Seq[SortOrder] = Nil, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends Collect[mutable.ArrayBuffer[Any]] + with SupportsOrderingWithinGroup + with ImplicitCastInputTypes { + + override def orderingFilled: Boolean = orderExpressions.nonEmpty + + override def isOrderingMandatory: Boolean = false + + override def isDistinctSupported: Boolean = true + + override def withOrderingWithinGroup(orderingWithinGroup: Seq[SortOrder]): AggregateFunction = + copy(orderExpressions = orderingWithinGroup) + + override protected lazy val bufferElementType: DataType = { + if (!needSaveOrderValue) { + child.dataType + } else { + StructType( + StructField("value", child.dataType) + +: orderValuesField + ) + } + } + /** Indicates that the result of [[child]] is not enough for evaluation */ + lazy val needSaveOrderValue: Boolean = !isOrderCompatible(orderExpressions) + + def this(child: Expression) = + this(child, Literal(null), Nil, 0, 0) + + def this(child: Expression, delimiter: Expression) = + this(child, delimiter, Nil, 0, 0) + + override def nullable: Boolean = true + + override def createAggregationBuffer(): mutable.ArrayBuffer[Any] = mutable.ArrayBuffer.empty + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def defaultResult: Option[Literal] = Option(Literal.create(null, dataType)) + + override def sql(isDistinct: Boolean): String = { + val distinct = if (isDistinct) "DISTINCT " else "" + val withinGroup = if (orderingFilled) { + s" WITHIN GROUP (ORDER BY ${orderExpressions.map(_.sql).mkString(", ")})" + } else { + "" + } + s"$prettyName($distinct${child.sql}, ${delimiter.sql})$withinGroup" + } + + override def inputTypes: Seq[AbstractDataType] = + TypeCollection( + StringTypeWithCollation(supportsTrimCollation = true), + BinaryType + ) +: + TypeCollection( + StringTypeWithCollation(supportsTrimCollation = true), + BinaryType, + NullType + ) +: + orderExpressions.map(_ => AnyDataType) + + override def checkInputDataTypes(): TypeCheckResult = { + val matchInputTypes = super.checkInputDataTypes() + if (matchInputTypes.isFailure) { + matchInputTypes + } else if (!delimiter.foldable) { + DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> toSQLId("delimiter"), + "inputType" -> toSQLType(delimiter.dataType), + "inputExpr" -> toSQLExpr(delimiter) + ) + ) + } else if (delimiter.dataType == NullType) { + // null is the default empty delimiter so type is not important + TypeCheckSuccess + } else { + TypeUtils.checkForSameTypeInputExpr(child.dataType :: delimiter.dataType :: Nil, prettyName) + } + } + + override def eval(buffer: mutable.ArrayBuffer[Any]): Any = { + if (buffer.nonEmpty) { + val sortedBufferWithoutNulls = sortBuffer(buffer) + concatSkippingNulls(sortedBufferWithoutNulls) + } else { + null + } + } + + /** + * Sort buffer according orderExpressions. + * If orderExpressions is empty then returns buffer as is. + * The format of buffer is determined by [[needSaveOrderValue]] + * @return sorted buffer containing only child's values + */ + private[this] def sortBuffer(buffer: mutable.ArrayBuffer[Any]): mutable.ArrayBuffer[Any] = { + if (!orderingFilled) { + // without order return as is. + return buffer + } + if (!needSaveOrderValue) { + // Here the buffer has structure [childValue0, childValue1, ...] + // and we want to sort it by childValues + val sortOrderExpression = orderExpressions.head + val ascendingOrdering = PhysicalDataType.ordering(sortOrderExpression.dataType) + val ordering = + if (sortOrderExpression.direction == Ascending) ascendingOrdering + else ascendingOrdering.reverse + buffer.sorted(ordering) + } else { + // Here the buffer has structure + // [[childValue, orderValue0, orderValue1, ...], + // [childValue, orderValue0, orderValue1, ...], + // ...] + // and we want to sort it by tuples (orderValue0, orderValue1, ...) + buffer + .asInstanceOf[mutable.ArrayBuffer[InternalRow]] + .sorted(bufferOrdering) + // drop orderValues after sort + .map(_.get(0, child.dataType)) + } + } + + /** + * @return ordering by (orderValue0, orderValue1, ...) + * for InternalRow with format [childValue, orderValue0, orderValue1, ...] + */ + private[this] def bufferOrdering: Ordering[InternalRow] = { + val bufferSortOrder = orderExpressions.zipWithIndex.map { + case (originalOrder, i) => + originalOrder.copy( + // first value is the evaluated child so add +1 for order's values + child = BoundReference(i + 1, originalOrder.dataType, originalOrder.child.nullable) + ) + } + new InterpretedOrdering(bufferSortOrder) + } + + private[this] def concatSkippingNulls(buffer: mutable.ArrayBuffer[Any]): Any = { + getDelimiterValue match { + case Right(delimiterValue: Array[Byte]) => + val inputs = buffer.filter(_ != null).map(_.asInstanceOf[Array[Byte]]) + ByteArray.concatWS(delimiterValue, inputs.toSeq: _*) + case Left(delimiterValue: UTF8String) => + val inputs = buffer.filter(_ != null).map(_.asInstanceOf[UTF8String]) + UTF8String.concatWs(delimiterValue, inputs.toSeq: _*) + } + } + + /** + * @return delimiter value or default empty value if delimiter is null. Type respects [[dataType]] + */ + private[this] def getDelimiterValue: Either[UTF8String, Array[Byte]] = { + val delimiterValue = delimiter.eval() + dataType match { + case _: StringType => + Left( + if (delimiterValue == null) UTF8String.fromString("") + else delimiterValue.asInstanceOf[UTF8String] + ) + case _: BinaryType => + Right( + if (delimiterValue == null) ByteArray.EMPTY_BYTE + else delimiterValue.asInstanceOf[Array[Byte]] + ) + } + } + + override def dataType: DataType = child.dataType + + override def update(buffer: ArrayBuffer[Any], input: InternalRow): ArrayBuffer[Any] = { + val value = child.eval(input) + if (value != null) { + val v = if (!needSaveOrderValue) { + convertToBufferElement(value) + } else { + InternalRow.fromSeq(convertToBufferElement(value) +: evalOrderValues(input)) + } + buffer += v + } + buffer + } + + private[this] def evalOrderValues(internalRow: InternalRow): Seq[Any] = { + orderExpressions.map(order => convertToBufferElement(order.child.eval(internalRow))) + } + + override protected def convertToBufferElement(value: Any): Any = InternalRow.copyValue(value) + + override def children: Seq[Expression] = child +: delimiter +: orderExpressions + + /** + * Utility func to check if given order is defined and different from [[child]]. + * + * @see [[QueryCompilationErrors.functionAndOrderExpressionMismatchError]] + * @see [[needSaveOrderValue]] + */ + private[this] def isOrderCompatible(someOrder: Seq[SortOrder]): Boolean = { + if (someOrder.isEmpty) { + return true + } + if (someOrder.size == 1 && someOrder.head.child.semanticEquals(child)) { + return true + } + false + } + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = + copy( + child = newChildren.head, + delimiter = newChildren(1), + orderExpressions = newChildren + .drop(2) + .map(_.asInstanceOf[SortOrder]) + ) + + private[this] def orderValuesField: Seq[StructField] = { + orderExpressions.zipWithIndex.map { + case (order, i) => StructField(s"sortOrderValue[$i]", order.dataType) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala index 89a6984b80852..6dfa1b499df23 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala @@ -378,7 +378,7 @@ case class PercentileCont(left: Expression, right: Expression, reverse: Boolean override def withOrderingWithinGroup(orderingWithinGroup: Seq[SortOrder]): AggregateFunction = { if (orderingWithinGroup.length != 1) { - throw QueryCompilationErrors.wrongNumOrderingsForInverseDistributionFunctionError( + throw QueryCompilationErrors.wrongNumOrderingsForFunctionError( nodeName, 1, orderingWithinGroup.length) } orderingWithinGroup.head match { @@ -390,6 +390,10 @@ case class PercentileCont(left: Expression, right: Expression, reverse: Boolean override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): PercentileCont = this.copy(left = newLeft, right = newRight) + + override def orderingFilled: Boolean = left != UnresolvedWithinGroup + override def isOrderingMandatory: Boolean = true + override def isDistinctSupported: Boolean = false } /** @@ -432,7 +436,7 @@ case class PercentileDisc( override def withOrderingWithinGroup(orderingWithinGroup: Seq[SortOrder]): AggregateFunction = { if (orderingWithinGroup.length != 1) { - throw QueryCompilationErrors.wrongNumOrderingsForInverseDistributionFunctionError( + throw QueryCompilationErrors.wrongNumOrderingsForFunctionError( nodeName, 1, orderingWithinGroup.length) } orderingWithinGroup.head match { @@ -467,6 +471,10 @@ case class PercentileDisc( toDoubleValue(higherKey) } } + + override def orderingFilled: Boolean = left != UnresolvedWithinGroup + override def isOrderingMandatory: Boolean = true + override def isDistinctSupported: Boolean = false } // scalastyle:off line.size.limit diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 03471ae8a3da5..7d7a490c97908 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, FunctionIdentif import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, FunctionAlreadyExistsException, NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchPartitionException, NoSuchTableException, Star, TableAlreadyExistsException, UnresolvedRegex} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, InvalidUDFClassException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, CreateMap, CreateStruct, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, CreateMap, CreateStruct, Expression, GroupingID, NamedExpression, SortOrder, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} import org.apache.spark.sql.catalyst.expressions.aggregate.AnyValue import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.logical.{Assignment, InputParameter, Join, LogicalPlan, SerdeInfo, Window} @@ -725,28 +725,32 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "windowExpr" -> toSQLExpr(windowExpr))) } - def distinctInverseDistributionFunctionUnsupportedError(funcName: String): Throwable = { + def distinctWithOrderingFunctionUnsupportedError(funcName: String): Throwable = { new AnalysisException( - errorClass = "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", - messageParameters = Map("funcName" -> toSQLId(funcName))) + errorClass = "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", + messageParameters = Map("funcName" -> toSQLId(funcName)) + ) } - def inverseDistributionFunctionMissingWithinGroupError(funcName: String): Throwable = { + def functionMissingWithinGroupError(funcName: String): Throwable = { new AnalysisException( - errorClass = "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WITHIN_GROUP_MISSING", - messageParameters = Map("funcName" -> toSQLId(funcName))) + errorClass = "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", + messageParameters = Map("funcName" -> toSQLId(funcName)) + ) } - def wrongNumOrderingsForInverseDistributionFunctionError( + def wrongNumOrderingsForFunctionError( funcName: String, validOrderingsNumber: Int, actualOrderingsNumber: Int): Throwable = { new AnalysisException( - errorClass = "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WRONG_NUM_ORDERINGS", + errorClass = "INVALID_WITHIN_GROUP_EXPRESSION.WRONG_NUM_ORDERINGS", messageParameters = Map( "funcName" -> toSQLId(funcName), "expectedNum" -> validOrderingsNumber.toString, - "actualNum" -> actualOrderingsNumber.toString)) + "actualNum" -> actualOrderingsNumber.toString + ) + ) } def aliasNumberNotMatchColumnNumberError( @@ -1049,6 +1053,18 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "operation" -> operation)) } + def functionAndOrderExpressionMismatchError( + functionName: String, + functionArg: Expression, + orderExpr: Seq[SortOrder]): Throwable = { + new AnalysisException( + errorClass = "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + messageParameters = Map( + "funcName" -> toSQLId(functionName), + "funcArg" -> toSQLExpr(functionArg), + "orderingExpr" -> orderExpr.map(order => toSQLExpr(order.child)).mkString(", "))) + } + def wrongCommandForObjectTypeError( operation: String, requiredType: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index c621c151c0bd6..e77c050fe8872 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -607,7 +607,12 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // [COUNT(DISTINCT bar), COUNT(DISTINCT foo)] is disallowed because those two distinct // aggregates have different column expressions. val distinctExpressions = - functionsWithDistinct.head.aggregateFunction.children.filterNot(_.foldable) + functionsWithDistinct.head.aggregateFunction.children + .filterNot(_.foldable) + .map { + case s: SortOrder => s.child + case e => e + } val normalizedNamedDistinctExpressions = distinctExpressions.map { e => // Ideally this should be done in `NormalizeFloatingNumbers`, but we do it here // because `distinctExpressions` is not extracted during logical phase. diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index c54e09735a9be..39cefdaa892b2 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -425,6 +425,8 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis | kurtosis | SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.ListAgg | listagg | SELECT listagg(col) FROM VALUES ('a'), ('b'), ('c') AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.ListAgg | string_agg | SELECT string_agg(col) FROM VALUES ('a'), ('b'), ('c') AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Max | max | SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy | max_by | SELECT max_by(x, y) FROM VALUES ('a', 10), ('b', 50), ('c', 20) AS tab(x, y) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Median | median | SELECT median(col) FROM VALUES (0), (10) AS tab(col) | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out new file mode 100644 index 0000000000000..ca471858a5416 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out @@ -0,0 +1,86 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query analysis +Aggregate [listagg(c1#x, null, collate(c1#x, utf8_binary) ASC NULLS FIRST, 0, 0) AS listagg(c1, NULL) WITHIN GROUP (ORDER BY collate(c1, utf8_binary) ASC NULLS FIRST)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query analysis +Aggregate [listagg(c1#x, null, collate(c1#x, utf8_lcase) ASC NULLS FIRST, 0, 0) AS listagg(c1, NULL) WITHIN GROUP (ORDER BY collate(c1, utf8_lcase) ASC NULLS FIRST)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query analysis +Aggregate [listagg(distinct collate(c1#x, utf8_binary), null, 0, 0) AS listagg(DISTINCT collate(c1, utf8_binary), NULL)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query analysis +Aggregate [listagg(distinct collate(c1#x, utf8_lcase), null, 0, 0) AS listagg(DISTINCT collate(c1, utf8_lcase), NULL)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('B'), ('b'), ('A')) AS t(c1) +-- !query analysis +Aggregate [listagg(distinct collate(c1#x, utf8_lcase), null, collate(c1#x, utf8_lcase) ASC NULLS FIRST, 0, 0) AS listagg(DISTINCT collate(c1, utf8_lcase), NULL) WITHIN GROUP (ORDER BY collate(c1, utf8_lcase) ASC NULLS FIRST)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc '), ('x'), ('abc')) AS t(c1) +-- !query analysis +Aggregate [listagg(distinct collate(c1#x, unicode_rtrim), null, 0, 0) AS listagg(DISTINCT collate(c1, unicode_rtrim), NULL)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x')) AS t(c1) +-- !query analysis +Aggregate [listagg(c1#x, null, c1#x ASC NULLS FIRST, 0, 0) AS listagg(c1, NULL) WITHIN GROUP (ORDER BY c1 ASC NULLS FIRST)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x')) AS t(c1) +-- !query analysis +Aggregate [listagg(c1#x, null, collate(c1#x, unicode_rtrim) ASC NULLS FIRST, 0, 0) AS listagg(c1, NULL) WITHIN GROUP (ORDER BY collate(c1, unicode_rtrim) ASC NULLS FIRST)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('b'), ('A'), ('B')) AS t(c1) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"collate(c1, utf8_lcase)\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"collate(c1, utf8_binary)\"" + } +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out new file mode 100644 index 0000000000000..71eb3f8ca76b3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg.sql.out @@ -0,0 +1,435 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMP VIEW df AS +SELECT * FROM (VALUES ('a', 'b'), ('a', 'c'), ('b', 'c'), ('b', 'd'), (NULL, NULL)) AS t(a, b) +-- !query analysis +CreateViewCommand `df`, SELECT * FROM (VALUES ('a', 'b'), ('a', 'c'), ('b', 'c'), ('b', 'd'), (NULL, NULL)) AS t(a, b), false, false, LocalTempView, UNSUPPORTED, true + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE TEMP VIEW df2 AS +SELECT * FROM (VALUES (1, true), (2, false), (3, false)) AS t(a, b) +-- !query analysis +CreateViewCommand `df2`, SELECT * FROM (VALUES (1, true), (2, false), (3, false)) AS t(a, b), false, false, LocalTempView, UNSUPPORTED, true + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(b) FROM df GROUP BY a +-- !query analysis +Aggregate [a#x], [listagg(b#x, null, 0, 0) AS listagg(b, NULL)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT string_agg(b) FROM df GROUP BY a +-- !query analysis +Aggregate [a#x], [string_agg(b#x, null, 0, 0) AS string_agg(b, NULL)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(b, NULL) FROM df GROUP BY a +-- !query analysis +Aggregate [a#x], [listagg(b#x, null, 0, 0) AS listagg(b, NULL)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(b) FROM df WHERE 1 != 1 +-- !query analysis +Aggregate [listagg(b#x, null, 0, 0) AS listagg(b, NULL)#x] ++- Filter NOT (1 = 1) + +- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(b, '|') FROM df GROUP BY a +-- !query analysis +Aggregate [a#x], [listagg(b#x, |, 0, 0) AS listagg(b, |)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) FROM df +-- !query analysis +Aggregate [listagg(a#x, null, 0, 0) AS listagg(a, NULL)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(DISTINCT a) FROM df +-- !query analysis +Aggregate [listagg(distinct a#x, null, 0, 0) AS listagg(DISTINCT a, NULL)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a) FROM df +-- !query analysis +Aggregate [listagg(a#x, null, a#x ASC NULLS FIRST, 0, 0) AS listagg(a, NULL) WITHIN GROUP (ORDER BY a ASC NULLS FIRST)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a DESC) FROM df +-- !query analysis +Aggregate [listagg(a#x, null, a#x DESC NULLS LAST, 0, 0) AS listagg(a, NULL) WITHIN GROUP (ORDER BY a DESC NULLS LAST)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a DESC) OVER (PARTITION BY b) FROM df +-- !query analysis +Project [listagg(a, NULL) WITHIN GROUP (ORDER BY a DESC NULLS LAST) OVER (PARTITION BY b ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] ++- Project [a#x, b#x, listagg(a, NULL) WITHIN GROUP (ORDER BY a DESC NULLS LAST) OVER (PARTITION BY b ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, listagg(a, NULL) WITHIN GROUP (ORDER BY a DESC NULLS LAST) OVER (PARTITION BY b ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [listagg(a#x, null, a#x DESC NULLS LAST, 0, 0) windowspecdefinition(b#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS listagg(a, NULL) WITHIN GROUP (ORDER BY a DESC NULLS LAST) OVER (PARTITION BY b ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [b#x] + +- Project [a#x, b#x] + +- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b) FROM df +-- !query analysis +Aggregate [listagg(a#x, null, b#x ASC NULLS FIRST, 0, 0) AS listagg(a, NULL) WITHIN GROUP (ORDER BY b ASC NULLS FIRST)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC) FROM df +-- !query analysis +Aggregate [listagg(a#x, null, b#x DESC NULLS LAST, 0, 0) AS listagg(a, NULL) WITHIN GROUP (ORDER BY b DESC NULLS LAST)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a, '|') WITHIN GROUP (ORDER BY b DESC) FROM df +-- !query analysis +Aggregate [listagg(a#x, |, b#x DESC NULLS LAST, 0, 0) AS listagg(a, |) WITHIN GROUP (ORDER BY b DESC NULLS LAST)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC, a ASC) FROM df +-- !query analysis +Aggregate [listagg(a#x, null, b#x DESC NULLS LAST, a#x ASC NULLS FIRST, 0, 0) AS listagg(a, NULL) WITHIN GROUP (ORDER BY b DESC NULLS LAST, a ASC NULLS FIRST)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC, a DESC) FROM df +-- !query analysis +Aggregate [listagg(a#x, null, b#x DESC NULLS LAST, a#x DESC NULLS LAST, 0, 0) AS listagg(a, NULL) WITHIN GROUP (ORDER BY b DESC NULLS LAST, a DESC NULLS LAST)#x] ++- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(c1) FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query analysis +Aggregate [listagg(c1#x, null, 0, 0) AS listagg(c1, NULL)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(c1, NULL) FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query analysis +Aggregate [listagg(c1#x, null, 0, 0) AS listagg(c1, NULL)#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(c1, X'42') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query analysis +Aggregate [listagg(c1#x, 0x42, 0, 0) AS listagg(c1, X'42')#x] ++- SubqueryAlias t + +- Project [col1#x AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT listagg(a), listagg(b, ',') FROM df2 +-- !query analysis +Aggregate [listagg(cast(a#x as string), null, 0, 0) AS listagg(a, NULL)#x, listagg(cast(b#x as string), ,, 0, 0) AS listagg(b, ,)#x] ++- SubqueryAlias df2 + +- View (`df2`, [a#x, b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as boolean) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(c1) FROM (VALUES (ARRAY('a', 'b'))) AS t(c1) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"c1\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "first", + "requiredType" : "(\"STRING\" or \"BINARY\")", + "sqlExpr" : "\"listagg(c1, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "listagg(c1)" + } ] +} + + +-- !query +SELECT listagg(c1, ', ') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"BINARY\" or \"STRING\")", + "functionName" : "`listagg`", + "sqlExpr" : "\"listagg(c1, , )\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "listagg(c1, ', ')" + } ] +} + + +-- !query +SELECT listagg(b, a) FROM df GROUP BY a +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"a\"", + "inputName" : "`delimiter`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"listagg(b, a)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "listagg(b, a)" + } ] +} + + +-- !query +SELECT listagg(a) OVER (ORDER BY a) FROM df +-- !query analysis +Project [listagg(a, NULL) OVER (ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [a#x, listagg(a, NULL) OVER (ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, listagg(a, NULL) OVER (ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [listagg(a#x, null, 0, 0) windowspecdefinition(a#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS listagg(a, NULL) OVER (ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [a#x ASC NULLS FIRST] + +- Project [a#x] + +- SubqueryAlias df + +- View (`df`, [a#x, b#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a) FROM df +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"listagg(a, NULL, a ASC NULLS FIRST)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "listagg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a)" + } ] +} + + +-- !query +SELECT string_agg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a) FROM df +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"listagg(a, NULL, a ASC NULLS FIRST)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "string_agg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a)" + } ] +} + + +-- !query +SELECT listagg(DISTINCT a) OVER (ORDER BY a) FROM df +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED", + "sqlState" : "0A000", + "messageParameters" : { + "windowExpr" : "\"listagg(DISTINCT a, NULL) OVER (ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "listagg(DISTINCT a) OVER (ORDER BY a)" + } ] +} + + +-- !query +SELECT listagg(DISTINCT a) WITHIN GROUP (ORDER BY b) FROM df +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"a\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"b\"" + } +} + + +-- !query +SELECT listagg(DISTINCT a) WITHIN GROUP (ORDER BY a, b) FROM df +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"a\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"a\", \"b\"" + } +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/mode.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/mode.sql.out index d6ecbc72a7178..8028c344140f5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/mode.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/mode.sql.out @@ -74,7 +74,7 @@ SELECT department, mode(DISTINCT salary) FROM basic_pays GROUP BY department ORD -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`mode`" @@ -379,7 +379,7 @@ FROM basic_pays -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`mode`" @@ -401,7 +401,7 @@ FROM basic_pays -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WITHIN_GROUP_MISSING", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`mode`" @@ -423,7 +423,7 @@ FROM basic_pays -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WRONG_NUM_ORDERINGS", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WRONG_NUM_ORDERINGS", "sqlState" : "42K0K", "messageParameters" : { "actualNum" : "1", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out index 4a31cff8c7d0f..eb8102afa47ef 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out @@ -248,7 +248,7 @@ FROM aggr -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -270,7 +270,7 @@ FROM aggr -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -342,7 +342,7 @@ FROM aggr -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WITHIN_GROUP_MISSING", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -364,7 +364,7 @@ FROM aggr -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WITHIN_GROUP_MISSING", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -386,7 +386,7 @@ FROM aggr -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WRONG_NUM_ORDERINGS", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WRONG_NUM_ORDERINGS", "sqlState" : "42K0K", "messageParameters" : { "actualNum" : "2", diff --git a/sql/core/src/test/resources/sql-tests/inputs/listagg-collations.sql b/sql/core/src/test/resources/sql-tests/inputs/listagg-collations.sql new file mode 100644 index 0000000000000..35f86183c37b3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/listagg-collations.sql @@ -0,0 +1,12 @@ +-- Test cases with collations +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1); +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1); +SELECT listagg(DISTINCT c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1); +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1); +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('B'), ('b'), ('A')) AS t(c1); +SELECT listagg(DISTINCT c1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc '), ('x'), ('abc')) AS t(c1); +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x')) AS t(c1); +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x')) AS t(c1); + +-- Error case with collations +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('b'), ('A'), ('B')) AS t(c1); \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/listagg.sql b/sql/core/src/test/resources/sql-tests/inputs/listagg.sql new file mode 100644 index 0000000000000..15c8cfa823e9b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/listagg.sql @@ -0,0 +1,38 @@ +-- Create temporary views +CREATE TEMP VIEW df AS +SELECT * FROM (VALUES ('a', 'b'), ('a', 'c'), ('b', 'c'), ('b', 'd'), (NULL, NULL)) AS t(a, b); + +CREATE TEMP VIEW df2 AS +SELECT * FROM (VALUES (1, true), (2, false), (3, false)) AS t(a, b); + +-- Test cases for listagg function +SELECT listagg(b) FROM df GROUP BY a; +SELECT string_agg(b) FROM df GROUP BY a; +SELECT listagg(b, NULL) FROM df GROUP BY a; +SELECT listagg(b) FROM df WHERE 1 != 1; +SELECT listagg(b, '|') FROM df GROUP BY a; +SELECT listagg(a) FROM df; +SELECT listagg(DISTINCT a) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY a) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY a DESC) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY a DESC) OVER (PARTITION BY b) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY b) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC) FROM df; +SELECT listagg(a, '|') WITHIN GROUP (ORDER BY b DESC) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC, a ASC) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC, a DESC) FROM df; +SELECT listagg(c1) FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1); +SELECT listagg(c1, NULL) FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1); +SELECT listagg(c1, X'42') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1); +SELECT listagg(a), listagg(b, ',') FROM df2; + +-- Error cases +SELECT listagg(c1) FROM (VALUES (ARRAY('a', 'b'))) AS t(c1); +SELECT listagg(c1, ', ') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1); +SELECT listagg(b, a) FROM df GROUP BY a; +SELECT listagg(a) OVER (ORDER BY a) FROM df; +SELECT listagg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a) FROM df; +SELECT string_agg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a) FROM df; +SELECT listagg(DISTINCT a) OVER (ORDER BY a) FROM df; +SELECT listagg(DISTINCT a) WITHIN GROUP (ORDER BY b) FROM df; +SELECT listagg(DISTINCT a) WITHIN GROUP (ORDER BY a, b) FROM df; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out new file mode 100644 index 0000000000000..cf3bac04f09ca --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out @@ -0,0 +1,82 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query schema +struct +-- !query output +ABab + + +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query schema +struct +-- !query output +aAbB + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query schema +struct +-- !query output +aAbB + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query schema +struct +-- !query output +ab + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase) FROM (VALUES ('a'), ('B'), ('b'), ('A')) AS t(c1) +-- !query schema +struct +-- !query output +aB + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc '), ('x'), ('abc')) AS t(c1) +-- !query schema +struct +-- !query output +abc x + + +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x')) AS t(c1) +-- !query schema +struct +-- !query output +abcabc +abc abc x + + +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x')) AS t(c1) +-- !query schema +struct +-- !query output +abc abc abcabc +x + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('b'), ('A'), ('B')) AS t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"collate(c1, utf8_lcase)\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"collate(c1, utf8_binary)\"" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/listagg.sql.out b/sql/core/src/test/resources/sql-tests/results/listagg.sql.out new file mode 100644 index 0000000000000..ef580704992ce --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/listagg.sql.out @@ -0,0 +1,368 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMP VIEW df AS +SELECT * FROM (VALUES ('a', 'b'), ('a', 'c'), ('b', 'c'), ('b', 'd'), (NULL, NULL)) AS t(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW df2 AS +SELECT * FROM (VALUES (1, true), (2, false), (3, false)) AS t(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT listagg(b) FROM df GROUP BY a +-- !query schema +struct +-- !query output +NULL +bc +cd + + +-- !query +SELECT string_agg(b) FROM df GROUP BY a +-- !query schema +struct +-- !query output +NULL +bc +cd + + +-- !query +SELECT listagg(b, NULL) FROM df GROUP BY a +-- !query schema +struct +-- !query output +NULL +bc +cd + + +-- !query +SELECT listagg(b) FROM df WHERE 1 != 1 +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT listagg(b, '|') FROM df GROUP BY a +-- !query schema +struct +-- !query output +NULL +b|c +c|d + + +-- !query +SELECT listagg(a) FROM df +-- !query schema +struct +-- !query output +aabb + + +-- !query +SELECT listagg(DISTINCT a) FROM df +-- !query schema +struct +-- !query output +ab + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a) FROM df +-- !query schema +struct +-- !query output +aabb + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a DESC) FROM df +-- !query schema +struct +-- !query output +bbaa + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a DESC) OVER (PARTITION BY b) FROM df +-- !query schema +struct +-- !query output +NULL +a +b +ba +ba + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b) FROM df +-- !query schema +struct +-- !query output +aabb + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC) FROM df +-- !query schema +struct +-- !query output +baba + + +-- !query +SELECT listagg(a, '|') WITHIN GROUP (ORDER BY b DESC) FROM df +-- !query schema +struct +-- !query output +b|a|b|a + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC, a ASC) FROM df +-- !query schema +struct +-- !query output +baba + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY b DESC, a DESC) FROM df +-- !query schema +struct +-- !query output +bbaa + + +-- !query +SELECT listagg(c1) FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query schema +struct +-- !query output +ޭ�� + + +-- !query +SELECT listagg(c1, NULL) FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query schema +struct +-- !query output +ޭ�� + + +-- !query +SELECT listagg(c1, X'42') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query schema +struct +-- !query output +ޭB�� + + +-- !query +SELECT listagg(a), listagg(b, ',') FROM df2 +-- !query schema +struct +-- !query output +123 true,false,false + + +-- !query +SELECT listagg(c1) FROM (VALUES (ARRAY('a', 'b'))) AS t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"c1\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "first", + "requiredType" : "(\"STRING\" or \"BINARY\")", + "sqlExpr" : "\"listagg(c1, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "listagg(c1)" + } ] +} + + +-- !query +SELECT listagg(c1, ', ') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"BINARY\" or \"STRING\")", + "functionName" : "`listagg`", + "sqlExpr" : "\"listagg(c1, , )\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "listagg(c1, ', ')" + } ] +} + + +-- !query +SELECT listagg(b, a) FROM df GROUP BY a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"a\"", + "inputName" : "`delimiter`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"listagg(b, a)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "listagg(b, a)" + } ] +} + + +-- !query +SELECT listagg(a) OVER (ORDER BY a) FROM df +-- !query schema +struct +-- !query output +NULL +aa +aa +aabb +aabb + + +-- !query +SELECT listagg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"listagg(a, NULL, a ASC NULLS FIRST)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "listagg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a)" + } ] +} + + +-- !query +SELECT string_agg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"listagg(a, NULL, a ASC NULLS FIRST)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "string_agg(a) WITHIN GROUP (ORDER BY a) OVER (ORDER BY a)" + } ] +} + + +-- !query +SELECT listagg(DISTINCT a) OVER (ORDER BY a) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED", + "sqlState" : "0A000", + "messageParameters" : { + "windowExpr" : "\"listagg(DISTINCT a, NULL) OVER (ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "listagg(DISTINCT a) OVER (ORDER BY a)" + } ] +} + + +-- !query +SELECT listagg(DISTINCT a) WITHIN GROUP (ORDER BY b) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"a\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"b\"" + } +} + + +-- !query +SELECT listagg(DISTINCT a) WITHIN GROUP (ORDER BY a, b) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"a\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"a\", \"b\"" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/mode.sql.out b/sql/core/src/test/resources/sql-tests/results/mode.sql.out index ad7d59eeb1634..70f253066d4f9 100644 --- a/sql/core/src/test/resources/sql-tests/results/mode.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/mode.sql.out @@ -51,7 +51,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`mode`" @@ -373,7 +373,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`mode`" @@ -397,7 +397,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WITHIN_GROUP_MISSING", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`mode`" @@ -421,7 +421,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WRONG_NUM_ORDERINGS", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WRONG_NUM_ORDERINGS", "sqlState" : "42K0K", "messageParameters" : { "actualNum" : "1", diff --git a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out index cd95eee186e12..55aaa8ee7378e 100644 --- a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out @@ -222,7 +222,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -246,7 +246,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.DISTINCT_UNSUPPORTED", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -324,7 +324,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WITHIN_GROUP_MISSING", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -348,7 +348,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WITHIN_GROUP_MISSING", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", "sqlState" : "42K0K", "messageParameters" : { "funcName" : "`percentile_cont`" @@ -372,7 +372,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_INVERSE_DISTRIBUTION_FUNCTION.WRONG_NUM_ORDERINGS", + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WRONG_NUM_ORDERINGS", "sqlState" : "42K0K", "messageParameters" : { "actualNum" : "2", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 6348e5f315395..ad80dc65926bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -621,6 +621,41 @@ class DataFrameAggregateSuite extends QueryTest ) } + test("listagg function") { + // normal case + val df = Seq(("a", "b"), ("b", "c"), ("c", "d")).toDF("a", "b") + checkAnswer( + df.selectExpr("listagg(a)", "listagg(b)"), + Seq(Row("abc", "bcd")) + ) + checkAnswer( + df.select(listagg($"a"), listagg($"b")), + Seq(Row("abc", "bcd")) + ) + + // distinct case + val df2 = Seq(("a", "b"), ("a", "b"), ("b", "d")).toDF("a", "b") + checkAnswer( + df2.select(listagg_distinct($"a"), listagg_distinct($"b")), + Seq(Row("ab", "bd")) + ) + + // null case + val df3 = Seq(("a", "b", null), ("a", "b", null), (null, null, null)).toDF("a", "b", "c") + checkAnswer( + df3.select(listagg_distinct($"a"), listagg($"a"), listagg_distinct($"b"), listagg($"b"), + listagg($"c")), + Seq(Row("a", "aa", "b", "bb", null)) + ) + + // custom delimiter + val df4 = Seq(("a", "b"), ("b", "c"), ("c", "d")).toDF("a", "b") + checkAnswer( + df4.selectExpr("listagg(a, '|')", "listagg(b, '|')"), + Seq(Row("a|b|c", "b|c|d")) + ) + } + test("SPARK-31500: collect_set() of BinaryType returns duplicate elements") { val bytesTest1 = "test1".getBytes val bytesTest2 = "test2".getBytes diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 975a82e26f4eb..4494057b1eefe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -73,7 +73,9 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { "sum_distinct", // equivalent to sum(distinct foo) "typedLit", "typedlit", // Scala only "udaf", "udf", // create function statement in sql - "call_function" // moot in SQL as you just call the function directly + "call_function", // moot in SQL as you just call the function directly + "listagg_distinct", // equivalent to listagg(distinct foo) + "string_agg_distinct" // equivalent to string_agg(distinct foo) ) val excludedSqlFunctions = Set.empty[String] diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 662f43fc00399..283454ad273ed 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -104,6 +104,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ "timestampNTZ/datetime-special-ansi.sql", // SPARK-47264 "collations.sql", + "listagg-collations.sql", "pipe-operators.sql", // VARIANT type "variant/named-function-arguments.sql" From 3fab712f69f0073d6e5481d43c455363431952fc Mon Sep 17 00:00:00 2001 From: Mihailo Timotic Date: Fri, 29 Nov 2024 21:46:32 +0800 Subject: [PATCH 060/438] [SPARK-50441][SQL] Fix parametrized identifiers not working when referencing CTEs ### What changes were proposed in this pull request? Fix parametrized identifiers not working when referencing CTEs ### Why are the changes needed? For a query: `with t1 as (select 1) select * from identifier(:cte) using cte as "t1"` the resolution fails because `BindParameters` can't resolve parameters because it waits for `ResolveIdentifierClause` to resolve `UnresolvedWithCTERelation`, but `ResolveIdentifierClause` can't resolve `UnresolvedWithCTERelation` until all `NamedParameters` in the plan are resolved. Instead of delaying CTE resolution with `UnresolvedWithCTERelation`, we can remove node entirely and delay the resolution by keeping the original `PlanWithUnresolvedIdentifier` and moving the CTE resolution to its `planBuilder`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a new test to `ParametersSuite` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48994 from mihailotim-db/mihailotim-db/cte_identifer. Authored-by: Mihailo Timotic Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 3 -- .../catalyst/analysis/CTESubstitution.scala | 41 +++++++++++++------ .../analysis/ResolveIdentifierClause.scala | 15 ++----- .../sql/catalyst/analysis/parameters.scala | 6 +-- .../sql/catalyst/analysis/unresolved.scala | 13 +----- .../sql/catalyst/trees/TreePatterns.scala | 1 - .../apache/spark/sql/ParametersSuite.scala | 11 +++++ 7 files changed, 47 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3af3565220bdb..089e18e3df4e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1610,9 +1610,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case s: Sort if !s.resolved || s.missingInput.nonEmpty => resolveReferencesInSort(s) - case u: UnresolvedWithCTERelations => - UnresolvedWithCTERelations(this.apply(u.unresolvedPlan), u.cteRelations) - case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString(conf.maxToStringFields)}") q.mapExpressions(resolveExpressionByPlanChildren(_, q, includeLastResort = true)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index ff0dbcd7ef153..d75e7d528d5b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -267,6 +267,25 @@ object CTESubstitution extends Rule[LogicalPlan] { resolvedCTERelations } + private def resolveWithCTERelations( + table: String, + alwaysInline: Boolean, + cteRelations: Seq[(String, CTERelationDef)], + unresolvedRelation: UnresolvedRelation): LogicalPlan = { + cteRelations + .find(r => conf.resolver(r._1, table)) + .map { + case (_, d) => + if (alwaysInline) { + d.child + } else { + // Add a `SubqueryAlias` for hint-resolving rules to match relation names. + SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) + } + } + .getOrElse(unresolvedRelation) + } + private def substituteCTE( plan: LogicalPlan, alwaysInline: Boolean, @@ -279,22 +298,20 @@ object CTESubstitution extends Rule[LogicalPlan] { throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(table)) case u @ UnresolvedRelation(Seq(table), _, _) => - cteRelations.find(r => plan.conf.resolver(r._1, table)).map { case (_, d) => - if (alwaysInline) { - d.child - } else { - // Add a `SubqueryAlias` for hint-resolving rules to match relation names. - SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) - } - }.getOrElse(u) + resolveWithCTERelations(table, alwaysInline, cteRelations, u) case p: PlanWithUnresolvedIdentifier => // We must look up CTE relations first when resolving `UnresolvedRelation`s, // but we can't do it here as `PlanWithUnresolvedIdentifier` is a leaf node - // and may produce `UnresolvedRelation` later. - // Here we wrap it with `UnresolvedWithCTERelations` so that we can - // delay the CTE relations lookup after `PlanWithUnresolvedIdentifier` is resolved. - UnresolvedWithCTERelations(p, cteRelations) + // and may produce `UnresolvedRelation` later. Instead, we delay CTE resolution + // by moving it to the planBuilder of the corresponding `PlanWithUnresolvedIdentifier`. + p.copy(planBuilder = (nameParts, children) => { + p.planBuilder.apply(nameParts, children) match { + case u @ UnresolvedRelation(Seq(table), _, _) => + resolveWithCTERelations(table, alwaysInline, cteRelations, u) + case other => other + } + }) case other => // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala index 0e1e71a658c8b..2cf3c6390d5fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.{AliasHelper, EvalHelper, Expression} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{CTERelationRef, LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.TreePattern.{UNRESOLVED_IDENTIFIER, UNRESOLVED_IDENTIFIER_WITH_CTE} +import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_IDENTIFIER import org.apache.spark.sql.types.StringType /** @@ -30,18 +30,9 @@ import org.apache.spark.sql.types.StringType object ResolveIdentifierClause extends Rule[LogicalPlan] with AliasHelper with EvalHelper { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning( - _.containsAnyPattern(UNRESOLVED_IDENTIFIER, UNRESOLVED_IDENTIFIER_WITH_CTE)) { + _.containsPattern(UNRESOLVED_IDENTIFIER)) { case p: PlanWithUnresolvedIdentifier if p.identifierExpr.resolved && p.childrenResolved => p.planBuilder.apply(evalIdentifierExpr(p.identifierExpr), p.children) - case u @ UnresolvedWithCTERelations(p, cteRelations) => - this.apply(p) match { - case u @ UnresolvedRelation(Seq(table), _, _) => - cteRelations.find(r => plan.conf.resolver(r._1, table)).map { case (_, d) => - // Add a `SubqueryAlias` for hint-resolving rules to match relation names. - SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) - }.getOrElse(u) - case other => other - } case other => other.transformExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_IDENTIFIER)) { case e: ExpressionWithUnresolvedIdentifier if e.identifierExpr.resolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index f24227abbb651..de73747769469 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.expressions.{Alias, CreateArray, CreateMap, CreateNamedStruct, Expression, LeafExpression, Literal, MapFromArrays, MapFromEntries, SubqueryExpression, Unevaluable, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SupervisingCommand} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, PARAMETER, PARAMETERIZED_QUERY, TreePattern, UNRESOLVED_IDENTIFIER_WITH_CTE, UNRESOLVED_WITH} +import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, PARAMETER, PARAMETERIZED_QUERY, TreePattern, UNRESOLVED_WITH} import org.apache.spark.sql.errors.QueryErrorsBase import org.apache.spark.sql.types.DataType @@ -189,7 +189,7 @@ object BindParameters extends ParameterizedQueryProcessor with QueryErrorsBase { // We should wait for `CTESubstitution` to resolve CTE before binding parameters, as CTE // relations are not children of `UnresolvedWith`. case NameParameterizedQuery(child, argNames, argValues) - if !child.containsAnyPattern(UNRESOLVED_WITH, UNRESOLVED_IDENTIFIER_WITH_CTE) && + if !child.containsPattern(UNRESOLVED_WITH) && argValues.forall(_.resolved) => if (argNames.length != argValues.length) { throw SparkException.internalError(s"The number of argument names ${argNames.length} " + @@ -200,7 +200,7 @@ object BindParameters extends ParameterizedQueryProcessor with QueryErrorsBase { bind(child) { case NamedParameter(name) if args.contains(name) => args(name) } case PosParameterizedQuery(child, args) - if !child.containsAnyPattern(UNRESOLVED_WITH, UNRESOLVED_IDENTIFIER_WITH_CTE) && + if !child.containsPattern(UNRESOLVED_WITH) && args.forall(_.resolved) => val indexedArgs = args.zipWithIndex checkArgs(indexedArgs.map(arg => (s"_${arg._2}", arg._1))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 7fc8aff72b81d..0a73b6b856740 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIden import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, LeafNode, LogicalPlan, UnaryNode} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId @@ -76,17 +76,6 @@ case class PlanWithUnresolvedIdentifier( copy(identifierExpr, newChildren, planBuilder) } -/** - * A logical plan placeholder which delays CTE resolution - * to moment when PlanWithUnresolvedIdentifier gets resolved - */ -case class UnresolvedWithCTERelations( - unresolvedPlan: LogicalPlan, - cteRelations: Seq[(String, CTERelationDef)]) - extends UnresolvedLeafNode { - final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_WITH_CTE) -} - /** * An expression placeholder that holds the identifier clause string expression. It will be * replaced by the actual expression with the evaluated identifier string. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 7435f4c527034..e95712281cb42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -154,7 +154,6 @@ object TreePattern extends Enumeration { val UNRESOLVED_FUNCTION: Value = Value val UNRESOLVED_HINT: Value = Value val UNRESOLVED_WINDOW_EXPRESSION: Value = Value - val UNRESOLVED_IDENTIFIER_WITH_CTE: Value = Value // Unresolved Plan patterns (Alphabetically ordered) val UNRESOLVED_FUNC: Value = Value diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 791bcc91d5094..2ac8ed26868a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -758,4 +758,15 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { checkAnswer(spark.sql(query("?"), args = Array("tt1")), Row(1)) } } + + test("SPARK-50441: parameterized identifier referencing a CTE") { + def query(p: String): String = { + s""" + |WITH t1 AS (SELECT 1) + |SELECT * FROM IDENTIFIER($p)""".stripMargin + } + + checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), Row(1)) + checkAnswer(spark.sql(query("?"), args = Array("t1")), Row(1)) + } } From faf74adad0f3d790a932d608e6ade639875d9e8f Mon Sep 17 00:00:00 2001 From: Stevo Mitric Date: Sun, 1 Dec 2024 23:02:36 +0800 Subject: [PATCH 061/438] [SPARK-50032][SQL] Allow use of fully qualified collation name ### What changes were proposed in this pull request? In this PR collations can now be identified by their fully qualified name, as per the collation project plan. The `Collation` expression has been changed to always return fully qualified name. Currently we only support predefined collations. ### Why are the changes needed? Make collation names behave as per the project spec. ### Does this PR introduce _any_ user-facing change? Yes. Two user-facing changes are made: 1. Collation expression now returns fully qualified name: ```sql select collation('a' collate utf8_lcase) -- returns `SYSTEM.BUILTIN.UTF8_LCASE` ``` 2. Collations can now be identified by their full qualified name: ```sql select contains('a' collate system.builtin.utf8_lcase, 'A') -- returns true ``` ### How was this patch tested? New tests in this PR. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48546 from stevomitric/stevomitric/fully-qualified-name. Lead-authored-by: Stevo Mitric Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/CollationFactory.java | 58 +++++++++++--- python/pyspark/sql/functions/builtin.py | 12 +-- python/pyspark/sql/tests/test_functions.py | 2 +- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../catalyst/parser/DataTypeAstBuilder.scala | 17 +++- .../sql/catalyst/analysis/Analyzer.scala | 1 + .../analysis/ResolveCollationName.scala | 36 +++++++++ .../expressions/collationExpressions.scala | 80 ++++++++++++++----- .../sql/catalyst/parser/AstBuilder.scala | 19 ++--- .../sql/catalyst/rules/RuleIdCollection.scala | 1 + .../sql/catalyst/trees/TreePatterns.scala | 1 + .../CollationExpressionSuite.scala | 38 ++++----- .../expressions/HashExpressionsSuite.scala | 4 +- .../function_collation.explain | 2 +- .../sql/CollationSQLExpressionsSuite.scala | 8 +- .../sql/CollationStringExpressionsSuite.scala | 6 +- .../org/apache/spark/sql/CollationSuite.scala | 77 ++++++++++++++---- .../CollationTypePrecedenceSuite.scala | 80 ++++++++++--------- .../collation/DefaultCollationTestSuite.scala | 52 +++++++----- .../sql/execution/command/DDLSuite.scala | 16 ++-- 20 files changed, 347 insertions(+), 165 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCollationName.scala diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java index 4064f830e92d8..ce416293131a1 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java @@ -415,18 +415,6 @@ private static Collation fetchCollation(int collationId) { } } - /** - * Method for constructing errors thrown on providing invalid collation name. - */ - protected static SparkException collationInvalidNameException(String collationName) { - Map params = new HashMap<>(); - final int maxSuggestions = 3; - params.put("collationName", collationName); - params.put("proposals", getClosestSuggestionsOnInvalidName(collationName, maxSuggestions)); - return new SparkException("COLLATION_INVALID_NAME", - SparkException.constructMessageParams(params), null); - } - private static int collationNameToId(String collationName) throws SparkException { // Collation names provided by user are treated as case-insensitive. String collationNameUpper = collationName.toUpperCase(); @@ -1185,6 +1173,52 @@ public static int collationNameToId(String collationName) throws SparkException return Collation.CollationSpec.collationNameToId(collationName); } + /** + * Returns the resolved fully qualified collation name. + */ + public static String resolveFullyQualifiedName(String[] collationName) throws SparkException { + // If collation name has only one part, then we don't need to do any name resolution. + if (collationName.length == 1) return collationName[0]; + else { + // Currently we only support builtin collation names with fixed catalog `SYSTEM` and + // schema `BUILTIN`. + if (collationName.length != 3 || + !CollationFactory.CATALOG.equalsIgnoreCase(collationName[0]) || + !CollationFactory.SCHEMA.equalsIgnoreCase(collationName[1])) { + // Throw exception with original (before case conversion) collation name. + throw CollationFactory.collationInvalidNameException( + collationName.length != 0 ? collationName[collationName.length - 1] : ""); + } + return collationName[2]; + } + } + + /** + * Method for constructing errors thrown on providing invalid collation name. + */ + public static SparkException collationInvalidNameException(String collationName) { + Map params = new HashMap<>(); + final int maxSuggestions = 3; + params.put("collationName", collationName); + params.put("proposals", getClosestSuggestionsOnInvalidName(collationName, maxSuggestions)); + return new SparkException("COLLATION_INVALID_NAME", + SparkException.constructMessageParams(params), null); + } + + + + /** + * Returns the fully qualified collation name for the given collation ID. + */ + public static String fullyQualifiedName(int collationId) { + Collation.CollationSpec.DefinitionOrigin definitionOrigin = + Collation.CollationSpec.getDefinitionOrigin(collationId); + // Currently only predefined collations are supported. + assert definitionOrigin == Collation.CollationSpec.DefinitionOrigin.PREDEFINED; + return String.format("%s.%s.%s", CATALOG, SCHEMA, + Collation.CollationSpec.fetchCollation(collationId).collationName); + } + public static boolean isCaseInsensitive(int collationId) { return Collation.CollationSpecICU.fromCollationId(collationId).caseSensitivity == Collation.CollationSpecICU.CaseSensitivity.CI; diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 11011c2f30252..21200ceb6f33e 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -16756,12 +16756,12 @@ def collation(col: "ColumnOrName") -> Column: Examples -------- >>> df = spark.createDataFrame([('name',)], ['dt']) - >>> df.select(collation('dt').alias('collation')).show() - +-----------+ - | collation| - +-----------+ - |UTF8_BINARY| - +-----------+ + >>> df.select(collation('dt').alias('collation')).show(truncate=False) + +--------------------------+ + |collation | + +--------------------------+ + |SYSTEM.BUILTIN.UTF8_BINARY| + +--------------------------+ """ return _invoke_function_over_columns("collation", col) diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 6c7ce80072923..e192366676ad8 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -456,7 +456,7 @@ def test_string_functions(self): def test_collation(self): df = self.spark.createDataFrame([("a",), ("b",)], ["name"]) actual = df.select(F.collation(F.collate("name", "UNICODE"))).distinct().collect() - self.assertEqual([Row("UNICODE")], actual) + self.assertEqual([Row("SYSTEM.BUILTIN.UNICODE")], actual) def test_try_make_interval(self): df = self.spark.createDataFrame([(2147483647,)], ["num"]) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 4b7b4634b74b2..93cf9974e654c 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -1233,7 +1233,7 @@ colPosition ; collateClause - : COLLATE collationName=identifier + : COLLATE collationName=multipartIdentifier ; type diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala index c2cb4a7154076..94e014fb77f1b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala @@ -57,6 +57,14 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { } } + /** + * Create a multi-part identifier. + */ + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = + withOrigin(ctx) { + ctx.parts.asScala.map(_.getText).toSeq + } + /** * Resolve/create a primitive type. */ @@ -78,8 +86,9 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { typeCtx.children.asScala.toSeq match { case Seq(_) => StringType case Seq(_, ctx: CollateClauseContext) => - val collationName = visitCollateClause(ctx) - val collationId = CollationFactory.collationNameToId(collationName) + val collationNameParts = visitCollateClause(ctx).toArray + val collationId = CollationFactory.collationNameToId( + CollationFactory.resolveFullyQualifiedName(collationNameParts)) StringType(collationId) } case (CHARACTER | CHAR, length :: Nil) => CharType(length.getText.toInt) @@ -219,8 +228,8 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { /** * Returns a collation name. */ - override def visitCollateClause(ctx: CollateClauseContext): String = withOrigin(ctx) { - ctx.identifier.getText + override def visitCollateClause(ctx: CollateClauseContext): Seq[String] = withOrigin(ctx) { + visitMultipartIdentifier(ctx.collationName) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 089e18e3df4e1..1c31c45d27296 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -299,6 +299,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolveFieldNameAndPosition :: AddMetadataColumns :: DeduplicateRelations :: + ResolveCollationName :: new ResolveReferences(catalogManager) :: // Please do not insert any other rules in between. See the TODO comments in rule // ResolveLateralColumnAliasReference for more details. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCollationName.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCollationName.scala new file mode 100644 index 0000000000000..50f36f78a4724 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCollationName.scala @@ -0,0 +1,36 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_COLLATION +import org.apache.spark.sql.catalyst.util.CollationFactory + +/** + * Resolves fully qualified collation name and replaces [[UnresolvedCollation]] with + * [[ResolvedCollation]]. + */ +object ResolveCollationName extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = + plan.resolveExpressionsWithPruning(_.containsPattern(UNRESOLVED_COLLATION), ruleId) { + case UnresolvedCollation(collationName) => + ResolvedCollation(CollationFactory.resolveFullyQualifiedName(collationName.toArray)) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala index c75bf30ad21f7..024bef08b5273 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.ExpressionBuilder +import org.apache.spark.sql.catalyst.analysis.{ExpressionBuilder, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.CollationFactory +import org.apache.spark.sql.catalyst.trees.TreePattern.{TreePattern, UNRESOLVED_COLLATION} +import org.apache.spark.sql.catalyst.util.{AttributeNameParser, CollationFactory} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation @@ -37,7 +39,7 @@ import org.apache.spark.sql.types._ examples = """ Examples: > SELECT COLLATION('Spark SQL' _FUNC_ UTF8_LCASE); - UTF8_LCASE + SYSTEM.BUILTIN.UTF8_LCASE """, since = "4.0.0", group = "string_funcs") @@ -56,7 +58,8 @@ object CollateExpressionBuilder extends ExpressionBuilder { evalCollation.toString.toUpperCase().contains("TRIM")) { throw QueryCompilationErrors.trimCollationNotEnabledError() } - Collate(e, evalCollation.toString) + Collate(e, UnresolvedCollation( + AttributeNameParser.parseAttributeName(evalCollation.toString))) } case (_: StringType, false) => throw QueryCompilationErrors.nonFoldableArgumentError( funcName, "collationName", StringType) @@ -73,24 +76,63 @@ object CollateExpressionBuilder extends ExpressionBuilder { * This function is pass-through, it will not modify the input data. * Only type metadata will be updated. */ -case class Collate(child: Expression, collationName: String) - extends UnaryExpression with ExpectsInputTypes { - private val collationId = CollationFactory.collationNameToId(collationName) - override def dataType: DataType = StringType(collationId) +case class Collate(child: Expression, collation: Expression) + extends BinaryExpression with ExpectsInputTypes { + override def left: Expression = child + override def right: Expression = collation + override def dataType: DataType = collation.dataType override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation(supportsTrimCollation = true)) - - override protected def withNewChildInternal( - newChild: Expression): Expression = copy(newChild) + Seq(StringTypeWithCollation(supportsTrimCollation = true), AnyDataType) override def eval(row: InternalRow): Any = child.eval(row) - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - defineCodeGen(ctx, ev, (in) => in) + /** Just a simple passthrough for code generation. */ + override def genCode(ctx: CodegenContext): ExprCode = child.genCode(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + throw SparkException.internalError("Collate.doGenCode should not be called.") + } + + override def sql: String = s"$prettyName(${child.sql}, $collation)" + + override def toString: String = + s"$prettyName($child, $collation)" + + override protected def withNewChildrenInternal( + newLeft: Expression, newRight: Expression): Expression = + copy(child = newLeft, collation = newRight) + + override def foldable: Boolean = child.foldable +} + +/** + * An expression that marks an unresolved collation name. + * + * This class is used to represent a collation name that has not yet been resolved from a fully + * qualified collation name. It is used during the analysis phase, where the collation name is + * specified but not yet validated or resolved. + */ +case class UnresolvedCollation(collationName: Seq[String]) + extends LeafExpression with Unevaluable { + override def dataType: DataType = throw new UnresolvedException("dataType") + + override def nullable: Boolean = false + + override lazy val resolved: Boolean = false + + final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_COLLATION) +} + +/** + * An expression that represents a resolved collation name. + */ +case class ResolvedCollation(collationName: String) extends LeafExpression with Unevaluable { + override def nullable: Boolean = false + + override def dataType: DataType = StringType(CollationFactory.collationNameToId(collationName)) - override def sql: String = s"$prettyName(${child.sql}, $collationName)" + override def toString: String = collationName - override def toString: String = s"$prettyName($child, $collationName)" + override def sql: String = collationName } // scalastyle:off line.contains.tab @@ -103,7 +145,7 @@ case class Collate(child: Expression, collationName: String) examples = """ Examples: > SELECT _FUNC_('Spark SQL'); - UTF8_BINARY + SYSTEM.BUILTIN.UTF8_BINARY """, since = "4.0.0", group = "string_funcs") @@ -113,8 +155,8 @@ case class Collation(child: Expression) override protected def withNewChildInternal(newChild: Expression): Collation = copy(newChild) override lazy val replacement: Expression = { val collationId = child.dataType.asInstanceOf[StringType].collationId - val collationName = CollationFactory.fetchCollation(collationId).collationName - Literal.create(collationName, SQLConf.get.defaultStringType) + val fullyQualifiedCollationName = CollationFactory.fullyQualifiedName(collationId) + Literal.create(fullyQualifiedCollationName, SQLConf.get.defaultStringType) } override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCollation(supportsTrimCollation = true)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3d74e9d314d57..a3fac7296dcc4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2286,14 +2286,6 @@ class AstBuilder extends DataTypeAstBuilder FunctionIdentifier(ctx.function.getText, Option(ctx.db).map(_.getText)) } - /** - * Create a multi-part identifier. - */ - override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = - withOrigin(ctx) { - ctx.parts.asScala.map(_.getText).toSeq - } - /* ******************************************************************************************** * Expression parsing * ******************************************************************************************** */ @@ -2706,15 +2698,16 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitCollate(ctx: CollateContext): Expression = withOrigin(ctx) { val collationName = visitCollateClause(ctx.collateClause()) - Collate(expression(ctx.primaryExpression), collationName) + + Collate(expression(ctx.primaryExpression), UnresolvedCollation(collationName)) } - override def visitCollateClause(ctx: CollateClauseContext): String = withOrigin(ctx) { - val collationName = ctx.collationName.getText - if (!SQLConf.get.trimCollationEnabled && collationName.toUpperCase().contains("TRIM")) { + override def visitCollateClause(ctx: CollateClauseContext): Seq[String] = withOrigin(ctx) { + val collationName = visitMultipartIdentifier(ctx.collationName) + if (!SQLConf.get.trimCollationEnabled && collationName.last.toUpperCase().contains("TRIM")) { throw QueryCompilationErrors.trimCollationNotEnabledError() } - ctx.identifier.getText + collationName } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index 5ae2ca0d532b7..0918306de62e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -51,6 +51,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveBinaryArithmetic" :: + "org.apache.spark.sql.catalyst.analysis.ResolveCollationName" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveDeserializer" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions" :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index e95712281cb42..95b5832392ec3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -146,6 +146,7 @@ object TreePattern extends Enumeration { // Unresolved expression patterns (Alphabetically ordered) val UNRESOLVED_ALIAS: Value = Value val UNRESOLVED_ATTRIBUTE: Value = Value + val UNRESOLVED_COLLATION: Value = Value val UNRESOLVED_DESERIALIZER: Value = Value val UNRESOLVED_DF_STAR: Value = Value val UNRESOLVED_HAVING: Value = Value diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala index 77a3d6df69221..9e9eeaf2f80d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala @@ -17,67 +17,63 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, CollationFactory, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + private val fullyQualifiedPrefix = s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}." + private val UTF8_BINARY_COLLATION_NAME = ResolvedCollation("UTF8_BINARY") + private val UTF8_LCASE_COLLATION_NAME = ResolvedCollation("UTF8_LCASE") + test("validate default collation") { val collationId = CollationFactory.collationNameToId("UTF8_BINARY") assert(collationId == 0) - val collateExpr = Collate(Literal("abc"), "UTF8_BINARY") + val collateExpr = Collate(Literal("abc"), UTF8_BINARY_COLLATION_NAME) assert(collateExpr.dataType === StringType(collationId)) assert(collateExpr.dataType.asInstanceOf[StringType].collationId == 0) checkEvaluation(collateExpr, "abc") } test("collate against literal") { - val collateExpr = Collate(Literal("abc"), "UTF8_LCASE") + val collateExpr = Collate(Literal("abc"), UTF8_LCASE_COLLATION_NAME) val collationId = CollationFactory.collationNameToId("UTF8_LCASE") assert(collateExpr.dataType === StringType(collationId)) checkEvaluation(collateExpr, "abc") } test("check input types") { - val collateExpr = Collate(Literal("abc"), "UTF8_BINARY") + val collateExpr = Collate(Literal("abc"), UTF8_BINARY_COLLATION_NAME) assert(collateExpr.checkInputDataTypes().isSuccess) val collateExprExplicitDefault = - Collate(Literal.create("abc", StringType(0)), "UTF8_BINARY") + Collate(Literal.create("abc", StringType(0)), UTF8_BINARY_COLLATION_NAME) assert(collateExprExplicitDefault.checkInputDataTypes().isSuccess) val collateExprExplicitNonDefault = - Collate(Literal.create("abc", StringType(1)), "UTF8_BINARY") + Collate(Literal.create("abc", StringType(1)), UTF8_BINARY_COLLATION_NAME) assert(collateExprExplicitNonDefault.checkInputDataTypes().isSuccess) - val collateOnNull = Collate(Literal.create(null, StringType(1)), "UTF8_BINARY") + val collateOnNull = Collate(Literal.create(null, StringType(1)), UTF8_BINARY_COLLATION_NAME) assert(collateOnNull.checkInputDataTypes().isSuccess) - val collateOnInt = Collate(Literal(1), "UTF8_BINARY") + val collateOnInt = Collate(Literal(1), UTF8_BINARY_COLLATION_NAME) assert(collateOnInt.checkInputDataTypes().isFailure) } - test("collate on non existing collation") { - checkError( - exception = intercept[SparkException] { Collate(Literal("abc"), "UTF8_BS") }, - condition = "COLLATION_INVALID_NAME", - sqlState = "42704", - parameters = Map("collationName" -> "UTF8_BS", "proposals" -> "UTF8_LCASE")) - } - test("collation on non-explicit default collation") { - checkEvaluation(Collation(Literal("abc")), "UTF8_BINARY") + checkEvaluation(Collation(Literal("abc")), fullyQualifiedPrefix + "UTF8_BINARY") } test("collation on explicitly collated string") { checkEvaluation( Collation(Literal.create("abc", StringType(CollationFactory.UTF8_LCASE_COLLATION_ID))), - "UTF8_LCASE") + fullyQualifiedPrefix + "UTF8_LCASE") checkEvaluation( - Collation(Collate(Literal("abc"), "UTF8_LCASE")), - "UTF8_LCASE") + Collation(Collate(Literal("abc"), UTF8_LCASE_COLLATION_NAME)), + fullyQualifiedPrefix + "UTF8_LCASE") } test("Array operations on arrays of collated strings") { @@ -222,7 +218,7 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { ).foreach { case (collation, normalized) => checkEvaluation(Collation(Literal.create("abc", StringType(collation))), - normalized) + fullyQualifiedPrefix + normalized) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 92ef24bb8ec63..019c953a3b0ac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -625,8 +625,8 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val s1 = "aaa" val s2 = "AAA" - val murmur3Hash1 = Murmur3Hash(Seq(Collate(Literal(s1), collation)), 42) - val murmur3Hash2 = Murmur3Hash(Seq(Collate(Literal(s2), collation)), 42) + val murmur3Hash1 = Murmur3Hash(Seq(Collate(Literal(s1), ResolvedCollation(collation))), 42) + val murmur3Hash2 = Murmur3Hash(Seq(Collate(Literal(s2), ResolvedCollation(collation))), 42) // Interpreted hash values for s1 and s2 val interpretedHash1 = murmur3Hash1.eval() diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_collation.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_collation.explain index a6a251505652a..a6bf9ae2e71c9 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_collation.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_collation.explain @@ -1,2 +1,2 @@ -Project [UTF8_BINARY AS collation(g)#0] +Project [SYSTEM.BUILTIN.UTF8_BINARY AS collation(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala index cf494fcd87451..b2bb3eaffd415 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.{SparkConf, SparkException, SparkIllegalArgumentExceptio import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Mode +import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.internal.{SqlApiConf, SQLConf} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -40,6 +41,7 @@ class CollationSQLExpressionsSuite private val testSuppCollations = Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI") private val testAdditionalCollations = Seq("UNICODE", "SR", "SR_CI", "SR_AI", "SR_CI_AI") + private val fullyQualifiedPrefix = s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}." test("Support Md5 hash expression with collation") { case class Md5TestCase( @@ -2027,11 +2029,11 @@ class CollationSQLExpressionsSuite val queryExtractor = s"select collation(map($mapKey, $mapVal)[$mapKey])" val queryElementAt = s"select collation(element_at(map($mapKey, $mapVal), $mapKey))" - checkAnswer(sql(queryExtractor), Row(collation)) - checkAnswer(sql(queryElementAt), Row(collation)) + checkAnswer(sql(queryExtractor), Row(fullyQualifiedPrefix + collation)) + checkAnswer(sql(queryElementAt), Row(fullyQualifiedPrefix + collation)) withSQLConf(SqlApiConf.DEFAULT_COLLATION -> defaultCollation) { - val res = if (collateVal) "UTF8_LCASE" else defaultCollation + val res = fullyQualifiedPrefix + (if (collateVal) "UTF8_LCASE" else defaultCollation) checkAnswer(sql(queryExtractor), Row(res)) checkAnswer(sql(queryElementAt), Row(res)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala index 626bd0b239366..ee9734ebaa5bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala @@ -198,8 +198,10 @@ class CollationStringExpressionsSuite checkError( exception = intercept[AnalysisException] { val expr = StringSplitSQL( - Collate(Literal.create("1a2", StringType("UTF8_BINARY")), "UTF8_BINARY"), - Collate(Literal.create("a", StringType("UTF8_BINARY")), "UTF8_LCASE")) + Collate(Literal.create("1a2", StringType("UTF8_BINARY")), + ResolvedCollation("UTF8_BINARY")), + Collate(Literal.create("a", StringType("UTF8_BINARY")), + ResolvedCollation("UTF8_LCASE"))) CollationTypeCasts.transform(expr) }, condition = "COLLATION_MISMATCH.EXPLICIT", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index f0f81e713457b..dc7d14b21becb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -43,6 +43,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { private val collationPreservingSources = Seq("parquet") private val collationNonPreservingSources = Seq("orc", "csv", "json", "text") private val allFileBasedDataSources = collationPreservingSources ++ collationNonPreservingSources + private val fullyQualifiedPrefix = s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}." @inline private def isSortMergeForced: Boolean = { @@ -117,7 +118,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ).foreach { collationName => checkAnswer( sql(s"select collation('aaa' collate $collationName)"), - Row(collationName.toUpperCase()) + Row(fullyQualifiedPrefix + collationName.toUpperCase()) ) } } @@ -209,7 +210,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } test("collation expression returns default collation") { - checkAnswer(sql(s"select collation('aaa')"), Row("UTF8_BINARY")) + checkAnswer(sql(s"select collation('aaa')"), Row(fullyQualifiedPrefix + "UTF8_BINARY")) } test("invalid collation name throws exception") { @@ -477,7 +478,8 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"INSERT INTO $tableName VALUES ('aaa')") sql(s"INSERT INTO $tableName VALUES ('AAA')") - checkAnswer(sql(s"SELECT DISTINCT COLLATION(c1) FROM $tableName"), Seq(Row(collationName))) + checkAnswer(sql(s"SELECT DISTINCT COLLATION(c1) FROM $tableName"), + Seq(Row(fullyQualifiedPrefix + collationName))) assert(sql(s"select c1 FROM $tableName").schema.head.dataType == StringType(collationId)) } } @@ -501,7 +503,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkAnswer(readback, Row("aaa")) checkAnswer( readback.selectExpr(s"collation(${readback.columns.head})"), - Row(readbackCollation)) + Row(fullyQualifiedPrefix + readbackCollation)) } } } @@ -523,7 +525,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"INSERT INTO $tableName VALUES ('AAA')") checkAnswer(sql(s"SELECT DISTINCT COLLATION(c1) FROM $tableName"), - Seq(Row(defaultCollation))) + Seq(Row(fullyQualifiedPrefix + defaultCollation))) sql( s""" @@ -535,7 +537,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"INSERT INTO $tableName VALUES ('AAA', 'AAA')") checkAnswer(sql(s"SELECT DISTINCT COLLATION(c2) FROM $tableName"), - Seq(Row(collationName))) + Seq(Row(fullyQualifiedPrefix + collationName))) assert(sql(s"select c2 FROM $tableName").schema.head.dataType == StringType(collationId)) } } @@ -558,7 +560,8 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"ALTER TABLE $tableName ALTER COLUMN c4.t TYPE STRING COLLATE UNICODE") checkAnswer(sql(s"SELECT collation(c1), collation(c2[0]), " + s"collation(c3[1]), collation(c4.t) FROM $tableName"), - Seq(Row("UTF8_LCASE", "UNICODE_CI", "UTF8_BINARY", "UNICODE"))) + Seq(Row(fullyQualifiedPrefix + "UTF8_LCASE", fullyQualifiedPrefix + "UNICODE_CI", + fullyQualifiedPrefix + "UTF8_BINARY", fullyQualifiedPrefix + "UNICODE"))) } } @@ -829,7 +832,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { """EXECUTE IMMEDIATE stmtStr1 USING | 'a' AS var1, | 'b' AS var2;""".stripMargin), - Seq(Row("UTF8_BINARY")) + Seq(Row(fullyQualifiedPrefix + "UTF8_BINARY")) ) withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { @@ -838,7 +841,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { """EXECUTE IMMEDIATE stmtStr1 USING | 'a' AS var1, | 'b' AS var2;""".stripMargin), - Seq(Row("UNICODE")) + Seq(Row(fullyQualifiedPrefix + "UNICODE")) ) } @@ -846,7 +849,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql( """EXECUTE IMMEDIATE stmtStr2 USING | 'a' AS var1;""".stripMargin), - Seq(Row("UNICODE")) + Seq(Row(fullyQualifiedPrefix + "UNICODE")) ) withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { @@ -854,7 +857,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql( """EXECUTE IMMEDIATE stmtStr2 USING | 'a' AS var1;""".stripMargin), - Seq(Row("UNICODE")) + Seq(Row(fullyQualifiedPrefix + "UNICODE")) ) } } @@ -941,7 +944,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"INSERT INTO $tableName VALUES ('a'), ('A')") checkAnswer(sql(s"SELECT DISTINCT COLLATION(c1) FROM $tableName"), - Seq(Row(collationName))) + Seq(Row(fullyQualifiedPrefix + collationName))) assert(sql(s"select c1 FROM $tableName").schema.head.dataType == StringType(collationId)) } } @@ -1053,7 +1056,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("SPARK-47431: Default collation set to UNICODE, literal test") { withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkAnswer(sql(s"SELECT collation('aa')"), Seq(Row("UNICODE"))) + checkAnswer(sql(s"SELECT collation('aa')"), Seq(Row(fullyQualifiedPrefix + "UNICODE"))) } } @@ -1092,7 +1095,8 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) checkAnswer(sql(s"SELECT cast(1 as string)"), Seq(Row("1"))) - checkAnswer(sql(s"SELECT collation(cast(1 as string))"), Seq(Row("UNICODE"))) + checkAnswer(sql(s"SELECT collation(cast(1 as string))"), + Seq(Row(fullyQualifiedPrefix + "UNICODE"))) } } @@ -2064,4 +2068,49 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkAnswer(sql("SELECT NAME FROM collations() WHERE ICU_VERSION is null"), Seq(Row("UTF8_BINARY"), Row("UTF8_LCASE"))) } + + test("fully qualified name") { + Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI_AI").foreach { collation => + // Make sure that the collation expression returns the correct fully qualified name. + val df = sql(s"SELECT collation('a' collate $collation)") + checkAnswer(df, + Seq(Row(s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}.$collation"))) + + // Make sure the user can specify the fully qualified name as a collation name. + Seq("contains", "startswith", "endswith").foreach{ binaryFunction => + val dfRegularName = sql( + s"SELECT $binaryFunction('a' collate $collation, 'A' collate $collation)") + val dfFullyQualifiedName = sql( + s"SELECT $binaryFunction('a' collate system.builtin.$collation, 'A' collate $collation)") + checkAnswer(dfRegularName, dfFullyQualifiedName) + } + } + + // Wrong collation names raise a Spark exception. + Seq( + ("system.builtin2.UTF8_BINARY", "UTF8_BINARY"), + ("system.UTF8_BINARY", "UTF8_BINARY"), + ("builtin.UTF8_LCASE", "UTF8_LCASE") + ).foreach { case(collationName, proposal) => + checkError( + exception = intercept[SparkException] { + sql(s"SELECT 'a' COLLATE ${collationName}") + }, + condition = "COLLATION_INVALID_NAME", + sqlState = "42704", + parameters = Map("collationName" -> collationName.split("\\.").last, + "proposals" -> proposal)) + } + + // Case insensitive fully qualified names are supported. + checkAnswer( + sql("SELECT 'a' collate sYstEm.bUiltIn.utf8_lCAse = 'A'"), + Seq(Row(true)) + ) + + // Make sure DDLs can use fully qualified names. + withTable("t") { + sql(s"CREATE TABLE t (c STRING COLLATE system.builtin.UTF8_LCASE)") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index 4a904a85e0a7b..93e36afae242b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -24,6 +24,10 @@ import org.apache.spark.sql.test.SharedSparkSession class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { val dataSource: String = "parquet" + val UNICODE_COLLATION_NAME = "SYSTEM.BUILTIN.UNICODE" + val UNICODE_CI_COLLATION_NAME = "SYSTEM.BUILTIN.UNICODE_CI" + val UTF8_BINARY_COLLATION_NAME = "SYSTEM.BUILTIN.UTF8_BINARY" + val UTF8_LCASE_COLLATION_NAME = "SYSTEM.BUILTIN.UTF8_LCASE" private def assertThrowsError(df: => DataFrame, errorClass: String): Unit = { val exception = intercept[SparkThrowable] { @@ -41,19 +45,19 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { test("explicit collation propagates up") { checkAnswer( sql(s"SELECT COLLATION('a' collate unicode)"), - Row("UNICODE")) + Row(UNICODE_COLLATION_NAME)) checkAnswer( sql(s"SELECT COLLATION('a' collate unicode || 'b')"), - Row("UNICODE")) + Row(UNICODE_COLLATION_NAME)) checkAnswer( sql(s"SELECT COLLATION(SUBSTRING('a' collate unicode, 0, 1))"), - Row("UNICODE")) + Row(UNICODE_COLLATION_NAME)) checkAnswer( sql(s"SELECT COLLATION(SUBSTRING('a' collate unicode, 0, 1) || 'b')"), - Row("UNICODE")) + Row(UNICODE_COLLATION_NAME)) assertExplicitMismatch( sql(s"SELECT COLLATION('a' collate unicode || 'b' collate utf8_lcase)")) @@ -68,9 +72,9 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { test("implicit collation in columns") { val tableName = "implicit_coll_tbl" - val c1Collation = "UNICODE" - val c2Collation = "UNICODE_CI" - val structCollation = "UTF8_LCASE" + val c1Collation = UNICODE_COLLATION_NAME + val c2Collation = UNICODE_CI_COLLATION_NAME + val structCollation = UTF8_LCASE_COLLATION_NAME withTable(tableName) { sql(s""" |CREATE TABLE $tableName ( @@ -100,8 +104,8 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { } test("variables have implicit collation") { - val v1Collation = "UTF8_BINARY" - val v2Collation = "UTF8_LCASE" + val v1Collation = UTF8_BINARY_COLLATION_NAME + val v2Collation = UTF8_LCASE_COLLATION_NAME sql(s"DECLARE v1 = 'a'") sql(s"DECLARE v2 = 'b' collate $v2Collation") @@ -115,7 +119,7 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT COLLATION(v2 || 'a' COLLATE UTF8_BINARY)"), - Row("UTF8_BINARY")) + Row(UTF8_BINARY_COLLATION_NAME)) checkAnswer( sql(s"SELECT COLLATION(SUBSTRING(v2, 0, 1) || 'a')"), @@ -137,30 +141,30 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { // Simple subquery with explicit collation checkAnswer( sql(s"SELECT COLLATION((SELECT 'text' COLLATE UTF8_BINARY) || 'suffix')"), - Row("UTF8_BINARY") + Row(UTF8_BINARY_COLLATION_NAME) ) checkAnswer( sql(s"SELECT COLLATION((SELECT 'text' COLLATE UTF8_LCASE) || 'suffix')"), - Row("UTF8_LCASE") + Row(UTF8_LCASE_COLLATION_NAME) ) // Nested subquery should retain the collation of the deepest expression checkAnswer( sql(s"SELECT COLLATION((SELECT (SELECT 'inner' COLLATE UTF8_LCASE) || 'outer'))"), - Row("UTF8_LCASE") + Row(UTF8_LCASE_COLLATION_NAME) ) checkAnswer( sql(s"SELECT COLLATION((SELECT (SELECT 'inner' COLLATE UTF8_BINARY) || 'outer'))"), - Row("UTF8_BINARY") + Row(UTF8_BINARY_COLLATION_NAME) ) // Subqueries with mixed collations should follow collation precedence rules checkAnswer( sql(s"SELECT COLLATION((SELECT 'string1' COLLATE UTF8_LCASE || " + s"(SELECT 'string2' COLLATE UTF8_BINARY)))"), - Row("UTF8_LCASE") + Row(UTF8_LCASE_COLLATION_NAME) ) } @@ -180,28 +184,28 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT COLLATION(c2.col1.col1 || 'a') FROM $tableName"), - Seq(Row(c2Collation))) + Seq(Row(UNICODE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(c1.col1 || 'a') FROM $tableName"), - Seq(Row(c1Collation))) + Seq(Row(UNICODE_CI_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(c1.col1 || 'a' collate UNICODE) FROM $tableName"), - Seq(Row("UNICODE"))) + Seq(Row(UNICODE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(struct('a').col1 || 'a' collate UNICODE) FROM $tableName"), - Seq(Row("UNICODE"))) + Seq(Row(UNICODE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(struct('a' collate UNICODE).col1 || 'a') FROM $tableName"), - Seq(Row("UNICODE"))) + Seq(Row(UNICODE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(struct('a').col1 collate UNICODE || 'a' collate UNICODE) " + s"FROM $tableName"), - Seq(Row("UNICODE"))) + Seq(Row(UNICODE_COLLATION_NAME))) assertExplicitMismatch( sql(s"SELECT COLLATION(struct('a').col1 collate UNICODE || 'a' collate UTF8_LCASE) " + @@ -229,7 +233,7 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT collation(element_at(array('a', 'b' collate utf8_lcase), 1))"), - Seq(Row("UTF8_LCASE"))) + Seq(Row(UTF8_LCASE_COLLATION_NAME))) assertExplicitMismatch( sql(s"SELECT collation(element_at(array('a' collate unicode, 'b' collate utf8_lcase), 1))") @@ -238,17 +242,17 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT collation(element_at(array('a', 'b' collate utf8_lcase), 1) || c1)" + s"from $tableName"), - Seq(Row("UTF8_LCASE"))) + Seq(Row(UTF8_LCASE_COLLATION_NAME))) checkAnswer( sql(s"SELECT collation(element_at(array_append(c2, 'd'), 1)) FROM $tableName"), - Seq(Row(arrayCollation)) + Seq(Row(UNICODE_CI_COLLATION_NAME)) ) checkAnswer( sql(s"SELECT collation(element_at(array_append(c2, 'd' collate utf8_lcase), 1))" + s"FROM $tableName"), - Seq(Row("UTF8_LCASE")) + Seq(Row(UTF8_LCASE_COLLATION_NAME)) ) } } @@ -262,21 +266,21 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT COLLATION(c1[0]) FROM $tableName"), - Seq(Row(columnCollation))) + Seq(Row(UNICODE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(cast(c1 AS ARRAY)[0]) FROM $tableName"), - Seq(Row("UTF8_BINARY"))) + Seq(Row(UTF8_BINARY_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(cast(c1 AS ARRAY)[0]) FROM $tableName"), - Seq(Row("UTF8_LCASE"))) + Seq(Row(UTF8_LCASE_COLLATION_NAME))) } } test("user defined cast") { val tableName = "dflt_coll_tbl" - val columnCollation = "UNICODE" + val columnCollation = UNICODE_COLLATION_NAME withTable(tableName) { sql(s"CREATE TABLE $tableName (c1 STRING COLLATE $columnCollation) USING $dataSource") sql(s"INSERT INTO $tableName VALUES ('a')") @@ -290,12 +294,12 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT COLLATION(CAST(to_char(DATE'2016-04-08', 'y') AS STRING)) " + s"FROM $tableName"), - Seq(Row("UTF8_BINARY"))) + Seq(Row(UTF8_BINARY_COLLATION_NAME))) // for string inputs collation is of the child expression checkAnswer( sql(s"SELECT COLLATION(CAST('a' AS STRING)) FROM $tableName"), - Seq(Row("UTF8_BINARY"))) + Seq(Row(UTF8_BINARY_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(CAST(c1 AS STRING)) FROM $tableName"), @@ -303,7 +307,7 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT COLLATION(CAST(c1 collate UTF8_LCASE AS STRING)) FROM $tableName"), - Seq(Row("UTF8_LCASE"))) + Seq(Row(UTF8_LCASE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(c1 || CAST('a' AS STRING)) FROM $tableName"), @@ -311,7 +315,7 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT COLLATION(c1 || CAST('a' collate UTF8_LCASE AS STRING)) FROM $tableName"), - Seq(Row("UTF8_LCASE"))) + Seq(Row(UTF8_LCASE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(c1 || CAST(c1 AS STRING)) FROM $tableName"), @@ -332,26 +336,26 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { checkAnswer( sql(s"SELECT COLLATION('a' collate utf8_lcase || current_database()) FROM $tableName"), - Seq(Row("UTF8_LCASE"))) + Seq(Row(UTF8_LCASE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION(c1 || current_database()) FROM $tableName"), - Seq(Row(columnCollation))) + Seq(Row(UNICODE_COLLATION_NAME))) checkAnswer( sql(s"SELECT COLLATION('a' || current_database()) FROM $tableName"), - Seq(Row("UTF8_BINARY"))) + Seq(Row(UTF8_BINARY_COLLATION_NAME))) } } test("functions that contain both string and non string params") { checkAnswer( sql(s"SELECT COLLATION(elt(2, 'a', 'b'))"), - Row("UTF8_BINARY")) + Row(UTF8_BINARY_COLLATION_NAME)) checkAnswer( sql(s"SELECT COLLATION(elt(2, 'a' collate UTF8_LCASE, 'b'))"), - Row("UTF8_LCASE")) + Row(UTF8_LCASE_COLLATION_NAME)) assertExplicitMismatch( sql(s"SELECT COLLATION(elt(2, 'a' collate UTF8_LCASE, 'b' collate UNICODE))")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala index 0de638d4e9bf9..69f1c6da65d12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.collation import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.connector.DatasourceV2SQLBase import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.test.SharedSparkSession @@ -28,6 +29,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi def dataSource: String = "parquet" def testTable: String = "test_tbl" def testView: String = "test_view" + protected val fullyQualifiedPrefix = s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}." def withSessionCollationAndTable(collation: String, testTables: String*)(f: => Unit): Unit = { withTable(testTables: _*) { @@ -144,9 +146,12 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi | array('a') AS c3 |""".stripMargin) - checkAnswer(sql(s"SELECT COLLATION(c1.col1) FROM $testTable"), Seq(Row("UTF8_BINARY"))) - checkAnswer(sql(s"SELECT COLLATION(c2['a']) FROM $testTable"), Seq(Row("UTF8_BINARY"))) - checkAnswer(sql(s"SELECT COLLATION(c3[0]) FROM $testTable"), Seq(Row("UTF8_BINARY"))) + checkAnswer(sql(s"SELECT COLLATION(c1.col1) FROM $testTable"), + Seq(Row(fullyQualifiedPrefix + "UTF8_BINARY"))) + checkAnswer(sql(s"SELECT COLLATION(c2['a']) FROM $testTable"), + Seq(Row(fullyQualifiedPrefix + "UTF8_BINARY"))) + checkAnswer(sql(s"SELECT COLLATION(c3[0]) FROM $testTable"), + Seq(Row(fullyQualifiedPrefix + "UTF8_BINARY"))) } } @@ -234,55 +239,62 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi test("literals with default collation") { val sessionCollation = "UTF8_LCASE" - withSessionCollation(sessionCollation) { + val sessionCollationFullyQualified = fullyQualifiedPrefix + sessionCollation + withSessionCollation(sessionCollation) { // literal without collation - checkAnswer(sql("SELECT COLLATION('a')"), Seq(Row(sessionCollation))) + checkAnswer(sql("SELECT COLLATION('a')"), Seq(Row(sessionCollationFullyQualified))) - checkAnswer(sql("SELECT COLLATION(map('a', 'b')['a'])"), Seq(Row(sessionCollation))) + checkAnswer(sql("SELECT COLLATION(map('a', 'b')['a'])"), + Seq(Row(sessionCollationFullyQualified))) - checkAnswer(sql("SELECT COLLATION(array('a')[0])"), Seq(Row(sessionCollation))) + checkAnswer(sql("SELECT COLLATION(array('a')[0])"), Seq(Row(sessionCollationFullyQualified))) - checkAnswer(sql("SELECT COLLATION(struct('a' as c)['c'])"), Seq(Row(sessionCollation))) + checkAnswer(sql("SELECT COLLATION(struct('a' as c)['c'])"), + Seq(Row(sessionCollationFullyQualified))) } } test("literals with explicit collation") { + val unicodeCollation = fullyQualifiedPrefix + "UNICODE" withSessionCollation("UTF8_LCASE") { - checkAnswer(sql("SELECT COLLATION('a' collate unicode)"), Seq(Row("UNICODE"))) + checkAnswer(sql("SELECT COLLATION('a' collate unicode)"), Seq(Row(unicodeCollation))) checkAnswer( sql("SELECT COLLATION(map('a', 'b' collate unicode)['a'])"), - Seq(Row("UNICODE"))) + Seq(Row(unicodeCollation))) - checkAnswer(sql("SELECT COLLATION(array('a' collate unicode)[0])"), Seq(Row("UNICODE"))) + checkAnswer(sql("SELECT COLLATION(array('a' collate unicode)[0])"), + Seq(Row(unicodeCollation))) checkAnswer( sql("SELECT COLLATION(struct('a' collate unicode as c)['c'])"), - Seq(Row("UNICODE"))) + Seq(Row(unicodeCollation))) } } test("cast is aware of session collation") { val sessionCollation = "UTF8_LCASE" + val sessionCollationFullyQualified = fullyQualifiedPrefix + sessionCollation withSessionCollation(sessionCollation) { - checkAnswer(sql("SELECT COLLATION(cast('a' as STRING))"), Seq(Row(sessionCollation))) + checkAnswer(sql("SELECT COLLATION(cast('a' as STRING))"), + Seq(Row(sessionCollationFullyQualified))) checkAnswer( sql("SELECT COLLATION(cast(map('a', 'b') as MAP)['a'])"), - Seq(Row(sessionCollation))) + Seq(Row(sessionCollationFullyQualified))) checkAnswer( sql("SELECT COLLATION(map_keys(cast(map('a', 'b') as MAP))[0])"), - Seq(Row(sessionCollation))) + Seq(Row(sessionCollationFullyQualified))) checkAnswer( sql("SELECT COLLATION(cast(array('a') as ARRAY)[0])"), - Seq(Row(sessionCollation))) + Seq(Row(sessionCollationFullyQualified))) checkAnswer( sql("SELECT COLLATION(cast(struct('a' as c) as STRUCT)['c'])"), - Seq(Row(sessionCollation))) + Seq(Row(sessionCollationFullyQualified))) } } @@ -395,7 +407,7 @@ class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { assertTableColumnCollation(testView, "c2", "UNICODE_CI") checkAnswer( sql(s"SELECT DISTINCT COLLATION(c1), COLLATION('a') FROM $testView"), - Row("UTF8_BINARY", sessionCollation)) + Row(fullyQualifiedPrefix + "UTF8_BINARY", fullyQualifiedPrefix + sessionCollation)) // filter should use session collation checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE 'a' = 'A'"), Row(2)) @@ -420,7 +432,7 @@ class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { assertTableColumnCollation(testView, "c2", "UNICODE_CI") checkAnswer( sql(s"SELECT DISTINCT COLLATION(c1), COLLATION('a') FROM $testView"), - Row("UNICODE_CI", sessionCollation)) + Row(fullyQualifiedPrefix + "UNICODE_CI", fullyQualifiedPrefix + sessionCollation)) // after alter both rows should be returned checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Row(2)) @@ -451,7 +463,7 @@ class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { |FROM $testView JOIN $joinTableName |ON $testView.c1 = $joinTableName.c1 COLLATE UNICODE_CI |""".stripMargin), - Row("UNICODE_CI", "UTF8_LCASE")) + Row(fullyQualifiedPrefix + "UNICODE_CI", fullyQualifiedPrefix + "UTF8_LCASE")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 32a63f5c61976..d91d762048d29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2324,9 +2324,9 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { // Plain `StringType`. sql("CREATE TABLE t1(col STRING) USING parquet") sql("INSERT INTO t1 VALUES ('a')") - checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("UTF8_BINARY")) + checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("SYSTEM.BUILTIN.UTF8_BINARY")) sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_LCASE") - checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("UTF8_LCASE")) + checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("SYSTEM.BUILTIN.UTF8_LCASE")) // Invalid "ALTER COLUMN" to Integer. val alterInt = "ALTER TABLE t1 ALTER COLUMN col TYPE INTEGER" @@ -2348,23 +2348,23 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { // `ArrayType` with collation. sql("CREATE TABLE t2(col ARRAY) USING parquet") sql("INSERT INTO t2 VALUES (ARRAY('a'))") - checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("UTF8_BINARY")) + checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("SYSTEM.BUILTIN.UTF8_BINARY")) assertThrows[AnalysisException] { sql("ALTER TABLE t2 ALTER COLUMN col TYPE ARRAY") } - checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("UTF8_BINARY")) + checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("SYSTEM.BUILTIN.UTF8_BINARY")) // `MapType` with collation. sql("CREATE TABLE t3(col MAP) USING parquet") sql("INSERT INTO t3 VALUES (MAP('k', 'v'))") - checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("UTF8_BINARY")) + checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("SYSTEM.BUILTIN.UTF8_BINARY")) assertThrows[AnalysisException] { sql( """ |ALTER TABLE t3 ALTER COLUMN col TYPE |MAP""".stripMargin) } - checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("UTF8_BINARY")) + checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("SYSTEM.BUILTIN.UTF8_BINARY")) // Invalid change of map key collation. val alterMap = @@ -2388,11 +2388,11 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { // `StructType` with collation. sql("CREATE TABLE t4(col STRUCT) USING parquet") sql("INSERT INTO t4 VALUES (NAMED_STRUCT('a', 'value'))") - checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("UTF8_BINARY")) + checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("SYSTEM.BUILTIN.UTF8_BINARY")) assertThrows[AnalysisException] { sql("ALTER TABLE t4 ALTER COLUMN col TYPE STRUCT") } - checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("UTF8_BINARY")) + checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("SYSTEM.BUILTIN.UTF8_BINARY")) } } From b45e3c01217af503735a735dabfc21c9b1e95126 Mon Sep 17 00:00:00 2001 From: Zhihong Yu Date: Mon, 2 Dec 2024 08:36:54 +0900 Subject: [PATCH 062/438] [MINOR] Use putAll to populate Properties ### What changes were proposed in this pull request? This PR uses the `putAll` method of `Properties` class in place of `put`. ### Why are the changes needed? In Scala 2.13, https://github.com/scala/bug/issues/10418 has been fixed. So we can avoid the workaround. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? There is no change in functionality. Existing tests suffice. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48993 from tedyu/put-all. Authored-by: Zhihong Yu Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/kafka010/KafkaTestUtils.scala | 4 +--- .../sql/hive/execution/HiveScriptTransformationExec.scala | 8 ++------ 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 9e06b6c6ff4a2..60de3705636ec 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -504,9 +504,7 @@ class KafkaTestUtils( props.put("sasl.enabled.mechanisms", "GSSAPI,SCRAM-SHA-512") } - // Can not use properties.putAll(propsMap.asJava) in scala-2.12 - // See https://github.com/scala/bug/issues/10418 - withBrokerProps.foreach { case (k, v) => props.put(k, v) } + props.putAll(withBrokerProps.asJava) props } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala index 0fcc43e5c3919..de2d15415837a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala @@ -283,9 +283,7 @@ object HiveScriptIOSchema extends HiveInspectors { propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames) val properties = new Properties() - // Can not use properties.putAll(propsMap.asJava) in scala-2.12 - // See https://github.com/scala/bug/issues/10418 - propsMap.foreach { case (k, v) => properties.put(k, v) } + properties.putAll(propsMap.asJava) serde.initialize(null, properties) serde @@ -299,9 +297,7 @@ object HiveScriptIOSchema extends HiveInspectors { val instance = Utils.classForName[RecordReader](klass).getConstructor(). newInstance() val props = new Properties() - // Can not use props.putAll(outputSerdeProps.toMap.asJava) in scala-2.12 - // See https://github.com/scala/bug/issues/10418 - ioschema.outputSerdeProps.toMap.foreach { case (k, v) => props.put(k, v) } + props.putAll(ioschema.outputSerdeProps.toMap.asJava) instance.initialize(inputStream, conf, props) instance } From 7d46fdb5cea2ac74a1c51a6e247a65949e947482 Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Mon, 2 Dec 2024 08:37:46 +0900 Subject: [PATCH 063/438] [SPARK-48148][FOLLOWUP] Fix JSON parser feature flag ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/46408 attempts to set the feature flag `INCLUDE_SOURCE_IN_LOCATION` in the JSON parser and reverts the flag to the original value. The reverting code is incorrect and accidentally sets the `AUTO_CLOSE_SOURCE` feature to false. The reason is that `overrideStdFeatures(value, mask)` sets the feature flags selected by `mask` to `value`. `originalMask` is a value of 0/1. When it is 1, it selects `AUTO_CLOSE_SOURCE`, whose ordinal is 0 ([reference](https://github.com/FasterXML/jackson-core/blob/172369cc390ace0f68a5032701634bdc984c2af8/src/main/java/com/fasterxml/jackson/core/JsonParser.java#L112)). The old code doesn't revert `INCLUDE_SOURCE_IN_LOCATION` to the original value either. As a result, when the JSON parser is closed, the underlying input stream is not closed, which can lead to memory leak. ### Why are the changes needed? Perform the originally intended feature, and avoid memory leak. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. It would fail without the change in the PR. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49018 from chenhao-db/fix_json_parser_flag. Authored-by: Chenhao Li Signed-off-by: Hyukjin Kwon --- .../spark/sql/catalyst/json/JacksonParser.scala | 11 +++-------- .../sql/catalyst/json/JacksonParserSuite.scala | 13 +++++++++++++ 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 13129d44fe0c2..19e2c4228236e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -292,13 +292,8 @@ class JacksonParser( case _: StringType => (parser: JsonParser) => { // This must be enabled if we will retrieve the bytes directly from the raw content: - val includeSourceInLocation = JsonParser.Feature.INCLUDE_SOURCE_IN_LOCATION - val originalMask = if (includeSourceInLocation.enabledIn(parser.getFeatureMask)) { - 1 - } else { - 0 - } - parser.overrideStdFeatures(includeSourceInLocation.getMask, includeSourceInLocation.getMask) + val oldFeature = parser.getFeatureMask + parser.setFeatureMask(oldFeature | JsonParser.Feature.INCLUDE_SOURCE_IN_LOCATION.getMask) val result = parseJsonToken[UTF8String](parser, dataType) { case VALUE_STRING => UTF8String.fromString(parser.getText) @@ -344,7 +339,7 @@ class JacksonParser( } } // Reset back to the original configuration: - parser.overrideStdFeatures(includeSourceInLocation.getMask, originalMask) + parser.setFeatureMask(oldFeature) result } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala index 587e22e787b87..89cdd38a3e7b4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonParserSuite.scala @@ -24,6 +24,19 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String class JacksonParserSuite extends SparkFunSuite { + test("feature mask should remain unchanged") { + val options = new JSONOptions(Map.empty[String, String], "GMT", "") + val parser = new JacksonParser(StructType.fromDDL("a string"), options, false, Nil) + val input = """{"a": {"b": 1}}""".getBytes + // The creating function is usually called inside `parser.parse`, but we need the JSON parser + // here for testing purpose. + val jsonParser = options.buildJsonFactory().createParser(input) + val oldFeature = jsonParser.getFeatureMask + val result = parser.parse[Array[Byte]](input, (_, _) => jsonParser, UTF8String.fromBytes) + assert(result === Seq(InternalRow(UTF8String.fromString("""{"b": 1}""")))) + assert(jsonParser.getFeatureMask == oldFeature) + } + test("skipping rows using pushdown filters") { def check( input: String = """{"i":1, "s": "a"}""", From dc73342db941a7a202acacc2a7e90ff245192712 Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Mon, 2 Dec 2024 08:42:08 +0900 Subject: [PATCH 064/438] [SPARK-50436][PYTHON][TESTS] Use assertDataFrameEqual in pyspark.sql.tests.test_udf ### What changes were proposed in this pull request? Use `assertDataFrameEqual` in pyspark.sql.tests.test_udf ### Why are the changes needed? `assertDataFrameEqual` is explicitly built to handle DataFrame-specific comparisons, including schema. So we propose to replace `assertEqual` with `assertDataFrameEqual` Part of https://issues.apache.org/jira/browse/SPARK-50435. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49001 from xinrong-meng/impr_test_udf. Authored-by: Xinrong Meng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/tests/test_udf.py | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 78aa2546128a1..8193913892376 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -220,7 +220,7 @@ def test_udf_in_filter_on_top_of_outer_join(self): right = self.spark.createDataFrame([Row(a=1)]) df = left.join(right, on="a", how="left_outer") df = df.withColumn("b", udf(lambda x: "x")(df.a)) - self.assertEqual(df.filter('b = "x"').collect(), [Row(a=1, b="x")]) + assertDataFrameEqual(df.filter('b = "x"'), [Row(a=1, b="x")]) def test_udf_in_filter_on_top_of_join(self): # regression test for SPARK-18589 @@ -228,7 +228,7 @@ def test_udf_in_filter_on_top_of_join(self): right = self.spark.createDataFrame([Row(b=1)]) f = udf(lambda a, b: a == b, BooleanType()) df = left.crossJoin(right).filter(f("a", "b")) - self.assertEqual(df.collect(), [Row(a=1, b=1)]) + assertDataFrameEqual(df, [Row(a=1, b=1)]) def test_udf_in_join_condition(self): # regression test for SPARK-25314 @@ -243,7 +243,7 @@ def test_udf_in_join_condition(self): df.collect() with self.sql_conf({"spark.sql.crossJoin.enabled": True}): df = left.join(right, f("a", "b")) - self.assertEqual(df.collect(), [Row(a=1, b=1)]) + assertDataFrameEqual(df, [Row(a=1, b=1)]) def test_udf_in_left_outer_join_condition(self): # regression test for SPARK-26147 @@ -256,7 +256,7 @@ def test_udf_in_left_outer_join_condition(self): # The Python UDF only refer to attributes from one side, so it's evaluable. df = left.join(right, f("a") == col("b").cast("string"), how="left_outer") with self.sql_conf({"spark.sql.crossJoin.enabled": True}): - self.assertEqual(df.collect(), [Row(a=1, b=1)]) + assertDataFrameEqual(df, [Row(a=1, b=1)]) def test_udf_and_common_filter_in_join_condition(self): # regression test for SPARK-25314 @@ -266,7 +266,7 @@ def test_udf_and_common_filter_in_join_condition(self): f = udf(lambda a, b: a == b, BooleanType()) df = left.join(right, [f("a", "b"), left.a1 == right.b1]) # do not need spark.sql.crossJoin.enabled=true for udf is not the only join condition. - self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1, b=1, b1=1, b2=1)]) + assertDataFrameEqual(df, [Row(a=1, a1=1, a2=1, b=1, b1=1, b2=1)]) def test_udf_not_supported_in_join_condition(self): # regression test for SPARK-25314 @@ -294,7 +294,7 @@ def test_udf_as_join_condition(self): f = udf(lambda a: a, IntegerType()) df = left.join(right, [f("a") == f("b"), left.a1 == right.b1]) - self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1, b=1, b1=1, b2=1)]) + assertDataFrameEqual(df, [Row(a=1, a1=1, a2=1, b=1, b1=1, b2=1)]) def test_udf_without_arguments(self): self.spark.catalog.registerFunction("foo", lambda: "bar") @@ -331,7 +331,7 @@ def test_udf_with_filter_function(self): my_filter = udf(lambda a: a < 2, BooleanType()) sel = df.select(col("key"), col("value")).filter((my_filter(col("key"))) & (df.value < "2")) - self.assertEqual(sel.collect(), [Row(key=1, value="1")]) + assertDataFrameEqual(sel, [Row(key=1, value="1")]) def test_udf_with_variant_input(self): df = self.spark.range(0, 10).selectExpr("parse_json(cast(id as string)) v") @@ -461,7 +461,7 @@ def test_udf_with_aggregate_function(self): my_filter = udf(lambda a: a == 1, BooleanType()) sel = df.select(col("key")).distinct().filter(my_filter(col("key"))) - self.assertEqual(sel.collect(), [Row(key=1)]) + assertDataFrameEqual(sel, [Row(key=1)]) my_copy = udf(lambda x: x, IntegerType()) my_add = udf(lambda a, b: int(a + b), IntegerType()) @@ -471,7 +471,7 @@ def test_udf_with_aggregate_function(self): .agg(sum(my_strlen(col("value"))).alias("s")) .select(my_add(col("k"), col("s")).alias("t")) ) - self.assertEqual(sel.collect(), [Row(t=4), Row(t=3)]) + assertDataFrameEqual(sel, [Row(t=4), Row(t=3)]) def test_udf_in_generate(self): from pyspark.sql.functions import explode @@ -505,7 +505,7 @@ def test_udf_with_order_by_and_limit(self): my_copy = udf(lambda x: x, IntegerType()) df = self.spark.range(10).orderBy("id") res = df.select(df.id, my_copy(df.id).alias("copy")).limit(1) - self.assertEqual(res.collect(), [Row(id=0, copy=0)]) + assertDataFrameEqual(res, [Row(id=0, copy=0)]) def test_udf_registration_returns_udf(self): df = self.spark.range(10) @@ -838,12 +838,12 @@ def test_datasource_with_udf(self): for df in [filesource_df, datasource_df, datasource_v2_df]: result = df.withColumn("c", c1) expected = df.withColumn("c", lit(2)) - self.assertEqual(expected.collect(), result.collect()) + assertDataFrameEqual(expected, result) for df in [filesource_df, datasource_df, datasource_v2_df]: result = df.withColumn("c", c2) expected = df.withColumn("c", col("i") + 1) - self.assertEqual(expected.collect(), result.collect()) + assertDataFrameEqual(expected, result) for df in [filesource_df, datasource_df, datasource_v2_df]: for f in [f1, f2]: @@ -902,7 +902,7 @@ def test_udf_in_subquery(self): result = self.spark.sql( "select i from values(0L) as data(i) where i in (select id from v)" ) - self.assertEqual(result.collect(), [Row(i=0)]) + assertDataFrameEqual(result, [Row(i=0)]) def test_udf_globals_not_overwritten(self): @udf("string") From e7071c0237da75967b2f1e222d9f3b8293a82f86 Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Mon, 2 Dec 2024 09:59:07 +0800 Subject: [PATCH 065/438] [SPARK-50435][PYTHON][TESTS] Use assertDataFrameEqual in pyspark.sql.tests.test_functions ### What changes were proposed in this pull request? Use `assertDataFrameEqual` in pyspark.sql.tests.test_functions ### Why are the changes needed? `assertDataFrameEqual` is explicitly built to handle DataFrame-specific comparisons, including schema. So we propose to replace `assertEqual` with `assertDataFrameEqual` Part of https://issues.apache.org/jira/browse/SPARK-50435. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49011 from xinrong-meng/impr_test_functions. Lead-authored-by: Xinrong Meng Co-authored-by: Hyukjin Kwon Signed-off-by: Xinrong Meng --- python/pyspark/sql/tests/test_functions.py | 196 ++++++++++----------- 1 file changed, 92 insertions(+), 104 deletions(-) diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index e192366676ad8..4607d5d3411fe 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -31,7 +31,7 @@ from pyspark.sql.column import Column from pyspark.sql.functions.builtin import nullifzero, randstr, uniform, zeroifnull from pyspark.testing.sqlutils import ReusedSQLTestCase, SQLTestUtils -from pyspark.testing.utils import have_numpy +from pyspark.testing.utils import have_numpy, assertDataFrameEqual class FunctionsTestsMixin: @@ -344,29 +344,29 @@ def test_try_parse_url(self): [("https://spark.apache.org/path?query=1", "QUERY", "query")], ["url", "part", "key"], ) - actual = df.select(F.try_parse_url(df.url, df.part, df.key)).collect() - self.assertEqual(actual, [Row("1")]) + actual = df.select(F.try_parse_url(df.url, df.part, df.key)) + assertDataFrameEqual(actual, [Row("1")]) df = self.spark.createDataFrame( [("inva lid://spark.apache.org/path?query=1", "QUERY", "query")], ["url", "part", "key"], ) - actual = df.select(F.try_parse_url(df.url, df.part, df.key)).collect() - self.assertEqual(actual, [Row(None)]) + actual = df.select(F.try_parse_url(df.url, df.part, df.key)) + assertDataFrameEqual(actual, [Row(None)]) def test_try_make_timestamp(self): data = [(2024, 5, 22, 10, 30, 0)] df = self.spark.createDataFrame(data, ["year", "month", "day", "hour", "minute", "second"]) actual = df.select( F.try_make_timestamp(df.year, df.month, df.day, df.hour, df.minute, df.second) - ).collect() - self.assertEqual(actual, [Row(datetime.datetime(2024, 5, 22, 10, 30))]) + ) + assertDataFrameEqual(actual, [Row(datetime.datetime(2024, 5, 22, 10, 30))]) data = [(2024, 13, 22, 10, 30, 0)] df = self.spark.createDataFrame(data, ["year", "month", "day", "hour", "minute", "second"]) actual = df.select( F.try_make_timestamp(df.year, df.month, df.day, df.hour, df.minute, df.second) - ).collect() - self.assertEqual(actual, [Row(None)]) + ) + assertDataFrameEqual(actual, [Row(None)]) def test_try_make_timestamp_ltz(self): # use local timezone here to avoid flakiness @@ -378,8 +378,8 @@ def test_try_make_timestamp_ltz(self): F.try_make_timestamp_ltz( df.year, df.month, df.day, df.hour, df.minute, df.second, df.timezone ) - ).collect() - self.assertEqual(actual, [Row(datetime.datetime(2024, 5, 22, 10, 30, 0))]) + ) + assertDataFrameEqual(actual, [Row(datetime.datetime(2024, 5, 22, 10, 30, 0))]) # use local timezone here to avoid flakiness data = [(2024, 13, 22, 10, 30, 0, datetime.datetime.now().astimezone().tzinfo.__str__())] @@ -390,23 +390,23 @@ def test_try_make_timestamp_ltz(self): F.try_make_timestamp_ltz( df.year, df.month, df.day, df.hour, df.minute, df.second, df.timezone ) - ).collect() - self.assertEqual(actual, [Row(None)]) + ) + assertDataFrameEqual(actual, [Row(None)]) def test_try_make_timestamp_ntz(self): data = [(2024, 5, 22, 10, 30, 0)] df = self.spark.createDataFrame(data, ["year", "month", "day", "hour", "minute", "second"]) actual = df.select( F.try_make_timestamp_ntz(df.year, df.month, df.day, df.hour, df.minute, df.second) - ).collect() - self.assertEqual(actual, [Row(datetime.datetime(2024, 5, 22, 10, 30))]) + ) + assertDataFrameEqual(actual, [Row(datetime.datetime(2024, 5, 22, 10, 30))]) data = [(2024, 13, 22, 10, 30, 0)] df = self.spark.createDataFrame(data, ["year", "month", "day", "hour", "minute", "second"]) actual = df.select( F.try_make_timestamp_ntz(df.year, df.month, df.day, df.hour, df.minute, df.second) - ).collect() - self.assertEqual(actual, [Row(None)]) + ) + assertDataFrameEqual(actual, [Row(None)]) def test_string_functions(self): string_functions = [ @@ -448,51 +448,51 @@ def test_string_functions(self): ) for name in string_functions: - self.assertEqual( - df.select(getattr(F, name)("name")).first()[0], - df.select(getattr(F, name)(F.col("name"))).first()[0], + assertDataFrameEqual( + df.select(getattr(F, name)("name")), + df.select(getattr(F, name)(F.col("name"))), ) def test_collation(self): df = self.spark.createDataFrame([("a",), ("b",)], ["name"]) - actual = df.select(F.collation(F.collate("name", "UNICODE"))).distinct().collect() - self.assertEqual([Row("SYSTEM.BUILTIN.UNICODE")], actual) + actual = df.select(F.collation(F.collate("name", "UNICODE"))).distinct() + assertDataFrameEqual([Row("SYSTEM.BUILTIN.UNICODE")], actual) def test_try_make_interval(self): df = self.spark.createDataFrame([(2147483647,)], ["num"]) - actual = df.select(F.isnull(F.try_make_interval("num"))).collect() - self.assertEqual([Row(True)], actual) + actual = df.select(F.isnull(F.try_make_interval("num"))) + assertDataFrameEqual([Row(True)], actual) def test_octet_length_function(self): # SPARK-36751: add octet length api for python df = self.spark.createDataFrame([("cat",), ("\U0001F408",)], ["cat"]) - actual = df.select(F.octet_length("cat")).collect() - self.assertEqual([Row(3), Row(4)], actual) + actual = df.select(F.octet_length("cat")) + assertDataFrameEqual([Row(3), Row(4)], actual) def test_bit_length_function(self): # SPARK-36751: add bit length api for python df = self.spark.createDataFrame([("cat",), ("\U0001F408",)], ["cat"]) - actual = df.select(F.bit_length("cat")).collect() - self.assertEqual([Row(24), Row(32)], actual) + actual = df.select(F.bit_length("cat")) + assertDataFrameEqual([Row(24), Row(32)], actual) def test_array_contains_function(self): df = self.spark.createDataFrame([(["1", "2", "3"],), ([],)], ["data"]) - actual = df.select(F.array_contains(df.data, "1").alias("b")).collect() - self.assertEqual([Row(b=True), Row(b=False)], actual) + actual = df.select(F.array_contains(df.data, "1").alias("b")) + assertDataFrameEqual([Row(b=True), Row(b=False)], actual) def test_levenshtein_function(self): df = self.spark.createDataFrame([("kitten", "sitting")], ["l", "r"]) - actual_without_threshold = df.select(F.levenshtein(df.l, df.r).alias("b")).collect() - self.assertEqual([Row(b=3)], actual_without_threshold) - actual_with_threshold = df.select(F.levenshtein(df.l, df.r, 2).alias("b")).collect() - self.assertEqual([Row(b=-1)], actual_with_threshold) + actual_without_threshold = df.select(F.levenshtein(df.l, df.r).alias("b")) + assertDataFrameEqual([Row(b=3)], actual_without_threshold) + actual_with_threshold = df.select(F.levenshtein(df.l, df.r, 2).alias("b")) + assertDataFrameEqual([Row(b=-1)], actual_with_threshold) def test_between_function(self): df = self.spark.createDataFrame( [Row(a=1, b=2, c=3), Row(a=2, b=1, c=3), Row(a=4, b=1, c=4)] ) - self.assertEqual( - [Row(a=2, b=1, c=3), Row(a=4, b=1, c=4)], df.filter(df.a.between(df.b, df.c)).collect() + assertDataFrameEqual( + [Row(a=2, b=1, c=3), Row(a=4, b=1, c=4)], df.filter(df.a.between(df.b, df.c)) ) def test_dayofweek(self): @@ -608,7 +608,7 @@ def test_first_last_ignorenulls(self): F.last(df2.id, False).alias("c"), F.last(df2.id, True).alias("d"), ) - self.assertEqual([Row(a=None, b=1, c=None, d=98)], df3.collect()) + assertDataFrameEqual([Row(a=None, b=1, c=None, d=98)], df3) def test_approxQuantile(self): df = self.spark.createDataFrame([Row(a=i, b=i + 10) for i in range(10)]) @@ -666,20 +666,20 @@ def test_sort_with_nulls_order(self): df = self.spark.createDataFrame( [("Tom", 80), (None, 60), ("Alice", 50)], ["name", "height"] ) - self.assertEqual( - df.select(df.name).orderBy(F.asc_nulls_first("name")).collect(), + assertDataFrameEqual( + df.select(df.name).orderBy(F.asc_nulls_first("name")), [Row(name=None), Row(name="Alice"), Row(name="Tom")], ) - self.assertEqual( - df.select(df.name).orderBy(F.asc_nulls_last("name")).collect(), + assertDataFrameEqual( + df.select(df.name).orderBy(F.asc_nulls_last("name")), [Row(name="Alice"), Row(name="Tom"), Row(name=None)], ) - self.assertEqual( - df.select(df.name).orderBy(F.desc_nulls_first("name")).collect(), + assertDataFrameEqual( + df.select(df.name).orderBy(F.desc_nulls_first("name")), [Row(name=None), Row(name="Tom"), Row(name="Alice")], ) - self.assertEqual( - df.select(df.name).orderBy(F.desc_nulls_last("name")).collect(), + assertDataFrameEqual( + df.select(df.name).orderBy(F.desc_nulls_last("name")), [Row(name="Tom"), Row(name="Alice"), Row(name=None)], ) @@ -716,20 +716,16 @@ def test_slice(self): ) expected = [Row(sliced=[2, 3]), Row(sliced=[5])] - self.assertEqual(df.select(F.slice(df.x, 2, 2).alias("sliced")).collect(), expected) - self.assertEqual( - df.select(F.slice(df.x, F.lit(2), F.lit(2)).alias("sliced")).collect(), expected - ) - self.assertEqual( - df.select(F.slice("x", "index", "len").alias("sliced")).collect(), expected - ) + assertDataFrameEqual(df.select(F.slice(df.x, 2, 2).alias("sliced")), expected) + assertDataFrameEqual(df.select(F.slice(df.x, F.lit(2), F.lit(2)).alias("sliced")), expected) + assertDataFrameEqual(df.select(F.slice("x", "index", "len").alias("sliced")), expected) - self.assertEqual( - df.select(F.slice(df.x, F.size(df.x) - 1, F.lit(1)).alias("sliced")).collect(), + assertDataFrameEqual( + df.select(F.slice(df.x, F.size(df.x) - 1, F.lit(1)).alias("sliced")), [Row(sliced=[2]), Row(sliced=[4])], ) - self.assertEqual( - df.select(F.slice(df.x, F.lit(1), F.size(df.x) - 1).alias("sliced")).collect(), + assertDataFrameEqual( + df.select(F.slice(df.x, F.lit(1), F.size(df.x) - 1).alias("sliced")), [Row(sliced=[1, 2]), Row(sliced=[4])], ) @@ -738,11 +734,9 @@ def test_array_repeat(self): df = df.withColumn("repeat_n", F.lit(3)) expected = [Row(val=[0, 0, 0])] - self.assertEqual(df.select(F.array_repeat("id", 3).alias("val")).collect(), expected) - self.assertEqual(df.select(F.array_repeat("id", F.lit(3)).alias("val")).collect(), expected) - self.assertEqual( - df.select(F.array_repeat("id", "repeat_n").alias("val")).collect(), expected - ) + assertDataFrameEqual(df.select(F.array_repeat("id", 3).alias("val")), expected) + assertDataFrameEqual(df.select(F.array_repeat("id", F.lit(3)).alias("val")), expected) + assertDataFrameEqual(df.select(F.array_repeat("id", "repeat_n").alias("val")), expected) def test_input_file_name_udf(self): df = self.spark.read.text("python/test_support/hello/hello.txt") @@ -754,11 +748,11 @@ def test_least(self): df = self.spark.createDataFrame([(1, 4, 3)], ["a", "b", "c"]) expected = [Row(least=1)] - self.assertEqual(df.select(F.least(df.a, df.b, df.c).alias("least")).collect(), expected) - self.assertEqual( - df.select(F.least(F.lit(3), F.lit(5), F.lit(1)).alias("least")).collect(), expected + assertDataFrameEqual(df.select(F.least(df.a, df.b, df.c).alias("least")), expected) + assertDataFrameEqual( + df.select(F.least(F.lit(3), F.lit(5), F.lit(1)).alias("least")), expected ) - self.assertEqual(df.select(F.least("a", "b", "c").alias("least")).collect(), expected) + assertDataFrameEqual(df.select(F.least("a", "b", "c").alias("least")), expected) with self.assertRaises(PySparkValueError) as pe: df.select(F.least(df.a).alias("least")).collect() @@ -800,11 +794,9 @@ def test_overlay(self): df = self.spark.createDataFrame([("SPARK_SQL", "CORE", 7, 0)], ("x", "y", "pos", "len")) exp = [Row(ol="SPARK_CORESQL")] - self.assertEqual(df.select(F.overlay(df.x, df.y, 7, 0).alias("ol")).collect(), exp) - self.assertEqual( - df.select(F.overlay(df.x, df.y, F.lit(7), F.lit(0)).alias("ol")).collect(), exp - ) - self.assertEqual(df.select(F.overlay("x", "y", "pos", "len").alias("ol")).collect(), exp) + assertDataFrameEqual(df.select(F.overlay(df.x, df.y, 7, 0).alias("ol")), exp) + assertDataFrameEqual(df.select(F.overlay(df.x, df.y, F.lit(7), F.lit(0)).alias("ol")), exp) + assertDataFrameEqual(df.select(F.overlay("x", "y", "pos", "len").alias("ol")), exp) with self.assertRaises(PySparkTypeError) as pe: df.select(F.overlay(df.x, df.y, 7.5, 0).alias("ol")).collect() @@ -1164,8 +1156,8 @@ def test_assert_true(self): def check_assert_true(self, tpe): df = self.spark.range(3) - self.assertEqual( - df.select(F.assert_true(df.id < 3)).toDF("val").collect(), + assertDataFrameEqual( + df.select(F.assert_true(df.id < 3)).toDF("val"), [Row(val=None), Row(val=None), Row(val=None)], ) @@ -1302,17 +1294,17 @@ def test_np_scalar_input(self): df = self.spark.createDataFrame([([1, 2, 3],), ([],)], ["data"]) for dtype in [np.int8, np.int16, np.int32, np.int64]: - res = df.select(F.array_contains(df.data, dtype(1)).alias("b")).collect() - self.assertEqual([Row(b=True), Row(b=False)], res) - res = df.select(F.array_position(df.data, dtype(1)).alias("c")).collect() - self.assertEqual([Row(c=1), Row(c=0)], res) + res = df.select(F.array_contains(df.data, dtype(1)).alias("b")) + assertDataFrameEqual([Row(b=True), Row(b=False)], res) + res = df.select(F.array_position(df.data, dtype(1)).alias("c")) + assertDataFrameEqual([Row(c=1), Row(c=0)], res) df = self.spark.createDataFrame([([1.0, 2.0, 3.0],), ([],)], ["data"]) for dtype in [np.float32, np.float64]: - res = df.select(F.array_contains(df.data, dtype(1)).alias("b")).collect() - self.assertEqual([Row(b=True), Row(b=False)], res) - res = df.select(F.array_position(df.data, dtype(1)).alias("c")).collect() - self.assertEqual([Row(c=1), Row(c=0)], res) + res = df.select(F.array_contains(df.data, dtype(1)).alias("b")) + assertDataFrameEqual([Row(b=True), Row(b=False)], res) + res = df.select(F.array_position(df.data, dtype(1)).alias("c")) + assertDataFrameEqual([Row(c=1), Row(c=0)], res) @unittest.skipIf(not have_numpy, "NumPy not installed") def test_ndarray_input(self): @@ -1729,46 +1721,42 @@ class IntEnum(Enum): def test_nullifzero_zeroifnull(self): df = self.spark.createDataFrame([(0,), (1,)], ["a"]) - result = df.select(nullifzero(df.a).alias("r")).collect() - self.assertEqual([Row(r=None), Row(r=1)], result) + result = df.select(nullifzero(df.a).alias("r")) + assertDataFrameEqual([Row(r=None), Row(r=1)], result) df = self.spark.createDataFrame([(None,), (1,)], ["a"]) - result = df.select(zeroifnull(df.a).alias("r")).collect() - self.assertEqual([Row(r=0), Row(r=1)], result) + result = df.select(zeroifnull(df.a).alias("r")) + assertDataFrameEqual([Row(r=0), Row(r=1)], result) def test_randstr_uniform(self): df = self.spark.createDataFrame([(0,)], ["a"]) - result = df.select(randstr(F.lit(5), F.lit(0)).alias("x")).selectExpr("length(x)").collect() - self.assertEqual([Row(5)], result) + result = df.select(randstr(F.lit(5), F.lit(0)).alias("x")).selectExpr("length(x)") + assertDataFrameEqual([Row(5)], result) # The random seed is optional. - result = df.select(randstr(F.lit(5)).alias("x")).selectExpr("length(x)").collect() - self.assertEqual([Row(5)], result) + result = df.select(randstr(F.lit(5)).alias("x")).selectExpr("length(x)") + assertDataFrameEqual([Row(5)], result) df = self.spark.createDataFrame([(0,)], ["a"]) - result = ( - df.select(uniform(F.lit(10), F.lit(20), F.lit(0)).alias("x")) - .selectExpr("x > 5") - .collect() - ) - self.assertEqual([Row(True)], result) + result = df.select(uniform(F.lit(10), F.lit(20), F.lit(0)).alias("x")).selectExpr("x > 5") + assertDataFrameEqual([Row(True)], result) # The random seed is optional. - result = df.select(uniform(F.lit(10), F.lit(20)).alias("x")).selectExpr("x > 5").collect() - self.assertEqual([Row(True)], result) + result = df.select(uniform(F.lit(10), F.lit(20)).alias("x")).selectExpr("x > 5") + assertDataFrameEqual([Row(True)], result) def test_string_validation(self): df = self.spark.createDataFrame([("abc",)], ["a"]) # test is_valid_utf8 - result_is_valid_utf8 = df.select(F.is_valid_utf8(df.a).alias("r")).collect() - self.assertEqual([Row(r=True)], result_is_valid_utf8) + result_is_valid_utf8 = df.select(F.is_valid_utf8(df.a).alias("r")) + assertDataFrameEqual([Row(r=True)], result_is_valid_utf8) # test make_valid_utf8 - result_make_valid_utf8 = df.select(F.make_valid_utf8(df.a).alias("r")).collect() - self.assertEqual([Row(r="abc")], result_make_valid_utf8) + result_make_valid_utf8 = df.select(F.make_valid_utf8(df.a).alias("r")) + assertDataFrameEqual([Row(r="abc")], result_make_valid_utf8) # test validate_utf8 - result_validate_utf8 = df.select(F.validate_utf8(df.a).alias("r")).collect() - self.assertEqual([Row(r="abc")], result_validate_utf8) + result_validate_utf8 = df.select(F.validate_utf8(df.a).alias("r")) + assertDataFrameEqual([Row(r="abc")], result_validate_utf8) # test try_validate_utf8 - result_try_validate_utf8 = df.select(F.try_validate_utf8(df.a).alias("r")).collect() - self.assertEqual([Row(r="abc")], result_try_validate_utf8) + result_try_validate_utf8 = df.select(F.try_validate_utf8(df.a).alias("r")) + assertDataFrameEqual([Row(r="abc")], result_try_validate_utf8) class FunctionsTests(ReusedSQLTestCase, FunctionsTestsMixin): From 05728e4ff64e6684d7c6501f8a079e3b9aded9ed Mon Sep 17 00:00:00 2001 From: xunxunmimi5577 <52647492+xunxunmimi5577@users.noreply.github.com> Date: Mon, 2 Dec 2024 16:38:14 +0800 Subject: [PATCH 066/438] [SPARK-49294][UI] Add width attribute for shuffle-write-time checkbox ### What changes were proposed in this pull request? The pr aims to add the style for `shuffle-write-time-checkbox-div` and set the width to be `155` pixels. ### Why are the changes needed? Fix bug for UI. The tip of `shuffle-write-time` appears in an strange position before this change. As shown below ![MEITU_20240819_105642523](https://github.com/user-attachments/assets/1e4e9639-a949-4fc3-86f4-7cb65d6d9c73) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually check. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47798 from xunxunmimi5577/add-width-style-for-shuffle_write_time-checkbox. Authored-by: xunxunmimi5577 <52647492+xunxunmimi5577@users.noreply.github.com> Signed-off-by: panbingkun --- core/src/main/resources/org/apache/spark/ui/static/webui.css | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index ca7c1f8ba65e2..bf9b230446b26 100755 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -361,6 +361,10 @@ a.downloadbutton { width: 170px; } +.shuffle-write-time-checkbox-div { + width: 155px; +} + .result-serialization-time-checkbox-div { width: 185px; } From 171e2ce7d1176e48e29112fb8e4390bc5426a1d8 Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Tue, 3 Dec 2024 02:15:57 +0800 Subject: [PATCH 067/438] [SPARK-50373] Prohibit Variant from set operations ### What changes were proposed in this pull request? Prior to this PR, Variant columns could be used with set operations like `DISTINCT`, `INTERSECT` and `EXCEPT`. This PR prohibits this behavior since Variant is not orderable. ### Why are the changes needed? Variant equality is not defined, and therefore, these operations are also undefined. ### Does this PR introduce _any_ user-facing change? Yes, users will now no longer be able to perform set operations on variant columns. ### How was this patch tested? Unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #48909 from harshmotw-db/harsh-motwani_data/variant_distinct_fix. Authored-by: Harsh Motwani Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 5 ++ .../sql/catalyst/analysis/CheckAnalysis.scala | 21 +++++ .../sql/DataFrameSetOperationsSuite.scala | 78 +++++++++++++++++++ 3 files changed, 104 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 024caf86cf94b..632d43b4d1056 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5360,6 +5360,11 @@ "Cannot have MAP type columns in DataFrame which calls set operations (INTERSECT, EXCEPT, etc.), but the type of column is ." ] }, + "SET_OPERATION_ON_VARIANT_TYPE" : { + "message" : [ + "Cannot have VARIANT type columns in DataFrame which calls set operations (INTERSECT, EXCEPT, etc.), but the type of column is ." + ] + }, "SET_PROPERTIES_AND_DBPROPERTIES" : { "message" : [ "set PROPERTIES and DBPROPERTIES at the same time." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 1c76fd7d00f76..c13da35334ba5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -76,6 +76,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB dt.existsRecursively(_.isInstanceOf[MapType]) } + protected def hasVariantType(dt: DataType): Boolean = { + dt.existsRecursively(_.isInstanceOf[VariantType]) + } + protected def mapColumnInSetOperation(plan: LogicalPlan): Option[Attribute] = plan match { case _: Intersect | _: Except | _: Distinct => plan.output.find(a => hasMapType(a.dataType)) @@ -84,6 +88,14 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case _ => None } + protected def variantColumnInSetOperation(plan: LogicalPlan): Option[Attribute] = plan match { + case _: Intersect | _: Except | _: Distinct => + plan.output.find(a => hasVariantType(a.dataType)) + case d: Deduplicate => + d.keys.find(a => hasVariantType(a.dataType)) + case _ => None + } + private def checkLimitLikeClause(name: String, limitExpr: Expression): Unit = { limitExpr match { case e if !e.foldable => limitExpr.failAnalysis( @@ -828,6 +840,15 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB "colName" -> toSQLId(mapCol.name), "dataType" -> toSQLType(mapCol.dataType))) + // TODO: Remove this type check once we support Variant ordering + case o if variantColumnInSetOperation(o).isDefined => + val variantCol = variantColumnInSetOperation(o).get + o.failAnalysis( + errorClass = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + messageParameters = Map( + "colName" -> toSQLId(variantCol.name), + "dataType" -> toSQLType(variantCol.dataType))) + case o if o.expressions.exists(!_.deterministic) && !operatorAllowsNonDeterministicExpressions(o) && !o.isInstanceOf[Project] && diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala index 5ff737d2b57cb..9c182be0f7dd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala @@ -350,6 +350,84 @@ class DataFrameSetOperationsSuite extends QueryTest dates.intersect(widenTypedRows).collect() } + test("SPARK-50373 - cannot run set operations with variant type") { + val df = sql("select parse_json(case when id = 0 then 'null' else '1' end)" + + " as v, id % 5 as id from range(0, 100, 1, 5)") + checkError( + exception = intercept[AnalysisException](df.intersect(df)), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + parameters = Map( + "colName" -> "`v`", + "dataType" -> "\"VARIANT\"") + ) + checkError( + exception = intercept[AnalysisException](df.except(df)), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + parameters = Map( + "colName" -> "`v`", + "dataType" -> "\"VARIANT\"") + ) + checkError( + exception = intercept[AnalysisException](df.distinct()), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + parameters = Map( + "colName" -> "`v`", + "dataType" -> "\"VARIANT\"")) + checkError( + exception = intercept[AnalysisException](df.dropDuplicates()), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + parameters = Map( + "colName" -> "`v`", + "dataType" -> "\"VARIANT\"")) + withTempView("tv") { + df.createOrReplaceTempView("tv") + checkError( + exception = intercept[AnalysisException](sql("SELECT DISTINCT v FROM tv")), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + parameters = Map( + "colName" -> "`v`", + "dataType" -> "\"VARIANT\""), + context = ExpectedContext( + fragment = "SELECT DISTINCT v FROM tv", + start = 0, + stop = 24) + ) + checkError( + exception = intercept[AnalysisException](sql("SELECT DISTINCT STRUCT(v) FROM tv")), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + parameters = Map( + "colName" -> "`struct(v)`", + "dataType" -> "\"STRUCT\""), + context = ExpectedContext( + fragment = "SELECT DISTINCT STRUCT(v) FROM tv", + start = 0, + stop = 32) + ) + checkError( + exception = intercept[AnalysisException](sql("SELECT DISTINCT ARRAY(v) FROM tv")), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_VARIANT_TYPE", + parameters = Map( + "colName" -> "`array(v)`", + "dataType" -> "\"ARRAY\""), + context = ExpectedContext( + fragment = "SELECT DISTINCT ARRAY(v) FROM tv", + start = 0, + stop = 31) + ) + checkError( + exception = intercept[AnalysisException](sql("SELECT DISTINCT MAP('m', v) FROM tv")), + condition = "UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE", + parameters = Map( + "colName" -> "`map(m, v)`", + "dataType" -> "\"MAP\""), + context = ExpectedContext( + fragment = "SELECT DISTINCT MAP('m', v) FROM tv", + start = 0, + stop = 34) + ) + } + } + test("SPARK-19893: cannot run set operations with map type") { val df = spark.range(1).select(map(lit("key"), $"id").as("m")) checkError( From f382cdf895360d86f83b0a728e9ae0387763d6a4 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 2 Dec 2024 11:34:49 -0800 Subject: [PATCH 068/438] [SPARK-50432][BUILD] Remove workaround for THRIFT-4805 ### What changes were proposed in this pull request? Remove workaround for THRIFT-4805 in log4j2 configuration files. ### Why are the changes needed? SPARK-47018 upgrades Thrift 0.16, and THRIFT-4805 was fixed in Thrift 0.13. ### Does this PR introduce _any_ user-facing change? Affects default logs a little. The existing logging filter rules use a regex to filter out the noisy logs caused by THRIFT-4805, while other problems may also produce similar logs and silently be dropped, this change restores that. ### How was this patch tested? Review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48980 from pan3793/SPARK-50432. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- conf/log4j2.properties.pattern-layout-template | 7 ------- conf/log4j2.properties.template | 7 ------- .../src/test/resources/log4j2.properties | 15 ++------------- 3 files changed, 2 insertions(+), 27 deletions(-) diff --git a/conf/log4j2.properties.pattern-layout-template b/conf/log4j2.properties.pattern-layout-template index ab96e03baed20..011fca58c9b2a 100644 --- a/conf/log4j2.properties.pattern-layout-template +++ b/conf/log4j2.properties.pattern-layout-template @@ -60,10 +60,3 @@ logger.RetryingHMSHandler.name = org.apache.hadoop.hive.metastore.RetryingHMSHan logger.RetryingHMSHandler.level = fatal logger.FunctionRegistry.name = org.apache.hadoop.hive.ql.exec.FunctionRegistry logger.FunctionRegistry.level = error - -# For deploying Spark ThriftServer -# SPARK-34128: Suppress undesirable TTransportException warnings involved in THRIFT-4805 -appender.console.filter.1.type = RegexFilter -appender.console.filter.1.regex = .*Thrift error occurred during processing of message.* -appender.console.filter.1.onMatch = deny -appender.console.filter.1.onMismatch = neutral diff --git a/conf/log4j2.properties.template b/conf/log4j2.properties.template index 8767245314449..76499bb6691e7 100644 --- a/conf/log4j2.properties.template +++ b/conf/log4j2.properties.template @@ -54,10 +54,3 @@ logger.RetryingHMSHandler.name = org.apache.hadoop.hive.metastore.RetryingHMSHan logger.RetryingHMSHandler.level = fatal logger.FunctionRegistry.name = org.apache.hadoop.hive.ql.exec.FunctionRegistry logger.FunctionRegistry.level = error - -# For deploying Spark ThriftServer -# SPARK-34128: Suppress undesirable TTransportException warnings involved in THRIFT-4805 -appender.console.filter.1.type = RegexFilter -appender.console.filter.1.regex = .*Thrift error occurred during processing of message.* -appender.console.filter.1.onMatch = deny -appender.console.filter.1.onMismatch = neutral diff --git a/sql/hive-thriftserver/src/test/resources/log4j2.properties b/sql/hive-thriftserver/src/test/resources/log4j2.properties index ebb3a6ccb2fca..e6753047c9055 100644 --- a/sql/hive-thriftserver/src/test/resources/log4j2.properties +++ b/sql/hive-thriftserver/src/test/resources/log4j2.properties @@ -32,12 +32,6 @@ appender.console.filter.1.type = Filters appender.console.filter.1.a.type = ThresholdFilter appender.console.filter.1.a.level = warn -# SPARK-34128: Suppress undesirable TTransportException warnings, due to THRIFT-4805 -appender.console.filter.1.b.type = RegexFilter -appender.console.filter.1.b.regex = .*Thrift error occurred during processing of message.* -appender.console.filter.1.b.onMatch = deny -appender.console.filter.1.b.onMismatch = neutral - #File Appender appender.file.type = File appender.file.name = File @@ -47,14 +41,9 @@ appender.file.layout.pattern = %d{HH:mm:ss.SSS} %t %p %c{1}: %m%n%ex appender.file.filter.1.type = Filters -appender.file.filter.1.a.type = RegexFilter -appender.file.filter.1.a.regex = .*Thrift error occurred during processing of message.* -appender.file.filter.1.a.onMatch = deny -appender.file.filter.1.a.onMismatch = neutral - # Set the logger level of File Appender to WARN -appender.file.filter.1.b.type = ThresholdFilter -appender.file.filter.1.b.level = debug +appender.file.filter.1.a.type = ThresholdFilter +appender.file.filter.1.a.level = debug # Some packages are noisy for no good reason. logger.parquet_recordreader.name = org.apache.parquet.hadoop.ParquetRecordReader From ae4625c9c375ebda82210ec17903b0941f060114 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 2 Dec 2024 11:37:06 -0800 Subject: [PATCH 069/438] [SPARK-50433][DOCS][TESTS] Fix confguring log4j2 guide docs for Spark on YARN and UT ### What changes were proposed in this pull request? As title. ### Why are the changes needed? SPARK-37814 (3.3.0) migrated logging system from log4j1 to log4j2, we should updated the docs as well. ### Does this PR introduce _any_ user-facing change? Yes, docs are updated. ### How was this patch tested? Review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48981 from pan3793/SPARK-50433. Lead-authored-by: Cheng Pan Co-authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/test/IntegrationTestUtils.scala | 2 +- .../spark/examples/streaming/KinesisWordCountASL.scala | 2 +- docs/running-on-yarn.md | 8 ++++---- .../spark/examples/streaming/StreamingExamples.scala | 2 +- .../org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/IntegrationTestUtils.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/IntegrationTestUtils.scala index 61d08912aec23..3ae9b9fc73b48 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/IntegrationTestUtils.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/IntegrationTestUtils.scala @@ -74,7 +74,7 @@ object IntegrationTestUtils { // Redirect server log into console "--conf", - s"spark.driver.extraJavaOptions=-Dlog4j.configuration=$log4j2") + s"spark.driver.extraJavaOptions=-Dlog4j.configurationFile=$log4j2") } else Seq.empty } diff --git a/connector/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/connector/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index 4835e9de086c4..cc24c378f4cbf 100644 --- a/connector/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/connector/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -275,7 +275,7 @@ private[streaming] object StreamingExamples extends Logging { // We first log something to initialize Spark's default logging, then we override the // logging level. logInfo("Setting log level to [WARN] for streaming example." + - " To override add a custom log4j.properties to the classpath.") + " To override add a custom log4j2.properties to the classpath.") Configurator.setRootLevel(Level.WARN) } } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d149f9196b345..aefa979946a6c 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -124,15 +124,15 @@ all environment variables used for launching each container. This process is use classpath problems in particular. (Note that enabling this requires admin privileges on cluster settings and a restart of all node managers. Thus, this is not applicable to hosted clusters). -To use a custom log4j configuration for the application master or executors, here are the options: +To use a custom log4j2 configuration for the application master or executors, here are the options: -- upload a custom `log4j.properties` using `spark-submit`, by adding it to the `--files` list of files +- upload a custom `log4j2.properties` using `spark-submit`, by adding it to the `--files` list of files to be uploaded with the application. -- add `-Dlog4j.configuration=` to `spark.driver.extraJavaOptions` +- add `-Dlog4j.configurationFile=` to `spark.driver.extraJavaOptions` (for the driver) or `spark.executor.extraJavaOptions` (for executors). Note that if using a file, the `file:` protocol should be explicitly provided, and the file needs to exist locally on all the nodes. -- update the `$SPARK_CONF_DIR/log4j.properties` file and it will be automatically uploaded along +- update the `$SPARK_CONF_DIR/log4j2.properties` file and it will be automatically uploaded along with the other configurations. Note that other 2 options has higher priority than this option if multiple options are specified. diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala index 20c5eb1700155..9289b005e3ba4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala @@ -31,7 +31,7 @@ object StreamingExamples extends Logging { // We first log something to initialize Spark's default logging, then we override the // logging level. logInfo("Setting log level to [WARN] for streaming example." + - " To override add a custom log4j.properties to the classpath.") + " To override add a custom log4j2.properties to the classpath.") Configurator.setRootLevel(Level.WARN) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 92d9f2d62d1c1..71843b7f90b1f 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -293,7 +293,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { } test("running Spark in yarn-cluster mode displays driver log links") { - val log4jConf = new File(tempDir, "log4j.properties") + val log4jConf = new File(tempDir, "log4j2.properties") val logOutFile = new File(tempDir, "logs") Files.asCharSink(log4jConf, StandardCharsets.UTF_8).write( s"""rootLogger.level = debug From 98350095d33c27dfe3e0c28fa81e3249b3d93cc4 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 2 Dec 2024 11:38:30 -0800 Subject: [PATCH 070/438] [SPARK-50452][BUILD] Upgrade jackson to 2.18.2 ### What changes were proposed in this pull request? The pr aims to upgrade `jackson` from `2.18.1` to `2.18.2` ### Why are the changes needed? The full release notes: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.18.2 image ### 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 #49010 from panbingkun/SPARK-50452. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- 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 a35e4eae8da12..8f04318fedeb0 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -105,16 +105,16 @@ 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 j2objc-annotations/3.0.0//j2objc-annotations-3.0.0.jar -jackson-annotations/2.18.1//jackson-annotations-2.18.1.jar +jackson-annotations/2.18.2//jackson-annotations-2.18.2.jar jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar -jackson-core/2.18.1//jackson-core-2.18.1.jar -jackson-databind/2.18.1//jackson-databind-2.18.1.jar -jackson-dataformat-cbor/2.18.1//jackson-dataformat-cbor-2.18.1.jar -jackson-dataformat-yaml/2.18.1//jackson-dataformat-yaml-2.18.1.jar +jackson-core/2.18.2//jackson-core-2.18.2.jar +jackson-databind/2.18.2//jackson-databind-2.18.2.jar +jackson-dataformat-cbor/2.18.2//jackson-dataformat-cbor-2.18.2.jar +jackson-dataformat-yaml/2.18.2//jackson-dataformat-yaml-2.18.2.jar jackson-datatype-jdk8/2.17.0//jackson-datatype-jdk8-2.17.0.jar -jackson-datatype-jsr310/2.18.1//jackson-datatype-jsr310-2.18.1.jar +jackson-datatype-jsr310/2.18.2//jackson-datatype-jsr310-2.18.2.jar jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.13/2.18.1//jackson-module-scala_2.13-2.18.1.jar +jackson-module-scala_2.13/2.18.2//jackson-module-scala_2.13-2.18.2.jar jakarta.annotation-api/2.1.1//jakarta.annotation-api-2.1.1.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 85c29aacab8be..2fc982c1d31f0 100644 --- a/pom.xml +++ b/pom.xml @@ -180,8 +180,8 @@ true true 1.9.13 - 2.18.1 - 2.18.1 + 2.18.2 + 2.18.2 2.3.1 1.1.10.7 3.0.3 From be12eb732b0b3deedb7cf8327c61f9308cda54e0 Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Mon, 2 Dec 2024 11:43:33 -0800 Subject: [PATCH 071/438] [SPARK-50465][PYTHON][TESTS] Use assertDataFrameEqual in pyspark.sql.tests.test_group and test_readwriter ### What changes were proposed in this pull request? Use `assertDataFrameEqual` in pyspark.sql.tests.test_group and test_readwriter ### Why are the changes needed? `assertDataFrameEqual` is explicitly built to handle DataFrame-specific comparisons, including schema. So we propose to replace `assertEqual` with `assertDataFrameEqual` Part of https://issues.apache.org/jira/browse/SPARK-50435. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49023 from xinrong-meng/impr_test_group. Authored-by: Xinrong Meng Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_group.py | 12 ++++++------ python/pyspark/sql/tests/test_readwriter.py | 21 +++++++++++---------- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/tests/test_group.py b/python/pyspark/sql/tests/test_group.py index 8e3d2d8d00033..bbc089b00c133 100644 --- a/python/pyspark/sql/tests/test_group.py +++ b/python/pyspark/sql/tests/test_group.py @@ -36,11 +36,11 @@ def test_agg_func(self): data = [Row(key=1, value=10), Row(key=1, value=20), Row(key=1, value=30)] df = self.spark.createDataFrame(data) g = df.groupBy("key") - self.assertEqual(g.max("value").collect(), [Row(**{"key": 1, "max(value)": 30})]) - self.assertEqual(g.min("value").collect(), [Row(**{"key": 1, "min(value)": 10})]) - self.assertEqual(g.sum("value").collect(), [Row(**{"key": 1, "sum(value)": 60})]) - self.assertEqual(g.count().collect(), [Row(key=1, count=3)]) - self.assertEqual(g.mean("value").collect(), [Row(**{"key": 1, "avg(value)": 20.0})]) + assertDataFrameEqual(g.max("value"), [Row(**{"key": 1, "max(value)": 30})]) + assertDataFrameEqual(g.min("value"), [Row(**{"key": 1, "min(value)": 10})]) + assertDataFrameEqual(g.sum("value"), [Row(**{"key": 1, "sum(value)": 60})]) + assertDataFrameEqual(g.count(), [Row(key=1, count=3)]) + assertDataFrameEqual(g.mean("value"), [Row(**{"key": 1, "avg(value)": 20.0})]) data = [ Row(electronic="Smartphone", year=2018, sales=150000), @@ -59,7 +59,7 @@ def test_aggregator(self): df = self.df g = df.groupBy() self.assertEqual([99, 100], sorted(g.agg({"key": "max", "value": "count"}).collect()[0])) - self.assertEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect()) + assertDataFrameEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect()) from pyspark.sql import functions diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index 2fca6b57decf9..683c925eefc23 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -23,6 +23,7 @@ from pyspark.sql.functions import col, lit from pyspark.sql.readwriter import DataFrameWriterV2 from pyspark.sql.types import StructType, StructField, StringType +from pyspark.testing import assertDataFrameEqual from pyspark.testing.sqlutils import ReusedSQLTestCase @@ -34,15 +35,15 @@ def test_save_and_load(self): try: df.write.json(tmpPath) actual = self.spark.read.json(tmpPath) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) schema = StructType([StructField("value", StringType(), True)]) actual = self.spark.read.json(tmpPath, schema) - self.assertEqual(sorted(df.select("value").collect()), sorted(actual.collect())) + assertDataFrameEqual(df.select("value"), actual) df.write.json(tmpPath, "overwrite") actual = self.spark.read.json(tmpPath) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) df.write.save( format="json", @@ -53,11 +54,11 @@ def test_save_and_load(self): actual = self.spark.read.load( format="json", path=tmpPath, noUse="this options will not be used in load." ) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) with self.sql_conf({"spark.sql.sources.default": "org.apache.spark.sql.json"}): actual = self.spark.read.load(path=tmpPath) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) csvpath = os.path.join(tempfile.mkdtemp(), "data") df.write.option("quote", None).format("csv").save(csvpath) @@ -71,15 +72,15 @@ def test_save_and_load_builder(self): try: df.write.json(tmpPath) actual = self.spark.read.json(tmpPath) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) schema = StructType([StructField("value", StringType(), True)]) actual = self.spark.read.json(tmpPath, schema) - self.assertEqual(sorted(df.select("value").collect()), sorted(actual.collect())) + assertDataFrameEqual(df.select("value"), actual) df.write.mode("overwrite").json(tmpPath) actual = self.spark.read.json(tmpPath) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) df.write.mode("overwrite").options( noUse="this options will not be used in save." @@ -89,11 +90,11 @@ def test_save_and_load_builder(self): actual = self.spark.read.format("json").load( path=tmpPath, noUse="this options will not be used in load." ) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) with self.sql_conf({"spark.sql.sources.default": "org.apache.spark.sql.json"}): actual = self.spark.read.load(path=tmpPath) - self.assertEqual(sorted(df.collect()), sorted(actual.collect())) + assertDataFrameEqual(df, actual) finally: shutil.rmtree(tmpPath) From 6c84f158469a70ae8d526a7eb65a4c7054516d55 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 2 Dec 2024 12:44:53 -0800 Subject: [PATCH 072/438] [SPARK-50468][BUILD] Upgrade Guava to 33.3.1-jre ### What changes were proposed in this pull request? This pr aims to upgrade guava from 33.2.1-jre to 33.3.1-jre ### Why are the changes needed? The full release notes as follows: - https://github.com/google/guava/releases/tag/v33.3.0 - https://github.com/google/guava/releases/tag/v33.3.1 ### 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 #48976 from LuciferYang/guava-33.3.1-jre. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 6 +++--- pom.xml | 4 ++-- project/SparkBuild.scala | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 8f04318fedeb0..d3ad905c47cf6 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -33,7 +33,7 @@ breeze-macros_2.13/2.1.0//breeze-macros_2.13-2.1.0.jar breeze_2.13/2.1.0//breeze_2.13-2.1.0.jar bundle/2.24.6//bundle-2.24.6.jar cats-kernel_2.13/2.8.0//cats-kernel_2.13-2.8.0.jar -checker-qual/3.42.0//checker-qual-3.42.0.jar +checker-qual/3.43.0//checker-qual-3.43.0.jar chill-java/0.10.0//chill-java-0.10.0.jar chill_2.13/0.10.0//chill_2.13-0.10.0.jar commons-cli/1.9.0//commons-cli-1.9.0.jar @@ -63,14 +63,14 @@ derby/10.16.1.1//derby-10.16.1.1.jar derbyshared/10.16.1.1//derbyshared-10.16.1.1.jar derbytools/10.16.1.1//derbytools-10.16.1.1.jar dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar -error_prone_annotations/2.26.1//error_prone_annotations-2.26.1.jar +error_prone_annotations/2.28.0//error_prone_annotations-2.28.0.jar esdk-obs-java/3.20.4.2//esdk-obs-java-3.20.4.2.jar failureaccess/1.0.2//failureaccess-1.0.2.jar flatbuffers-java/24.3.25//flatbuffers-java-24.3.25.jar gcs-connector/hadoop3-2.2.25/shaded/gcs-connector-hadoop3-2.2.25-shaded.jar gmetric4j/1.0.10//gmetric4j-1.0.10.jar gson/2.11.0//gson-2.11.0.jar -guava/33.2.1-jre//guava-33.2.1-jre.jar +guava/33.3.1-jre//guava-33.3.1-jre.jar hadoop-aliyun/3.4.1//hadoop-aliyun-3.4.1.jar hadoop-annotations/3.4.1//hadoop-annotations-3.4.1.jar hadoop-aws/3.4.1//hadoop-aws-3.4.1.jar diff --git a/pom.xml b/pom.xml index 2fc982c1d31f0..06c3eca083bdb 100644 --- a/pom.xml +++ b/pom.xml @@ -195,7 +195,7 @@ 2.12.0 4.1.17 - 33.2.1-jre + 33.3.1-jre 2.11.0 3.1.9 3.0.16 @@ -295,7 +295,7 @@ true - 33.2.1-jre + 33.3.1-jre 1.0.2 1.67.1 1.1.4 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 48b243618eea3..b060483c89c9b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1057,7 +1057,7 @@ object KubernetesIntegrationTests { * Overrides to work around sbt's dependency resolution being different from Maven's. */ object DependencyOverrides { - lazy val guavaVersion = sys.props.get("guava.version").getOrElse("33.1.0-jre") + lazy val guavaVersion = sys.props.get("guava.version").getOrElse("33.3.1-jre") lazy val settings = Seq( dependencyOverrides += "com.google.guava" % "guava" % guavaVersion, dependencyOverrides += "jline" % "jline" % "2.14.6", From d0e2c06e874a2a61fb95450bbb9085be2ba2c167 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Tue, 3 Dec 2024 08:21:53 +0900 Subject: [PATCH 073/438] [SPARK-50372][CONNECT][SQL] Make all DF execution path collect observed metrics ### What changes were proposed in this pull request? This PR fixes an issue that some of DataFrame execution paths would not process `ObservedMetrics`. The fix is done by injecting a lazy processing logic into the result iterator. The following private execution APIs are affected by this issue: - `SparkSession.execute(proto.Relation.Builder)` - `SparkSession.execute(proto.Command)` - `SparkSession.execute(proto.Plan)` The following user-facing API is affected by this issue: - `DataFrame.write.format("...").mode("...").save()` This PR also fixes an issue in which on the Server side, two observed metrics can be assigned to the same Plan ID when they are in the same plan (e.g., one observation is used as the input of another). The fix is to traverse the plan and find all observations with correct IDs. Another bug is discovered as a byproduct of introducing a new test case. Copying the PR comment here from SparkConnectPlanner.scala: > This fixes a bug where the input of a `CollectMetrics` can be processed two times, once in Line 1190 and once here/below. > > When the `input` contains another `CollectMetrics`, transforming it twice will cause two `Observation` objects (in the input) to be initialised and registered two times to the system. Since only one of them will be fulfilled when the query finishes, the one we'll be looking at may not have any data. > > This issue is highlighted in the test case `Observation.get is blocked until the query is finished ...`, where we specifically execute `observedObservedDf`, which is a `CollectMetrics` that has another `CollectMetrics` as its input. ### Why are the changes needed? To fix a bug. ### Does this PR introduce _any_ user-facing change? Yes, this bug is user-facing. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48920 from xupefei/observation-notify-fix. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/SparkSession.scala | 45 +++++++++----- .../apache/spark/sql/ClientE2ETestSuite.scala | 59 +++++++++++++------ .../connect/client/CloseableIterator.scala | 10 ++++ .../sql/connect/client/SparkResult.scala | 41 +++++++------ .../execution/ExecuteThreadRunner.scala | 2 +- .../execution/SparkConnectPlanExecution.scala | 14 +++-- .../connect/planner/SparkConnectPlanner.scala | 4 +- .../sql/connect/service/ExecuteHolder.scala | 26 ++++++++ 8 files changed, 137 insertions(+), 64 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index b74d0c2ff2243..3183a155c16a3 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -34,6 +34,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse +import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.Catalog @@ -371,13 +372,8 @@ class SparkSession private[sql] ( private[sql] def timeZoneId: String = conf.get(SqlApiConf.SESSION_LOCAL_TIMEZONE_KEY) private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { - val value = client.execute(plan) - new SparkResult( - value, - allocator, - encoder, - timeZoneId, - Some(setMetricsAndUnregisterObservation)) + val value = executeInternal(plan) + new SparkResult(value, allocator, encoder, timeZoneId) } private[sql] def execute(f: proto.Relation.Builder => Unit): Unit = { @@ -386,7 +382,7 @@ class SparkSession private[sql] ( builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) val plan = proto.Plan.newBuilder().setRoot(builder).build() // .foreach forces that the iterator is consumed and closed - client.execute(plan).foreach(_ => ()) + executeInternal(plan).foreach(_ => ()) } @Since("4.0.0") @@ -395,11 +391,26 @@ class SparkSession private[sql] ( val plan = proto.Plan.newBuilder().setCommand(command).build() // .toSeq forces that the iterator is consumed and closed. On top, ignore all // progress messages. - client.execute(plan).filter(!_.hasExecutionProgress).toSeq + executeInternal(plan).filter(!_.hasExecutionProgress).toSeq } - private[sql] def execute(plan: proto.Plan): CloseableIterator[ExecutePlanResponse] = - client.execute(plan) + /** + * The real `execute` method that calls into `SparkConnectClient`. + * + * Here we inject a lazy map to process registered observed metrics, so consumers of the + * returned iterator does not need to worry about it. + * + * Please make sure all `execute` methods call this method. + */ + private[sql] def executeInternal(plan: proto.Plan): CloseableIterator[ExecutePlanResponse] = { + client + .execute(plan) + .map { response => + // Note, this map() is lazy. + processRegisteredObservedMetrics(response.getObservedMetricsList) + response + } + } private[sql] def registerUdf(udf: proto.CommonInlineUserDefinedFunction): Unit = { val command = proto.Command.newBuilder().setRegisterFunction(udf).build() @@ -541,10 +552,14 @@ class SparkSession private[sql] ( observationRegistry.putIfAbsent(planId, observation) } - private[sql] def setMetricsAndUnregisterObservation(planId: Long, metrics: Row): Unit = { - val observationOrNull = observationRegistry.remove(planId) - if (observationOrNull != null) { - observationOrNull.setMetricsAndNotify(metrics) + private def processRegisteredObservedMetrics(metrics: java.util.List[ObservedMetrics]): Unit = { + metrics.asScala.map { metric => + // Here we only process metrics that belong to a registered Observation object. + // All metrics, whether registered or not, will be collected by `SparkResult`. + val observationOrNull = observationRegistry.remove(metric.getPlanId) + if (observationOrNull != null) { + observationOrNull.setMetricsAndNotify(SparkResult.transformObservedMetrics(metric)) + } } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index 0371981b728d1..92b5808f4d626 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -1536,28 +1536,49 @@ class ClientE2ETestSuite val ob1Metrics = Map("ob1" -> new GenericRowWithSchema(Array(0, 49, 98), ob1Schema)) val ob2Metrics = Map("ob2" -> new GenericRowWithSchema(Array(-1, 48, 97), ob2Schema)) + val obMetrics = observedDf.collectResult().getObservedMetrics assert(df.collectResult().getObservedMetrics === Map.empty) assert(observedDf.collectResult().getObservedMetrics === ob1Metrics) - assert(observedObservedDf.collectResult().getObservedMetrics === ob1Metrics ++ ob2Metrics) - } - - test("Observation.get is blocked until the query is finished") { - val df = spark.range(99).withColumn("extra", col("id") - 1) - val observation = new Observation("ob1") - val observedDf = df.observe(observation, min("id"), avg("id"), max("id")) - - // Start a new thread to get the observation - val future = Future(observation.get)(ExecutionContext.global) - // make sure the thread is blocked right now - val e = intercept[java.util.concurrent.TimeoutException] { - SparkThreadUtils.awaitResult(future, 2.seconds) + assert(obMetrics.map(_._2.schema) === Seq(ob1Schema)) + + val obObMetrics = observedObservedDf.collectResult().getObservedMetrics + assert(obObMetrics === ob1Metrics ++ ob2Metrics) + assert(obObMetrics.map(_._2.schema).exists(_.equals(ob1Schema))) + assert(obObMetrics.map(_._2.schema).exists(_.equals(ob2Schema))) + } + + for (collectFunc <- Seq( + ("collect", (df: DataFrame) => df.collect()), + ("collectAsList", (df: DataFrame) => df.collectAsList()), + ("collectResult", (df: DataFrame) => df.collectResult().length), + ("write", (df: DataFrame) => df.write.format("noop").mode("append").save()))) + test( + "Observation.get is blocked until the query is finished, " + + s"collect using method ${collectFunc._1}") { + val df = spark.range(99).withColumn("extra", col("id") - 1) + val ob1 = new Observation("ob1") + val ob2 = new Observation("ob2") + val observedDf = df.observe(ob1, min("id"), avg("id"), max("id")) + val observedObservedDf = observedDf.observe(ob2, min("extra"), avg("extra"), max("extra")) + // Start new threads to get observations + val future1 = Future(ob1.get)(ExecutionContext.global) + val future2 = Future(ob2.get)(ExecutionContext.global) + // make sure the threads are blocked right now + val e1 = intercept[java.util.concurrent.TimeoutException] { + SparkThreadUtils.awaitResult(future1, 2.seconds) + } + assert(e1.getMessage.contains("timed out after")) + val e2 = intercept[java.util.concurrent.TimeoutException] { + SparkThreadUtils.awaitResult(future2, 2.seconds) + } + assert(e2.getMessage.contains("timed out after")) + collectFunc._2(observedObservedDf) + // make sure the threads are unblocked after the query is finished + val metrics1 = SparkThreadUtils.awaitResult(future1, 5.seconds) + assert(metrics1 === Map("min(id)" -> 0, "avg(id)" -> 49, "max(id)" -> 98)) + val metrics2 = SparkThreadUtils.awaitResult(future2, 5.seconds) + assert(metrics2 === Map("min(extra)" -> -1, "avg(extra)" -> 48, "max(extra)" -> 97)) } - assert(e.getMessage.contains("Future timed out")) - observedDf.collect() - // make sure the thread is unblocked after the query is finished - val metrics = SparkThreadUtils.awaitResult(future, 2.seconds) - assert(metrics === Map("min(id)" -> 0, "avg(id)" -> 49, "max(id)" -> 98)) - } test("SPARK-48852: trim function on a string column returns correct results") { val session: SparkSession = spark diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala index 4ec6828d885ab..9de585503a500 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala @@ -25,6 +25,16 @@ private[sql] trait CloseableIterator[E] extends Iterator[E] with AutoCloseable { override def close() = self.close() } + + override def map[B](f: E => B): CloseableIterator[B] = { + new CloseableIterator[B] { + override def next(): B = f(self.next()) + + override def hasNext: Boolean = self.hasNext + + override def close(): Unit = self.close() + } + } } private[sql] abstract class WrappedCloseableIterator[E] extends CloseableIterator[E] { diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 3aad90e96f8cd..959779b357c2d 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -41,8 +41,7 @@ private[sql] class SparkResult[T]( responses: CloseableIterator[proto.ExecutePlanResponse], allocator: BufferAllocator, encoder: AgnosticEncoder[T], - timeZoneId: String, - setObservationMetricsOpt: Option[(Long, Row) => Unit] = None) + timeZoneId: String) extends AutoCloseable { self => case class StageInfo( @@ -122,7 +121,8 @@ private[sql] class SparkResult[T]( while (!stop && responses.hasNext) { val response = responses.next() - // Collect metrics for this response + // Collect **all** metrics for this response, whether or not registered to an Observation + // object. observedMetrics ++= processObservedMetrics(response.getObservedMetricsList) // Save and validate operationId @@ -209,23 +209,7 @@ private[sql] class SparkResult[T]( private def processObservedMetrics( metrics: java.util.List[ObservedMetrics]): Iterable[(String, Row)] = { metrics.asScala.map { metric => - assert(metric.getKeysCount == metric.getValuesCount) - var schema = new StructType() - val values = mutable.ArrayBuilder.make[Any] - values.sizeHint(metric.getKeysCount) - (0 until metric.getKeysCount).foreach { i => - val key = metric.getKeys(i) - val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) - schema = schema.add(key, LiteralValueProtoConverter.toDataType(value.getClass)) - values += value - } - val row = new GenericRowWithSchema(values.result(), schema) - // If the metrics is registered by an Observation object, attach them and unblock any - // blocked thread. - setObservationMetricsOpt.foreach { setObservationMetrics => - setObservationMetrics(metric.getPlanId, row) - } - metric.getName -> row + metric.getName -> SparkResult.transformObservedMetrics(metric) } } @@ -387,8 +371,23 @@ private[sql] class SparkResult[T]( } } -private object SparkResult { +private[sql] object SparkResult { private val cleaner: Cleaner = Cleaner.create() + + /** Return value is a Seq of pairs, to preserve the order of values. */ + private[sql] def transformObservedMetrics(metric: ObservedMetrics): Row = { + assert(metric.getKeysCount == metric.getValuesCount) + var schema = new StructType() + val values = mutable.ArrayBuilder.make[Any] + values.sizeHint(metric.getKeysCount) + (0 until metric.getKeysCount).foreach { i => + val key = metric.getKeys(i) + val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) + schema = schema.add(key, LiteralValueProtoConverter.toDataType(value.getClass)) + values += value + } + new GenericRowWithSchema(values.result(), schema) + } } private[client] class SparkResultCloseable( diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala index d27f390a23f95..05e3395a53169 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala @@ -245,7 +245,7 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends .createObservedMetricsResponse( executeHolder.sessionHolder.sessionId, executeHolder.sessionHolder.serverSessionId, - executeHolder.request.getPlan.getRoot.getCommon.getPlanId, + executeHolder.allObservationAndPlanIds, observedMetrics ++ accumulatedInPython)) } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala index c0fd00b2eeaa7..5e3499573e9d9 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala @@ -77,8 +77,10 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) responseObserver.onNext(createSchemaResponse(request.getSessionId, dataframe.schema)) processAsArrowBatches(dataframe, responseObserver, executeHolder) responseObserver.onNext(MetricGenerator.createMetricsResponse(sessionHolder, dataframe)) - createObservedMetricsResponse(request.getSessionId, dataframe).foreach( - responseObserver.onNext) + createObservedMetricsResponse( + request.getSessionId, + executeHolder.allObservationAndPlanIds, + dataframe).foreach(responseObserver.onNext) } type Batch = (Array[Byte], Long) @@ -255,6 +257,7 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) private def createObservedMetricsResponse( sessionId: String, + observationAndPlanIds: Map[String, Long], dataframe: DataFrame): Option[ExecutePlanResponse] = { val observedMetrics = dataframe.queryExecution.observedMetrics.collect { case (name, row) if !executeHolder.observations.contains(name) => @@ -264,13 +267,12 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) name -> values } if (observedMetrics.nonEmpty) { - val planId = executeHolder.request.getPlan.getRoot.getCommon.getPlanId Some( SparkConnectPlanExecution .createObservedMetricsResponse( sessionId, sessionHolder.serverSessionId, - planId, + observationAndPlanIds, observedMetrics)) } else None } @@ -280,17 +282,17 @@ object SparkConnectPlanExecution { def createObservedMetricsResponse( sessionId: String, serverSessionId: String, - planId: Long, + observationAndPlanIds: Map[String, Long], metrics: Map[String, Seq[(Option[String], Any)]]): ExecutePlanResponse = { val observedMetrics = metrics.map { case (name, values) => val metrics = ExecutePlanResponse.ObservedMetrics .newBuilder() .setName(name) - .setPlanId(planId) values.foreach { case (key, value) => metrics.addValues(toLiteralProto(value)) key.foreach(metrics.addKeys) } + observationAndPlanIds.get(name).foreach(metrics.setPlanId) metrics.build() } // Prepare a response with the observed metrics. diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 979fd83612e7b..ee030a52b221a 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1190,14 +1190,14 @@ class SparkConnectPlanner( val input = transformRelation(rel.getInput) if (input.isStreaming || executeHolderOpt.isEmpty) { - CollectMetrics(name, metrics.map(_.named), transformRelation(rel.getInput), planId) + CollectMetrics(name, metrics.map(_.named), input, planId) } else { // TODO this might be too complex for no good reason. It might // be easier to inspect the plan after it completes. val observation = Observation(name) session.observationManager.register(observation, planId) executeHolderOpt.get.addObservation(name, observation) - CollectMetrics(name, metrics.map(_.named), transformRelation(rel.getInput), planId) + CollectMetrics(name, metrics.map(_.named), input, planId) } } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala index 821ddb2c85d58..94638151f7f18 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala @@ -22,6 +22,8 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable import scala.jdk.CollectionConverters._ +import com.google.protobuf.GeneratedMessage + import org.apache.spark.SparkEnv import org.apache.spark.connect.proto import org.apache.spark.internal.Logging @@ -81,6 +83,10 @@ private[connect] class ExecuteHolder( val observations: mutable.Map[String, Observation] = mutable.Map.empty + lazy val allObservationAndPlanIds: Map[String, Long] = { + ExecuteHolder.collectAllObservationAndPlanIds(request.getPlan).toMap + } + private val runner: ExecuteThreadRunner = new ExecuteThreadRunner(this) /** System.currentTimeMillis when this ExecuteHolder was created. */ @@ -289,6 +295,26 @@ private[connect] class ExecuteHolder( def operationId: String = key.operationId } +private object ExecuteHolder { + private def collectAllObservationAndPlanIds( + planOrMessage: GeneratedMessage, + collected: mutable.Map[String, Long] = mutable.Map.empty): mutable.Map[String, Long] = { + planOrMessage match { + case relation: proto.Relation if relation.hasCollectMetrics => + collected += relation.getCollectMetrics.getName -> relation.getCommon.getPlanId + collectAllObservationAndPlanIds(relation.getCollectMetrics.getInput, collected) + case _ => + planOrMessage.getAllFields.values().asScala.foreach { + case message: GeneratedMessage => + collectAllObservationAndPlanIds(message, collected) + case _ => + // not a message (probably a primitive type), do nothing + } + } + collected + } +} + /** Used to identify ExecuteHolder jobTag among SparkContext.SPARK_JOB_TAGS. */ object ExecuteJobTag { private val prefix = "SparkConnect_OperationTag" From 31abad98972bd4e421c34115e45a89f0d5c28896 Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Tue, 3 Dec 2024 08:37:01 +0900 Subject: [PATCH 074/438] [SPARK-50471][PYTHON] Support Arrow-based Python Data Source Writer MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR introduces a new Python Data Source Writer that leverages PyArrow’s RecordBatch format. Unlike the current DataSourceWriter, which operates on iterators of Spark Rows, this new writer takes in an iterator of PyArrow `RecordBatch` as input. ### Why are the changes needed? Make Python data source write more performant when interfacing with systems or libraries that natively support Arrow. ### Does this PR introduce _any_ user-facing change? Yes. This PR adds a new user-facing class `DataSourceArrowWriter`. ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49028 from allisonwang-db/spark-50471-arrow-writer. Authored-by: Allison Wang Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/datasource.py | 39 +++++++++++++++++++ .../sql/tests/test_python_datasource.py | 34 +++++++++++++++- .../sql/worker/write_into_data_source.py | 6 ++- 3 files changed, 77 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/datasource.py b/python/pyspark/sql/datasource.py index a51c96a9d178f..06b853ce5b4e0 100644 --- a/python/pyspark/sql/datasource.py +++ b/python/pyspark/sql/datasource.py @@ -32,6 +32,7 @@ "DataSourceStreamReader", "SimpleDataSourceStreamReader", "DataSourceWriter", + "DataSourceArrowWriter", "DataSourceStreamWriter", "DataSourceRegistration", "InputPartition", @@ -666,6 +667,44 @@ def abort(self, messages: List[Optional["WriterCommitMessage"]]) -> None: ... +class DataSourceArrowWriter(DataSourceWriter): + """ + A base class for data source writers that process data using PyArrow’s `RecordBatch`. + + Unlike :class:`DataSourceWriter`, which works with an iterator of Spark Rows, this class + is optimized for using the Arrow format when writing data. It can offer better performance + when interfacing with systems or libraries that natively support Arrow. + + .. versionadded: 4.0.0 + """ + + @abstractmethod + def write(self, iterator: Iterator["RecordBatch"]) -> "WriterCommitMessage": + """ + Writes an iterator of PyArrow `RecordBatch` objects to the sink. + + This method is called once on each executor to write data to the data source. + It accepts an iterator of PyArrow `RecordBatch`\\s and returns a single row + representing a commit message, or None if there is no commit message. + + The driver collects commit messages, if any, from all executors and passes them + to the :class:`DataSourceWriter.commit` method if all tasks run successfully. If any + task fails, the :class:`DataSourceWriter.abort` method will be called with the + collected commit messages. + + Parameters + ---------- + iterator : iterator of :class:`RecordBatch`\\s + An iterator of PyArrow `RecordBatch` objects representing the input data. + + Returns + ------- + :class:`WriterCommitMessage` + a serializable commit message + """ + ... + + class DataSourceStreamWriter(ABC): """ A base class for data stream writers. Data stream writers are responsible for writing diff --git a/python/pyspark/sql/tests/test_python_datasource.py b/python/pyspark/sql/tests/test_python_datasource.py index 140c7680b181b..a636b852a1e50 100644 --- a/python/pyspark/sql/tests/test_python_datasource.py +++ b/python/pyspark/sql/tests/test_python_datasource.py @@ -25,6 +25,7 @@ DataSourceReader, InputPartition, DataSourceWriter, + DataSourceArrowWriter, WriterCommitMessage, CaseInsensitiveDict, ) @@ -277,7 +278,7 @@ def write(self, iterator): from pyspark import TaskContext context = TaskContext.get() - output_path = os.path.join(self.path, f"{context.partitionId}.json") + output_path = os.path.join(self.path, f"{context.partitionId()}.json") count = 0 with open(output_path, "w") as file: for row in iterator: @@ -436,6 +437,37 @@ def partitions(self): ): self.spark.read.format("arrowbatch").schema("key int, dummy string").load().show() + def test_arrow_batch_sink(self): + class TestDataSource(DataSource): + @classmethod + def name(cls): + return "arrow_sink" + + def writer(self, schema, overwrite): + return TestArrowWriter(self.options["path"]) + + class TestArrowWriter(DataSourceArrowWriter): + def __init__(self, path): + self.path = path + + def write(self, iterator): + from pyspark import TaskContext + + context = TaskContext.get() + output_path = os.path.join(self.path, f"{context.partitionId()}.json") + with open(output_path, "w") as file: + for batch in iterator: + df = batch.to_pandas() + df.to_json(file, orient="records", lines=True) + return WriterCommitMessage() + + self.spark.dataSource.register(TestDataSource) + df = self.spark.range(3) + with tempfile.TemporaryDirectory(prefix="test_arrow_batch_sink") as d: + df.write.format("arrow_sink").mode("append").save(d) + df2 = self.spark.read.format("json").load(d) + assertDataFrameEqual(df2, df) + def test_data_source_type_mismatch(self): class TestDataSource(DataSource): @classmethod diff --git a/python/pyspark/sql/worker/write_into_data_source.py b/python/pyspark/sql/worker/write_into_data_source.py index a114a3facc467..91a1f4d3b1b34 100644 --- a/python/pyspark/sql/worker/write_into_data_source.py +++ b/python/pyspark/sql/worker/write_into_data_source.py @@ -32,6 +32,7 @@ from pyspark.sql.datasource import ( DataSource, DataSourceWriter, + DataSourceArrowWriter, WriterCommitMessage, CaseInsensitiveDict, ) @@ -194,7 +195,10 @@ def batch_to_rows() -> Iterator[Row]: ] yield _create_row(fields=fields, values=values) - res = writer.write(batch_to_rows()) + if isinstance(writer, DataSourceArrowWriter): + res = writer.write(iterator) + else: + res = writer.write(batch_to_rows()) # Check the commit message has the right type. if not isinstance(res, WriterCommitMessage): From 758b5c9ba94164252438faba243973b3168d3cf4 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 3 Dec 2024 08:38:57 +0900 Subject: [PATCH 075/438] [SPARK-50466][PYTHON] Refine the docstring for string functions - part 1 ### What changes were proposed in this pull request? Refine the docstring for string functions ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? new doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49025 from zhengruifeng/py_doc_12. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 398 ++++++++++++++++-------- 1 file changed, 272 insertions(+), 126 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 21200ceb6f33e..13b2e12bca330 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -13216,7 +13216,7 @@ def upper(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -13224,17 +13224,22 @@ def upper(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` upper case values. + See Also + -------- + :meth:`pyspark.sql.functions.lower` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame(["Spark", "PySpark", "Pandas API"], "STRING") - >>> df.select(upper("value")).show() - +------------+ - |upper(value)| - +------------+ - | SPARK| - | PYSPARK| - | PANDAS API| - +------------+ + >>> df.select("*", sf.upper("value")).show() + +----------+------------+ + | value|upper(value)| + +----------+------------+ + | Spark| SPARK| + | PySpark| PYSPARK| + |Pandas API| PANDAS API| + +----------+------------+ """ return _invoke_function_over_columns("upper", col) @@ -13251,7 +13256,7 @@ def lower(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -13259,17 +13264,22 @@ def lower(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` lower case values. + See Also + -------- + :meth:`pyspark.sql.functions.upper` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame(["Spark", "PySpark", "Pandas API"], "STRING") - >>> df.select(lower("value")).show() - +------------+ - |lower(value)| - +------------+ - | spark| - | pyspark| - | pandas api| - +------------+ + >>> df.select("*", sf.lower("value")).show() + +----------+------------+ + | value|lower(value)| + +----------+------------+ + | Spark| spark| + | PySpark| pyspark| + |Pandas API| pandas api| + +----------+------------+ """ return _invoke_function_over_columns("lower", col) @@ -13286,7 +13296,7 @@ def ascii(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -13296,15 +13306,16 @@ def ascii(col: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame(["Spark", "PySpark", "Pandas API"], "STRING") - >>> df.select(ascii("value")).show() - +------------+ - |ascii(value)| - +------------+ - | 83| - | 80| - | 80| - +------------+ + >>> df.select("*", sf.ascii("value")).show() + +----------+------------+ + | value|ascii(value)| + +----------+------------+ + | Spark| 83| + | PySpark| 80| + |Pandas API| 80| + +----------+------------+ """ return _invoke_function_over_columns("ascii", col) @@ -13321,7 +13332,7 @@ def base64(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -13329,17 +13340,22 @@ def base64(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` BASE64 encoding of string value. + See Also + -------- + :meth:`pyspark.sql.functions.unbase64` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame(["Spark", "PySpark", "Pandas API"], "STRING") - >>> df.select(base64("value")).show() - +----------------+ - | base64(value)| - +----------------+ - | U3Bhcms=| - | UHlTcGFyaw==| - |UGFuZGFzIEFQSQ==| - +----------------+ + >>> df.select("*", sf.base64("value")).show() + +----------+----------------+ + | value| base64(value)| + +----------+----------------+ + | Spark| U3Bhcms=| + | PySpark| UHlTcGFyaw==| + |Pandas API|UGFuZGFzIEFQSQ==| + +----------+----------------+ """ return _invoke_function_over_columns("base64", col) @@ -13356,7 +13372,7 @@ def unbase64(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -13364,19 +13380,22 @@ def unbase64(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` encoded string value. + See Also + -------- + :meth:`pyspark.sql.functions.base64` + Examples -------- - >>> df = spark.createDataFrame(["U3Bhcms=", - ... "UHlTcGFyaw==", - ... "UGFuZGFzIEFQSQ=="], "STRING") - >>> df.select(unbase64("value")).show() - +--------------------+ - | unbase64(value)| - +--------------------+ - | [53 70 61 72 6B]| - |[50 79 53 70 61 7...| - |[50 61 6E 64 61 7...| - +--------------------+ + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame(["U3Bhcms=", "UHlTcGFyaw==", "UGFuZGFzIEFQSQ=="], "STRING") + >>> df.select("*", sf.unbase64("value")).show(truncate=False) + +----------------+-------------------------------+ + |value |unbase64(value) | + +----------------+-------------------------------+ + |U3Bhcms= |[53 70 61 72 6B] | + |UHlTcGFyaw== |[50 79 53 70 61 72 6B] | + |UGFuZGFzIEFQSQ==|[50 61 6E 64 61 73 20 41 50 49]| + +----------------+-------------------------------+ """ return _invoke_function_over_columns("unbase64", col) @@ -13393,9 +13412,9 @@ def ltrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. - trim : :class:`~pyspark.sql.Column` or str, optional + trim : :class:`~pyspark.sql.Column` or column name, optional The trim string characters to trim, the default value is a single space .. versionadded:: 4.0.0 @@ -13405,6 +13424,11 @@ def ltrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: :class:`~pyspark.sql.Column` left trimmed values. + See Also + -------- + :meth:`pyspark.sql.functions.trim` + :meth:`pyspark.sql.functions.rtrim` + Examples -------- Example 1: Trim the spaces @@ -13432,6 +13456,18 @@ def ltrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: | Spark**| Spark**| | *Spark| Spark| +--------+--------------------------+ + + Example 3: Trim a column containing different characters + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([("**Spark*", "*"), ("==Spark=", "=")], ["value", "t"]) + >>> df.select("*", sf.ltrim("value", "t")).show() + +--------+---+--------------------------+ + | value| t|TRIM(LEADING t FROM value)| + +--------+---+--------------------------+ + |**Spark*| *| Spark*| + |==Spark=| =| Spark=| + +--------+---+--------------------------+ """ if trim is not None: return _invoke_function_over_columns("ltrim", col, trim) @@ -13451,9 +13487,9 @@ def rtrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. - trim : :class:`~pyspark.sql.Column` or str, optional + trim : :class:`~pyspark.sql.Column` or column name, optional The trim string characters to trim, the default value is a single space .. versionadded:: 4.0.0 @@ -13463,6 +13499,11 @@ def rtrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: :class:`~pyspark.sql.Column` right trimmed values. + See Also + -------- + :meth:`pyspark.sql.functions.trim` + :meth:`pyspark.sql.functions.ltrim` + Examples -------- Example 1: Trim the spaces @@ -13490,6 +13531,18 @@ def rtrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: | Spark**| Spark| | *Spark| *Spark| +--------+---------------------------+ + + Example 3: Trim a column containing different characters + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([("**Spark*", "*"), ("==Spark=", "=")], ["value", "t"]) + >>> df.select("*", sf.rtrim("value", "t")).show() + +--------+---+---------------------------+ + | value| t|TRIM(TRAILING t FROM value)| + +--------+---+---------------------------+ + |**Spark*| *| **Spark| + |==Spark=| =| ==Spark| + +--------+---+---------------------------+ """ if trim is not None: return _invoke_function_over_columns("rtrim", col, trim) @@ -13509,9 +13562,9 @@ def trim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. - trim : :class:`~pyspark.sql.Column` or str, optional + trim : :class:`~pyspark.sql.Column` or column name, optional The trim string characters to trim, the default value is a single space .. versionadded:: 4.0.0 @@ -13521,6 +13574,11 @@ def trim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: :class:`~pyspark.sql.Column` trimmed values from both sides. + See Also + -------- + :meth:`pyspark.sql.functions.ltrim` + :meth:`pyspark.sql.functions.rtrim` + Examples -------- Example 1: Trim the spaces @@ -13548,6 +13606,18 @@ def trim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: | Spark**| Spark| | *Spark| Spark| +--------+-----------------------+ + + Example 3: Trim a column containing different characters + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([("**Spark*", "*"), ("==Spark=", "=")], ["value", "t"]) + >>> df.select("*", sf.trim("value", "t")).show() + +--------+---+-----------------------+ + | value| t|TRIM(BOTH t FROM value)| + +--------+---+-----------------------+ + |**Spark*| *| Spark| + |==Spark=| =| Spark| + +--------+---+-----------------------+ """ if trim is not None: return _invoke_function_over_columns("trim", col, trim) @@ -13568,9 +13638,9 @@ def concat_ws(sep: str, *cols: "ColumnOrName") -> Column: Parameters ---------- - sep : str + sep : literal string words separator. - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name list of columns to work on. Returns @@ -13578,11 +13648,20 @@ def concat_ws(sep: str, *cols: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` string of concatenated words. + See Also + -------- + :meth:`pyspark.sql.functions.concat` + Examples -------- - >>> df = spark.createDataFrame([('abcd','123')], ['s', 'd']) - >>> df.select(concat_ws('-', df.s, df.d).alias('s')).collect() - [Row(s='abcd-123')] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([("abcd", "123")], ["s", "d"]) + >>> df.select("*", sf.concat_ws("-", df.s, "d", sf.lit("xyz"))).show() + +----+---+-----------------------+ + | s| d|concat_ws(-, s, d, xyz)| + +----+---+-----------------------+ + |abcd|123| abcd-123-xyz| + +----+---+-----------------------+ """ from pyspark.sql.classic.column import _to_seq, _to_java_column @@ -13603,9 +13682,9 @@ def decode(col: "ColumnOrName", charset: str) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. - charset : str + charset : literal string charset to use to decode to. Returns @@ -13613,15 +13692,20 @@ def decode(col: "ColumnOrName", charset: str) -> Column: :class:`~pyspark.sql.Column` the column for computed results. + See Also + -------- + :meth:`pyspark.sql.functions.encode` + Examples -------- - >>> df = spark.createDataFrame([('abcd',)], ['a']) - >>> df.select(decode("a", "UTF-8")).show() - +----------------+ - |decode(a, UTF-8)| - +----------------+ - | abcd| - +----------------+ + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(b"\x61\x62\x63\x64",)], ["a"]) + >>> df.select("*", sf.decode("a", "UTF-8")).show() + +-------------+----------------+ + | a|decode(a, UTF-8)| + +-------------+----------------+ + |[61 62 63 64]| abcd| + +-------------+----------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -13641,9 +13725,9 @@ def encode(col: "ColumnOrName", charset: str) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. - charset : str + charset : literal string charset to use to encode. Returns @@ -13651,15 +13735,20 @@ def encode(col: "ColumnOrName", charset: str) -> Column: :class:`~pyspark.sql.Column` the column for computed results. + See Also + -------- + :meth:`pyspark.sql.functions.decode` + Examples -------- - >>> df = spark.createDataFrame([('abcd',)], ['c']) - >>> df.select(encode("c", "UTF-8")).show() - +----------------+ - |encode(c, UTF-8)| - +----------------+ - | [61 62 63 64]| - +----------------+ + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([("abcd",)], ["c"]) + >>> df.select("*", sf.encode("c", "UTF-8")).show() + +----+----------------+ + | c|encode(c, UTF-8)| + +----+----------------+ + |abcd| [61 62 63 64]| + +----+----------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -13675,7 +13764,7 @@ def is_valid_utf8(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name A column of strings, each representing a UTF-8 byte sequence. Returns @@ -13683,6 +13772,12 @@ def is_valid_utf8(str: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` whether the input string is a valid UTF-8 string. + See Also + -------- + :meth:`pyspark.sql.functions.make_valid_utf8` + :meth:`pyspark.sql.functions.validate_utf8` + :meth:`pyspark.sql.functions.try_validate_utf8` + Examples -------- >>> import pyspark.sql.functions as sf @@ -13706,7 +13801,7 @@ def make_valid_utf8(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name A column of strings, each representing a UTF-8 byte sequence. Returns @@ -13714,6 +13809,12 @@ def make_valid_utf8(str: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the valid UTF-8 version of the given input string. + See Also + -------- + :meth:`pyspark.sql.functions.is_valid_utf8` + :meth:`pyspark.sql.functions.validate_utf8` + :meth:`pyspark.sql.functions.try_validate_utf8` + Examples -------- >>> import pyspark.sql.functions as sf @@ -13736,7 +13837,7 @@ def validate_utf8(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name A column of strings, each representing a UTF-8 byte sequence. Returns @@ -13744,6 +13845,12 @@ def validate_utf8(str: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the input string if it is a valid UTF-8 string, error otherwise. + See Also + -------- + :meth:`pyspark.sql.functions.is_valid_utf8` + :meth:`pyspark.sql.functions.make_valid_utf8` + :meth:`pyspark.sql.functions.try_validate_utf8` + Examples -------- >>> import pyspark.sql.functions as sf @@ -13766,7 +13873,7 @@ def try_validate_utf8(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name A column of strings, each representing a UTF-8 byte sequence. Returns @@ -13774,6 +13881,12 @@ def try_validate_utf8(str: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the input string if it is a valid UTF-8 string, null otherwise. + See Also + -------- + :meth:`pyspark.sql.functions.is_valid_utf8` + :meth:`pyspark.sql.functions.make_valid_utf8` + :meth:`pyspark.sql.functions.validate_utf8` + Examples -------- >>> import pyspark.sql.functions as sf @@ -13800,7 +13913,7 @@ def format_number(col: "ColumnOrName", d: int) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name the column name of the numeric value to be formatted d : int the N decimal places @@ -13812,8 +13925,14 @@ def format_number(col: "ColumnOrName", d: int) -> Column: Examples -------- - >>> spark.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect() - [Row(v='5.0000')] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(5,)], ["a"]) + >>> df.select("*", sf.format_number("a", 4), sf.format_number(df.a, 6)).show() + +---+-------------------+-------------------+ + | a|format_number(a, 4)|format_number(a, 6)| + +---+-------------------+-------------------+ + | 5| 5.0000| 5.000000| + +---+-------------------+-------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -13832,9 +13951,9 @@ def format_string(format: str, *cols: "ColumnOrName") -> Column: Parameters ---------- - format : str + format : literal string string that can contain embedded format tags and used as result column's value - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name column names or :class:`~pyspark.sql.Column`\\s to be used in formatting Returns @@ -13842,11 +13961,20 @@ def format_string(format: str, *cols: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the column of formatted results. + See Also + -------- + :meth:`pyspark.sql.functions.printf` + Examples -------- - >>> df = spark.createDataFrame([(5, "hello")], ['a', 'b']) - >>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect() - [Row(v='5 hello')] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(5, "hello")], ["a", "b"]) + >>> df.select("*", sf.format_string('%d %s', "a", df.b)).show() + +---+-----+--------------------------+ + | a| b|format_string(%d %s, a, b)| + +---+-----+--------------------------+ + | 5|hello| 5 hello| + +---+-----+--------------------------+ """ from pyspark.sql.classic.column import _to_seq, _to_java_column @@ -13934,14 +14062,14 @@ def overlay( Parameters ---------- - src : :class:`~pyspark.sql.Column` or str - column name or column containing the string that will be replaced - replace : :class:`~pyspark.sql.Column` or str - column name or column containing the substitution string - pos : :class:`~pyspark.sql.Column` or str or int - column name, column, or int containing the starting position in src - len : :class:`~pyspark.sql.Column` or str or int, optional - column name, column, or int containing the number of bytes to replace in src + src : :class:`~pyspark.sql.Column` or column name + the string that will be replaced + replace : :class:`~pyspark.sql.Column` or column name + the substitution string + pos : :class:`~pyspark.sql.Column` or column name or int + the starting position in src + len : :class:`~pyspark.sql.Column` or column name or int, optional + the number of bytes to replace in src string by 'replace' defaults to -1, which represents the length of the 'replace' string Returns @@ -13951,13 +14079,28 @@ def overlay( Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("SPARK_SQL", "CORE")], ("x", "y")) - >>> df.select(overlay("x", "y", 7).alias("overlayed")).collect() - [Row(overlayed='SPARK_CORE')] - >>> df.select(overlay("x", "y", 7, 0).alias("overlayed")).collect() - [Row(overlayed='SPARK_CORESQL')] - >>> df.select(overlay("x", "y", 7, 2).alias("overlayed")).collect() - [Row(overlayed='SPARK_COREL')] + >>> df.select("*", sf.overlay("x", df.y, 7)).show() + +---------+----+--------------------+ + | x| y|overlay(x, y, 7, -1)| + +---------+----+--------------------+ + |SPARK_SQL|CORE| SPARK_CORE| + +---------+----+--------------------+ + + >>> df.select("*", sf.overlay("x", df.y, 7, 0)).show() + +---------+----+-------------------+ + | x| y|overlay(x, y, 7, 0)| + +---------+----+-------------------+ + |SPARK_SQL|CORE| SPARK_CORESQL| + +---------+----+-------------------+ + + >>> df.select("*", sf.overlay("x", "y", 7, 2)).show() + +---------+----+-------------------+ + | x| y|overlay(x, y, 7, 2)| + +---------+----+-------------------+ + |SPARK_SQL|CORE| SPARK_COREL| + +---------+----+-------------------+ """ pos = _enum_to_value(pos) if not isinstance(pos, (int, str, Column)): @@ -14011,11 +14154,11 @@ def sentences( Parameters ---------- - string : :class:`~pyspark.sql.Column` or str + string : :class:`~pyspark.sql.Column` or column name a string to be split - language : :class:`~pyspark.sql.Column` or str, optional + language : :class:`~pyspark.sql.Column` or column name, optional a language of the locale - country : :class:`~pyspark.sql.Column` or str, optional + country : :class:`~pyspark.sql.Column` or column name, optional a country of the locale Returns @@ -14025,26 +14168,28 @@ def sentences( Examples -------- - >>> df = spark.createDataFrame([["This is an example sentence."]], ["string"]) - >>> df.select(sentences(df.string, lit("en"), lit("US"))).show(truncate=False) - +-----------------------------------+ - |sentences(string, en, US) | - +-----------------------------------+ - |[[This, is, an, example, sentence]]| - +-----------------------------------+ - >>> df.select(sentences(df.string, lit("en"))).show(truncate=False) - +-----------------------------------+ - |sentences(string, en, ) | - +-----------------------------------+ - |[[This, is, an, example, sentence]]| - +-----------------------------------+ - >>> df = spark.createDataFrame([["Hello world. How are you?"]], ["s"]) - >>> df.select(sentences("s")).show(truncate=False) - +---------------------------------+ - |sentences(s, , ) | - +---------------------------------+ - |[[Hello, world], [How, are, you]]| - +---------------------------------+ + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([("This is an example sentence.", )], ["s"]) + >>> df.select("*", sf.sentences(df.s, sf.lit("en"), sf.lit("US"))).show(truncate=False) + +----------------------------+-----------------------------------+ + |s |sentences(s, en, US) | + +----------------------------+-----------------------------------+ + |This is an example sentence.|[[This, is, an, example, sentence]]| + +----------------------------+-----------------------------------+ + + >>> df.select("*", sf.sentences(df.s, sf.lit("en"))).show(truncate=False) + +----------------------------+-----------------------------------+ + |s |sentences(s, en, ) | + +----------------------------+-----------------------------------+ + |This is an example sentence.|[[This, is, an, example, sentence]]| + +----------------------------+-----------------------------------+ + + >>> df.select("*", sf.sentences(df.s)).show(truncate=False) + +----------------------------+-----------------------------------+ + |s |sentences(s, , ) | + +----------------------------+-----------------------------------+ + |This is an example sentence.|[[This, is, an, example, sentence]]| + +----------------------------+-----------------------------------+ """ if language is None: language = lit("") @@ -17404,6 +17549,7 @@ def concat(*cols: "ColumnOrName") -> Column: See Also -------- + :meth:`pyspark.sql.functions.concat_ws` :meth:`pyspark.sql.functions.array_join` : to concatenate string columns with delimiter Examples From 33f248ce2364b3cacfb830561bd299270358fcc7 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 2 Dec 2024 15:46:35 -0800 Subject: [PATCH 076/438] [SPARK-50458][CORE][SQL] Proper error handling for unsupported file system when reading files ### What changes were proposed in this pull request? Add FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM for reading a valid file but on an unsupported file system. ### Why are the changes needed? For these errors, we report ambiguous errors now. For example, querying json.`https://a/b/c.json`, will report UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY but json is actually supported. ### Does this PR introduce _any_ user-facing change? Yes, a meaningful error is given when visiting an unsupported file system. ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49016 from yaooqinn/SPARK-50458. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 5 ++++ .../org/apache/spark/util/HadoopFSUtils.scala | 7 ++++++ .../sql/execution/datasources/rules.scala | 5 ++++ .../analyzer-results/sql-on-files.sql.out | 22 +++++++++++++++++ .../sql-tests/inputs/sql-on-files.sql | 2 ++ .../sql-tests/results/sql-on-files.sql.out | 24 +++++++++++++++++++ .../datasources/DataSourceSuite.scala | 13 ++++++++++ 7 files changed, 78 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 632d43b4d1056..157989c09d098 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1476,6 +1476,11 @@ "message" : [ "Data type mismatches when reading Parquet column . Expected Spark type , actual Parquet type ." ] + }, + "UNSUPPORTED_FILE_SYSTEM" : { + "message" : [ + "The file system hasn't implemented ." + ] } }, "sqlState" : "KD001" diff --git a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala index f8f5bb4f72a40..5e50361b278aa 100644 --- a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala @@ -241,6 +241,13 @@ private[spark] object HadoopFSUtils extends Logging { logWarning(log"The directory ${MDC(PATH, path)} " + log"was not found. Was it deleted very recently?") Array.empty[FileStatus] + case u: UnsupportedOperationException => + throw new SparkUnsupportedOperationException( + errorClass = "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM", + messageParameters = Map( + "path" -> path.toString, + "fileSystemClass" -> fs.getClass.getName, + "method" -> u.getStackTrace.head.getMethodName)) } val filteredStatuses = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index fcc3a257cd2dd..713161cc49ce8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -22,6 +22,7 @@ import java.util.Locale import scala.collection.mutable.{HashMap, HashSet} import scala.jdk.CollectionConverters._ +import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ @@ -54,6 +55,10 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { val ds = resolveDataSource(u) Some(LogicalRelation(ds.resolveRelation())) } catch { + case e: SparkUnsupportedOperationException => + u.failAnalysis( + errorClass = e.getCondition, + messageParameters = e.getMessageParameters.asScala.toMap) case _: ClassNotFoundException => None case e: Exception if !e.isInstanceOf[AnalysisException] => // the provider is valid, but failed to create a logical plan diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out index b098a9758fe4e..329fbd49b92d8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out @@ -231,3 +231,25 @@ DROP DATABASE sql_on_files -- !query analysis DropNamespace false, false +- ResolvedNamespace V2SessionCatalog(spark_catalog), [sql_on_files] + + +-- !query +SELECT * FROM json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM", + "sqlState" : "KD001", + "messageParameters" : { + "fileSystemClass" : "org.apache.hadoop.fs.http.HttpsFileSystem", + "method" : "listStatus", + "path" : "https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 128, + "fragment" : "json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql index 8a00e4400e6b0..c3a16ca577ee9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql @@ -28,3 +28,5 @@ SELECT * FROM json.`${spark.sql.warehouse.dir}/sql_on_files.db/test_json`; DROP TABLE sql_on_files.test_json; DROP DATABASE sql_on_files; + +SELECT * FROM json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`; diff --git a/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out index fc8f44bc22fee..71d4216ea2078 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out @@ -257,3 +257,27 @@ DROP DATABASE sql_on_files struct<> -- !query output + + +-- !query +SELECT * FROM json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM", + "sqlState" : "KD001", + "messageParameters" : { + "fileSystemClass" : "org.apache.hadoop.fs.http.HttpsFileSystem", + "method" : "listStatus", + "path" : "https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 128, + "fragment" : "json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`" + } ] +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala index fd9d31e7a594d..d2acdcfc62053 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -207,6 +208,18 @@ class DataSourceSuite extends SharedSparkSession with PrivateMethodTester { Utils.deleteRecursively(baseDir) } } + + test("SPARK-50458: Proper error handling for unsupported file system") { + val loc = "https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/" + + "src/main/resources/employees.json" + checkError(exception = intercept[SparkUnsupportedOperationException]( + sql(s"CREATE TABLE HTTP USING JSON LOCATION '$loc'")), + condition = "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM", + parameters = Map( + "path" -> loc, + "fileSystemClass" -> "org.apache.hadoop.fs.http.HttpsFileSystem", + "method" -> "listStatus")) + } } object TestPaths { From 75397c9207b77eb6406f22b21ca4fb991232b9d9 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 2 Dec 2024 15:57:23 -0800 Subject: [PATCH 077/438] [MINOR][INFRA] Add a space in the JIRA user assignment message ### What changes were proposed in this pull request? This PR proposes to add a space at the message at the merge script below: ``` Enter number of user, or userid, to assign to (blank to leave unassigned):0 ``` ### Why are the changes needed? To be consistent with other messages in this script. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49034 from HyukjinKwon/minor-add-space. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- dev/merge_spark_pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 4ebd3e4b951f5..415f468a11577 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -419,7 +419,7 @@ def choose_jira_assignee(issue): annotations.append("Commentator") print("[%d] %s (%s)" % (idx, author.displayName, ",".join(annotations))) raw_assignee = bold_input( - "Enter number of user, or userid, to assign to (blank to leave unassigned):" + "Enter number of user, or userid, to assign to (blank to leave unassigned): " ) if raw_assignee == "": return None From 2041519cd9d99eb11e78477f66b7b712de1b06e1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 Dec 2024 08:58:14 +0900 Subject: [PATCH 078/438] [SPARK-47993][DOCS][FOLLOWUP] Update RDD programing guide to Python 3.9+ ### What changes were proposed in this pull request? This is a follow-up of - #46228 ### Why are the changes needed? To update the RDD programing guide consistently. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49035 from dongjoon-hyun/SPARK-47993. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- docs/rdd-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index a1adcc2f6eb03..400f8a512e7a7 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -39,7 +39,7 @@ along with if you launch Spark's interactive shell -- either `bin/spark-shell` f
-Spark {{site.SPARK_VERSION}} works with Python 3.8+. It can use the standard CPython interpreter, +Spark {{site.SPARK_VERSION}} works with Python 3.9+. It can use the standard CPython interpreter, so C libraries like NumPy can be used. It also works with PyPy 7.3.6+. Spark applications in Python can either be run with the `bin/spark-submit` script which includes Spark at runtime, or by including it in your setup.py as: From 4abaab3ffeba5a3d39216e7224928bb82b254e22 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 3 Dec 2024 15:37:43 +0900 Subject: [PATCH 079/438] [SPARK-50430][CORE][FOLLOW-UP] Keep the logic of manual putting key and values in Properties ### What changes were proposed in this pull request? This PR proposes to actually more conservatively preserve the original code of creating new properties instead of cloning. ### Why are the changes needed? Previous codes only copied the key and values but `clone` actually copies more fields in `Properties`. `cloneProperties` is being used in Spark Core, and all other components so I propose to keep the logic as is. ### Does this PR introduce _any_ user-facing change? This is more a fix of a potential bug. ### How was this patch tested? No, it is difficult to add a test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49036 from HyukjinKwon/SPARK-50430-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 109db36d40695..b2cf99241fdee 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2982,7 +2982,9 @@ private[spark] object Utils if (props == null) { return props } - props.clone().asInstanceOf[Properties] + val resultProps = new Properties() + resultProps.putAll(props.clone().asInstanceOf[Properties]) + resultProps } /** From 38ab95bed4eb88feed372ddfb209ca4c8c3a2be3 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 3 Dec 2024 19:55:31 +0800 Subject: [PATCH 080/438] [SPARK-50467][PYTHON] Add `__all__` for builtin functions ### What changes were proposed in this pull request? Add `__all__` for builtin function ### Why are the changes needed? to control the wild import ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci and manual test ### Was this patch authored or co-authored using generative AI tooling? no Closes #49024 from zhengruifeng/py_func_all. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/__init__.py | 484 +++++++++++++++++++++++ 1 file changed, 484 insertions(+) diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py index dd09c4aa5c774..98db2a7b091dd 100644 --- a/python/pyspark/sql/functions/__init__.py +++ b/python/pyspark/sql/functions/__init__.py @@ -19,3 +19,487 @@ from pyspark.sql.functions.builtin import * # noqa: F401,F403 from pyspark.sql.functions import partitioning # noqa: F401,F403 + +__all__ = [ # noqa: F405 + # Normal functions + "broadcast", + "call_function", + "col", + "column", + "lit", + "expr", + # Conditional Functions + "coalesce", + "ifnull", + "nanvl", + "nullif", + "nullifzero", + "nvl", + "nvl2", + "when", + "zeroifnull", + # Predicate Functions + "equal_null", + "ilike", + "isnan", + "isnotnull", + "isnull", + "like", + "regexp", + "regexp_like", + "rlike", + # Sort Functions + "asc", + "asc_nulls_first", + "asc_nulls_last", + "desc", + "desc_nulls_first", + "desc_nulls_last", + # Mathematical Functions + "abs", + "acos", + "acosh", + "asin", + "asinh", + "atan", + "atan2", + "atanh", + "bin", + "bround", + "cbrt", + "ceil", + "ceiling", + "conv", + "cos", + "cosh", + "cot", + "csc", + "degrees", + "e", + "exp", + "expm1", + "factorial", + "floor", + "greatest", + "hex", + "hypot", + "least", + "ln", + "log", + "log10", + "log1p", + "log2", + "negate", + "negative", + "pi", + "pmod", + "positive", + "pow", + "power", + "radians", + "rand", + "randn", + "rint", + "round", + "sec", + "sign", + "signum", + "sin", + "sinh", + "sqrt", + "tan", + "tanh", + "try_add", + "try_divide", + "try_mod", + "try_multiply", + "try_subtract", + "unhex", + "uniform", + "width_bucket", + # String Functions + "ascii", + "base64", + "bit_length", + "btrim", + "char", + "char_length", + "character_length", + "collate", + "collation", + "concat_ws", + "contains", + "decode", + "elt", + "encode", + "endswith", + "find_in_set", + "format_number", + "format_string", + "initcap", + "instr", + "is_valid_utf8", + "lcase", + "left", + "length", + "levenshtein", + "locate", + "lower", + "lpad", + "ltrim", + "make_valid_utf8", + "mask", + "octet_length", + "overlay", + "position", + "printf", + "randstr", + "regexp_count", + "regexp_extract", + "regexp_extract_all", + "regexp_instr", + "regexp_replace", + "regexp_substr", + "repeat", + "replace", + "right", + "rpad", + "rtrim", + "sentences", + "soundex", + "split", + "split_part", + "startswith", + "substr", + "substring", + "substring_index", + "to_binary", + "to_char", + "to_number", + "to_varchar", + "translate", + "trim", + "try_to_binary", + "try_to_number", + "try_validate_utf8", + "ucase", + "unbase64", + "upper", + "validate_utf8", + # Bitwise Functions + "bit_count", + "bit_get", + "bitwise_not", + "getbit", + "shiftleft", + "shiftright", + "shiftrightunsigned", + # Date and Timestamp Functions + "add_months", + "convert_timezone", + "curdate", + "current_date", + "current_timestamp", + "current_timezone", + "date_add", + "date_diff", + "date_format", + "date_from_unix_date", + "date_part", + "date_sub", + "date_trunc", + "dateadd", + "datediff", + "datepart", + "day", + "dayname", + "dayofmonth", + "dayofweek", + "dayofyear", + "extract", + "from_unixtime", + "from_utc_timestamp", + "hour", + "last_day", + "localtimestamp", + "make_date", + "make_dt_interval", + "make_interval", + "make_timestamp", + "make_timestamp_ltz", + "make_timestamp_ntz", + "make_ym_interval", + "minute", + "month", + "monthname", + "months_between", + "next_day", + "now", + "quarter", + "second", + "session_window", + "timestamp_add", + "timestamp_diff", + "timestamp_micros", + "timestamp_millis", + "timestamp_seconds", + "to_date", + "to_timestamp", + "to_timestamp_ltz", + "to_timestamp_ntz", + "to_unix_timestamp", + "to_utc_timestamp", + "trunc", + "try_make_interval", + "try_make_timestamp", + "try_make_timestamp_ltz", + "try_make_timestamp_ntz", + "try_to_timestamp", + "unix_date", + "unix_micros", + "unix_millis", + "unix_seconds", + "unix_timestamp", + "weekday", + "weekofyear", + "window", + "window_time", + "year", + # Hash Functions + "crc32", + "hash", + "md5", + "sha", + "sha1", + "sha2", + "xxhash64", + # Collection Functions + "aggregate", + "array_sort", + "cardinality", + "concat", + "element_at", + "exists", + "filter", + "forall", + "map_filter", + "map_zip_with", + "reduce", + "reverse", + "size", + "transform", + "transform_keys", + "transform_values", + "try_element_at", + "zip_with", + # Array Functions + "array", + "array_append", + "array_compact", + "array_contains", + "array_distinct", + "array_except", + "array_insert", + "array_intersect", + "array_join", + "array_max", + "array_min", + "array_position", + "array_prepend", + "array_remove", + "array_repeat", + "array_size", + "array_union", + "arrays_overlap", + "arrays_zip", + "flatten", + "get", + "sequence", + "shuffle", + "slice", + "sort_array", + # Struct Functions + "named_struct", + "struct", + # Map Functions + "create_map", + "map_concat", + "map_contains_key", + "map_entries", + "map_from_arrays", + "map_from_entries", + "map_keys", + "map_values", + "str_to_map", + # Aggregate Functions + "any_value", + "approx_count_distinct", + "approx_percentile", + "array_agg", + "avg", + "bit_and", + "bit_or", + "bit_xor", + "bitmap_construct_agg", + "bitmap_or_agg", + "bool_and", + "bool_or", + "collect_list", + "collect_set", + "corr", + "count", + "count_distinct", + "count_if", + "count_min_sketch", + "covar_pop", + "covar_samp", + "every", + "first", + "first_value", + "grouping", + "grouping_id", + "histogram_numeric", + "hll_sketch_agg", + "hll_union_agg", + "kurtosis", + "last", + "last_value", + "max", + "max_by", + "mean", + "median", + "min", + "min_by", + "mode", + "percentile", + "percentile_approx", + "product", + "regr_avgx", + "regr_avgy", + "regr_count", + "regr_intercept", + "regr_r2", + "regr_slope", + "regr_sxx", + "regr_sxy", + "regr_syy", + "skewness", + "some", + "std", + "stddev", + "stddev_pop", + "stddev_samp", + "sum", + "sum_distinct", + "try_avg", + "try_sum", + "var_pop", + "var_samp", + "variance", + # Window Functions + "cume_dist", + "dense_rank", + "lag", + "lead", + "nth_value", + "ntile", + "percent_rank", + "rank", + "row_number", + # Generator Functions + "explode", + "explode_outer", + "inline", + "inline_outer", + "posexplode", + "posexplode_outer", + "stack", + # Partition Transformation Functions + "years", + "months", + "days", + "hours", + "bucket", + # CSV Functions + "from_csv", + "schema_of_csv", + "to_csv", + # JSON Functions + "from_json", + "get_json_object", + "json_array_length", + "json_object_keys", + "json_tuple", + "schema_of_json", + "to_json", + # VARIANT Functions + "is_variant_null", + "parse_json", + "schema_of_variant", + "schema_of_variant_agg", + "try_variant_get", + "variant_get", + "try_parse_json", + "to_variant_object", + # XML Functions + "from_xml", + "schema_of_xml", + "to_xml", + "xpath", + "xpath_boolean", + "xpath_double", + "xpath_float", + "xpath_int", + "xpath_long", + "xpath_number", + "xpath_short", + "xpath_string", + # URL Functions + "parse_url", + "try_parse_url", + "url_decode", + "url_encode", + "try_url_decode", + # Misc Functions + "aes_decrypt", + "aes_encrypt", + "assert_true", + "bitmap_bit_position", + "bitmap_bucket_number", + "bitmap_count", + "current_catalog", + "current_database", + "current_schema", + "current_user", + "hll_sketch_estimate", + "hll_union", + "input_file_block_length", + "input_file_block_start", + "input_file_name", + "java_method", + "monotonically_increasing_id", + "raise_error", + "reflect", + "session_user", + "spark_partition_id", + "try_aes_decrypt", + "try_reflect", + "typeof", + "user", + "version", + # UDF, UDTF and UDT + "AnalyzeArgument", + "AnalyzeResult", + "OrderingColumn", + "PandasUDFType", + "PartitioningColumn", + "SelectedColumn", + "SkipRestOfInputTableException", + "UserDefinedFunction", + "UserDefinedTableFunction", + "call_udf", + "pandas_udf", + "udf", + "udtf", + "unwrap_udt", +] From f572ad1a722aef2aa1e06fc85aad168ed542cee9 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 3 Dec 2024 21:37:49 +0800 Subject: [PATCH 081/438] [SPARK-50474][BUILD] Upgrade checkstyle to 10.20.2 ### What changes were proposed in this pull request? This pr upgrades `checkstyle` from version 10.20.0 to 10.20.2, and simultaneously updates the Guava library used by `checkstyle` in `project/plugins.sbt` to version `33.3.1-jre`. This is due to the fact https://github.com/checkstyle/checkstyle/blob/checkstyle-10.20.2/pom.xml#L291-L301 ![image](https://github.com/user-attachments/assets/903c02c1-fd07-4efe-8e13-8bdfd479be03) ### Why are the changes needed? The new version will bring some bug fixes: - https://checkstyle.org/releasenotes.html#Release_10.20.1 - https://checkstyle.org/releasenotes.html#Release_10.20.2 ### Does this PR introduce _any_ user-facing change? No, dev only ### How was this patch tested? Manually by: ``` ./dev/lint-java ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #49040 from LuciferYang/checkstyle-10.20.2. Authored-by: yangjie01 Signed-off-by: yangjie01 --- pom.xml | 2 +- project/plugins.sbt | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 06c3eca083bdb..807b08847ebd6 100644 --- a/pom.xml +++ b/pom.xml @@ -3225,7 +3225,7 @@ --> com.puppycrawl.tools checkstyle - 10.20.0 + 10.20.2 diff --git a/project/plugins.sbt b/project/plugins.sbt index 8ae03c0995132..2f5fcf79740bf 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,10 +19,10 @@ addSbtPlugin("software.purpledragon" % "sbt-checkstyle-plugin" % "4.0.1") // If you are changing the dependency setting for checkstyle plugin, // please check pom.xml in the root of the source tree too. -libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "10.20.0" +libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "10.20.2" -// checkstyle uses guava 33.1.0-jre. -libraryDependencies += "com.google.guava" % "guava" % "33.1.0-jre" +// checkstyle uses guava 33.3.1-jre. +libraryDependencies += "com.google.guava" % "guava" % "33.3.1-jre" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "2.3.0") From 98e94af58f1614e74b040057e71bd6402b85e67c Mon Sep 17 00:00:00 2001 From: cashmand Date: Tue, 3 Dec 2024 23:08:22 +0800 Subject: [PATCH 082/438] [SPARK-48898][SQL] Fix Variant shredding bug ### What changes were proposed in this pull request? In VariantShreddingWriter, there are two calls to `variantBuilder.appendVariant` that were left over from an earlier version of the shredding spec where we constructed new metadata for every shredded value. This method rebuilds the Variant value to refer to the new metadata dictionary in the builder, so we should not be using it in shredding, where all dictionary IDs now refer to the original Variant metadata. 1) When writing a Variant value that does not match the shredding type. The code was doing the right thing, but unnecessarily calling `variantBuilder.appendVariant` and then discarding the result. The PR removes that dead code. 2) When reconstructing a Variant object that contains only the fields of the original object that don't appear in the shredding schema. This is a correctness bug, since we would modify the value to use new dictionary IDs that do not correspond to the ones in the original metadata. ### Why are the changes needed? Variant shredding correctness. ### Does this PR introduce _any_ user-facing change? No, shredding has not yet been released. ### How was this patch tested? Added a unit test that fails without the fix. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49031 from cashmand/SPARK-48898-bugfix. Authored-by: cashmand Signed-off-by: Wenchen Fan --- .../spark/types/variant/VariantShreddingWriter.java | 6 +++--- .../apache/spark/sql/VariantWriteShreddingSuite.scala | 11 +++++++++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/common/variant/src/main/java/org/apache/spark/types/variant/VariantShreddingWriter.java b/common/variant/src/main/java/org/apache/spark/types/variant/VariantShreddingWriter.java index b5f8ea0a1484b..bbee7ee0dca38 100644 --- a/common/variant/src/main/java/org/apache/spark/types/variant/VariantShreddingWriter.java +++ b/common/variant/src/main/java/org/apache/spark/types/variant/VariantShreddingWriter.java @@ -101,7 +101,9 @@ public static ShreddedResult castShredded( int id = v.getDictionaryIdAtIndex(i); fieldEntries.add(new VariantBuilder.FieldEntry( field.key, id, variantBuilder.getWritePos() - start)); - variantBuilder.appendVariant(field.value); + // shallowAppendVariant is needed for correctness, since we're relying on the metadata IDs + // being unchanged. + variantBuilder.shallowAppendVariant(field.value); } } if (numFieldsMatched < objectSchema.length) { @@ -133,8 +135,6 @@ public static ShreddedResult castShredded( // Store the typed value. result.addScalar(typedValue); } else { - VariantBuilder variantBuilder = new VariantBuilder(false); - variantBuilder.appendVariant(v); result.addVariantValue(v.getValue()); } } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala index ed66ddb1f0f44..a62c6e4462464 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala @@ -179,6 +179,17 @@ class VariantWriteShreddingSuite extends SparkFunSuite with ExpressionEvalHelper // Not an object testWithSchema(obj, ArrayType(StructType.fromDDL("a int, b string")), Row(obj.getMetadata, untypedValue(obj), null)) + + // Similar to the case above where "b" was not in the shredding schema, but with the unshredded + // value being an object. Check that the copied value has correct dictionary IDs. + val obj2 = parseJson("""{"a": 1, "b": {"c": "hello"}}""") + val residual2 = untypedValue("""{"b": {"c": "hello"}}""") + // First byte is the type, second is number of fields, and the third is the ID for "b" + residual2(2) = 1 + // Followed by 2 bytes for offsets, inner object type and number of fields, then ID for "c". + residual2(7) = 2 + testWithSchema(obj2, StructType.fromDDL("a int, c string"), + Row(obj2.getMetadata, residual2, Row(Row(null, 1), Row(null, null)))) } test("shredding as array") { From d427aa3c6f473e33c60eb9f8c7211dfb76cc3e3d Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 3 Dec 2024 11:42:21 -0400 Subject: [PATCH 083/438] [SPARK-50473][SQL] Simplify classic Column handling ### What changes were proposed in this pull request? We added a couple of helper functions that make it easier to work Columns in the Classic API. This covers functionality that creates a Column from an Expression, and creating (named) Expressions from a Column. There are currently multiple ways of doing the same thing and this is confusing. This PR attempts to simplify this a bit, by making the following changes: - `ExpressionUtils` is moved to the background. `ClassicConversions`/`ColumnConversion` are now predominantly used. The benefit of this is that most code now looks like pre-Spark 4 code. - `ExpressionUtils.expression(..)` and `ExpressionUtils.column(..)` are not implicit anymore. This was confusing. - `testImplicits` now supports both Expression -> Column and Column -> Expression conversions. ### Why are the changes needed? Easier to understand code. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49038 from hvanhovell/SPARK-50473. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../CheckConnectJvmClientCompatibility.scala | 11 ++-- .../org/apache/spark/ml/stat/Summarizer.scala | 11 ++-- .../connect/planner/SparkConnectPlanner.scala | 56 +++++++++---------- .../spark/sql/DataFrameNaFunctions.scala | 2 +- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../spark/sql/RelationalGroupedDataset.scala | 6 +- .../org/apache/spark/sql/SparkSession.scala | 26 +++------ .../spark/sql/api/python/PythonSQLUtils.scala | 6 +- ...sicConversions.scala => conversions.scala} | 54 +++++++++++++++++- .../python/UserDefinedPythonFunction.scala | 7 ++- .../sql/execution/stat/FrequentItems.scala | 8 ++- .../sql/internal/MergeIntoWriterImpl.scala | 2 +- .../sql/internal/columnNodeSupport.scala | 11 ++-- .../spark/sql/DataFrameComplexTypeSuite.scala | 5 +- .../spark/sql/DataFrameFunctionsSuite.scala | 7 +-- .../spark/sql/DataFrameSelfJoinSuite.scala | 7 +-- .../org/apache/spark/sql/DataFrameSuite.scala | 11 ++-- .../sql/DataFrameWindowFunctionsSuite.scala | 3 +- .../spark/sql/IntegratedUDFTestUtils.scala | 5 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 3 +- .../sql/TypedImperativeAggregateSuite.scala | 8 +-- .../functions/V2FunctionBenchmark.scala | 8 ++- .../datasources/orc/OrcFilterSuite.scala | 9 ++- .../execution/datasources/orc/OrcTest.scala | 6 +- .../datasources/orc/OrcV1FilterSuite.scala | 12 ++-- .../parquet/ParquetFilterSuite.scala | 14 +++-- .../spark/sql/sources/BucketedReadSuite.scala | 3 +- .../apache/spark/sql/test/SQLTestUtils.scala | 10 +++- .../ObjectHashAggregateExecBenchmark.scala | 5 +- .../OptimizeHiveMetadataOnlyQuerySuite.scala | 2 +- .../execution/ObjectHashAggregateSuite.scala | 5 +- 31 files changed, 187 insertions(+), 138 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/classic/{ClassicConversions.scala => conversions.scala} (56%) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index d9ff8d9122ead..b5ea973aa1d70 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -233,9 +233,11 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.artifact.ArtifactManager$SparkContextResourceType$"), // ColumnNode conversions + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.SparkSession"), ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.sql.SparkSession.Converter"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SparkSession$Converter$"), + "org.apache.spark.sql.SparkSession.expression"), + ProblemFilters.exclude[DirectMissingMethodProblem]( + "org.apache.spark.sql.SparkSession.toRichColumn"), // UDFRegistration ProblemFilters.exclude[DirectMissingMethodProblem]( @@ -295,10 +297,9 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.KeyValueGroupedDatasetImpl$"), // ColumnNode conversions - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.sql.SparkSession#RichColumn.expr"), ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.sql.SparkSession#RichColumn.typedExpr"), + "org.apache.spark.sql.SparkSession.RichColumn"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SparkSession$RichColumn"), // New public APIs added in the client // Dataset diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala index 4c3242c132090..e67b72e090601 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala @@ -29,8 +29,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, ImplicitCastInputTypes} import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate import org.apache.spark.sql.catalyst.trees.BinaryLike +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.internal.ExpressionUtils.{column, expression} +import org.apache.spark.sql.internal.ExpressionUtils.expression import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -249,13 +250,13 @@ private[ml] class SummaryBuilderImpl( ) extends SummaryBuilder { override def summary(featuresCol: Column, weightCol: Column): Column = { - SummaryBuilderImpl.MetricsAggregate( + Column(SummaryBuilderImpl.MetricsAggregate( requestedMetrics, requestedCompMetrics, - featuresCol, - weightCol, + expression(featuresCol), + expression(weightCol), mutableAggBufferOffset = 0, - inputAggBufferOffset = 0) + inputAggBufferOffset = 0)) } } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index ee030a52b221a..6ecf3ce110389 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -43,7 +43,7 @@ import org.apache.spark.connect.proto.WriteStreamOperationStart.TriggerCase import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} -import org.apache.spark.sql.{Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} +import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} @@ -58,6 +58,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry @@ -77,7 +78,6 @@ import org.apache.spark.sql.execution.streaming.GroupStateImpl.groupStateTimeout import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper import org.apache.spark.sql.expressions.{Aggregator, ReduceAggregator, SparkUserDefinedFunction, UserDefinedAggregator, UserDefinedFunction} import org.apache.spark.sql.internal.{CatalogImpl, MergeIntoWriterImpl, TypedAggUtils} -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode, StreamingQuery, StreamingQueryListener, StreamingQueryProgress, Trigger} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -106,7 +106,7 @@ class SparkConnectPlanner( @Since("4.0.0") @DeveloperApi def session: SparkSession = sessionHolder.session - import sessionHolder.session.RichColumn + import sessionHolder.session.toRichColumn private[connect] def parser = session.sessionState.sqlParser @@ -554,7 +554,7 @@ class SparkConnectPlanner( .ofRows(session, transformRelation(rel.getInput)) .stat .sampleBy( - col = column(transformExpression(rel.getCol)), + col = Column(transformExpression(rel.getCol)), fractions = fractions.toMap, seed = if (rel.hasSeed) rel.getSeed else Utils.random.nextLong) .logicalPlan @@ -646,17 +646,17 @@ class SparkConnectPlanner( val pythonUdf = transformPythonUDF(commonUdf) val cols = rel.getGroupingExpressionsList.asScala.toSeq.map(expr => - column(transformExpression(expr))) + Column(transformExpression(expr))) val group = Dataset .ofRows(session, transformRelation(rel.getInput)) .groupBy(cols: _*) pythonUdf.evalType match { case PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF => - group.flatMapGroupsInPandas(column(pythonUdf)).logicalPlan + group.flatMapGroupsInPandas(Column(pythonUdf)).logicalPlan case PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF => - group.flatMapGroupsInArrow(column(pythonUdf)).logicalPlan + group.flatMapGroupsInArrow(Column(pythonUdf)).logicalPlan case _ => throw InvalidPlanInput( @@ -765,10 +765,10 @@ class SparkConnectPlanner( case proto.CommonInlineUserDefinedFunction.FunctionCase.PYTHON_UDF => val inputCols = rel.getInputGroupingExpressionsList.asScala.toSeq.map(expr => - column(transformExpression(expr))) + Column(transformExpression(expr))) val otherCols = rel.getOtherGroupingExpressionsList.asScala.toSeq.map(expr => - column(transformExpression(expr))) + Column(transformExpression(expr))) val input = Dataset .ofRows(session, transformRelation(rel.getInput)) @@ -783,10 +783,10 @@ class SparkConnectPlanner( pythonUdf.evalType match { case PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF => - input.flatMapCoGroupsInPandas(other, pythonUdf).logicalPlan + input.flatMapCoGroupsInPandas(other, Column(pythonUdf)).logicalPlan case PythonEvalType.SQL_COGROUPED_MAP_ARROW_UDF => - input.flatMapCoGroupsInArrow(other, pythonUdf).logicalPlan + input.flatMapCoGroupsInArrow(other, Column(pythonUdf)).logicalPlan case _ => throw InvalidPlanInput( @@ -982,7 +982,7 @@ class SparkConnectPlanner( private def transformApplyInPandasWithState(rel: proto.ApplyInPandasWithState): LogicalPlan = { val pythonUdf = transformPythonUDF(rel.getFunc) val cols = - rel.getGroupingExpressionsList.asScala.toSeq.map(expr => column(transformExpression(expr))) + rel.getGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) val outputSchema = parseSchema(rel.getOutputSchema) @@ -992,7 +992,7 @@ class SparkConnectPlanner( .ofRows(session, transformRelation(rel.getInput)) .groupBy(cols: _*) .applyInPandasWithState( - column(pythonUdf), + Column(pythonUdf), outputSchema, stateSchema, rel.getOutputMode, @@ -1080,7 +1080,7 @@ class SparkConnectPlanner( Metadata.empty } - (alias.getName(0), column(transformExpression(alias.getExpr)), metadata) + (alias.getName(0), Column(transformExpression(alias.getExpr)), metadata) }.unzip3 Dataset @@ -1142,7 +1142,7 @@ class SparkConnectPlanner( private def transformUnpivot(rel: proto.Unpivot): LogicalPlan = { val ids = rel.getIdsList.asScala.toArray.map { expr => - column(transformExpression(expr)) + Column(transformExpression(expr)) } if (!rel.hasValues) { @@ -1155,7 +1155,7 @@ class SparkConnectPlanner( transformRelation(rel.getInput)) } else { val values = rel.getValues.getValuesList.asScala.toArray.map { expr => - column(transformExpression(expr)) + Column(transformExpression(expr)) } Unpivot( @@ -1184,7 +1184,7 @@ class SparkConnectPlanner( private def transformCollectMetrics(rel: proto.CollectMetrics, planId: Long): LogicalPlan = { val metrics = rel.getMetricsList.asScala.toSeq.map { expr => - column(transformExpression(expr)) + Column(transformExpression(expr)) } val name = rel.getName val input = transformRelation(rel.getInput) @@ -2112,10 +2112,10 @@ class SparkConnectPlanner( private def transformAsOfJoin(rel: proto.AsOfJoin): LogicalPlan = { val left = Dataset.ofRows(session, transformRelation(rel.getLeft)) val right = Dataset.ofRows(session, transformRelation(rel.getRight)) - val leftAsOf = column(transformExpression(rel.getLeftAsOf)) - val rightAsOf = column(transformExpression(rel.getRightAsOf)) + val leftAsOf = Column(transformExpression(rel.getLeftAsOf)) + val rightAsOf = Column(transformExpression(rel.getRightAsOf)) val joinType = rel.getJoinType - val tolerance = if (rel.hasTolerance) column(transformExpression(rel.getTolerance)) else null + val tolerance = if (rel.hasTolerance) Column(transformExpression(rel.getTolerance)) else null val allowExactMatches = rel.getAllowExactMatches val direction = rel.getDirection @@ -2131,7 +2131,7 @@ class SparkConnectPlanner( allowExactMatches = allowExactMatches, direction = direction) } else { - val joinExprs = if (rel.hasJoinExpr) column(transformExpression(rel.getJoinExpr)) else null + val joinExprs = if (rel.hasJoinExpr) Column(transformExpression(rel.getJoinExpr)) else null left.joinAsOf( other = right, leftAsOf = leftAsOf, @@ -2172,7 +2172,7 @@ class SparkConnectPlanner( private def transformDrop(rel: proto.Drop): LogicalPlan = { var output = Dataset.ofRows(session, transformRelation(rel.getInput)) if (rel.getColumnsCount > 0) { - val cols = rel.getColumnsList.asScala.toSeq.map(expr => column(transformExpression(expr))) + val cols = rel.getColumnsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) output = output.drop(cols.head, cols.tail: _*) } if (rel.getColumnNamesCount > 0) { @@ -2247,7 +2247,7 @@ class SparkConnectPlanner( rel.getPivot.getValuesList.asScala.toSeq.map(transformLiteral) } else { RelationalGroupedDataset - .collectPivotValues(Dataset.ofRows(session, input), column(pivotExpr)) + .collectPivotValues(Dataset.ofRows(session, input), Column(pivotExpr)) .map(expressions.Literal.apply) } logical.Pivot( @@ -2574,12 +2574,12 @@ class SparkConnectPlanner( if (!namedArguments.isEmpty) { session.sql( sql.getQuery, - namedArguments.asScala.toMap.transform((_, e) => column(transformExpression(e))), + namedArguments.asScala.toMap.transform((_, e) => Column(transformExpression(e))), tracker) } else if (!posArguments.isEmpty) { session.sql( sql.getQuery, - posArguments.asScala.map(e => column(transformExpression(e))).toArray, + posArguments.asScala.map(e => Column(transformExpression(e))).toArray, tracker) } else if (!args.isEmpty) { session.sql( @@ -2830,7 +2830,7 @@ class SparkConnectPlanner( if (writeOperation.getPartitioningColumnsCount > 0) { val names = writeOperation.getPartitioningColumnsList.asScala .map(transformExpression) - .map(column) + .map(Column(_)) .toSeq w.partitionedBy(names.head, names.tail: _*) } @@ -2848,7 +2848,7 @@ class SparkConnectPlanner( w.create() } case proto.WriteOperationV2.Mode.MODE_OVERWRITE => - w.overwrite(column(transformExpression(writeOperation.getOverwriteCondition))) + w.overwrite(Column(transformExpression(writeOperation.getOverwriteCondition))) case proto.WriteOperationV2.Mode.MODE_OVERWRITE_PARTITIONS => w.overwritePartitions() case proto.WriteOperationV2.Mode.MODE_APPEND => @@ -3410,7 +3410,7 @@ class SparkConnectPlanner( val sourceDs = Dataset.ofRows(session, transformRelation(cmd.getSourceTablePlan)) val mergeInto = sourceDs - .mergeInto(cmd.getTargetTableName, column(transformExpression(cmd.getMergeCondition))) + .mergeInto(cmd.getTargetTableName, Column(transformExpression(cmd.getMergeCondition))) .asInstanceOf[MergeIntoWriterImpl[Row]] mergeInto.matchedActions ++= matchedActions mergeInto.notMatchedActions ++= notMatchedActions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 53e12f58edd69..0d49e850b4637 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.types._ @Stable final class DataFrameNaFunctions private[sql](df: DataFrame) extends api.DataFrameNaFunctions { - import df.sparkSession.RichColumn + import df.sparkSession.toRichColumn protected def drop(minNonNulls: Option[Int]): Dataset[Row] = { drop0(minNonNulls, outputAttributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index a74d93b44db98..846d97b257869 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -225,7 +225,7 @@ class Dataset[T] private[sql]( queryExecution.sparkSession } - import sparkSession.RichColumn + import sparkSession.toRichColumn // A globally unique id of this Dataset. private[sql] val id = Dataset.curId.getAndIncrement() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 6f0db42ec1f5e..b8c4b03fc13d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.internal.ExpressionUtils.{column, generateAlias} +import org.apache.spark.sql.internal.ExpressionUtils.generateAlias import org.apache.spark.sql.internal.TypedAggUtils.withInputType import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{NumericType, StructType} @@ -114,7 +114,7 @@ class RelationalGroupedDataset protected[sql]( namedExpr } } - columnExprs.map(column) + columnExprs.map(Column(_)) } /** @inheritdoc */ @@ -238,7 +238,7 @@ class RelationalGroupedDataset protected[sql]( broadcastVars: Array[Broadcast[Object]], outputSchema: StructType): DataFrame = { val groupingNamedExpressions = groupingExprs.map(alias) - val groupingCols = groupingNamedExpressions.map(column) + val groupingCols = groupingNamedExpressions.map(Column(_)) val groupingDataFrame = df.select(groupingCols : _*) val groupingAttributes = groupingNamedExpressions.map(_.toAttribute) Dataset.ofRows( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index dbe4543c33101..878fdc8e267a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis.{NameParameterizedQuery, PosParameterizedQuery, UnresolvedRelation} import org.apache.spark.sql.catalyst.encoders._ -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan, Range} import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -98,7 +98,7 @@ class SparkSession private( @transient private[sql] val extensions: SparkSessionExtensions, @transient private[sql] val initialSessionOptions: Map[String, String], @transient private val parentManagedJobTags: Map[String, String]) - extends api.SparkSession with Logging { self => + extends api.SparkSession with Logging with classic.ColumnConversions { self => // The call site where this SparkSession was constructed. private val creationSite: CallSite = Utils.getCallSite() @@ -797,23 +797,11 @@ class SparkSession private( .getOrElse(sparkContext.defaultParallelism) } - private[sql] object Converter extends ColumnNodeToExpressionConverter with Serializable { - override protected def parser: ParserInterface = sessionState.sqlParser - override protected def conf: SQLConf = sessionState.conf - } - - private[sql] def expression(e: Column): Expression = Converter(e.node) - - private[sql] implicit class RichColumn(val column: Column) { - /** - * Returns the expression for this column. - */ - def expr: Expression = Converter(column.node) - /** - * Returns the expression for this column either with an existing or auto assigned name. - */ - def named: NamedExpression = ExpressionUtils.toNamed(expr) - } + override protected[sql] val converter: ColumnNodeToExpressionConverter = + new ColumnNodeToExpressionConverter with Serializable { + override protected def parser: ParserInterface = sessionState.sqlParser + override protected def conf: SQLConf = sessionState.conf + } private[sql] lazy val observationManager = new ObservationManager(this) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index a66a6e54a7c8a..da03293ce743b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -33,10 +33,11 @@ import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TableFunctionRe import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.execution.{ExplainMode, QueryExecution} import org.apache.spark.sql.execution.arrow.ArrowConverters import org.apache.spark.sql.execution.python.EvaluatePython -import org.apache.spark.sql.internal.ExpressionUtils.{column, expression} +import org.apache.spark.sql.internal.ExpressionUtils.expression import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{MutableURLClassLoader, Utils} @@ -152,7 +153,8 @@ private[sql] object PythonSQLUtils extends Logging { Column(internal.LambdaFunction(function.node, arguments)) } - def namedArgumentExpression(name: String, e: Column): Column = NamedArgumentExpression(name, e) + def namedArgumentExpression(name: String, e: Column): Column = + Column(NamedArgumentExpression(name, expression(e))) @scala.annotation.varargs def fn(name: String, arguments: Column*): Column = Column.fn(name, arguments: _*) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/ClassicConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/conversions.scala similarity index 56% rename from sql/core/src/main/scala/org/apache/spark/sql/classic/ClassicConversions.scala rename to sql/core/src/main/scala/org/apache/spark/sql/classic/conversions.scala index 8c3223fa72f55..e90fd4b6a6032 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/ClassicConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/conversions.scala @@ -20,8 +20,8 @@ import scala.language.implicitConversions import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.internal.ExpressionUtils +import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} +import org.apache.spark.sql.internal.{ColumnNodeToExpressionConverter, ExpressionUtils} /** * Conversions from sql interfaces to the Classic specific implementation. @@ -56,4 +56,54 @@ trait ClassicConversions { } } +@DeveloperApi object ClassicConversions extends ClassicConversions + +/** + * Conversions from a [[Column]] to an [[Expression]]. + */ +@DeveloperApi +trait ColumnConversions { + protected def converter: ColumnNodeToExpressionConverter + + /** + * Convert a [[Column]] into an [[Expression]]. + */ + @DeveloperApi + def expression(column: Column): Expression = converter(column.node) + + /** + * Wrap a [[Column]] with a [[RichColumn]] to provide the `expr` and `named` methods. + */ + @DeveloperApi + implicit def toRichColumn(column: Column): RichColumn = new RichColumn(column, converter) +} + +/** + * Automatic conversions from a Column to an Expression. This uses the active SparkSession for + * parsing, and the active SQLConf for fetching configurations. + * + * This functionality is not part of the ClassicConversions because it is generally better to use + * `SparkSession.toRichColumn(...)` or `SparkSession.expression(...)` directly. + */ +@DeveloperApi +object ColumnConversions extends ColumnConversions { + override protected def converter: ColumnNodeToExpressionConverter = + ColumnNodeToExpressionConverter +} + +/** + * Helper class that adds the `expr` and `named` methods to a Column. This can be used to reinstate + * the pre-Spark 4 Column functionality. + */ +@DeveloperApi +class RichColumn(column: Column, converter: ColumnNodeToExpressionConverter) { + /** + * Returns the expression for this column. + */ + def expr: Expression = converter(column.node) + /** + * Returns the expression for this column either with an existing or auto assigned name. + */ + def named: NamedExpression = ExpressionUtils.toNamed(expr) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala index ea1f5e6ae1340..388ede5d062e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala @@ -28,8 +28,9 @@ import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Descending, Expression, FunctionTableSubqueryArgumentExpression, NamedArgumentExpression, NullsFirst, NullsLast, PythonUDAF, PythonUDF, PythonUDTF, PythonUDTFAnalyzeResult, PythonUDTFSelectedExpression, SortOrder, UnresolvedPolymorphicPythonUDTF} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, NamedParametersSupport, OneRowRelation} +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.internal.ExpressionUtils.{column, expression} +import org.apache.spark.sql.internal.ExpressionUtils.expression import org.apache.spark.sql.types.{DataType, StructType} /** @@ -75,10 +76,10 @@ case class UserDefinedPythonFunction( * Returns a [[Column]] that will evaluate the UDF expression with the given input. */ def fromUDFExpr(expr: Expression): Column = { - expr match { + Column(expr match { case udaf: PythonUDAF => udaf.toAggregateExpression() case _ => expr - } + }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index 148766f9d0026..221ca17ddf19d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -22,13 +22,13 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, Da import scala.collection.mutable import org.apache.spark.internal.Logging -import org.apache.spark.sql.{functions, DataFrame} +import org.apache.spark.sql.{functions, Column, DataFrame} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.aggregate.{ImperativeAggregate, TypedImperativeAggregate} import org.apache.spark.sql.catalyst.trees.UnaryLike import org.apache.spark.sql.catalyst.util.GenericArrayData -import org.apache.spark.sql.internal.ExpressionUtils.{column, expression} +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -52,13 +52,15 @@ object FrequentItems extends Logging { df: DataFrame, cols: Seq[String], support: Double): DataFrame = { + import df.sparkSession.expression require(support >= 1e-4 && support <= 1.0, s"Support must be in [1e-4, 1], but got $support.") // number of max items to keep counts for val sizeOfMap = (1 / support).toInt val frequentItemCols = cols.map { col => - column(new CollectFrequentItems(functions.col(col), sizeOfMap)).as(s"${col}_freqItems") + Column(new CollectFrequentItems(expression(functions.col(col)), sizeOfMap)) + .as(s"${col}_freqItems") } df.select(frequentItemCols: _*) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/MergeIntoWriterImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/MergeIntoWriterImpl.scala index bb8146e3e0e33..2f1a34648a470 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/MergeIntoWriterImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/MergeIntoWriterImpl.scala @@ -44,7 +44,7 @@ class MergeIntoWriterImpl[T] private[sql] (table: String, ds: Dataset[T], on: Co private val df: DataFrame = ds.toDF() private[sql] val sparkSession = ds.sparkSession - import sparkSession.RichColumn + import sparkSession.toRichColumn private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala index 00e9a01f33c1d..8b4726114890f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala @@ -16,8 +16,6 @@ */ package org.apache.spark.sql.internal -import scala.language.implicitConversions - import UserDefinedFunctionUtils.toScalaUDF import org.apache.spark.SparkException @@ -302,13 +300,14 @@ private[spark] object ExpressionUtils { /** * Create an Expression backed Column. */ - implicit def column(e: Expression): Column = Column(ExpressionColumnNode(e)) + def column(e: Expression): Column = Column(ExpressionColumnNode(e)) /** - * Create an ColumnNode backed Expression. Please not that this has to be converted to an actual - * Expression before it is used. + * Create an ColumnNode backed Expression. This can only be used for expressions that will be + * used to construct a [[Column]]. In all other cases please use `SparkSession.expression(...)`, + * `SparkSession.toRichColumn(...)`, or `org.apache.spark.sql.classic.ColumnConversions`. */ - implicit def expression(c: Column): Expression = ColumnNodeExpression(c.node) + def expression(c: Column): Expression = ColumnNodeExpression(c.node) /** * Returns the expression either with an existing or auto assigned name. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 48ea0e01a4372..8024b579e5d0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.objects.MapObjects import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{ArrayType, BooleanType, Decimal, DoubleType, IntegerType, MapType, StringType, StructField, StructType} @@ -92,8 +91,8 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession { // items: Seq[Int] => items.map { item => Seq(Struct(item)) } val result = df.select( - column(MapObjects( - (item: Expression) => array(struct(column(item))).expr, + Column(MapObjects( + (item: Expression) => array(struct(Column(item))).expr, $"items".expr, df.schema("items").dataType.asInstanceOf[ArrayType].elementType )) as "items" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 4494057b1eefe..ce34db47c6dfa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, UTC} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -406,7 +405,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { callSitePattern = "", startIndex = 0, stopIndex = 0)) - expr = nullifzero(Literal.create(20201231, DateType)) + expr = nullifzero(Column(Literal.create(20201231, DateType))) checkError( intercept[AnalysisException](df.select(expr)), condition = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", @@ -588,7 +587,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { callSitePattern = "", startIndex = 0, stopIndex = 0)) - expr = zeroifnull(Literal.create(20201231, DateType)) + expr = zeroifnull(Column(Literal.create(20201231, DateType))) checkError( intercept[AnalysisException](df.select(expr)), condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", @@ -5737,7 +5736,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { import DataFrameFunctionsSuite.CodegenFallbackExpr for ((codegenFallback, wholeStage) <- Seq((true, false), (false, false), (false, true))) { val c = if (codegenFallback) { - column(CodegenFallbackExpr(v.expr)) + Column(CodegenFallbackExpr(v.expr)) } else { v } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala index f0ed2241fd286..0e9b1c9d2104e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, AttributeRef import org.apache.spark.sql.catalyst.plans.logical.{Expand, Generate, ScriptInputOutputSchema, ScriptTransformation, Window => WindowPlan} import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions.{col, count, explode, sum, year} -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.TestData @@ -375,7 +374,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { Seq.empty, PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, true) - val df7 = df1.mapInPandas(mapInPandasUDF) + val df7 = df1.mapInPandas(Column(mapInPandasUDF)) val df8 = df7.filter($"x" > 0) assertAmbiguousSelfJoin(df7.join(df8, df7("x") === df8("y"))) assertAmbiguousSelfJoin(df8.join(df7, df7("x") === df8("y"))) @@ -386,7 +385,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { Seq.empty, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, true) - val df9 = df1.groupBy($"key1").flatMapGroupsInPandas(flatMapGroupsInPandasUDF) + val df9 = df1.groupBy($"key1").flatMapGroupsInPandas(Column(flatMapGroupsInPandasUDF)) val df10 = df9.filter($"x" > 0) assertAmbiguousSelfJoin(df9.join(df10, df9("x") === df10("y"))) assertAmbiguousSelfJoin(df10.join(df9, df9("x") === df10("y"))) @@ -398,7 +397,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, true) val df11 = df1.groupBy($"key1").flatMapCoGroupsInPandas( - df1.groupBy($"key2"), flatMapCoGroupsInPandasUDF) + df1.groupBy($"key2"), Column(flatMapCoGroupsInPandasUDF)) val df12 = df11.filter($"x" > 0) assertAmbiguousSelfJoin(df11.join(df12, df11("x") === df12("y"))) assertAmbiguousSelfJoin(df12.join(df11, df11("x") === df12("y"))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index ff251ddbbfb52..c1d977dad82d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -43,7 +43,6 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.expressions.{Aggregator, Window} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} import org.apache.spark.sql.test.SQLTestData.{ArrayStringWrapper, ContainerStringWrapper, StringWrapper, TestData2} @@ -1567,7 +1566,7 @@ class DataFrameSuite extends QueryTest test("SPARK-46794: exclude subqueries from LogicalRDD constraints") { withTempDir { checkpointDir => val subquery = - column(ScalarSubquery(spark.range(10).selectExpr("max(id)").logicalPlan)) + Column(ScalarSubquery(spark.range(10).selectExpr("max(id)").logicalPlan)) val df = spark.range(1000).filter($"id" === subquery) assert(df.logicalPlan.constraints.exists(_.exists(_.isInstanceOf[ScalarSubquery]))) @@ -2054,18 +2053,18 @@ class DataFrameSuite extends QueryTest // the number of keys must match val exception1 = intercept[IllegalArgumentException] { df1.groupBy($"key1", $"key2").flatMapCoGroupsInPandas( - df2.groupBy($"key2"), flatMapCoGroupsInPandasUDF) + df2.groupBy($"key2"), Column(flatMapCoGroupsInPandasUDF)) } assert(exception1.getMessage.contains("Cogroup keys must have same size: 2 != 1")) val exception2 = intercept[IllegalArgumentException] { df1.groupBy($"key1").flatMapCoGroupsInPandas( - df2.groupBy($"key1", $"key2"), flatMapCoGroupsInPandasUDF) + df2.groupBy($"key1", $"key2"), Column(flatMapCoGroupsInPandasUDF)) } assert(exception2.getMessage.contains("Cogroup keys must have same size: 1 != 2")) // but different keys are allowed val actual = df1.groupBy($"key1").flatMapCoGroupsInPandas( - df2.groupBy($"key2"), flatMapCoGroupsInPandasUDF) + df2.groupBy($"key2"), Column(flatMapCoGroupsInPandasUDF)) // can't evaluate the DataFrame as there is no PythonFunction given assert(actual != null) } @@ -2419,7 +2418,7 @@ class DataFrameSuite extends QueryTest | SELECT a, b FROM (SELECT a, b FROM VALUES (1, 2) AS t(a, b)) |) |""".stripMargin) - val stringCols = df.logicalPlan.output.map(column(_).cast(StringType)) + val stringCols = df.logicalPlan.output.map(Column(_).cast(StringType)) val castedDf = df.select(stringCols: _*) checkAnswer(castedDf, Row("1", "1") :: Row("1", "2") :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 8a86aa10887c0..01e72daead440 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, S import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -862,7 +861,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest lead($"value", 2, null, true).over(window), lead($"value", 3, null, true).over(window), lead(concat($"value", $"key"), 1, null, true).over(window), - column(Lag($"value".expr, NonFoldableLiteral(1), Literal(null), true)).over(window), + Column(Lag($"value".expr, NonFoldableLiteral(1), Literal(null), true)).over(window), lag($"value", 2).over(window), lag($"value", 0, null, true).over(window), lag($"value", 1, null, true).over(window), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index cdea4446d9461..22f55819d1d4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -31,10 +31,11 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, ExprId, PythonUDF} import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.execution.datasources.v2.python.UserDefinedPythonDataSource import org.apache.spark.sql.execution.python.{UserDefinedPythonFunction, UserDefinedPythonTableFunction} import org.apache.spark.sql.expressions.SparkUserDefinedFunction -import org.apache.spark.sql.internal.ExpressionUtils.{column, expression} +import org.apache.spark.sql.internal.ExpressionUtils.expression import org.apache.spark.sql.internal.UserDefinedFunctionUtils.toScalaUDF import org.apache.spark.sql.types.{DataType, IntegerType, NullType, StringType, StructType, VariantType} import org.apache.spark.util.ArrayImplicits._ @@ -1592,7 +1593,7 @@ object IntegratedUDFTestUtils extends SQLHelper { Cast(toScalaUDF(udf, Cast(expr, StringType) :: Nil), rt) } - def apply(exprs: Column*): Column = builder(exprs.map(expression)) + def apply(exprs: Column*): Column = Column(builder(exprs.map(expression))) val prettyName: String = "Scala UDF" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 84408d8e2495d..3803360f2da4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.Cast._ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -1394,7 +1393,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { val df = Seq(1).toDF("a") val schema = StructType(StructField("b", ObjectType(classOf[java.lang.Integer])) :: Nil) val row = InternalRow.fromSeq(Seq(Integer.valueOf(1))) - val structData = column(Literal.create(row, schema)) + val structData = Column(Literal.create(row, schema)) checkError( exception = intercept[AnalysisException] { df.select($"a").withColumn("c", to_json(structData)).collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala index 624bae70ce09c..662eead137c40 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.trees.UnaryLike import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.ExpressionUtils.{column => toColumn, expression} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -89,7 +88,7 @@ class TypedImperativeAggregateSuite extends QueryTest with SharedSparkSession { test("dataframe aggregate with object aggregate buffer, should not use HashAggregate") { val df = data.toDF("a", "b") - val max = TypedMax($"a") + val max = Column(TypedMax($"a".expr)) // Always uses SortAggregateExec val sparkPlan = df.select(max).queryExecution.sparkPlan @@ -212,9 +211,10 @@ class TypedImperativeAggregateSuite extends QueryTest with SharedSparkSession { checkAnswer(query, expected) } - private def typedMax(column: Column): Column = TypedMax(column) + private def typedMax(column: Column): Column = Column(TypedMax(column.expr)) - private def nullableTypedMax(column: Column): Column = TypedMax(column, nullable = true) + private def nullableTypedMax(column: Column): Column = + Column(TypedMax(column.expr, nullable = true)) } object TypedImperativeAggregateSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/functions/V2FunctionBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/functions/V2FunctionBenchmark.scala index 1401048cf705d..a5f0285bf2eff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/functions/V2FunctionBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/functions/V2FunctionBenchmark.scala @@ -21,15 +21,16 @@ import test.org.apache.spark.sql.connector.catalog.functions.JavaLongAdd import test.org.apache.spark.sql.connector.catalog.functions.JavaLongAdd.{JavaLongAddDefault, JavaLongAddMagic, JavaLongAddStaticMagic} import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BinaryArithmetic, EvalMode, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryCatalog} import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction} import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions.col -import org.apache.spark.sql.internal.ExpressionUtils.{column, expression} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{AbstractDataType, DataType, LongType, NumericType, StructType} @@ -64,6 +65,7 @@ object V2FunctionBenchmark extends SqlBasedBenchmark { N: Long, codegenEnabled: Boolean, resultNullable: Boolean): Unit = { + import spark.toRichColumn withSQLConf(s"spark.sql.catalog.$catalogName" -> classOf[InMemoryCatalog].getName) { createFunction("java_long_add_default", new JavaLongAdd(new JavaLongAddDefault(resultNullable))) @@ -81,7 +83,9 @@ object V2FunctionBenchmark extends SqlBasedBenchmark { s"codegen = $codegenEnabled" val benchmark = new Benchmark(name, N, output = output) benchmark.addCase(s"native_long_add", numIters = 3) { _ => - spark.range(N).select(NativeAdd(col("id"), col("id"), resultNullable)).noop() + spark.range(N) + .select(Column(NativeAdd(col("id").expr, col("id").expr, resultNullable))) + .noop() } Seq("java_long_add_default", "java_long_add_magic", "java_long_add_static_magic", "scala_long_add_default", "scala_long_add_magic").foreach { functionName => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 500c0647bcb2a..bf9740970a667 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -28,14 +28,13 @@ import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument, SearchA import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.spark.{SparkConf, SparkException, SparkRuntimeException} -import org.apache.spark.sql.{AnalysisException, DataFrame, Row} +import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Row} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.functions.col -import org.apache.spark.sql.internal.ExpressionUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -47,7 +46,7 @@ import org.apache.spark.util.ArrayImplicits._ */ @ExtendedSQLTest class OrcFilterSuite extends OrcTest with SharedSparkSession { - import testImplicits.toRichColumn + import testImplicits.{toRichColumn, ColumnConstructorExt} override protected def sparkConf: SparkConf = super @@ -60,8 +59,8 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { checker: (SearchArgument) => Unit): Unit = { val output = predicate.collect { case a: Attribute => a }.distinct val query = df - .select(output.map(e => ExpressionUtils.column(e)): _*) - .where(ExpressionUtils.column(predicate)) + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, DataSourceV2ScanRelation(_, o: OrcScan, _, _, _)) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index b8669ee4d1ef1..9fbc872ad262b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -28,10 +28,10 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.execution.datasources.FileBasedDataSourceTest import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION import org.apache.spark.util.ArrayImplicits._ @@ -118,8 +118,8 @@ trait OrcTest extends QueryTest with FileBasedDataSourceTest with BeforeAndAfter (implicit df: DataFrame): Unit = { val output = predicate.collect { case a: Attribute => a }.distinct val query = df - .select(output.map(e => column(e)): _*) - .where(predicate) + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, DataSourceV2ScanRelation(_, o: OrcScan, _, _, _)) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala index 5260ebf15e4f3..8018417f923af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -21,12 +21,12 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl import org.apache.spark.SparkConf -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Column, DataFrame} import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Predicate} import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelationWithTable} import org.apache.spark.sql.execution.datasources.orc.OrcShimUtils.{Operator, SearchArgument} -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.tags.ExtendedSQLTest @@ -44,8 +44,8 @@ class OrcV1FilterSuite extends OrcFilterSuite { checker: (SearchArgument) => Unit): Unit = { val output = predicate.collect { case a: Attribute => a }.distinct val query = df - .select(output.map(e => column(e)): _*) - .where(predicate) + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { @@ -90,8 +90,8 @@ class OrcV1FilterSuite extends OrcFilterSuite { (implicit df: DataFrame): Unit = { val output = predicate.collect { case a: Attribute => a }.distinct val query = df - .select(output.map(e => column(e)): _*) - .where(predicate) + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 37edb9ea2315e..5f7a0c9e7e749 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -49,7 +49,7 @@ import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsR import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.{ExpressionUtils, LegacyBehaviorPolicy, SQLConf} +import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.internal.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.{INT96, TIMESTAMP_MICROS, TIMESTAMP_MILLIS} import org.apache.spark.sql.test.SharedSparkSession @@ -2233,6 +2233,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared @ExtendedSQLTest class ParquetV1FilterSuite extends ParquetFilterSuite { + import testImplicits.ColumnConstructorExt + override protected def sparkConf: SparkConf = super .sparkConf @@ -2260,8 +2262,8 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", SQLConf.NESTED_PREDICATE_PUSHDOWN_FILE_SOURCE_LIST.key -> pushdownDsList) { val query = df - .select(output.map(ExpressionUtils.column): _*) - .where(ExpressionUtils.column(predicate)) + .select(output.map(Column(_)): _*) + .where(Column(predicate)) val nestedOrAttributes = predicate.collectFirst { case g: GetStructField => g @@ -2313,6 +2315,8 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { @ExtendedSQLTest class ParquetV2FilterSuite extends ParquetFilterSuite { + import testImplicits.ColumnConstructorExt + // TODO: enable Parquet V2 write path after file source V2 writers are workable. override protected def sparkConf: SparkConf = super @@ -2339,8 +2343,8 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { val query = df - .select(output.map(ExpressionUtils.column): _*) - .where(ExpressionUtils.column(predicate)) + .select(output.map(Column(_)): _*) + .where(Column(predicate)) query.queryExecution.optimizedPlan.collectFirst { case PhysicalOperation(_, filters, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 24732223c6698..c4b09c4b289e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.execution.datasources.BucketingUtils import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.ExpressionUtils.column import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -229,7 +228,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti checkPrunedAnswers( bucketSpec, bucketValues = Seq(bucketValue, bucketValue + 1, bucketValue + 2, bucketValue + 3), - filterCondition = column(inSetExpr), + filterCondition = Column(inSetExpr), df) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index fe5a0f8ee257a..c93f17701c620 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -41,10 +41,11 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.classic.{ClassicConversions, ColumnConversions} import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{ColumnNodeToExpressionConverter, SQLConf} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.UninterruptibleThread import org.apache.spark.util.Utils @@ -239,9 +240,12 @@ private[sql] trait SQLTestUtilsBase * This is because we create the `SparkSession` immediately before the first test is run, * but the implicits import is needed in the constructor. */ - protected object testImplicits extends SQLImplicits { + protected object testImplicits + extends SQLImplicits + with ClassicConversions + with ColumnConversions { override protected def session: SparkSession = self.spark - implicit def toRichColumn(c: Column): SparkSession#RichColumn = session.RichColumn(c) + override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } protected override def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala index 700a4984a4e39..f5bf49439d3f9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -23,10 +23,11 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Column, DataFrame, SparkSession} +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.functions.{lit, percentile_approx => pa} import org.apache.spark.sql.hive.execution.TestingTypedCount import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.internal.ExpressionUtils.{column => toCol, expression} +import org.apache.spark.sql.internal.ExpressionUtils.expression import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.LongType @@ -117,7 +118,7 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { output = output ) - def typed_count(column: Column): Column = TestingTypedCount(column) + def typed_count(column: Column): Column = Column(TestingTypedCount(expression(column))) val df = spark.range(N) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/OptimizeHiveMetadataOnlyQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/OptimizeHiveMetadataOnlyQuerySuite.scala index 2152a29b17ff4..6709a139dcf96 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/OptimizeHiveMetadataOnlyQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/OptimizeHiveMetadataOnlyQuerySuite.scala @@ -32,7 +32,7 @@ class OptimizeHiveMetadataOnlyQuerySuite extends QueryTest with TestHiveSingleto with BeforeAndAfter with SQLTestUtils { import spark.implicits._ - import spark.RichColumn + import spark.toRichColumn override def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala index bcd0644af0782..008a324f73dac 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -23,12 +23,11 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax import org.scalatest.matchers.must.Matchers._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper} +import org.apache.spark.sql.catalyst.expressions.ExpressionEvalHelper import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.functions.{col, count_distinct, first, lit, max, percentile_approx => pa} import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.ExpressionUtils.{column => toCol, expression} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ @@ -181,7 +180,7 @@ class ObjectHashAggregateSuite pa(column, lit(percentage), lit(10000)) } - private def typed_count(column: Column): Column = TestingTypedCount(column) + private def typed_count(column: Column): Column = Column(TestingTypedCount(column.expr)) // Generates 50 random rows for a given schema. private def generateRandomRows(schemaForGenerator: StructType): Seq[Row] = { From 6cd13344bffd411289ec20170ae2da2035bb2859 Mon Sep 17 00:00:00 2001 From: Fokko Date: Tue, 3 Dec 2024 08:08:34 -0800 Subject: [PATCH 084/438] [SPARK-50425][BUILD] Bump Apache Parquet to 1.15.0 ### What changes were proposed in this pull request? Bumps to the latest version of Parquet. For the full list of changes, please check the pre-release: https://github.com/apache/parquet-java/releases/tag/apache-parquet-1.15.0 Including some interesting patches for Spark, such as https://github.com/apache/parquet-java/pull/3030 ### Why are the changes needed? To bring the latest features and bug fixes for 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 #48970 from Fokko/fd-parquet-1-15-0. Authored-by: Fokko Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 13 ++++++------- pom.xml | 2 +- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index d3ad905c47cf6..4721941937a2d 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -111,7 +111,6 @@ jackson-core/2.18.2//jackson-core-2.18.2.jar jackson-databind/2.18.2//jackson-databind-2.18.2.jar jackson-dataformat-cbor/2.18.2//jackson-dataformat-cbor-2.18.2.jar jackson-dataformat-yaml/2.18.2//jackson-dataformat-yaml-2.18.2.jar -jackson-datatype-jdk8/2.17.0//jackson-datatype-jdk8-2.17.0.jar jackson-datatype-jsr310/2.18.2//jackson-datatype-jsr310-2.18.2.jar jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.13/2.18.2//jackson-module-scala_2.13-2.18.2.jar @@ -241,12 +240,12 @@ orc-shims/2.0.3//orc-shims-2.0.3.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar -parquet-column/1.14.4//parquet-column-1.14.4.jar -parquet-common/1.14.4//parquet-common-1.14.4.jar -parquet-encoding/1.14.4//parquet-encoding-1.14.4.jar -parquet-format-structures/1.14.4//parquet-format-structures-1.14.4.jar -parquet-hadoop/1.14.4//parquet-hadoop-1.14.4.jar -parquet-jackson/1.14.4//parquet-jackson-1.14.4.jar +parquet-column/1.15.0//parquet-column-1.15.0.jar +parquet-common/1.15.0//parquet-common-1.15.0.jar +parquet-encoding/1.15.0//parquet-encoding-1.15.0.jar +parquet-format-structures/1.15.0//parquet-format-structures-1.15.0.jar +parquet-hadoop/1.15.0//parquet-hadoop-1.15.0.jar +parquet-jackson/1.15.0//parquet-jackson-1.15.0.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 diff --git a/pom.xml b/pom.xml index 807b08847ebd6..5f8556ce0c211 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ 3.9.0 10.16.1.1 - 1.14.4 + 1.15.0 2.0.3 shaded-protobuf 11.0.24 From 7b974ca758961668a26a1d0c60c91614dac38742 Mon Sep 17 00:00:00 2001 From: Richard Chen Date: Tue, 3 Dec 2024 08:36:59 -0800 Subject: [PATCH 085/438] [SPARK-50463][SQL] Fix `ConstantColumnVector` with Columnar to Row conversion ### What changes were proposed in this pull request? https://github.com/apache/spark/commit/800faf0abfa368ad0a5ef1e0fa44b74dbaab724e frees column vector resources between batches in columnar to row conversion. However, like `WritableColumnVector`, `ConstantColumnVector` should not free resources between batches because the same data is used across batches ### Why are the changes needed? Without this change, ConstantColumnVectors with string values, for example, will fail if used with column->row conversion. For instance, reading a parquet table partitioned by a string column with multiple batches. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? added UT that failed before and now passes ### Was this patch authored or co-authored using generative AI tooling? no Closes #49021 from richardc-db/col_to_row_const_col_vec_fix. Authored-by: Richard Chen Signed-off-by: Dongjoon Hyun --- .../spark/sql/vectorized/ColumnVector.java | 12 +++++------ .../spark/sql/vectorized/ColumnarBatch.java | 8 ++++---- .../vectorized/ConstantColumnVector.java | 5 +++++ .../vectorized/WritableColumnVector.java | 2 +- .../apache/spark/sql/execution/Columnar.scala | 2 +- .../parquet/ParquetQuerySuite.scala | 20 +++++++++++++++++++ 6 files changed, 37 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java index bfb1833b731a7..54b62c00283fa 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java @@ -69,14 +69,14 @@ public abstract class ColumnVector implements AutoCloseable { public abstract void close(); /** - * Cleans up memory for this column vector if it's not writable. The column vector is not usable - * after this. + * Cleans up memory for this column vector if it's resources are freeable between batches. + * The column vector is not usable after this. * - * If this is a writable column vector, it is a no-op. + * If this is a writable column vector or constant column vector, it is a no-op. */ - public void closeIfNotWritable() { - // By default, we just call close() for all column vectors. If a column vector is writable, it - // should override this method and do nothing. + public void closeIfFreeable() { + // By default, we just call close() for all column vectors. If a column vector is writable or + // constant, it should override this method and do nothing. close(); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java index 52e4115af336a..7ef570a212292 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java @@ -46,12 +46,12 @@ public void close() { } /** - * Called to close all the columns if they are not writable. This is used to clean up memory - * allocated during columnar processing. + * Called to close all the columns if their resources are freeable between batches. + * This is used to clean up memory allocated during columnar processing. */ - public void closeIfNotWritable() { + public void closeIfFreeable() { for (ColumnVector c: columns) { - c.closeIfNotWritable(); + c.closeIfFreeable(); } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ConstantColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ConstantColumnVector.java index 8b24973ad3d87..cd2a821698853 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ConstantColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ConstantColumnVector.java @@ -77,6 +77,11 @@ public ConstantColumnVector(int numRows, DataType type) { } } + public void closeIfFreeable() { + // no-op: `ConstantColumnVector`s reuse the data backing its value across multiple batches and + // are freed at the end of execution in `close`. + } + @Override public void close() { stringData = null; diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index 696e20525cdac..fc465e73006be 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -97,7 +97,7 @@ public void close() { } @Override - public void closeIfNotWritable() { + public void closeIfFreeable() { // no-op } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index 64163da50e13a..a67648f24b4c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -194,7 +194,7 @@ case class ColumnarToRowExec(child: SparkPlan) extends ColumnarToRowTransition w | $shouldStop | } | $idx = $numRows; - | $batch.closeIfNotWritable(); + | $batch.closeIfFreeable(); | $batch = null; | $nextBatchFuncName(); |} 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 22a02447e720f..bba71f1c48dec 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 @@ -473,6 +473,26 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } } + test("SPARK-50463: Partition values can be read over multiple batches") { + withTempDir { dir => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE.key -> "1") { + val path = dir.getAbsolutePath + spark.range(0, 5) + .selectExpr("concat(cast(id % 2 as string), 'a') as partCol", "id") + .write + .format("parquet") + .mode("overwrite") + .partitionBy("partCol").save(path) + val df = spark.read.format("parquet").load(path).selectExpr("partCol") + val expected = spark.range(0, 5) + .selectExpr("concat(cast(id % 2 as string), 'a') as partCol") + .collect() + + checkAnswer(df, expected) + } + } + } + test("SPARK-10301 requested schema clipping - same schema") { withTempPath { dir => val path = dir.getCanonicalPath From d0dbc6c5e5c44f64c3f13e676e0fb468a3ae7f57 Mon Sep 17 00:00:00 2001 From: Aleksei Shishkin Date: Tue, 3 Dec 2024 23:33:08 +0100 Subject: [PATCH 086/438] [SPARK-50470][SQL] Block usage of collations for map keys ### What changes were proposed in this pull request? According to the issue description collation usage on keys of map may lead for unexpected effects (key duplication). This PR blocks this usage, but it is still available with Spark conf flag. Old behavior is enabled by `spark.sql.collation.allowInMapKeys`. ### Why are the changes needed? Because usage may create unclear situation. For example if we have `map('A' -> 1, 'a' -> 2)` then changing collation from UTF8_BINARY to UTF8_LCASE break key uniqueness. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49030 from Alexvsalexvsalex/SPARK-50470_block_collations_on_maps_key. Authored-by: Aleksei Shishkin Signed-off-by: Max Gekk --- .../resources/error/error-conditions.json | 5 ++ .../sql/catalyst/analysis/CheckAnalysis.scala | 8 ++ .../sql/errors/QueryCompilationErrors.scala | 6 ++ .../apache/spark/sql/internal/SQLConf.scala | 9 ++ .../apache/spark/sql/util/SchemaUtils.scala | 11 +++ .../spark/sql/execution/command/tables.scala | 3 + .../InsertIntoHadoopFsRelationCommand.scala | 3 + .../sql/execution/datasources/rules.scala | 3 + .../sql/CollationExpressionWalkerSuite.scala | 75 ++++++++-------- .../sql/CollationSQLExpressionsSuite.scala | 8 +- .../org/apache/spark/sql/CollationSuite.scala | 87 ++++++++++++++----- ...CollatedFilterPushDownToParquetSuite.scala | 8 +- .../CollationTypePrecedenceSuite.scala | 46 +++++----- 13 files changed, 187 insertions(+), 85 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 157989c09d098..63c4d18c99de9 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5193,6 +5193,11 @@ "The SQL pipe operator syntax using |> does not support ." ] }, + "COLLATIONS_IN_MAP_KEYS" : { + "message" : [ + "Collated strings for keys of maps" + ] + }, "COMBINATION_QUERY_RESULT_CLAUSES" : { "message" : [ "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c13da35334ba5..d4b97ff037f36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1560,15 +1560,23 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB alter.conf.resolver) } + def checkNoCollationsInMapKeys(colsToAdd: Seq[QualifiedColType]): Unit = { + if (!alter.conf.allowCollationsInMapKeys) { + colsToAdd.foreach(col => SchemaUtils.checkNoCollationsInMapKeys(col.dataType)) + } + } + alter match { case AddColumns(table: ResolvedTable, colsToAdd) => colsToAdd.foreach { colToAdd => checkColumnNotExists("add", colToAdd.name, table.schema) } checkColumnNameDuplication(colsToAdd) + checkNoCollationsInMapKeys(colsToAdd) case ReplaceColumns(_: ResolvedTable, colsToAdd) => checkColumnNameDuplication(colsToAdd) + checkNoCollationsInMapKeys(colsToAdd) case RenameColumn(table: ResolvedTable, col: ResolvedFieldName, newName) => checkColumnNotExists("rename", col.path :+ newName, table.schema) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 7d7a490c97908..b673d5a043153 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -351,6 +351,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat ) } + def collatedStringsInMapKeysNotSupportedError(): Throwable = { + new AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.COLLATIONS_IN_MAP_KEYS", + messageParameters = Map.empty) + } + def trimCollationNotEnabledError(): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE.TRIM_COLLATION", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 2a05508a17544..c0d35fa0ce2b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -767,6 +767,13 @@ object SQLConf { .checkValue(_ > 0, "The initial number of partitions must be positive.") .createOptional + lazy val ALLOW_COLLATIONS_IN_MAP_KEYS = + buildConf("spark.sql.collation.allowInMapKeys") + .doc("Allow for non-UTF8_BINARY collated strings inside of map's keys") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + lazy val TRIM_COLLATION_ENABLED = buildConf("spark.sql.collation.trim.enabled") .internal() @@ -5585,6 +5592,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { } } + def allowCollationsInMapKeys: Boolean = getConf(ALLOW_COLLATIONS_IN_MAP_KEYS) + def trimCollationEnabled: Boolean = getConf(TRIM_COLLATION_ENABLED) override def defaultStringType: StringType = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 1e0bac331dc75..0aadd3cd3a449 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -304,6 +304,17 @@ private[spark] object SchemaUtils { } } + def checkNoCollationsInMapKeys(schema: DataType): Unit = schema match { + case m: MapType => + if (hasNonUTF8BinaryCollation(m.keyType)) { + throw QueryCompilationErrors.collatedStringsInMapKeysNotSupportedError() + } + checkNoCollationsInMapKeys(m.valueType) + case s: StructType => s.fields.foreach(field => checkNoCollationsInMapKeys(field.dataType)) + case a: ArrayType => checkNoCollationsInMapKeys(a.elementType) + case _ => + } + /** * Replaces any collated string type with non collated StringType * recursively in the given data type. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 9ecd3fd19aa64..84b73a74f3ab2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -241,6 +241,9 @@ case class AlterTableAddColumnsCommand( SchemaUtils.checkColumnNameDuplication( (colsWithProcessedDefaults ++ catalogTable.schema).map(_.name), conf.caseSensitiveAnalysis) + if (!conf.allowCollationsInMapKeys) { + colsToAdd.foreach(col => SchemaUtils.checkNoCollationsInMapKeys(col.dataType)) + } DDLUtils.checkTableColumns(catalogTable, StructType(colsWithProcessedDefaults)) val existingSchema = CharVarcharUtils.getRawSchema(catalogTable.dataSchema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index aed129c7dccc4..8a795f0748811 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -84,6 +84,9 @@ case class InsertIntoHadoopFsRelationCommand( outputColumnNames, sparkSession.sessionState.conf.caseSensitiveAnalysis) } + if (!conf.allowCollationsInMapKeys) { + SchemaUtils.checkNoCollationsInMapKeys(query.schema) + } val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) val fs = outputPath.getFileSystem(hadoopConf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 713161cc49ce8..23596861a647c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -343,6 +343,9 @@ case class PreprocessTableCreation(catalog: SessionCatalog) extends Rule[Logical SchemaUtils.checkSchemaColumnNameDuplication( schema, conf.caseSensitiveAnalysis) + if (!conf.allowCollationsInMapKeys) { + SchemaUtils.checkNoCollationsInMapKeys(schema) + } val normalizedPartCols = normalizePartitionColumns(schema, table) val normalizedBucketSpec = normalizeBucketSpec(schema, table) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala index bc62fa5fdd331..e3622c3101854 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.{SparkFunSuite, SparkRuntimeException} import org.apache.spark.sql.catalyst.analysis.ExpressionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.variant.ParseJson -import org.apache.spark.sql.internal.SqlApiConf +import org.apache.spark.sql.internal.{SqlApiConf, SQLConf} import org.apache.spark.sql.internal.types._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -636,48 +636,49 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi val expr = headConstructor.newInstance(args: _*).asInstanceOf[ExpectsInputTypes] withTable("tbl", "tbl_lcase") { + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + val utf8_df = generateTableData(expr.inputTypes.take(2), Utf8Binary) + val utf8_lcase_df = generateTableData(expr.inputTypes.take(2), Utf8Lcase) + + val utf8BinaryResult = try { + val df = utf8_df.selectExpr(transformExpressionToString(expr, Utf8Binary)) + df.getRows(1, 0) + scala.util.Right(df) + } catch { + case e: Throwable => scala.util.Left(e) + } + val utf8LcaseResult = try { + val df = utf8_lcase_df.selectExpr(transformExpressionToString(expr, Utf8Lcase)) + df.getRows(1, 0) + scala.util.Right(df) + } catch { + case e: Throwable => scala.util.Left(e) + } - val utf8_df = generateTableData(expr.inputTypes.take(2), Utf8Binary) - val utf8_lcase_df = generateTableData(expr.inputTypes.take(2), Utf8Lcase) - - val utf8BinaryResult = try { - val df = utf8_df.selectExpr(transformExpressionToString(expr, Utf8Binary)) - df.getRows(1, 0) - scala.util.Right(df) - } catch { - case e: Throwable => scala.util.Left(e) - } - val utf8LcaseResult = try { - val df = utf8_lcase_df.selectExpr(transformExpressionToString(expr, Utf8Lcase)) - df.getRows(1, 0) - scala.util.Right(df) - } catch { - case e: Throwable => scala.util.Left(e) - } - - assert(utf8BinaryResult.isLeft === utf8LcaseResult.isLeft) + assert(utf8BinaryResult.isLeft === utf8LcaseResult.isLeft) - if (utf8BinaryResult.isRight) { - val utf8BinaryResultChecked = utf8BinaryResult.getOrElse(null) - val utf8LcaseResultChecked = utf8LcaseResult.getOrElse(null) + if (utf8BinaryResult.isRight) { + val utf8BinaryResultChecked = utf8BinaryResult.getOrElse(null) + val utf8LcaseResultChecked = utf8LcaseResult.getOrElse(null) - val dt = utf8BinaryResultChecked.schema.fields.head.dataType + val dt = utf8BinaryResultChecked.schema.fields.head.dataType - dt match { - case st if utf8BinaryResultChecked != null && utf8LcaseResultChecked != null && - hasStringType(st) => - // scalastyle:off caselocale - assert(utf8BinaryResultChecked.getRows(1, 0).map(_.map(_.toLowerCase))(1) === - utf8LcaseResultChecked.getRows(1, 0).map(_.map(_.toLowerCase))(1)) + dt match { + case st if utf8BinaryResultChecked != null && utf8LcaseResultChecked != null && + hasStringType(st) => + // scalastyle:off caselocale + assert(utf8BinaryResultChecked.getRows(1, 0).map(_.map(_.toLowerCase))(1) === + utf8LcaseResultChecked.getRows(1, 0).map(_.map(_.toLowerCase))(1)) // scalastyle:on caselocale - case _ => - assert(utf8BinaryResultChecked.getRows(1, 0)(1) === - utf8LcaseResultChecked.getRows(1, 0)(1)) + case _ => + assert(utf8BinaryResultChecked.getRows(1, 0)(1) === + utf8LcaseResultChecked.getRows(1, 0)(1)) + } + } + else { + assert(utf8BinaryResult.getOrElse(new Exception()).getClass + == utf8LcaseResult.getOrElse(new Exception()).getClass) } - } - else { - assert(utf8BinaryResult.getOrElse(new Exception()).getClass - == utf8LcaseResult.getOrElse(new Exception()).getClass) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala index b2bb3eaffd415..4e91fd721a075 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala @@ -2006,9 +2006,11 @@ class CollationSQLExpressionsSuite } val tableName = s"t_${t1.collationId}_mode_nested_map_struct1" withTable(tableName) { - sql(s"CREATE TABLE ${tableName}(" + - s"i STRUCT>) USING parquet") - sql(s"INSERT INTO ${tableName} VALUES ${getValuesToAdd(t1)}") + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + sql(s"CREATE TABLE ${tableName}(" + + s"i STRUCT>) USING parquet") + sql(s"INSERT INTO ${tableName} VALUES ${getValuesToAdd(t1)}") + } val query = "SELECT lower(cast(mode(i).m1 as string))" + s" FROM ${tableName}" val queryResult = sql(query) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index dc7d14b21becb..11f2c4b997a4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -221,23 +221,54 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { parameters = Map("collationName" -> "UTF8_BS", "proposals" -> "UTF8_LCASE")) } + test("fail on table creation with collated strings as map key") { + withTable("table_1", "table_2") { + checkError( + exception = intercept[AnalysisException] { + sql("CREATE TABLE table_1 (col MAP) USING parquet") + }, + condition = "UNSUPPORTED_FEATURE.COLLATIONS_IN_MAP_KEYS" + ) + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + sql("CREATE TABLE table_2 (col MAP) USING parquet") + } + } + } + + test("fail on adding column with collated map key") { + withTable("table_1") { + sql("CREATE TABLE table_1 (id INTEGER) USING parquet") + checkError( + exception = intercept[AnalysisException] { + sql("ALTER TABLE table_1 ADD COLUMN col1 MAP, INTEGER>") + }, + condition = "UNSUPPORTED_FEATURE.COLLATIONS_IN_MAP_KEYS" + ) + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + sql("ALTER TABLE table_1 ADD COLUMN col1 MAP, INTEGER>") + } + } + } + test("disable bucketing on collated string column") { def createTable(bucketColumns: String*): Unit = { val tableName = "test_partition_tbl" withTable(tableName) { - sql( - s""" - |CREATE TABLE $tableName ( - | id INT, - | c1 STRING COLLATE UNICODE, - | c2 STRING, - | struct_col STRUCT, - | array_col ARRAY, - | map_col MAP - |) USING parquet - |CLUSTERED BY (${bucketColumns.mkString(",")}) - |INTO 4 BUCKETS""".stripMargin - ) + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + sql( + s""" + |CREATE TABLE $tableName ( + | id INT, + | c1 STRING COLLATE UNICODE, + | c2 STRING, + | struct_col STRUCT, + | array_col ARRAY, + | map_col MAP + |) USING parquet + |CLUSTERED BY (${bucketColumns.mkString(",")}) + |INTO 4 BUCKETS""".stripMargin + ) + } } } // should work fine on default collated columns @@ -1124,7 +1155,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } // map doesn't support aggregation withTable(table) { - sql(s"create table $table (m map) using parquet") + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + sql(s"create table $table (m map) using parquet") + } val query = s"select distinct m from $table" checkError( exception = intercept[ExtendedAnalysisException](sql(query)), @@ -1166,8 +1199,10 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } // map doesn't support joins withTable(tableLeft, tableRight) { - Seq(tableLeft, tableRight).map(tab => - sql(s"create table $tab (m map) using parquet")) + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + Seq(tableLeft, tableRight).map(tab => + sql(s"create table $tab (m map) using parquet")) + } val query = s"select $tableLeft.m from $tableLeft join $tableRight on $tableLeft.m = $tableRight.m" val ctx = s"$tableLeft.m = $tableRight.m" @@ -1418,7 +1453,10 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val tableName = "t" withTable(tableName) { - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen) { + withSQLConf( + SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen, + SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true" + ) { sql(s"create table $tableName" + s" (m map)") sql(s"insert into $tableName values (map('aaa', 'AAA'))") @@ -1443,7 +1481,10 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val tableName = "t" withTable(tableName) { - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen) { + withSQLConf( + SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen, + SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true" + ) { sql(s"create table $tableName" + s" (m map, " + s"struct>)") @@ -1470,7 +1511,10 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val tableName = "t" withTable(tableName) { - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen) { + withSQLConf( + SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen, + SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true" + ) { sql(s"create table $tableName " + s"(m map, array>)") sql(s"insert into $tableName values (map(array('aaa', 'bbb'), array('ccc', 'ddd')))") @@ -1493,7 +1537,10 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test(s"Check that order by on map with$collationSetup strings fails ($codeGen)") { val tableName = "t" withTable(tableName) { - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen) { + withSQLConf( + SQLConf.CODEGEN_FACTORY_MODE.key -> codeGen, + SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true" + ) { sql(s"create table $tableName" + s" (m map, " + s" c integer)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollatedFilterPushDownToParquetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollatedFilterPushDownToParquetSuite.scala index 9b54fe4bb052c..8bb4a1c803e8e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollatedFilterPushDownToParquetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollatedFilterPushDownToParquetSuite.scala @@ -43,7 +43,7 @@ abstract class CollatedFilterPushDownToParquetSuite extends QueryTest val collatedStructNestedCol = "f1" val collatedStructFieldAccess = s"$collatedStructCol.$collatedStructNestedCol" val collatedArrayCol = "c3" - val collatedMapCol = "c4" + val nonCollatedMapCol = "c4" val lcaseCollation = "'UTF8_LCASE'" @@ -69,7 +69,7 @@ abstract class CollatedFilterPushDownToParquetSuite extends QueryTest | named_struct('$collatedStructNestedCol', | COLLATE(c, $lcaseCollation)) as $collatedStructCol, | array(COLLATE(c, $lcaseCollation)) as $collatedArrayCol, - | map(COLLATE(c, $lcaseCollation), 1) as $collatedMapCol + | map(c, 1) as $nonCollatedMapCol |FROM VALUES ('aaa'), ('AAA'), ('bbb') |as data(c) |""".stripMargin) @@ -215,9 +215,9 @@ abstract class CollatedFilterPushDownToParquetSuite extends QueryTest test("map - parquet does not support null check on complex types") { testPushDown( - filterString = s"map_keys($collatedMapCol) != array(collate('aaa', $lcaseCollation))", + filterString = s"map_keys($nonCollatedMapCol) != array('aaa')", expectedPushedFilters = Seq.empty, - expectedRowCount = 1) + expectedRowCount = 2) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index 93e36afae242b..bb6fce1fb1b66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.collation import org.apache.spark.SparkThrowable import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { @@ -397,27 +398,30 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { sql(s"SELECT c1 FROM $tableName WHERE $condition = 'B'") withTable(tableName) { - sql(s""" - |CREATE TABLE $tableName ( - | c1 MAP, - | c2 STRING - |) USING $dataSource - |""".stripMargin) - - sql(s"INSERT INTO $tableName VALUES (map('a', 'b'), 'a')") - - Seq("c1['A']", - "c1['A' COLLATE UNICODE_CI]", - "c1[c2 COLLATE UNICODE_CI]").foreach { condition => - checkAnswer(selectQuery(condition), Seq(Row(Map("a" -> "b")))) - } - - Seq( - // different explicit collation - "c1['A' COLLATE UNICODE]", - // different implicit collation - "c1[c2]").foreach { condition => - assertThrowsError(selectQuery(condition), "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { + sql( + s""" + |CREATE TABLE $tableName ( + | c1 MAP, + | c2 STRING + |) USING $dataSource + |""".stripMargin) + + sql(s"INSERT INTO $tableName VALUES (map('a', 'b'), 'a')") + + Seq("c1['A']", + "c1['A' COLLATE UNICODE_CI]", + "c1[c2 COLLATE UNICODE_CI]").foreach { condition => + checkAnswer(selectQuery(condition), Seq(Row(Map("a" -> "b")))) + } + + Seq( + // different explicit collation + "c1['A' COLLATE UNICODE]", + // different implicit collation + "c1[c2]").foreach { condition => + assertThrowsError(selectQuery(condition), "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + } } } } From ecc33d26e8fa5654cb19ce59c7d0155a8db8c139 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 4 Dec 2024 10:31:13 +0900 Subject: [PATCH 087/438] [SPARK-50482][CORE] Deprecated no-op `spark.shuffle.spill` config ### What changes were proposed in this pull request? This PR aims to deprecated no-op `spark.shuffle.spill` config in Apache Spark 4.0.0. - Previously, it complains only `spark.shuffle.spill=false`. - We had better deprecate this configuration explicitly to simplify Apache Spark 4+ configuration space. ### Why are the changes needed? Since Apache Spark 1.6.0, `spark.shuffle.spill` has been ignored completely. - https://github.com/apache/spark/pull/8831 ### Does this PR introduce _any_ user-facing change? There is no Spark shuffle behavior change. Only it will show additional deprecation message when `spark.shuffle.spill=true` is used by users explicitly. **BEFORE: Only spark.shuffle.spill=false** ``` 24/12/03 10:37:23 WARN SortShuffleManager: spark.shuffle.spill was set to false, but this configuration is ignored as of Spark 1.6+. Shuffle will continue to spill to disk when necessary. ``` **AFTER: Both spark.shuffle.spill=false and spark.shuffle.spill=true** ``` 24/12/03 10:37:47 WARN SparkConf: The configuration key 'spark.shuffle.spill' has been deprecated as of Spark 1.6 and may be removed in the future. Not used anymore. ``` ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49047 from dongjoon-hyun/SPARK-50482. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- core/src/main/scala/org/apache/spark/SparkConf.scala | 1 + .../org/apache/spark/shuffle/sort/SortShuffleManager.scala | 6 ------ 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index cfb514913694b..74abd5cbe0ae5 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -608,6 +608,7 @@ private[spark] object SparkConf extends Logging { "Please use spark.kryoserializer.buffer instead. The default value for " + "spark.kryoserializer.buffer.mb was previously specified as '0.064'. Fractional values " + "are no longer accepted. To specify the equivalent now, one may use '64k'."), + DeprecatedConfig("spark.shuffle.spill", "1.6", "Not used anymore."), DeprecatedConfig("spark.rpc", "2.0", "Not used anymore."), DeprecatedConfig("spark.scheduler.executorTaskBlacklistTime", "2.1.0", "Please use the new excludedOnFailure options, spark.excludeOnFailure.*"), diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index efffda43695cc..6902fb6d236de 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -74,12 +74,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager import SortShuffleManager._ - if (!conf.getBoolean("spark.shuffle.spill", true)) { - logWarning( - "spark.shuffle.spill was set to false, but this configuration is ignored as of Spark 1.6+." + - " Shuffle will continue to spill to disk when necessary.") - } - /** * A mapping from shuffle ids to the task ids of mappers producing output for those shuffles. */ From 13315eeec07e2aebcc05dfee762bbd060ae192ec Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 4 Dec 2024 10:32:12 +0900 Subject: [PATCH 088/438] [SPARK-50481][CORE] Improve `SortShuffleManager.unregisterShuffle` to skip checksum file logic if checksum is disabled ### What changes were proposed in this pull request? This PR aims to improve `SortShuffleManager.unregisterShuffle` to skip checksum file logic if checksum is disabled. ### Why are the changes needed? `SortShuffleManager.unregisterShuffle` depends on `IndexShuffleBlockResolver.removeDataByMap`. https://github.com/apache/spark/blob/7b974ca758961668a26a1d0c60c91614dac38742/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala#L185-L192 It always tries to check and delete the checksum files even when they doesn't exists. https://github.com/apache/spark/blob/7b974ca758961668a26a1d0c60c91614dac38742/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala#L198-L201 This PR aims to improve Spark by removing these operations when `checksum` is disabled. ### 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 #49046 from dongjoon-hyun/SPARK-50481. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- .../spark/shuffle/IndexShuffleBlockResolver.scala | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 30bc1382fb021..bf3117a9a9b12 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -85,6 +85,9 @@ private[spark] class IndexShuffleBlockResolver( private val remoteShuffleMaxDisk: Option[Long] = conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE) + private val checksumEnabled = conf.get(config.SHUFFLE_CHECKSUM_ENABLED) + private lazy val algorithm = conf.get(config.SHUFFLE_CHECKSUM_ALGORITHM) + def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None) /** @@ -195,9 +198,11 @@ private[spark] class IndexShuffleBlockResolver( logWarning(log"Error deleting index ${MDC(PATH, file.getPath())}") } - file = getChecksumFile(shuffleId, mapId, conf.get(config.SHUFFLE_CHECKSUM_ALGORITHM)) - if (file.exists() && !file.delete()) { - logWarning(log"Error deleting checksum ${MDC(PATH, file.getPath())}") + if (checksumEnabled) { + file = getChecksumFile(shuffleId, mapId, algorithm) + if (file.exists() && !file.delete()) { + logWarning(log"Error deleting checksum ${MDC(PATH, file.getPath())}") + } } } @@ -396,8 +401,7 @@ private[spark] class IndexShuffleBlockResolver( val (checksumFileOpt, checksumTmpOpt) = if (checksumEnabled) { assert(lengths.length == checksums.length, "The size of partition lengths and checksums should be equal") - val checksumFile = - getChecksumFile(shuffleId, mapId, conf.get(config.SHUFFLE_CHECKSUM_ALGORITHM)) + val checksumFile = getChecksumFile(shuffleId, mapId, algorithm) (Some(checksumFile), Some(createTempFile(checksumFile))) } else { (None, None) From 0e45e217996248a093cfcb892407bb523f885c08 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 4 Dec 2024 13:13:12 +0900 Subject: [PATCH 089/438] [SPARK-50486][PYTHON][DOCS] Refine the docstring for string functions - part 2 ### What changes were proposed in this pull request? Refine the docstring for string functions ### Why are the changes needed? to improve the docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? new doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49049 from zhengruifeng/py_doc_13. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 491 ++++++++++++++++-------- 1 file changed, 340 insertions(+), 151 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 13b2e12bca330..391bc3db7a86f 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -7343,6 +7343,12 @@ def rand(seed: Optional[int] = None) -> Column: :class:`~pyspark.sql.Column` A column of random values. + See Also + -------- + :meth:`pyspark.sql.functions.randn` + :meth:`pyspark.sql.functions.randstr` + :meth:`pyspark.sql.functions.uniform` + Examples -------- Example 1: Generate a random column without a seed @@ -7396,6 +7402,12 @@ def randn(seed: Optional[int] = None) -> Column: :class:`~pyspark.sql.Column` A column of random values. + See Also + -------- + :meth:`pyspark.sql.functions.rand` + :meth:`pyspark.sql.functions.randstr` + :meth:`pyspark.sql.functions.uniform` + Examples -------- Example 1: Generate a random column without a seed @@ -14015,6 +14027,13 @@ def instr(str: "ColumnOrName", substr: Union[Column, str]) -> Column: :class:`~pyspark.sql.Column` location of the first occurrence of the substring as integer. + See Also + -------- + :meth:`pyspark.sql.functions.locate` + :meth:`pyspark.sql.functions.substr` + :meth:`pyspark.sql.functions.substring` + :meth:`pyspark.sql.functions.substring_index` + Examples -------- Example 1: Using a literal string as the 'substring' @@ -14166,6 +14185,11 @@ def sentences( :class:`~pyspark.sql.Column` arrays of split sentences. + See Also + -------- + :meth:`pyspark.sql.functions.split` + :meth:`pyspark.sql.functions.split_part` + Examples -------- >>> from pyspark.sql import functions as sf @@ -14221,15 +14245,15 @@ def substring( Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - pos : :class:`~pyspark.sql.Column` or str or int + pos : :class:`~pyspark.sql.Column` or column name or int starting position in str. .. versionchanged:: 4.0.0 `pos` now accepts column and column name. - len : :class:`~pyspark.sql.Column` or str or int + len : :class:`~pyspark.sql.Column` or column name or int length of chars. .. versionchanged:: 4.0.0 @@ -14240,6 +14264,14 @@ def substring( :class:`~pyspark.sql.Column` substring of given value. + See Also + -------- + :meth:`pyspark.sql.functions.instr` + :meth:`pyspark.sql.functions.locate` + :meth:`pyspark.sql.functions.substr` + :meth:`pyspark.sql.functions.substring_index` + :meth:`pyspark.sql.Column.substr` + Examples -------- Example 1: Using literal integers as arguments @@ -14318,9 +14350,9 @@ def substring_index(str: "ColumnOrName", delim: str, count: int) -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - delim : str + delim : literal string delimiter of values. count : int number of occurrences. @@ -14330,13 +14362,31 @@ def substring_index(str: "ColumnOrName", delim: str, count: int) -> Column: :class:`~pyspark.sql.Column` substring of given value. + See Also + -------- + :meth:`pyspark.sql.functions.instr` + :meth:`pyspark.sql.functions.locate` + :meth:`pyspark.sql.functions.substr` + :meth:`pyspark.sql.functions.substring` + :meth:`pyspark.sql.Column.substr` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('a.b.c.d',)], ['s']) - >>> df.select(substring_index(df.s, '.', 2).alias('s')).collect() - [Row(s='a.b')] - >>> df.select(substring_index(df.s, '.', -3).alias('s')).collect() - [Row(s='b.c.d')] + >>> df.select('*', sf.substring_index(df.s, '.', 2)).show() + +-------+------------------------+ + | s|substring_index(s, ., 2)| + +-------+------------------------+ + |a.b.c.d| a.b| + +-------+------------------------+ + + >>> df.select('*', sf.substring_index('s', '.', -3)).show() + +-------+-------------------------+ + | s|substring_index(s, ., -3)| + +-------+-------------------------+ + |a.b.c.d| b.c.d| + +-------+-------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -14358,16 +14408,15 @@ def levenshtein( Parameters ---------- - left : :class:`~pyspark.sql.Column` or str + left : :class:`~pyspark.sql.Column` or column name first column value. - right : :class:`~pyspark.sql.Column` or str + right : :class:`~pyspark.sql.Column` or column name second column value. threshold : int, optional if set when the levenshtein distance of the two given strings less than or equal to a given threshold then return result distance, or -1 - .. versionchanged: 3.5.0 - Added ``threshold`` argument. + .. versionadded: 3.5.0 Returns ------- @@ -14376,11 +14425,21 @@ def levenshtein( Examples -------- - >>> df0 = spark.createDataFrame([('kitten', 'sitting',)], ['l', 'r']) - >>> df0.select(levenshtein('l', 'r').alias('d')).collect() - [Row(d=3)] - >>> df0.select(levenshtein('l', 'r', 2).alias('d')).collect() - [Row(d=-1)] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('kitten', 'sitting',)], ['l', 'r']) + >>> df.select('*', sf.levenshtein('l', 'r')).show() + +------+-------+-----------------+ + | l| r|levenshtein(l, r)| + +------+-------+-----------------+ + |kitten|sitting| 3| + +------+-------+-----------------+ + + >>> df.select('*', sf.levenshtein(df.l, df.r, 2)).show() + +------+-------+--------------------+ + | l| r|levenshtein(l, r, 2)| + +------+-------+--------------------+ + |kitten|sitting| -1| + +------+-------+--------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -14404,9 +14463,9 @@ def locate(substr: str, str: "ColumnOrName", pos: int = 1) -> Column: Parameters ---------- - substr : str + substr : literal string a string - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name a Column of :class:`pyspark.sql.types.StringType` pos : int, optional start position (zero based) @@ -14421,11 +14480,31 @@ def locate(substr: str, str: "ColumnOrName", pos: int = 1) -> Column: The position is not zero based, but 1 based index. Returns 0 if substr could not be found in str. + See Also + -------- + :meth:`pyspark.sql.functions.instr` + :meth:`pyspark.sql.functions.substr` + :meth:`pyspark.sql.functions.substring` + :meth:`pyspark.sql.functions.substring_index` + :meth:`pyspark.sql.Column.substr` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('abcd',)], ['s',]) - >>> df.select(locate('b', df.s, 1).alias('s')).collect() - [Row(s=2)] + >>> df.select('*', sf.locate('b', 's', 1)).show() + +----+---------------+ + | s|locate(b, s, 1)| + +----+---------------+ + |abcd| 2| + +----+---------------+ + + >>> df.select('*', sf.locate('b', df.s, 3)).show() + +----+---------------+ + | s|locate(b, s, 3)| + +----+---------------+ + |abcd| 0| + +----+---------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -14469,6 +14548,10 @@ def lpad( :class:`~pyspark.sql.Column` left padded result. + See Also + -------- + :meth:`pyspark.sql.functions.rpad` + Examples -------- Example 1: Pad with a literal string @@ -14535,6 +14618,10 @@ def rpad( :class:`~pyspark.sql.Column` right padded result. + See Also + -------- + :meth:`pyspark.sql.functions.lpad` + Examples -------- Example 1: Pad with a literal string @@ -14578,9 +14665,9 @@ def repeat(col: "ColumnOrName", n: Union["ColumnOrName", int]) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. - n : :class:`~pyspark.sql.Column` or str or int + n : :class:`~pyspark.sql.Column` or column name or int number of times to repeat value. .. versionchanged:: 4.0.0 @@ -14593,35 +14680,35 @@ def repeat(col: "ColumnOrName", n: Union["ColumnOrName", int]) -> Column: Examples -------- - >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [('ab',)], ['s',] - ... ).select(sf.repeat("s", 3)).show() - +------------+ - |repeat(s, 3)| - +------------+ - | ababab| - +------------+ + Example 1: Repeat with a constant number of times >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [('ab',)], ['s',] - ... ).select(sf.repeat("s", sf.lit(4))).show() - +------------+ - |repeat(s, 4)| - +------------+ - | abababab| - +------------+ + >>> df = spark.createDataFrame([('ab',)], ['s',]) + >>> df.select("*", sf.repeat("s", 3)).show() + +---+------------+ + | s|repeat(s, 3)| + +---+------------+ + | ab| ababab| + +---+------------+ + + >>> df.select("*", sf.repeat(df.s, sf.lit(4))).show() + +---+------------+ + | s|repeat(s, 4)| + +---+------------+ + | ab| abababab| + +---+------------+ + + Example 2: Repeat with a column containing different number of times >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [('ab', 5,)], ['s', 't'] - ... ).select(sf.repeat("s", 't')).show() - +------------+ - |repeat(s, t)| - +------------+ - | ababababab| - +------------+ + >>> df = spark.createDataFrame([('ab', 5,), ('abc', 6,)], ['s', 't']) + >>> df.select("*", sf.repeat("s", "t")).show() + +---+---+------------------+ + | s| t| repeat(s, t)| + +---+---+------------------+ + | ab| 5| ababababab| + |abc| 6|abcabcabcabcabcabc| + +---+---+------------------+ """ n = _enum_to_value(n) n = lit(n) if isinstance(n, int) else n @@ -14644,12 +14731,18 @@ def split( Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name a string expression to split - pattern : :class:`~pyspark.sql.Column` or str + pattern : :class:`~pyspark.sql.Column` or literal string a string representing a regular expression. The regex string should be a Java regular expression. - limit : :class:`~pyspark.sql.Column` or str or int + + .. versionchanged:: 4.0.0 + `pattern` now accepts column. Does not accept column name since string type remain + accepted as a regular expression representation, for backwards compatibility. + In addition to int, `limit` now accepts column and column name. + + limit : :class:`~pyspark.sql.Column` or column name or int an integer which controls the number of times `pattern` is applied. * ``limit > 0``: The resulting array's length will not be more than `limit`, and the @@ -14661,61 +14754,66 @@ def split( .. versionchanged:: 3.0 `split` now takes an optional `limit` field. If not provided, default limit value is -1. - .. versionchanged:: 4.0.0 - `pattern` now accepts column. Does not accept column name since string type remain - accepted as a regular expression representation, for backwards compatibility. - In addition to int, `limit` now accepts column and column name. - Returns ------- :class:`~pyspark.sql.Column` array of separated strings. + See Also + -------- + :meth:`pyspark.sql.functions.sentences` + :meth:`pyspark.sql.functions.split_part` + Examples -------- - >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame([('oneAtwoBthreeC',)], ['s',]) - >>> df.select(sf.split(df.s, '[ABC]', 2).alias('s')).show() - +-----------------+ - | s| - +-----------------+ - |[one, twoBthreeC]| - +-----------------+ + Example 1: Repeat with a constant pattern >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([('oneAtwoBthreeC',)], ['s',]) - >>> df.select(sf.split(df.s, '[ABC]', -1).alias('s')).show() - +-------------------+ - | s| - +-------------------+ - |[one, two, three, ]| - +-------------------+ - - >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame( - ... [('oneAtwoBthreeC', '[ABC]'), ('1A2B3C', '[1-9]+'), ('aa2bb3cc4', '[1-9]+')], - ... ['s', 'pattern'] - ... ) - >>> df.select(sf.split(df.s, df.pattern).alias('s')).show() - +-------------------+ - | s| - +-------------------+ - |[one, two, three, ]| - | [, A, B, C]| - | [aa, bb, cc, ]| - +-------------------+ + >>> df.select('*', sf.split(df.s, '[ABC]')).show() + +--------------+-------------------+ + | s|split(s, [ABC], -1)| + +--------------+-------------------+ + |oneAtwoBthreeC|[one, two, three, ]| + +--------------+-------------------+ + + >>> df.select('*', sf.split(df.s, '[ABC]', 2)).show() + +--------------+------------------+ + | s|split(s, [ABC], 2)| + +--------------+------------------+ + |oneAtwoBthreeC| [one, twoBthreeC]| + +--------------+------------------+ + + >>> df.select('*', sf.split('s', '[ABC]', -2)).show() + +--------------+-------------------+ + | s|split(s, [ABC], -2)| + +--------------+-------------------+ + |oneAtwoBthreeC|[one, two, three, ]| + +--------------+-------------------+ + + Example 2: Repeat with a column containing different patterns and limits >>> import pyspark.sql.functions as sf - >>> df = spark.createDataFrame( - ... [('oneAtwoBthreeC', '[ABC]', 2), ('1A2B3C', '[1-9]+', -1)], - ... ['s', 'pattern', 'expected_parts'] - ... ) - >>> df.select(sf.split(df.s, df.pattern, df.expected_parts).alias('s')).show() + >>> df = spark.createDataFrame([ + ... ('oneAtwoBthreeC', '[ABC]', 2), + ... ('1A2B3C', '[1-9]+', 1), + ... ('aa2bb3cc4', '[1-9]+', -1)], ['s', 'p', 'l']) + >>> df.select('*', sf.split(df.s, df.p)).show() + +--------------+------+---+-------------------+ + | s| p| l| split(s, p, -1)| + +--------------+------+---+-------------------+ + |oneAtwoBthreeC| [ABC]| 2|[one, two, three, ]| + | 1A2B3C|[1-9]+| 1| [, A, B, C]| + | aa2bb3cc4|[1-9]+| -1| [aa, bb, cc, ]| + +--------------+------+---+-------------------+ + + >>> df.select(sf.split('s', df.p, 'l')).show() +-----------------+ - | s| + | split(s, p, l)| +-----------------+ |[one, twoBthreeC]| - | [, A, B, C]| + | [1A2B3C]| + | [aa, bb, cc, ]| +-----------------+ """ limit = _enum_to_value(limit) @@ -14880,16 +14978,29 @@ def randstr(length: Union[Column, int], seed: Optional[Union[Column, int]] = Non :class:`~pyspark.sql.Column` The generated random string with the specified length. + See Also + -------- + :meth:`pyspark.sql.functions.rand` + :meth:`pyspark.sql.functions.randn` + Examples -------- - >>> spark.createDataFrame([('3',)], ['a']) \\ - ... .select(randstr(lit(5), lit(0)).alias('result')) \\ - ... .selectExpr("length(result) > 0").show() - +--------------------+ - |(length(result) > 0)| - +--------------------+ - | true| - +--------------------+ + >>> import pyspark.sql.functions as sf + >>> spark.range(0, 10, 1, 1).select(sf.randstr(16, 3)).show() + +----------------+ + | randstr(16, 3)| + +----------------+ + |nurJIpH4cmmMnsCG| + |fl9YtT5m01trZtIt| + |PD19rAgscTHS7qQZ| + |2CuAICF5UJOruVv4| + |kNZEs8nDpJEoz3Rl| + |OXiU0KN5eaXfjXFs| + |qfnTM1BZAHtN0gBV| + |1p8XiSKwg33KnRPK| + |od5y5MucayQq1bKK| + |tklYPmKmc5sIppWM| + +----------------+ """ length = _enum_to_value(length) length = lit(length) @@ -15137,7 +15248,7 @@ def initcap(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -15147,8 +15258,14 @@ def initcap(col: "ColumnOrName") -> Column: Examples -------- - >>> spark.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect() - [Row(v='Ab Cd')] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('ab cd',)], ['a']) + >>> df.select("*", sf.initcap("a")).show() + +-----+----------+ + | a|initcap(a)| + +-----+----------+ + |ab cd| Ab Cd| + +-----+----------+ """ return _invoke_function_over_columns("initcap", col) @@ -15165,7 +15282,7 @@ def soundex(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -15175,9 +15292,15 @@ def soundex(col: "ColumnOrName") -> Column: Examples -------- - >>> df = spark.createDataFrame([("Peters",),("Uhrbach",)], ['name']) - >>> df.select(soundex(df.name).alias("soundex")).collect() - [Row(soundex='P362'), Row(soundex='U612')] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([("Peters",),("Uhrbach",)], ["s"]) + >>> df.select("*", sf.soundex("s")).show() + +-------+----------+ + | s|soundex(s)| + +-------+----------+ + | Peters| P362| + |Uhrbach| U612| + +-------+----------+ """ return _invoke_function_over_columns("soundex", col) @@ -15193,7 +15316,7 @@ def bin(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -15203,9 +15326,22 @@ def bin(col: "ColumnOrName") -> Column: Examples -------- - >>> df = spark.createDataFrame([2,5], "INT") - >>> df.select(bin(df.value).alias('c')).collect() - [Row(c='10'), Row(c='101')] + >>> import pyspark.sql.functions as sf + >>> spark.range(10).select("*", sf.bin("id")).show() + +---+-------+ + | id|bin(id)| + +---+-------+ + | 0| 0| + | 1| 1| + | 2| 10| + | 3| 11| + | 4| 100| + | 5| 101| + | 6| 110| + | 7| 111| + | 8| 1000| + | 9| 1001| + +---+-------+ """ return _invoke_function_over_columns("bin", col) @@ -15223,9 +15359,13 @@ def hex(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. + See Also + -------- + :meth:`pyspark.sql.functions.unhex` + Returns ------- :class:`~pyspark.sql.Column` @@ -15233,8 +15373,14 @@ def hex(col: "ColumnOrName") -> Column: Examples -------- - >>> spark.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() - [Row(hex(a)='414243', hex(b)='3')] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('ABC', 3)], ['a', 'b']) + >>> df.select('*', sf.hex('a'), sf.hex(df.b)).show() + +---+---+------+------+ + | a| b|hex(a)|hex(b)| + +---+---+------+------+ + |ABC| 3|414243| 3| + +---+---+------+------+ """ return _invoke_function_over_columns("hex", col) @@ -15251,9 +15397,13 @@ def unhex(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. + See Also + -------- + :meth:`pyspark.sql.functions.hex` + Returns ------- :class:`~pyspark.sql.Column` @@ -15261,8 +15411,14 @@ def unhex(col: "ColumnOrName") -> Column: Examples -------- - >>> spark.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect() - [Row(unhex(a)=bytearray(b'ABC'))] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('414243',)], ['a']) + >>> df.select('*', sf.unhex('a')).show() + +------+----------+ + | a| unhex(a)| + +------+----------+ + |414243|[41 42 43]| + +------+----------+ """ return _invoke_function_over_columns("unhex", col) @@ -15297,14 +15453,22 @@ def uniform( Examples -------- - >>> spark.createDataFrame([('3',)], ['a']) \\ - ... .select(uniform(lit(0), lit(10), lit(0)).alias('result')) \\ - ... .selectExpr("result < 15").show() - +-------------+ - |(result < 15)| - +-------------+ - | true| - +-------------+ + >>> import pyspark.sql.functions as sf + >>> spark.range(0, 10, 1, 1).select(sf.uniform(5, 105, 3)).show() + +------------------+ + |uniform(5, 105, 3)| + +------------------+ + | 30| + | 71| + | 99| + | 77| + | 16| + | 25| + | 89| + | 80| + | 51| + | 83| + +------------------+ """ min = _enum_to_value(min) min = lit(min) @@ -15667,18 +15831,35 @@ def split_part(src: "ColumnOrName", delimiter: "ColumnOrName", partNum: "ColumnO Parameters ---------- - src : :class:`~pyspark.sql.Column` or str + src : :class:`~pyspark.sql.Column` or column name A column of string to be splited. - delimiter : :class:`~pyspark.sql.Column` or str + delimiter : :class:`~pyspark.sql.Column` or column name A column of string, the delimiter used for split. - partNum : :class:`~pyspark.sql.Column` or str + partNum : :class:`~pyspark.sql.Column` or column name A column of string, requested part of the split (1-based). + See Also + -------- + :meth:`pyspark.sql.functions.sentences` + :meth:`pyspark.sql.functions.split` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("11.12.13", ".", 3,)], ["a", "b", "c"]) - >>> df.select(split_part(df.a, df.b, df.c).alias('r')).collect() - [Row(r='13')] + >>> df.select("*", sf.split_part("a", "b", "c")).show() + +--------+---+---+-------------------+ + | a| b| c|split_part(a, b, c)| + +--------+---+---+-------------------+ + |11.12.13| .| 3| 13| + +--------+---+---+-------------------+ + + >>> df.select("*", sf.split_part(df.a, df.b, sf.lit(-2))).show() + +--------+---+---+--------------------+ + | a| b| c|split_part(a, b, -2)| + +--------+---+---+--------------------+ + |11.12.13| .| 3| 12| + +--------+---+---+--------------------+ """ return _invoke_function_over_columns("split_part", src, delimiter, partNum) @@ -15695,34 +15876,42 @@ def substr( Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name A column of string. - pos : :class:`~pyspark.sql.Column` or str + pos : :class:`~pyspark.sql.Column` or column name A column of string, the substring of `str` that starts at `pos`. - len : :class:`~pyspark.sql.Column` or str, optional + len : :class:`~pyspark.sql.Column` or column name, optional A column of string, the substring of `str` is of length `len`. + Returns + ------- + :class:`~pyspark.sql.Column` + substring of given value. + + See Also + -------- + :meth:`pyspark.sql.functions.instr` + :meth:`pyspark.sql.functions.substring` + :meth:`pyspark.sql.functions.substring_index` + :meth:`pyspark.sql.Column.substr` + Examples -------- - >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [("Spark SQL", 5, 1,)], ["a", "b", "c"] - ... ).select(sf.substr("a", "b", "c")).show() - +---------------+ - |substr(a, b, c)| - +---------------+ - | k| - +---------------+ + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([("Spark SQL", 5, 1,)], ["a", "b", "c"]) + >>> df.select("*", sf.substr("a", "b", "c")).show() + +---------+---+---+---------------+ + | a| b| c|substr(a, b, c)| + +---------+---+---+---------------+ + |Spark SQL| 5| 1| k| + +---------+---+---+---------------+ - >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [("Spark SQL", 5, 1,)], ["a", "b", "c"] - ... ).select(sf.substr("a", "b")).show() - +------------------------+ - |substr(a, b, 2147483647)| - +------------------------+ - | k SQL| - +------------------------+ + >>> df.select("*", sf.substr(df.a, df.b)).show() + +---------+---+---+------------------------+ + | a| b| c|substr(a, b, 2147483647)| + +---------+---+---+------------------------+ + |Spark SQL| 5| 1| k SQL| + +---------+---+---+------------------------+ """ if len is not None: return _invoke_function_over_columns("substr", str, pos, len) From 784a97b3ee8c63a023747f85e57a2f2d89cae3f6 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 4 Dec 2024 16:16:30 +0900 Subject: [PATCH 090/438] [MINOR][INFRA] Update labeler ### What changes were proposed in this pull request? Update labeler ### Why are the changes needed? some labels are not up-to-date, e.g. `python/pyspark/sql/connect/avro` is not covered by `AVRO` ### 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 #49051 from zhengruifeng/update_labeler. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .github/labeler.yml | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/.github/labeler.yml b/.github/labeler.yml index 6617acbf9187e..be78eba4baf8b 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -93,9 +93,9 @@ SQL: - changed-files: - all-globs-to-any-file: [ '**/sql/**/*', - '!python/pyspark/sql/avro/**/*', - '!python/pyspark/sql/streaming/**/*', - '!python/pyspark/sql/tests/streaming/test_streaming*.py' + '!python/**/avro/**/*', + '!python/**/protobuf/**/*', + '!python/**/streaming/**/*' ] - any-glob-to-any-file: [ 'common/unsafe/**/*', @@ -119,7 +119,7 @@ AVRO: - changed-files: - any-glob-to-any-file: [ 'connector/avro/**/*', - 'python/pyspark/sql/avro/**/*' + 'python/**/avro/**/*' ] DSTREAM: @@ -152,9 +152,8 @@ ML: MLLIB: - changed-files: - any-glob-to-any-file: [ - '**/spark/mllib/**/*', - 'mllib-local/**/*', - 'python/pyspark/mllib/**/*' + '**/mllib/**/*', + 'mllib-local/**/*' ] STRUCTURED STREAMING: @@ -162,8 +161,7 @@ STRUCTURED STREAMING: - any-glob-to-any-file: [ '**/sql/**/streaming/**/*', 'connector/kafka-0-10-sql/**/*', - 'python/pyspark/sql/streaming/**/*', - 'python/pyspark/sql/tests/streaming/test_streaming*.py', + 'python/pyspark/sql/**/streaming/**/*', '**/*streaming.R' ] @@ -226,13 +224,12 @@ CONNECT: - any-glob-to-any-file: [ 'sql/connect/**/*', 'connector/connect/**/*', - 'python/pyspark/sql/**/connect/**/*', - 'python/pyspark/ml/**/connect/**/*' + 'python/**/connect/**/*' ] PROTOBUF: - changed-files: - any-glob-to-any-file: [ 'connector/protobuf/**/*', - 'python/pyspark/sql/protobuf/**/*' + 'python/**/protobuf/**/*' ] From 5fc6b713874a0617cc8ff4149f01e17c40623528 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Wed, 4 Dec 2024 09:29:40 +0100 Subject: [PATCH 091/438] [SPARK-50405][SQL] Handle collation type coercion of complex data types properly ### What changes were proposed in this pull request? This pull request generalizes collation type coercion to support not just casting all children to a single string type, but also handling complex data types such as structs, maps, and arrays (arrays partially worked already). The core idea is to recursively analyze the entire data type of an expression, annotating each StringType within it with the highest-priority collation and its strength. This annotation propagates upward through the expression tree. Once the root of the expression is reached, the annotations are removed, and the expression is cast to the desired data type. For the root expression `e`, the collation data type context is computed by first calculating the context for all its children and then merging those results into the data type of `e`. ### Why are the changes needed? In #48663, a new approach to calculating collation precedence was introduced. This approach recursively examines the children of an expression and propagates the collation with the highest priority upward. However, the current implementation of collation coercion is limited to determining the StringType that all children should be cast to. This approach falls short when dealing with complex types like structs, maps, and arrays, which can also contain collations. To address this limitation, we need a more general mechanism that allows coercion of any data type, not just simple strings. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? With new unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48936 from stefankandic/collationCoercionComplex. Lead-authored-by: Stefan Kandic Co-authored-by: Stefan Kandic <154237371+stefankandic@users.noreply.github.com> Signed-off-by: Max Gekk --- .../analysis/CollationTypeCoercion.scala | 350 +++++++++++------- .../analyzer-results/collations.sql.out | 78 ++-- .../sql-tests/results/collations.sql.out | 28 +- .../spark/sql/CollationSQLRegexpSuite.scala | 3 +- .../CollationTypePrecedenceSuite.scala | 158 +++++++- 5 files changed, 433 insertions(+), 184 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index cca1d21df3a7e..02640aba2d283 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -17,14 +17,12 @@ package org.apache.spark.sql.catalyst.analysis -import scala.annotation.tailrec - import org.apache.spark.sql.catalyst.analysis.CollationStrength.{Default, Explicit, Implicit} -import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType, haveSameType} +import org.apache.spark.sql.catalyst.analysis.TypeCoercion.haveSameType import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{ArrayType, DataType, StringType} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType, StructType} import org.apache.spark.sql.util.SchemaUtils /** @@ -32,7 +30,7 @@ import org.apache.spark.sql.util.SchemaUtils */ object CollationTypeCoercion { - private val COLLATION_CONTEXT_TAG = new TreeNodeTag[CollationContext]("collationContext") + private val COLLATION_CONTEXT_TAG = new TreeNodeTag[DataType]("collationContext") private def hasCollationContextTag(expr: Expression): Boolean = { expr.getTagValue(COLLATION_CONTEXT_TAG).isDefined @@ -53,10 +51,10 @@ object CollationTypeCoercion { outputStringType match { case Some(st) => val newBranches = caseWhenExpr.branches.map { case (condition, value) => - (condition, castStringType(value, st)) + (condition, changeType(value, st)) } val newElseValue = - caseWhenExpr.elseValue.map(e => castStringType(e, st)) + caseWhenExpr.elseValue.map(e => changeType(e, st)) CaseWhen(newBranches, newElseValue) case _ => @@ -105,11 +103,9 @@ object CollationTypeCoercion { val newValues = collateToSingleType(mapCreate.values) mapCreate.withNewChildren(newKeys.zip(newValues).flatMap(pair => Seq(pair._1, pair._2))) - case namedStruct: CreateNamedStruct if namedStruct.children.size % 2 == 0 => - val newNames = collateToSingleType(namedStruct.nameExprs) - val newValues = collateToSingleType(namedStruct.valExprs) - val interleaved = newNames.zip(newValues).flatMap(pair => Seq(pair._1, pair._2)) - namedStruct.withNewChildren(interleaved) + case namedStruct: CreateNamedStruct => + // since each child is separate we should not coerce them at all + namedStruct case splitPart: SplitPart => val Seq(str, delimiter, partNum) = splitPart.children @@ -164,31 +160,89 @@ object CollationTypeCoercion { } /** - * Extracts StringTypes from filtered hasStringType + * Changes the data type of the expression to the given `newType`. */ - @tailrec - private def extractStringType(dt: DataType): Option[StringType] = dt match { - case st: StringType => Some(st) - case ArrayType(et, _) => extractStringType(et) - case _ => None + private def changeType(expr: Expression, newType: DataType): Expression = { + mergeTypes(expr.dataType, newType) match { + case Some(newDataType) if newDataType != expr.dataType => + assert(!newDataType.existsRecursively(_.isInstanceOf[StringTypeWithContext])) + + val exprWithNewType = expr match { + case lit: Literal => lit.copy(dataType = newDataType) + case cast: Cast => cast.copy(dataType = newDataType) + case _ => Cast(expr, newDataType) + } + + // also copy the collation context tag + if (hasCollationContextTag(expr)) { + exprWithNewType.setTagValue( + COLLATION_CONTEXT_TAG, expr.getTagValue(COLLATION_CONTEXT_TAG).get) + } + exprWithNewType + + case _ => + expr + } } /** - * Casts given expression to collated StringType with id equal to collationId only - * if expression has StringType in the first place. + * If possible, returns the new data type from `inType` by applying + * the collation of `castType`. */ - def castStringType(expr: Expression, st: StringType): Expression = { - castStringType(expr.dataType, st) - .map(dt => Cast(expr, dt)) - .getOrElse(expr) + private def mergeTypes(inType: DataType, castType: DataType): Option[DataType] = { + val outType = mergeStructurally(inType, castType) { + case (_: StringType, right: StringTypeWithContext) => + right.stringType + } + + outType } - private def castStringType(inType: DataType, castType: StringType): Option[DataType] = { - inType match { - case st: StringType if st.collationId != castType.collationId => - Some(castType) - case ArrayType(arrType, nullable) => - castStringType(arrType, castType).map(ArrayType(_, nullable)) + /** + * Merges two data types structurally according to the given base case. + */ + private def mergeStructurally( + leftType: DataType, + rightType: DataType) + (baseCase: PartialFunction[(DataType, DataType), DataType]): Option[DataType] = { + (leftType, rightType) match { + + // handle the base cases first + case _ if baseCase.isDefinedAt((leftType, rightType)) => + Option(baseCase(leftType, rightType)) + + case _ if leftType == rightType => + Some(leftType) + + case (ArrayType(leftElemType, nullable), ArrayType(rightElemType, _)) => + mergeStructurally(leftElemType, rightElemType)(baseCase).map(ArrayType(_, nullable)) + + case (MapType(leftKey, leftValue, nullable), MapType(rightKey, rightValue, _)) => + for { + newKeyType <- mergeStructurally(leftKey, rightKey)(baseCase) + newValueType <- mergeStructurally(leftValue, rightValue)(baseCase) + } yield MapType(newKeyType, newValueType, nullable) + + case (ArrayType(elementType, nullable), right) => + mergeStructurally(elementType, right)(baseCase).map(ArrayType(_, nullable)) + + case (left, ArrayType(elementType, _)) => + mergeStructurally(left, elementType)(baseCase) + + case (StructType(leftFields), StructType(rightFields)) => + if (leftFields.length != rightFields.length) { + return None + } + val newFields = leftFields.zip(rightFields).map { + case (leftField, rightField) => + val newType = mergeStructurally(leftField.dataType, rightField.dataType)(baseCase) + if (newType.isEmpty) { + return None + } + leftField.copy(dataType = newType.get) + } + Some(StructType(newFields)) + case _ => None } } @@ -201,7 +255,7 @@ object CollationTypeCoercion { lctOpt match { case Some(lct) => - expressions.map(e => castStringType(e, lct)) + expressions.map(e => changeType(e, lct)) case _ => expressions } @@ -210,7 +264,7 @@ object CollationTypeCoercion { /** * Tries to find the least common StringType among the given expressions. */ - private def findLeastCommonStringType(expressions: Seq[Expression]): Option[StringType] = { + private def findLeastCommonStringType(expressions: Seq[Expression]): Option[DataType] = { if (!expressions.exists(e => SchemaUtils.hasNonUTF8BinaryCollation(e.dataType))) { // if there are no collated types we don't need to do anything return None @@ -223,62 +277,70 @@ object CollationTypeCoercion { val collationContextWinner = expressions.foldLeft(findCollationContext(expressions.head)) { case (Some(left), right) => findCollationContext(right).flatMap { ctx => - collationPrecedenceWinner(left, ctx) + mergeWinner(left, ctx) } - case (None, _) => return None - } - - collationContextWinner.flatMap { cc => - extractStringType(cc.dataType) + case (None, _) => None } + collationContextWinner } /** - * Tries to find the collation context for the given expression. + * Tries to find the data type with the collation context for the given expression. * If found, it will also set the [[COLLATION_CONTEXT_TAG]] on the expression, * so that the context can be reused later. */ - private def findCollationContext(expr: Expression): Option[CollationContext] = { + private def findCollationContext(expr: Expression): Option[DataType] = { val contextOpt = expr match { - case _ if hasCollationContextTag(expr) => - Some(expr.getTagValue(COLLATION_CONTEXT_TAG).get) - - // if `expr` doesn't have a string in its dataType then it doesn't - // have the collation context either - case _ if !expr.dataType.existsRecursively(_.isInstanceOf[StringType]) => - None - case collate: Collate => - Some(CollationContext(collate.dataType, Explicit)) + case _ if collationStrengthBaseCases.isDefinedAt(expr) => + collationStrengthBaseCases(expr) - case _: Alias | _: SubqueryExpression | _: AttributeReference | _: VariableReference => - Some(CollationContext(expr.dataType, Implicit)) + case getStruct: GetStructField => + val childContext = findCollationContext(getStruct.child) + childContext match { + case Some(struct: StructType) => + val field = struct.fields(getStruct.ordinal) + Some(field.dataType) + case _ => None + } - case _: Literal => - Some(CollationContext(expr.dataType, Default)) + case getMapValue: GetMapValue => + findCollationContext(getMapValue.child) match { + case Some(MapType(_, valueType, _)) => + mergeWinner(getMapValue.dataType, valueType) + case _ => + None + } - // if it does have a string type but none of its children do - // then the collation context strength is default - case _ if !expr.children.exists(_.dataType.existsRecursively(_.isInstanceOf[StringType])) => - Some(CollationContext(expr.dataType, Default)) + case struct: CreateNamedStruct => + val childrenContexts = struct.valExprs.map(findCollationContext) + if (childrenContexts.isEmpty) { + return None + } + val newFields = struct.dataType.fields.zip(childrenContexts).map { + case (field, Some(context)) => + field.copy(dataType = context) + case (field, None) => field + } + Some(StructType(newFields)) - case _ => - val contextWinnerOpt = getContextRelevantChildren(expr) - .flatMap(findCollationContext) - .foldLeft(Option.empty[CollationContext]) { - case (Some(left), right) => - collationPrecedenceWinner(left, right) - case (None, right) => - Some(right) - } + case map: CreateMap => + val keyContexts = map.keys.flatMap(findCollationContext) + val valueContexts = map.values.flatMap(findCollationContext) + if (keyContexts.length + valueContexts.length != map.children.length) { + return None + } - contextWinnerOpt.map { context => - if (hasStringType(expr.dataType)) { - CollationContext(expr.dataType, context.strength) - } else { - context - } + val keyContextWinner = mergeWinners(map.dataType.keyType, keyContexts) + val valueContextWinner = mergeWinners(map.dataType.valueType, valueContexts) + if (keyContextWinner.isEmpty || valueContextWinner.isEmpty) { + return None } + Some(MapType(keyContextWinner.get, valueContextWinner.get)) + + case _ => + val childContexts = expr.children.flatMap(findCollationContext) + mergeWinners(expr.dataType, childContexts) } contextOpt.foreach(expr.setTagValue(COLLATION_CONTEXT_TAG, _)) @@ -286,69 +348,91 @@ object CollationTypeCoercion { } /** - * Returns the children of the given expression that should be used for calculating the - * winning collation context. + * Base cases for determining the strength of the collation. */ - private def getContextRelevantChildren(expression: Expression): Seq[Expression] = { - expression match { - // collation context for named struct should be calculated based on its values only - case createStruct: CreateNamedStruct => - createStruct.valExprs + private def collationStrengthBaseCases: PartialFunction[Expression, Option[DataType]] = { + case expr if hasCollationContextTag(expr) => + Some(expr.getTagValue(COLLATION_CONTEXT_TAG).get) - // collation context does not depend on the key for extracting the value - case extract: ExtractValue => - Seq(extract.child) + // if `expr` doesn't have a string in its dataType then it doesn't + // have the collation context either + case expr if !expr.dataType.existsRecursively(_.isInstanceOf[StringType]) => + None - // we currently don't support collation precedence for maps, - // as this would involve calculating them for keys and values separately - case _: CreateMap => - Seq.empty + case collate: Collate => + Some(addContextToStringType(collate.dataType, Explicit)) - case _ => - expression.children + case expr @ (_: Alias | _: SubqueryExpression | _: AttributeReference | _: VariableReference) => + Some(addContextToStringType(expr.dataType, Implicit)) + + case lit: Literal => + Some(addContextToStringType(lit.dataType, Default)) + + // if it does have a string type but none of its children do + // then the collation context strength is default + case expr if !expr.children.exists(_.dataType.existsRecursively(_.isInstanceOf[StringType])) => + Some(addContextToStringType(expr.dataType, Default)) + } + + /** + * Adds collation context to the given string type so we can know its strength. + */ + private def addContextToStringType(dt: DataType, strength: CollationStrength): DataType = { + dt.transformRecursively { + case st: StringType => StringTypeWithContext(st, strength) } } /** - * Returns the collation context that wins in precedence between left and right. + * Merges multiple data types structurally according to strength of the collations into the + * data type of the `start`. + * + * If any of the data types cannot be merged, it returns None. */ - private def collationPrecedenceWinner( - left: CollationContext, - right: CollationContext): Option[CollationContext] = { - - val (leftStringType, rightStringType) = - (extractStringType(left.dataType), extractStringType(right.dataType)) match { - case (Some(l), Some(r)) => - (l, r) - case (None, None) => - return None - case (Some(_), None) => - return Some(left) - case (None, Some(_)) => - return Some(right) - } + private def mergeWinners(start: DataType, rest: Seq[DataType]): Option[DataType] = { + rest.foldLeft(Option(start)) { + case (Some(acc), childContext) => + mergeWinner(acc, childContext) + case (None, _) => + None + } + } - (left.strength, right.strength) match { - case (Explicit, Explicit) if leftStringType != rightStringType => - throw QueryCompilationErrors.explicitCollationMismatchError( - Seq(leftStringType, rightStringType)) + /** + * Merges two data types structurally according to strength of the collations. + */ + private def mergeWinner(left: DataType, right: DataType): Option[DataType] = { + mergeStructurally(left, right) { + case (left: StringTypeWithContext, right: StringTypeWithContext) => + getWinningStringType(left, right) - case (Explicit, _) => Some(left) - case (_, Explicit) => Some(right) + case (_: StringType, right: StringTypeWithContext) => + right + } + } - case (Implicit, Implicit) if leftStringType != rightStringType => + /** Determines the winning StringTypeWithContext based on the strength of the collation. */ + private def getWinningStringType( + left: StringTypeWithContext, + right: StringTypeWithContext): StringTypeWithContext = { + def handleMismatch(): Nothing = { + if (left.strength == Explicit) { + throw QueryCompilationErrors.explicitCollationMismatchError( + Seq(left.stringType, right.stringType)) + } else { throw QueryCompilationErrors.implicitCollationMismatchError( - Seq(leftStringType, rightStringType)) - - case (Implicit, _) => Some(left) - case (_, Implicit) => Some(right) + Seq(left.stringType, right.stringType)) + } + } - case (Default, Default) if leftStringType != rightStringType => - throw QueryCompilationErrors.implicitCollationMismatchError( - Seq(leftStringType, rightStringType)) + (left.strength.priority, right.strength.priority) match { + case (leftPriority, rightPriority) if leftPriority == rightPriority => + if (left.sameType(right)) left + else handleMismatch() - case _ => - Some(left) + case (leftPriority, rightPriority) => + if (leftPriority < rightPriority) left + else right } } } @@ -356,18 +440,32 @@ object CollationTypeCoercion { /** * Represents the strength of collation used for determining precedence in collation resolution. */ -private sealed trait CollationStrength {} +private sealed trait CollationStrength { + val priority: Int +} private object CollationStrength { - case object Explicit extends CollationStrength {} - case object Implicit extends CollationStrength {} - case object Default extends CollationStrength {} + case object Explicit extends CollationStrength { + override val priority: Int = 0 + } + case object Implicit extends CollationStrength { + override val priority: Int = 1 + } + case object Default extends CollationStrength { + override val priority: Int = 2 + } } /** * Encapsulates the context for collation, including data type and strength. * - * @param dataType The data type associated with this collation context. + * @param stringType StringType. * @param strength The strength level of the collation, which determines its precedence. */ -private case class CollationContext(dataType: DataType, strength: CollationStrength) {} +private case class StringTypeWithContext(stringType: StringType, strength: CollationStrength) + extends DataType { + + override def defaultSize: Int = stringType.defaultSize + + override private[spark] def asNullable: DataType = this +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index 45ab1cdcff79e..0d5c414416d40 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -442,77 +442,77 @@ Project [array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_l -- !query select 'a' collate unicode < 'A' -- !query analysis -Project [(collate(a, unicode) < cast(A as string collate UNICODE)) AS (collate(a, unicode) < A)#x] +Project [(collate(a, unicode) < A) AS (collate(a, unicode) < 'A' collate UNICODE)#x] +- OneRowRelation -- !query select 'a' collate unicode_ci = 'A' -- !query analysis -Project [(collate(a, unicode_ci) = cast(A as string collate UNICODE_CI)) AS (collate(a, unicode_ci) = A)#x] +Project [(collate(a, unicode_ci) = A) AS (collate(a, unicode_ci) = 'A' collate UNICODE_CI)#x] +- OneRowRelation -- !query select 'a' collate unicode_ai = 'å' -- !query analysis -Project [(collate(a, unicode_ai) = cast(å as string collate UNICODE_AI)) AS (collate(a, unicode_ai) = å)#x] +Project [(collate(a, unicode_ai) = å) AS (collate(a, unicode_ai) = 'å' collate UNICODE_AI)#x] +- OneRowRelation -- !query select 'a' collate unicode_ci_ai = 'Å' -- !query analysis -Project [(collate(a, unicode_ci_ai) = cast(Å as string collate UNICODE_CI_AI)) AS (collate(a, unicode_ci_ai) = Å)#x] +Project [(collate(a, unicode_ci_ai) = Å) AS (collate(a, unicode_ci_ai) = 'Å' collate UNICODE_CI_AI)#x] +- OneRowRelation -- !query select 'a' collate en < 'A' -- !query analysis -Project [(collate(a, en) < cast(A as string collate en)) AS (collate(a, en) < A)#x] +Project [(collate(a, en) < A) AS (collate(a, en) < 'A' collate en)#x] +- OneRowRelation -- !query select 'a' collate en_ci = 'A' -- !query analysis -Project [(collate(a, en_ci) = cast(A as string collate en_CI)) AS (collate(a, en_ci) = A)#x] +Project [(collate(a, en_ci) = A) AS (collate(a, en_ci) = 'A' collate en_CI)#x] +- OneRowRelation -- !query select 'a' collate en_ai = 'å' -- !query analysis -Project [(collate(a, en_ai) = cast(å as string collate en_AI)) AS (collate(a, en_ai) = å)#x] +Project [(collate(a, en_ai) = å) AS (collate(a, en_ai) = 'å' collate en_AI)#x] +- OneRowRelation -- !query select 'a' collate en_ci_ai = 'Å' -- !query analysis -Project [(collate(a, en_ci_ai) = cast(Å as string collate en_CI_AI)) AS (collate(a, en_ci_ai) = Å)#x] +Project [(collate(a, en_ci_ai) = Å) AS (collate(a, en_ci_ai) = 'Å' collate en_CI_AI)#x] +- OneRowRelation -- !query select 'Kypper' collate sv < 'Köpfe' -- !query analysis -Project [(collate(Kypper, sv) < cast(Köpfe as string collate sv)) AS (collate(Kypper, sv) < Köpfe)#x] +Project [(collate(Kypper, sv) < Köpfe) AS (collate(Kypper, sv) < 'Köpfe' collate sv)#x] +- OneRowRelation -- !query select 'Kypper' collate de > 'Köpfe' -- !query analysis -Project [(collate(Kypper, de) > cast(Köpfe as string collate de)) AS (collate(Kypper, de) > Köpfe)#x] +Project [(collate(Kypper, de) > Köpfe) AS (collate(Kypper, de) > 'Köpfe' collate de)#x] +- OneRowRelation -- !query select 'I' collate tr_ci = 'ı' -- !query analysis -Project [(collate(I, tr_ci) = cast(ı as string collate tr_CI)) AS (collate(I, tr_ci) = ı)#x] +Project [(collate(I, tr_ci) = ı) AS (collate(I, tr_ci) = 'ı' collate tr_CI)#x] +- OneRowRelation @@ -826,7 +826,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d -- !query select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 -- !query analysis -Project [concat_ws(cast( as string collate UTF8_LCASE), utf8_lcase#x, utf8_lcase#x) AS concat_ws( , utf8_lcase, utf8_lcase)#x] +Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws( , utf8_lcase, utf8_lcase)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -868,7 +868,7 @@ Project [concat_ws(collate( , utf8_lcase), cast(utf8_binary#x as string collate -- !query select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t5 -- !query analysis -Project [concat_ws(cast(, as string collate UTF8_LCASE), utf8_lcase#x, cast(word as string collate UTF8_LCASE)) AS concat_ws(,, utf8_lcase, word)#x, concat_ws(,, utf8_binary#x, word) AS concat_ws(,, utf8_binary, word)#x] +Project [concat_ws(,, utf8_lcase#x, word) AS concat_ws(,, utf8_lcase, word)#x, concat_ws(,, utf8_binary#x, word) AS concat_ws(,, utf8_binary, word)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -876,7 +876,7 @@ Project [concat_ws(cast(, as string collate UTF8_LCASE), utf8_lcase#x, cast(word -- !query select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t5 -- !query analysis -Project [concat_ws(,, cast(utf8_lcase#x as string), collate(word, utf8_binary)) AS concat_ws(,, utf8_lcase, collate(word, utf8_binary))#x, concat_ws(cast(, as string collate UTF8_LCASE), cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase)) AS concat_ws(,, utf8_binary, collate(word, utf8_lcase))#x] +Project [concat_ws(,, cast(utf8_lcase#x as string), collate(word, utf8_binary)) AS concat_ws(,, utf8_lcase, collate(word, utf8_binary))#x, concat_ws(,, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase)) AS concat_ws(,, utf8_binary, collate(word, utf8_lcase))#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -934,7 +934,7 @@ Project [elt(1, collate(utf8_binary#x, utf8_binary), cast(utf8_lcase#x as string -- !query select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5 -- !query analysis -Project [elt(1, utf8_binary#x, word, true) AS elt(1, utf8_binary, word)#x, elt(1, utf8_lcase#x, cast(word as string collate UTF8_LCASE), true) AS elt(1, utf8_lcase, word)#x] +Project [elt(1, utf8_binary#x, word, true) AS elt(1, utf8_binary, word)#x, elt(1, utf8_lcase#x, word, true) AS elt(1, utf8_lcase, 'word' collate UTF8_LCASE)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1024,7 +1024,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t5 -- !query analysis -Project [split_part(utf8_binary#x, a, 3) AS split_part(utf8_binary, a, 3)#x, split_part(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 3) AS split_part(utf8_lcase, a, 3)#x] +Project [split_part(utf8_binary#x, a, 3) AS split_part(utf8_binary, a, 3)#x, split_part(utf8_lcase#x, a, 3) AS split_part(utf8_lcase, a, 3)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1122,7 +1122,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t5 -- !query analysis -Project [Contains(utf8_binary#x, a) AS contains(utf8_binary, a)#x, Contains(utf8_lcase#x, cast(a as string collate UTF8_LCASE)) AS contains(utf8_lcase, a)#x] +Project [Contains(utf8_binary#x, a) AS contains(utf8_binary, a)#x, Contains(utf8_lcase#x, a) AS contains(utf8_lcase, a)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1220,7 +1220,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t5 -- !query analysis -Project [substring_index(utf8_binary#x, a, 2) AS substring_index(utf8_binary, a, 2)#x, substring_index(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 2) AS substring_index(utf8_lcase, a, 2)#x] +Project [substring_index(utf8_binary#x, a, 2) AS substring_index(utf8_binary, a, 2)#x, substring_index(utf8_lcase#x, a, 2) AS substring_index(utf8_lcase, a, 2)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1318,7 +1318,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t5 -- !query analysis -Project [instr(utf8_binary#x, a) AS instr(utf8_binary, a)#x, instr(utf8_lcase#x, cast(a as string collate UTF8_LCASE)) AS instr(utf8_lcase, a)#x] +Project [instr(utf8_binary#x, a) AS instr(utf8_binary, a)#x, instr(utf8_lcase#x, a) AS instr(utf8_lcase, a)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1384,7 +1384,7 @@ Project [find_in_set(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, u -- !query select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t5 -- !query analysis -Project [find_in_set(utf8_binary#x, aaAaaAaA,i̇o) AS find_in_set(utf8_binary, aaAaaAaA,i̇o)#x, find_in_set(utf8_lcase#x, cast(aaAaaAaA,i̇o as string collate UTF8_LCASE)) AS find_in_set(utf8_lcase, aaAaaAaA,i̇o)#x] +Project [find_in_set(utf8_binary#x, aaAaaAaA,i̇o) AS find_in_set(utf8_binary, aaAaaAaA,i̇o)#x, find_in_set(utf8_lcase#x, aaAaaAaA,i̇o) AS find_in_set(utf8_lcase, aaAaaAaA,i̇o)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1482,7 +1482,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t5 -- !query analysis -Project [StartsWith(utf8_binary#x, aaAaaAaA) AS startswith(utf8_binary, aaAaaAaA)#x, StartsWith(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE)) AS startswith(utf8_lcase, aaAaaAaA)#x] +Project [StartsWith(utf8_binary#x, aaAaaAaA) AS startswith(utf8_binary, aaAaaAaA)#x, StartsWith(utf8_lcase#x, aaAaaAaA) AS startswith(utf8_lcase, aaAaaAaA)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1506,7 +1506,7 @@ Project [StartsWith(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), coll -- !query select translate(utf8_lcase, utf8_lcase, '12345') from t5 -- !query analysis -Project [translate(utf8_lcase#x, utf8_lcase#x, cast(12345 as string collate UTF8_LCASE)) AS translate(utf8_lcase, utf8_lcase, 12345)#x] +Project [translate(utf8_lcase#x, utf8_lcase#x, 12345) AS translate(utf8_lcase, utf8_lcase, 12345)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1572,7 +1572,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t5 -- !query analysis -Project [translate(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE), cast(12345 as string collate UTF8_LCASE)) AS translate(utf8_lcase, aaAaaAaA, 12345)#x, translate(utf8_binary#x, aaAaaAaA, 12345) AS translate(utf8_binary, aaAaaAaA, 12345)#x] +Project [translate(utf8_lcase#x, aaAaaAaA, 12345) AS translate(utf8_lcase, aaAaaAaA, 12345)#x, translate(utf8_binary#x, aaAaaAaA, 12345) AS translate(utf8_binary, aaAaaAaA, 12345)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1580,7 +1580,7 @@ Project [translate(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE), ca -- !query select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5 -- !query analysis -Project [translate(cast(utf8_lcase#x as string), collate(aBc, utf8_binary), 12345) AS translate(utf8_lcase, collate(aBc, utf8_binary), 12345)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), cast(12345 as string collate UTF8_LCASE)) AS translate(utf8_binary, collate(aBc, utf8_lcase), 12345)#x] +Project [translate(cast(utf8_lcase#x as string), collate(aBc, utf8_binary), 12345) AS translate(utf8_lcase, collate(aBc, utf8_binary), 12345)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), 12345) AS translate(utf8_binary, collate(aBc, utf8_lcase), 12345)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1588,7 +1588,7 @@ Project [translate(cast(utf8_lcase#x as string), collate(aBc, utf8_binary), 1234 -- !query select translate(utf8_lcase, 'aBc ' collate utf8_binary_rtrim, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5 -- !query analysis -Project [translate(cast(utf8_lcase#x as string collate UTF8_BINARY_RTRIM), collate(aBc , utf8_binary_rtrim), cast(12345 as string collate UTF8_BINARY_RTRIM)) AS translate(utf8_lcase, collate(aBc , utf8_binary_rtrim), 12345)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), cast(12345 as string collate UTF8_LCASE)) AS translate(utf8_binary, collate(aBc, utf8_lcase), 12345)#x] +Project [translate(cast(utf8_lcase#x as string collate UTF8_BINARY_RTRIM), collate(aBc , utf8_binary_rtrim), 12345) AS translate(utf8_lcase, collate(aBc , utf8_binary_rtrim), 12345)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), 12345) AS translate(utf8_binary, collate(aBc, utf8_lcase), 12345)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1638,7 +1638,7 @@ Project [replace(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), abc) AS repl -- !query select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t5 -- !query analysis -Project [replace(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), cast(abc as string collate UTF8_LCASE)) AS replace(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), abc)#x] +Project [replace(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), abc) AS replace(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), abc)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1655,7 +1655,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "inputType" : "\"STRING COLLATE UNICODE_AI\"", "paramIndex" : "first", "requiredType" : "\"STRING\"", - "sqlExpr" : "\"replace(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), abc)\"" + "sqlExpr" : "\"replace(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 'abc' collate UNICODE_AI)\"" }, "queryContext" : [ { "objectType" : "", @@ -1670,7 +1670,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t5 -- !query analysis -Project [replace(utf8_binary#x, aaAaaAaA, abc) AS replace(utf8_binary, aaAaaAaA, abc)#x, replace(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE), cast(abc as string collate UTF8_LCASE)) AS replace(utf8_lcase, aaAaaAaA, abc)#x] +Project [replace(utf8_binary#x, aaAaaAaA, abc) AS replace(utf8_binary, aaAaaAaA, abc)#x, replace(utf8_lcase#x, aaAaaAaA, abc) AS replace(utf8_lcase, aaAaaAaA, abc)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1678,7 +1678,7 @@ Project [replace(utf8_binary#x, aaAaaAaA, abc) AS replace(utf8_binary, aaAaaAaA, -- !query select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5 -- !query analysis -Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase), cast(abc as string collate UTF8_LCASE)) AS replace(utf8_binary, collate(aaAaaAaA, utf8_lcase), abc)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x] +Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase), abc) AS replace(utf8_binary, collate(aaAaaAaA, utf8_lcase), abc)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1686,7 +1686,7 @@ Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaa -- !query select replace(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5 -- !query analysis -Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim), cast(abc as string collate UTF8_LCASE_RTRIM)) AS replace(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim), abc)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x] +Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim), abc) AS replace(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim), abc)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -1768,7 +1768,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t5 -- !query analysis -Project [EndsWith(utf8_binary#x, aaAaaAaA) AS endswith(utf8_binary, aaAaaAaA)#x, EndsWith(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE)) AS endswith(utf8_lcase, aaAaaAaA)#x] +Project [EndsWith(utf8_binary#x, aaAaaAaA) AS endswith(utf8_binary, aaAaaAaA)#x, EndsWith(utf8_lcase#x, aaAaaAaA) AS endswith(utf8_lcase, aaAaaAaA)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2042,7 +2042,7 @@ Project [overlay(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_ -- !query select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5 -- !query analysis -Project [overlay(utf8_binary#x, a, 2, -1) AS overlay(utf8_binary, a, 2, -1)#x, overlay(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 2, -1) AS overlay(utf8_lcase, a, 2, -1)#x] +Project [overlay(utf8_binary#x, a, 2, -1) AS overlay(utf8_binary, a, 2, -1)#x, overlay(utf8_lcase#x, a, 2, -1) AS overlay(utf8_lcase, 'a' collate UTF8_LCASE, 2, -1)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2204,7 +2204,7 @@ Project [levenshtein(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, u -- !query select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t5 -- !query analysis -Project [levenshtein(utf8_binary#x, a, None) AS levenshtein(utf8_binary, a)#x, levenshtein(utf8_lcase#x, cast(a as string collate UTF8_LCASE), None) AS levenshtein(utf8_lcase, a)#x] +Project [levenshtein(utf8_binary#x, a, None) AS levenshtein(utf8_binary, a)#x, levenshtein(utf8_lcase#x, a, None) AS levenshtein(utf8_lcase, a)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2390,7 +2390,7 @@ Project [lpad(collate(utf8_binary#x, utf8_binary_rtrim), 8, collate(utf8_lcase#x -- !query select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t5 -- !query analysis -Project [rpad(utf8_binary#x, 8, a) AS rpad(utf8_binary, 8, a)#x, rpad(utf8_lcase#x, 8, cast(a as string collate UTF8_LCASE)) AS rpad(utf8_lcase, 8, a)#x] +Project [rpad(utf8_binary#x, 8, a) AS rpad(utf8_binary, 8, a)#x, rpad(utf8_lcase#x, 8, a) AS rpad(utf8_lcase, 8, a)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2464,7 +2464,7 @@ Project [lpad(collate(utf8_binary#x, utf8_binary_rtrim), 8, collate(utf8_lcase#x -- !query select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t5 -- !query analysis -Project [lpad(utf8_binary#x, 8, a) AS lpad(utf8_binary, 8, a)#x, lpad(utf8_lcase#x, 8, cast(a as string collate UTF8_LCASE)) AS lpad(utf8_lcase, 8, a)#x] +Project [lpad(utf8_binary#x, 8, a) AS lpad(utf8_binary, 8, a)#x, lpad(utf8_lcase#x, 8, a) AS lpad(utf8_lcase, 8, a)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2554,7 +2554,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t5 -- !query analysis -Project [locate(utf8_binary#x, a, 1) AS locate(utf8_binary, a, 1)#x, locate(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 1) AS locate(utf8_lcase, a, 1)#x] +Project [locate(utf8_binary#x, a, 1) AS locate(utf8_binary, a, 1)#x, locate(utf8_lcase#x, a, 1) AS locate(utf8_lcase, a, 1)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2660,7 +2660,7 @@ Project [trim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binary -- !query select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t5 -- !query analysis -Project [trim(utf8_binary#x, Some(ABc)) AS TRIM(BOTH ABc FROM utf8_binary)#x, trim(utf8_lcase#x, Some(cast(ABc as string collate UTF8_LCASE))) AS TRIM(BOTH ABc FROM utf8_lcase)#x] +Project [trim(utf8_binary#x, Some(ABc)) AS TRIM(BOTH ABc FROM utf8_binary)#x, trim(utf8_lcase#x, Some(ABc)) AS TRIM(BOTH ABc FROM utf8_lcase)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2856,7 +2856,7 @@ Project [ltrim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binar -- !query select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t5 -- !query analysis -Project [ltrim(utf8_binary#x, Some(ABc)) AS TRIM(LEADING ABc FROM utf8_binary)#x, ltrim(utf8_lcase#x, Some(cast(ABc as string collate UTF8_LCASE))) AS TRIM(LEADING ABc FROM utf8_lcase)#x] +Project [ltrim(utf8_binary#x, Some(ABc)) AS TRIM(LEADING ABc FROM utf8_binary)#x, ltrim(utf8_lcase#x, Some(ABc)) AS TRIM(LEADING ABc FROM utf8_lcase)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -2954,7 +2954,7 @@ Project [rtrim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binar -- !query select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t5 -- !query analysis -Project [rtrim(utf8_binary#x, Some(ABc)) AS TRIM(TRAILING ABc FROM utf8_binary)#x, rtrim(utf8_lcase#x, Some(cast(ABc as string collate UTF8_LCASE))) AS TRIM(TRAILING ABc FROM utf8_lcase)#x] +Project [rtrim(utf8_binary#x, Some(ABc)) AS TRIM(TRAILING ABc FROM utf8_binary)#x, rtrim(utf8_lcase#x, Some(ABc)) AS TRIM(TRAILING ABc FROM utf8_lcase)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/results/collations.sql.out b/sql/core/src/test/resources/sql-tests/results/collations.sql.out index f92fc5de8c3f4..e96549f00d6ec 100644 --- a/sql/core/src/test/resources/sql-tests/results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/collations.sql.out @@ -479,7 +479,7 @@ struct +struct<(collate(a, unicode) < 'A' collate UNICODE):boolean> -- !query output true @@ -487,7 +487,7 @@ true -- !query select 'a' collate unicode_ci = 'A' -- !query schema -struct<(collate(a, unicode_ci) = A):boolean> +struct<(collate(a, unicode_ci) = 'A' collate UNICODE_CI):boolean> -- !query output true @@ -495,7 +495,7 @@ true -- !query select 'a' collate unicode_ai = 'å' -- !query schema -struct<(collate(a, unicode_ai) = å):boolean> +struct<(collate(a, unicode_ai) = 'å' collate UNICODE_AI):boolean> -- !query output true @@ -503,7 +503,7 @@ true -- !query select 'a' collate unicode_ci_ai = 'Å' -- !query schema -struct<(collate(a, unicode_ci_ai) = Å):boolean> +struct<(collate(a, unicode_ci_ai) = 'Å' collate UNICODE_CI_AI):boolean> -- !query output true @@ -511,7 +511,7 @@ true -- !query select 'a' collate en < 'A' -- !query schema -struct<(collate(a, en) < A):boolean> +struct<(collate(a, en) < 'A' collate en):boolean> -- !query output true @@ -519,7 +519,7 @@ true -- !query select 'a' collate en_ci = 'A' -- !query schema -struct<(collate(a, en_ci) = A):boolean> +struct<(collate(a, en_ci) = 'A' collate en_CI):boolean> -- !query output true @@ -527,7 +527,7 @@ true -- !query select 'a' collate en_ai = 'å' -- !query schema -struct<(collate(a, en_ai) = å):boolean> +struct<(collate(a, en_ai) = 'å' collate en_AI):boolean> -- !query output true @@ -535,7 +535,7 @@ true -- !query select 'a' collate en_ci_ai = 'Å' -- !query schema -struct<(collate(a, en_ci_ai) = Å):boolean> +struct<(collate(a, en_ci_ai) = 'Å' collate en_CI_AI):boolean> -- !query output true @@ -543,7 +543,7 @@ true -- !query select 'Kypper' collate sv < 'Köpfe' -- !query schema -struct<(collate(Kypper, sv) < Köpfe):boolean> +struct<(collate(Kypper, sv) < 'Köpfe' collate sv):boolean> -- !query output true @@ -551,7 +551,7 @@ true -- !query select 'Kypper' collate de > 'Köpfe' -- !query schema -struct<(collate(Kypper, de) > Köpfe):boolean> +struct<(collate(Kypper, de) > 'Köpfe' collate de):boolean> -- !query output true @@ -559,7 +559,7 @@ true -- !query select 'I' collate tr_ci = 'ı' -- !query schema -struct<(collate(I, tr_ci) = ı):boolean> +struct<(collate(I, tr_ci) = 'ı' collate tr_CI):boolean> -- !query output true @@ -1109,7 +1109,7 @@ kitten -- !query select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5 -- !query schema -struct +struct -- !query output Hello, world! Nice day. Hello, world! Nice day. Something else. Nothing here. Something else. Nothing here. @@ -2492,7 +2492,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "inputType" : "\"STRING COLLATE UNICODE_AI\"", "paramIndex" : "first", "requiredType" : "\"STRING\"", - "sqlExpr" : "\"replace(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), abc)\"" + "sqlExpr" : "\"replace(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 'abc' collate UNICODE_AI)\"" }, "queryContext" : [ { "objectType" : "", @@ -3342,7 +3342,7 @@ ksitTing -- !query select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5 -- !query schema -struct +struct -- !query output Hallo, world! Nice day. Hallo, world! Nice day. Saark SaL diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala index 7cafb999ffcf0..8d831e4ca1668 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala @@ -450,7 +450,8 @@ class CollationSQLRegexpSuite }, condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( - "sqlExpr" -> "\"regexp_replace(collate(ABCDE, UNICODE_CI), .c., FFF, 1)\"", + "sqlExpr" -> + """"regexp_replace(collate(ABCDE, UNICODE_CI), .c., 'FFF' collate UNICODE_CI, 1)"""", "paramIndex" -> "first", "inputSql" -> "\"collate(ABCDE, UNICODE_CI)\"", "inputType" -> "\"STRING COLLATE UNICODE_CI\"", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index bb6fce1fb1b66..23d0d4ad8c21c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.SparkThrowable import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { @@ -43,6 +44,11 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { private def assertImplicitMismatch(df: => DataFrame): Unit = assertThrowsError(df, "COLLATION_MISMATCH.IMPLICIT") + private def assertQuerySchema(df: => DataFrame, expectedSchema: DataType): Unit = { + val querySchema = df.schema.fields.head.dataType + assert(DataType.equalsIgnoreNullability(querySchema, expectedSchema)) + } + test("explicit collation propagates up") { checkAnswer( sql(s"SELECT COLLATION('a' collate unicode)"), @@ -382,13 +388,157 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { s"'name2' collate utf8_lcase, 'value2' collate unicode)"), Row(Row("value1", "value2"))) - assertExplicitMismatch( + checkAnswer( sql(s"SELECT named_struct" + - s"('name1' collate unicode, 'value1', 'name2' collate utf8_lcase, 'value2')")) + s"('name1' collate unicode, 'value1', 'name2' collate utf8_lcase, 'value2')"), + Row(Row("value1", "value2"))) - assertExplicitMismatch( + checkAnswer( sql(s"SELECT named_struct" + - s"('name1', 'value1' collate unicode, 'name2', 'value2' collate utf8_lcase)")) + s"('name1', 'value1' collate unicode, 'name2', 'value2' collate utf8_lcase)"), + Row(Row("value1", "value2"))) + } + + test("coercing structs") { + assertQuerySchema( + sql(s"SELECT array(struct(1, 'a'), struct(2, 'b' collate utf8_lcase))"), + ArrayType( + StructType( + Seq(StructField("col1", IntegerType), StructField("col2", StringType("UTF8_LCASE")))))) + + assertQuerySchema( + sql(s"SELECT array(struct(1, 'a' collate utf8_lcase), struct(2, 'b' collate utf8_lcase))"), + ArrayType( + StructType( + Seq(StructField("col1", IntegerType), StructField("col2", StringType("UTF8_LCASE")))))) + + assertExplicitMismatch( + sql(s"SELECT array(struct(1, 'a' collate utf8_lcase), struct(2, 'b' collate unicode))")) + + assertImplicitMismatch(sql(s""" + |SELECT array(struct(1, c1), struct(2, c2)) + |FROM VALUES ('a' collate unicode, 'b' collate utf8_lcase) AS t(c1, c2) + |""".stripMargin)) + } + + test("coercing maps") { + assertQuerySchema( + sql(s"SELECT map('key1', 'val1', 'key2', 'val2')"), + MapType(StringType, StringType)) + + assertQuerySchema( + sql(s"SELECT map('key1' collate utf8_lcase, 'val1', 'key2', 'val2' collate unicode)"), + MapType(StringType("UTF8_LCASE"), StringType("UNICODE"))) + + assertQuerySchema( + sql(s"SELECT ARRAY(map('key1', 'val1'), map('key2' collate UNICODE, 'val2'))"), + ArrayType(MapType(StringType("UNICODE"), StringType))) + + assertExplicitMismatch( + sql(s"SELECT map('key1', 'val1' collate utf8_lcase, 'key2', 'val2' collate unicode)")) + } + + test("maps of structs") { + assertQuerySchema( + sql(s"SELECT map('key1', struct(1, 'a' collate unicode), 'key2', struct(2, 'b'))"), + MapType( + StringType, + StructType( + Seq(StructField("col1", IntegerType), StructField("col2", StringType("UNICODE")))))) + + checkAnswer( + sql( + s"SELECT map('key1', struct(1, 'a' collate unicode_ci)," + + s"'key2', struct(2, 'b'))['key1'].col2 = 'A'"), + Seq(Row(true))) + } + + test("coercing arrays") { + assertQuerySchema(sql(s"SELECT array('a', 'b')"), ArrayType(StringType)) + + assertQuerySchema( + sql(s"SELECT array('a' collate utf8_lcase, 'b')"), + ArrayType(StringType("UTF8_LCASE"))) + + assertQuerySchema( + sql(s"SELECT array('a' collate utf8_lcase, 'b' collate utf8_lcase)"), + ArrayType(StringType("UTF8_LCASE"))) + + assertExplicitMismatch(sql(s"SELECT array('a' collate utf8_lcase, 'b' collate unicode)")) + + assertQuerySchema( + sql(s"SELECT array(array('a', 'b'), array('c' collate utf8_lcase, 'd'))"), + ArrayType(ArrayType(StringType("UTF8_LCASE")))) + + checkAnswer( + sql(s"SELECT array('a', 'b') = array('A' collate utf8_lcase, 'B')"), + Seq(Row(true))) + + checkAnswer( + sql(s"SELECT array('a', 'b')[0] = array('A' collate utf8_lcase, 'B')[1]"), + Seq(Row(false))) + + assertExplicitMismatch( + sql(s"SELECT array('a', 'b' collate unicode) = array('A' collate utf8_lcase, 'B')")) + } + + test("array of structs") { + assertQuerySchema( + sql(s"SELECT array(struct(1, 'a' collate unicode), struct(2, 'b'))[0]"), + StructType( + Seq(StructField("col1", IntegerType), StructField("col2", StringType("UNICODE"))))) + + checkAnswer( + sql(s"SELECT array(struct(1, 'a' collate unicode_ci), struct(2, 'b'))[0].col2 = 'A'"), + Seq(Row(true))) + } + + test("coercing deeply nested complex types") { + assertQuerySchema( + sql(s""" + |SELECT struct( + | struct(1, 'nested' collate unicode), + | array( + | struct(1, 'a' collate utf8_lcase), + | struct(2, 'b' collate utf8_lcase) + | ) + |) + |""".stripMargin), + StructType( + Seq( + StructField( + "col1", + StructType( + Seq(StructField("col1", IntegerType), StructField("col2", StringType("UNICODE"))))), + StructField( + "col2", + ArrayType( + StructType(Seq( + StructField("col1", IntegerType), + StructField("col2", StringType("UTF8_LCASE"))))))))) + + assertQuerySchema( + sql(s""" + |SELECT struct( + | struct( + | array( + | map('key1' collate utf8_lcase, 'val1', + | 'key2', 'val2'), + | map('key3', 'val3' collate unicode) + | ) + | ), + | 42 + |) + |""".stripMargin), + StructType( + Seq( + StructField( + "col1", + StructType( + Seq(StructField( + "col1", + ArrayType(MapType(StringType("UTF8_LCASE"), StringType("UNICODE"))))))), + StructField("col2", IntegerType)))) } test("access collated map via literal") { From 45da6f66d155fff4024840f804d80f335e66360c Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 4 Dec 2024 16:41:21 +0800 Subject: [PATCH 092/438] [SPARK-50477][INFRA] Add a separate docker file for python 3.9 daily build ### What changes were proposed in this pull request? Add a separate docker file for python 3.9 daily build ### Why are the changes needed? to isolate the environments ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? ci, the second commit and the 4-th commit tested this PR against the new image https://github.com/zhengruifeng/spark/actions/runs/12135050296/job/33835846375 https://github.com/zhengruifeng/spark/actions/runs/12140138335/job/33850700922 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49042 from zhengruifeng/infra_py_images. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 36 +++++++- .../workflows/build_infra_images_cache.yml | 14 ++++ .github/workflows/build_python_3.9.yml | 1 + dev/spark-test-image/python-309/Dockerfile | 82 +++++++++++++++++++ 4 files changed, 132 insertions(+), 1 deletion(-) create mode 100644 dev/spark-test-image/python-309/Dockerfile diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 3117872e21680..cf49316fafbbf 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -55,6 +55,7 @@ jobs: runs-on: ubuntu-latest env: GITHUB_PREV_SHA: ${{ github.event.before }} + PYSPARK_IMAGE_TO_TEST: '' outputs: required: ${{ steps.set-outputs.outputs.required }} image_url: ${{ steps.infra-image-outputs.outputs.image_url }} @@ -64,6 +65,8 @@ jobs: image_lint_url_link: ${{ steps.infra-image-link.outputs.image_lint_url_link }} image_sparkr_url: ${{ steps.infra-image-sparkr-outputs.outputs.image_sparkr_url }} image_sparkr_url_link: ${{ steps.infra-image-link.outputs.image_sparkr_url_link }} + image_pyspark_url: ${{ steps.infra-image-pyspark-outputs.outputs.image_pyspark_url }} + image_pyspark_url_link: ${{ steps.infra-image-link.outputs.image_pyspark_url_link }} steps: - name: Checkout Spark repository uses: actions/checkout@v4 @@ -164,8 +167,19 @@ jobs: IMG_NAME="apache-spark-ci-image-sparkr:${{ inputs.branch }}-${{ github.run_id }}" IMG_URL="ghcr.io/$REPO_OWNER/$IMG_NAME" echo "image_sparkr_url=$IMG_URL" >> $GITHUB_OUTPUT + - name: Generate infra image URL (PySpark ${{ env.PYSPARK_IMAGE_TO_TEST }}) + id: infra-image-pyspark-outputs + if: ${{ env.PYSPARK_IMAGE_TO_TEST }} + env: ${{ fromJSON(inputs.envs) }} + run: | + # Convert to lowercase to meet Docker repo name requirement + REPO_OWNER=$(echo "${{ github.repository_owner }}" | tr '[:upper:]' '[:lower:]') + IMG_NAME="apache-spark-ci-image-pyspark-${{ env.PYSPARK_IMAGE_TO_TEST }}:${{ inputs.branch }}-${{ github.run_id }}" + IMG_URL="ghcr.io/$REPO_OWNER/$IMG_NAME" + echo "image_pyspark_url=$IMG_URL" >> $GITHUB_OUTPUT - name: Link the docker images id: infra-image-link + env: ${{ fromJSON(inputs.envs) }} run: | # Set the image URL for job "docs" # Should delete the link and directly use image_docs_url after SPARK 3.x EOL @@ -173,10 +187,16 @@ jobs: echo "image_docs_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT echo "image_lint_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT echo "image_sparkr_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT + echo "image_pyspark_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT else echo "image_docs_url_link=${{ steps.infra-image-docs-outputs.outputs.image_docs_url }}" >> $GITHUB_OUTPUT echo "image_lint_url_link=${{ steps.infra-image-lint-outputs.outputs.image_lint_url }}" >> $GITHUB_OUTPUT echo "image_sparkr_url_link=${{ steps.infra-image-sparkr-outputs.outputs.image_sparkr_url }}" >> $GITHUB_OUTPUT + if [[ "${{ env.PYSPARK_IMAGE_TO_TEST }}" != "" ]]; then + echo "image_pyspark_url_link=${{ steps.infra-image-pyspark-outputs.outputs.image_pyspark_url }}" >> $GITHUB_OUTPUT + else + echo "image_pyspark_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT + fi fi # Build: build Spark and run the tests for specified modules. @@ -360,6 +380,8 @@ jobs: runs-on: ubuntu-latest permissions: packages: write + env: + PYSPARK_IMAGE_TO_TEST: '' steps: - name: Login to GitHub Container Registry uses: docker/login-action@v3 @@ -428,6 +450,18 @@ jobs: ${{ needs.precondition.outputs.image_sparkr_url }} # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ inputs.branch }} + - name: Build and push (PySpark ${{ env.PYSPARK_IMAGE_TO_TEST }}) + if: ${{ env.PYSPARK_IMAGE_TO_TEST }} + id: docker_build_pyspark + env: ${{ fromJSON(inputs.envs) }} + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/${{ env.PYSPARK_IMAGE_TO_TEST }}/ + push: true + tags: | + ${{ needs.precondition.outputs.image_pyspark_url }} + # Use the infra image cache to speed up + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-${{ env.PYSPARK_IMAGE_TO_TEST }}-cache:${{ inputs.branch }} pyspark: @@ -438,7 +472,7 @@ jobs: runs-on: ubuntu-latest timeout-minutes: 180 container: - image: ${{ needs.precondition.outputs.image_url }} + image: ${{ needs.precondition.outputs.image_pyspark_url_link }} strategy: fail-fast: false matrix: diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index a6beacedeebd4..3d5a8306aca80 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -30,6 +30,7 @@ on: - 'dev/spark-test-image/docs/Dockerfile' - 'dev/spark-test-image/lint/Dockerfile' - 'dev/spark-test-image/sparkr/Dockerfile' + - 'dev/spark-test-image/python-309/Dockerfile' - '.github/workflows/build_infra_images_cache.yml' # Create infra image when cutting down branches/tags create: @@ -102,3 +103,16 @@ jobs: - name: Image digest (SparkR) if: hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' run: echo ${{ steps.docker_build_sparkr.outputs.digest }} + - name: Build and push (PySpark with Python 3.9) + if: hashFiles('dev/spark-test-image/python-309/Dockerfile') != '' + id: docker_build_pyspark_python_309 + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/python-309/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-309-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-309-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-309-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark with Python 3.9) + if: hashFiles('dev/spark-test-image/python-309/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_python_309.outputs.digest }} diff --git a/.github/workflows/build_python_3.9.yml b/.github/workflows/build_python_3.9.yml index b2401fcf2aa14..744e18cc8db39 100644 --- a/.github/workflows/build_python_3.9.yml +++ b/.github/workflows/build_python_3.9.yml @@ -36,6 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-309", "PYTHON_TO_TEST": "python3.9" } jobs: >- diff --git a/dev/spark-test-image/python-309/Dockerfile b/dev/spark-test-image/python-309/Dockerfile new file mode 100644 index 0000000000000..dbab99c1441bc --- /dev/null +++ b/dev/spark-test-image/python-309/Dockerfile @@ -0,0 +1,82 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark with Python 3.09" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE 20241119 + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pandoc \ + pkg-config \ + qpdf \ + software-properties-common \ + wget \ + zlib1g-dev \ + && rm -rf /var/lib/apt/lists/* + +# Install Python 3.9 +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt-get update && apt-get install -y \ + python3.9 python3.9-distutils \ + && rm -rf /var/lib/apt/lists/* + +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +# Python deps for Spark Connect +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" + +# Install Python 3.9 +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt-get update && apt-get install -y \ + python3.9 python3.9-distutils \ + && rm -rf /var/lib/apt/lists/* +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 +RUN python3.9 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.9 -m pip install --force $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ + python3.9 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ + python3.9 -m pip install torcheval && \ + python3.9 -m pip cache purge From 3d063a01d7c2a6d9613e11dec882739daa7eeb71 Mon Sep 17 00:00:00 2001 From: huangxiaoping <1754789345@qq.com> Date: Wed, 4 Dec 2024 18:24:35 +0800 Subject: [PATCH 093/438] [SPARK-50487][DOCS] Update broken jira link ### What changes were proposed in this pull request? Update broken jira link ### Why are the changes needed? The old link is not accessible ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No testing required ### Was this patch authored or co-authored using generative AI tooling? No Closes #49052 from huangxiaopingRD/SPARK-50487. Lead-authored-by: huangxiaoping <1754789345@qq.com> Co-authored-by: Dongjoon Hyun Signed-off-by: Kent Yao --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 5 +++-- python/pyspark/core/rdd.py | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 0db0133f632bf..80db818b77e42 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1834,8 +1834,9 @@ abstract class RDD[T: ClassTag]( * Please read the linked SPIP and design docs to understand the limitations and future plans. * @return an [[RDDBarrier]] instance that provides actions within a barrier stage * @see [[org.apache.spark.BarrierTaskContext]] - * @see SPIP: Barrier Execution Mode - * @see Design Doc + * @see + * SPIP: Barrier Execution Mode + * @see Design Doc */ @Experimental @Since("2.4.0") diff --git a/python/pyspark/core/rdd.py b/python/pyspark/core/rdd.py index a40af3e551584..68287a4dbdd39 100644 --- a/python/pyspark/core/rdd.py +++ b/python/pyspark/core/rdd.py @@ -4998,8 +4998,8 @@ def barrier(self: "RDD[T]") -> "RDDBarrier[T]": ----- For additional information see - - `SPIP: Barrier Execution Mode `_ - - `Design Doc `_ + - `SPIP: Barrier Execution Mode `_ + - `Design Doc `_ This API is experimental """ From 74c3757513c9f580d060a88982463f3a8b1745b4 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 4 Dec 2024 14:00:11 +0100 Subject: [PATCH 094/438] [MINOR][DOCS] Add a migration guide for encode/decode unmappable characters ### What changes were proposed in this pull request? Add a migration guide for encode/decode unmappable characters ### Why are the changes needed? Providing upgrading guides for users ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? passing doc build ### Was this patch authored or co-authored using generative AI tooling? no Closes #49058 from yaooqinn/minor. Authored-by: Kent Yao Signed-off-by: Max Gekk --- docs/sql-migration-guide.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index ea4dbe926d146..717d27befef02 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -33,6 +33,7 @@ license: | - Since Spark 4.0, `spark.sql.parquet.compression.codec` drops the support of codec name `lz4raw`, please use `lz4_raw` instead. - Since Spark 4.0, when overflowing during casting timestamp to byte/short/int under non-ansi mode, Spark will return null instead a wrapping value. - Since Spark 4.0, the `encode()` and `decode()` functions support only the following charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32'. To restore the previous behavior when the function accepts charsets of the current JDK used by Spark, set `spark.sql.legacy.javaCharsets` to `true`. +- Since Spark 4.0, the `encode()` and `decode()` functions raise `MALFORMED_CHARACTER_CODING` error when handling unmappable characters, while in Spark 3.5 and earlier versions, these characters will be replaced with mojibakes. To restore the previous behavior, set `spark.sql.legacy.codingErrorAction` to `true`. For example, if you try to `decode` a string value `tést` / [116, -23, 115, 116] (encoded in latin1) with 'UTF-8', you get `t�st`. - Since Spark 4.0, the legacy datetime rebasing SQL configs with the prefix `spark.sql.legacy` are removed. To restore the previous behavior, use the following configs: - `spark.sql.parquet.int96RebaseModeInWrite` instead of `spark.sql.legacy.parquet.int96RebaseModeInWrite` - `spark.sql.parquet.datetimeRebaseModeInWrite` instead of `spark.sql.legacy.parquet.datetimeRebaseModeInWrite` From 10e0b619092b9683f08846409b4083dcd7624478 Mon Sep 17 00:00:00 2001 From: Jovan Pavlovic Date: Wed, 4 Dec 2024 15:06:26 +0100 Subject: [PATCH 095/438] [SPARK-49670][SQL] Enable trim collation for all passthrough expressions ### What changes were proposed in this pull request? Enabling usage of passthrough expressions for trim collation. As with this change there will be more expressions that will support trim collation then those who don't in follow up default for support trim collation will be set on true. **NOTE: it looks like a tons of changes but only changes are: for each expression set supportsTrimCollation=true and add tests.** ### Why are the changes needed? So that all expressions could be used with trim collation ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Add tests to CollationSqlExpressionsSuite ### Was this patch authored or co-authored using generative AI tooling? No Closes #48739 from jovanpavl-db/implement_passthrough_functions. Authored-by: Jovan Pavlovic Signed-off-by: Max Gekk --- .../analysis/TypeCoercionHelper.scala | 3 +- .../expressions/CallMethodViaReflection.scala | 2 +- .../sql/catalyst/expressions/ExprUtils.scala | 4 +- .../expressions/collectionOperations.scala | 14 +- .../catalyst/expressions/csvExpressions.scala | 3 +- .../expressions/datetimeExpressions.scala | 46 +- .../expressions/jsonExpressions.scala | 16 +- .../expressions/maskExpressions.scala | 10 +- .../expressions/mathExpressions.scala | 7 +- .../spark/sql/catalyst/expressions/misc.scala | 15 +- .../expressions/numberFormatExpressions.scala | 6 +- .../expressions/stringExpressions.scala | 6 +- .../catalyst/expressions/urlExpressions.scala | 12 +- .../variant/variantExpressions.scala | 5 +- .../sql/catalyst/expressions/xml/xpath.scala | 6 +- .../catalyst/expressions/xmlExpressions.scala | 3 +- .../analysis/AnsiTypeCoercionSuite.scala | 16 +- .../analyzer-results/collations.sql.out | 40 ++ .../resources/sql-tests/inputs/collations.sql | 5 + .../sql-tests/results/collations.sql.out | 110 +++++ .../sql/CollationSQLExpressionsSuite.scala | 400 ++++++++++++++++-- .../org/apache/spark/sql/CollationSuite.scala | 15 + 22 files changed, 634 insertions(+), 110 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala index 3fc4b71c986ff..ab2ab50cb33ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala @@ -318,7 +318,8 @@ abstract class TypeCoercionHelper { } case aj @ ArrayJoin(arr, d, nr) - if !AbstractArrayType(StringTypeWithCollation).acceptsType(arr.dataType) && + if !AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true)). + acceptsType(arr.dataType) && ArrayType.acceptsType(arr.dataType) => val containsNull = arr.dataType.asInstanceOf[ArrayType].containsNull implicitCast(arr, ArrayType(StringType, containsNull)) match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala index d38ee01485288..4eb14fb9e7b86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala @@ -115,7 +115,7 @@ case class CallMethodViaReflection( "requiredType" -> toSQLType( TypeCollection(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, - StringTypeWithCollation)), + StringTypeWithCollation(supportsTrimCollation = true))), "inputSql" -> toSQLExpr(e), "inputType" -> toSQLType(e.dataType)) ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala index e65a0200b064f..8b7d641828ba1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala @@ -61,7 +61,9 @@ object ExprUtils extends EvalHelper with QueryErrorsBase { def convertToMapData(exp: Expression): Map[String, String] = exp match { case m: CreateMap - if AbstractMapType(StringTypeWithCollation, StringTypeWithCollation) + if AbstractMapType( + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true)) .acceptsType(m.dataType) => val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData] ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index fb130574d3474..9843e844ad169 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -1354,7 +1354,7 @@ case class Reverse(child: Expression) override def nullIntolerant: Boolean = true // Input types are utilized by type coercion in ImplicitTypeCasts. override def inputTypes: Seq[AbstractDataType] = - Seq(TypeCollection(StringTypeWithCollation, ArrayType)) + Seq(TypeCollection(StringTypeWithCollation(supportsTrimCollation = true), ArrayType)) override def dataType: DataType = child.dataType @@ -2127,12 +2127,12 @@ case class ArrayJoin( this(array, delimiter, Some(nullReplacement)) override def inputTypes: Seq[AbstractDataType] = if (nullReplacement.isDefined) { - Seq(AbstractArrayType(StringTypeWithCollation), - StringTypeWithCollation, - StringTypeWithCollation) + Seq(AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true)), + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true)) } else { - Seq(AbstractArrayType(StringTypeWithCollation), - StringTypeWithCollation) + Seq(AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true)), + StringTypeWithCollation(supportsTrimCollation = true)) } override def children: Seq[Expression] = if (nullReplacement.isDefined) { @@ -2855,7 +2855,7 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio with QueryErrorsBase { private def allowedTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, BinaryType, ArrayType) + Seq(StringTypeWithCollation(supportsTrimCollation = true), BinaryType, ArrayType) final override val nodePatterns: Seq[TreePattern] = Seq(CONCAT) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index 21b6295a59f02..04fb9bc133c67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -87,7 +87,8 @@ case class CsvToStructs( copy(timeZoneId = Option(timeZoneId)) } - override def inputTypes: Seq[AbstractDataType] = StringTypeWithCollation :: Nil + override def inputTypes: Seq[AbstractDataType] = + StringTypeWithCollation(supportsTrimCollation = true) :: Nil override def prettyName: String = "from_csv" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index fba3927a0bc9c..55e6c7f1503fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -971,7 +971,7 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti override def dataType: DataType = SQLConf.get.defaultStringType override def inputTypes: Seq[AbstractDataType] = - Seq(TimestampType, StringTypeWithCollation) + Seq(TimestampType, StringTypeWithCollation(supportsTrimCollation = true)) override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) @@ -1279,10 +1279,13 @@ abstract class ToTimestamp override def forTimestampNTZ: Boolean = left.dataType == TimestampNTZType override def inputTypes: Seq[AbstractDataType] = - Seq(TypeCollection( - StringTypeWithCollation, DateType, TimestampType, TimestampNTZType - ), - StringTypeWithCollation) + Seq( + TypeCollection( + StringTypeWithCollation(supportsTrimCollation = true), + DateType, + TimestampType, + TimestampNTZType), + StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = LongType override def nullable: Boolean = if (failOnError) children.exists(_.nullable) else true @@ -1454,7 +1457,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ override def nullable: Boolean = true override def inputTypes: Seq[AbstractDataType] = - Seq(LongType, StringTypeWithCollation) + Seq(LongType, StringTypeWithCollation(supportsTrimCollation = true)) override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) @@ -1566,7 +1569,7 @@ case class NextDay( def this(left: Expression, right: Expression) = this(left, right, SQLConf.get.ansiEnabled) override def inputTypes: Seq[AbstractDataType] = - Seq(DateType, StringTypeWithCollation) + Seq(DateType, StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = DateType override def nullable: Boolean = true @@ -1781,7 +1784,7 @@ sealed trait UTCTimestamp extends BinaryExpression with ImplicitCastInputTypes { val funcName: String override def inputTypes: Seq[AbstractDataType] = - Seq(TimestampType, StringTypeWithCollation) + Seq(TimestampType, StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = TimestampType override def nullSafeEval(time: Any, timezone: Any): Any = { @@ -2123,8 +2126,11 @@ case class ParseToDate( // Note: ideally this function should only take string input, but we allow more types here to // be backward compatible. TypeCollection( - StringTypeWithCollation, DateType, TimestampType, TimestampNTZType) +: - format.map(_ => StringTypeWithCollation).toSeq + StringTypeWithCollation(supportsTrimCollation = true), + DateType, + TimestampType, + TimestampNTZType) +: + format.map(_ => StringTypeWithCollation(supportsTrimCollation = true)).toSeq } override protected def withNewChildrenInternal( @@ -2195,10 +2201,15 @@ case class ParseToTimestamp( override def inputTypes: Seq[AbstractDataType] = { // Note: ideally this function should only take string input, but we allow more types here to // be backward compatible. - val types = Seq(StringTypeWithCollation, DateType, TimestampType, TimestampNTZType) + val types = Seq( + StringTypeWithCollation( + supportsTrimCollation = true), + DateType, + TimestampType, + TimestampNTZType) TypeCollection( (if (dataType.isInstanceOf[TimestampType]) types :+ NumericType else types): _* - ) +: format.map(_ => StringTypeWithCollation).toSeq + ) +: format.map(_ => StringTypeWithCollation(supportsTrimCollation = true)).toSeq } override protected def withNewChildrenInternal( @@ -2329,7 +2340,7 @@ case class TruncDate(date: Expression, format: Expression) override def right: Expression = format override def inputTypes: Seq[AbstractDataType] = - Seq(DateType, StringTypeWithCollation) + Seq(DateType, StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = DateType override def prettyName: String = "trunc" override val instant = date @@ -2399,7 +2410,7 @@ case class TruncTimestamp( override def right: Expression = timestamp override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, TimestampType) + Seq(StringTypeWithCollation(supportsTrimCollation = true), TimestampType) override def dataType: TimestampType = TimestampType override def prettyName: String = "date_trunc" override val instant = timestamp @@ -2800,7 +2811,7 @@ case class MakeTimestamp( // casted into decimal safely, we use DecimalType(16, 6) which is wider than DecimalType(10, 0). override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType, IntegerType, IntegerType, IntegerType, IntegerType, DecimalType(16, 6)) ++ - timezone.map(_ => StringTypeWithCollation) + timezone.map(_ => StringTypeWithCollation(supportsTrimCollation = true)) override def nullable: Boolean = if (failOnError) children.exists(_.nullable) else true override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = @@ -3333,7 +3344,10 @@ case class ConvertTimezone( override def third: Expression = sourceTs override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, StringTypeWithCollation, TimestampNTZType) + Seq( + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true), + TimestampNTZType) override def dataType: DataType = TimestampNTZType override def nullSafeEval(srcTz: Any, tgtTz: Any, micros: Any): Any = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 68cce1c2a138b..affc8261dc883 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -133,7 +133,9 @@ case class GetJsonObject(json: Expression, path: Expression) override def left: Expression = json override def right: Expression = path override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, StringTypeWithCollation) + Seq( + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = SQLConf.get.defaultStringType override def nullable: Boolean = true override def prettyName: String = "get_json_object" @@ -490,7 +492,8 @@ case class JsonTuple(children: Seq[Expression]) ) } else if ( children.forall( - child => StringTypeWithCollation.acceptsType(child.dataType))) { + child => StringTypeWithCollation(supportsTrimCollation = true) + .acceptsType(child.dataType))) { TypeCheckResult.TypeCheckSuccess } else { DataTypeMismatch( @@ -709,7 +712,8 @@ case class JsonToStructs( |""".stripMargin) } - override def inputTypes: Seq[AbstractDataType] = StringTypeWithCollation :: Nil + override def inputTypes: Seq[AbstractDataType] = + StringTypeWithCollation(supportsTrimCollation = true) :: Nil override def sql: String = schema match { case _: MapType => "entries" @@ -922,7 +926,8 @@ case class LengthOfJsonArray(child: Expression) with ExpectsInputTypes with RuntimeReplaceable { - override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCollation) + override def inputTypes: Seq[AbstractDataType] = + Seq(StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = IntegerType override def nullable: Boolean = true override def prettyName: String = "json_array_length" @@ -967,7 +972,8 @@ case class JsonObjectKeys(child: Expression) with ExpectsInputTypes with RuntimeReplaceable { - override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCollation) + override def inputTypes: Seq[AbstractDataType] = + Seq(StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = ArrayType(SQLConf.get.defaultStringType) override def nullable: Boolean = true override def prettyName: String = "json_object_keys" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala index 7be6df14194fc..5b17d2029ed1b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala @@ -193,11 +193,11 @@ case class Mask( */ override def inputTypes: Seq[AbstractDataType] = Seq( - StringTypeWithCollation, - StringTypeWithCollation, - StringTypeWithCollation, - StringTypeWithCollation, - StringTypeWithCollation) + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true)) override def nullable: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 30f07dcc1e67e..317a08b8c64c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -455,7 +455,7 @@ case class Conv( override def second: Expression = fromBaseExpr override def third: Expression = toBaseExpr override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, IntegerType, IntegerType) + Seq(StringTypeWithCollation(supportsTrimCollation = true), IntegerType, IntegerType) override def dataType: DataType = first.dataType override def nullable: Boolean = true @@ -1118,7 +1118,7 @@ case class Hex(child: Expression) override def nullIntolerant: Boolean = true override def inputTypes: Seq[AbstractDataType] = - Seq(TypeCollection(LongType, BinaryType, StringTypeWithCollation)) + Seq(TypeCollection(LongType, BinaryType, StringTypeWithCollation(supportsTrimCollation = true))) override def dataType: DataType = child.dataType match { case st: StringType => st @@ -1163,7 +1163,8 @@ case class Unhex(child: Expression, failOnError: Boolean = false) def this(expr: Expression) = this(expr, false) - override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCollation) + override def inputTypes: Seq[AbstractDataType] = + Seq(StringTypeWithCollation(supportsTrimCollation = true)) override def nullable: Boolean = true override def dataType: DataType = BinaryType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 622a0e0aa5bb7..fb30eab327d4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -85,7 +85,12 @@ case class RaiseError(errorClass: Expression, errorParms: Expression, dataType: override def foldable: Boolean = false override def nullable: Boolean = true override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, AbstractMapType(StringTypeWithCollation, StringTypeWithCollation)) + Seq( + StringTypeWithCollation(supportsTrimCollation = true), + AbstractMapType( + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true) + )) override def left: Expression = errorClass override def right: Expression = errorParms @@ -416,8 +421,8 @@ case class AesEncrypt( override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, BinaryType, - StringTypeWithCollation, - StringTypeWithCollation, + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true), BinaryType, BinaryType) override def children: Seq[Expression] = Seq(input, key, mode, padding, iv, aad) @@ -493,8 +498,8 @@ case class AesDecrypt( override def inputTypes: Seq[AbstractDataType] = { Seq(BinaryType, BinaryType, - StringTypeWithCollation, - StringTypeWithCollation, BinaryType) + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true), BinaryType) } override def prettyName: String = "aes_decrypt" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala index d4dcfdc5e72fb..fd6399d65271e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala @@ -51,7 +51,9 @@ abstract class ToNumberBase(left: Expression, right: Expression, errorOnFail: Bo } override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, StringTypeWithCollation) + Seq( + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true)) override def checkInputDataTypes(): TypeCheckResult = { val inputTypeCheck = super.checkInputDataTypes() @@ -288,7 +290,7 @@ case class ToCharacter(left: Expression, right: Expression) override def dataType: DataType = SQLConf.get.defaultStringType override def inputTypes: Seq[AbstractDataType] = - Seq(DecimalType, StringTypeWithCollation) + Seq(DecimalType, StringTypeWithCollation(supportsTrimCollation = true)) override def checkInputDataTypes(): TypeCheckResult = { val inputTypeCheck = super.checkInputDataTypes() if (inputTypeCheck.isSuccess) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 2ea53350fea36..efd7e5c07de40 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -3557,9 +3557,9 @@ case class Sentences( ArrayType(ArrayType(str.dataType, containsNull = false), containsNull = false) override def inputTypes: Seq[AbstractDataType] = Seq( - StringTypeWithCollation, - StringTypeWithCollation, - StringTypeWithCollation + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true) ) override def first: Expression = str override def second: Expression = language diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/urlExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/urlExpressions.scala index 22dcd33937dfb..845ca0b608ef3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/urlExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/urlExpressions.scala @@ -57,13 +57,14 @@ case class UrlEncode(child: Expression) SQLConf.get.defaultStringType, "encode", Seq(child), - Seq(StringTypeWithCollation)) + Seq(StringTypeWithCollation(supportsTrimCollation = true))) override protected def withNewChildInternal(newChild: Expression): Expression = { copy(child = newChild) } - override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCollation) + override def inputTypes: Seq[AbstractDataType] = + Seq(StringTypeWithCollation(supportsTrimCollation = true)) override def prettyName: String = "url_encode" } @@ -96,13 +97,14 @@ case class UrlDecode(child: Expression, failOnError: Boolean = true) SQLConf.get.defaultStringType, "decode", Seq(child, Literal(failOnError)), - Seq(StringTypeWithCollation, BooleanType)) + Seq(StringTypeWithCollation(supportsTrimCollation = true), BooleanType)) override protected def withNewChildInternal(newChild: Expression): Expression = { copy(child = newChild) } - override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCollation) + override def inputTypes: Seq[AbstractDataType] = + Seq(StringTypeWithCollation(supportsTrimCollation = true)) override def prettyName: String = "url_decode" } @@ -211,7 +213,7 @@ case class ParseUrl( override def nullable: Boolean = true override def inputTypes: Seq[AbstractDataType] = - Seq.fill(children.size)(StringTypeWithCollation) + Seq.fill(children.size)(StringTypeWithCollation(supportsTrimCollation = true)) override def dataType: DataType = SQLConf.get.defaultStringType override def prettyName: String = "parse_url" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala index 06aec93912984..1639a161df4cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala @@ -66,7 +66,8 @@ case class ParseJson(child: Expression, failOnError: Boolean = true) inputTypes :+ BooleanType :+ BooleanType, returnNullable = !failOnError) - override def inputTypes: Seq[AbstractDataType] = StringTypeWithCollation :: Nil + override def inputTypes: Seq[AbstractDataType] = + StringTypeWithCollation(supportsTrimCollation = true) :: Nil override def dataType: DataType = VariantType @@ -270,7 +271,7 @@ case class VariantGet( final override def nodePatternsInternal(): Seq[TreePattern] = Seq(VARIANT_GET) override def inputTypes: Seq[AbstractDataType] = - Seq(VariantType, StringTypeWithCollation) + Seq(VariantType, StringTypeWithCollation(supportsTrimCollation = true)) override def prettyName: String = if (failOnError) "variant_get" else "try_variant_get" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala index 2c18ffa2abecb..2e591288a21cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala @@ -41,7 +41,9 @@ abstract class XPathExtract override def nullable: Boolean = true override def inputTypes: Seq[AbstractDataType] = - Seq(StringTypeWithCollation, StringTypeWithCollation) + Seq( + StringTypeWithCollation(supportsTrimCollation = true), + StringTypeWithCollation(supportsTrimCollation = true)) override def checkInputDataTypes(): TypeCheckResult = { if (!path.foldable) { @@ -49,7 +51,7 @@ abstract class XPathExtract errorSubClass = "NON_FOLDABLE_INPUT", messageParameters = Map( "inputName" -> toSQLId("path"), - "inputType" -> toSQLType(StringTypeWithCollation), + "inputType" -> toSQLType(StringTypeWithCollation(supportsTrimCollation = true)), "inputExpr" -> toSQLExpr(path) ) ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala index 66f7f25e4abe8..d8254f04b4d94 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala @@ -126,7 +126,8 @@ case class XmlToStructs( defineCodeGen(ctx, ev, input => s"(InternalRow) $expr.nullSafeEval($input)") } - override def inputTypes: Seq[AbstractDataType] = StringTypeWithCollation :: Nil + override def inputTypes: Seq[AbstractDataType] = + StringTypeWithCollation(supportsTrimCollation = true) :: Nil override def prettyName: String = "from_xml" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala index 8cf7d78b510be..139e89828f8e5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercionSuite.scala @@ -1057,11 +1057,11 @@ class AnsiTypeCoercionSuite extends TypeCoercionSuiteBase { ArrayType(IntegerType)) shouldCast( ArrayType(StringType), - AbstractArrayType(StringTypeWithCollation), + AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true)), ArrayType(StringType)) shouldCast( ArrayType(IntegerType), - AbstractArrayType(StringTypeWithCollation), + AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true)), ArrayType(StringType)) shouldCast( ArrayType(StringType), @@ -1075,11 +1075,11 @@ class AnsiTypeCoercionSuite extends TypeCoercionSuiteBase { ArrayType(ArrayType(IntegerType))) shouldCast( ArrayType(ArrayType(StringType)), - AbstractArrayType(AbstractArrayType(StringTypeWithCollation)), + AbstractArrayType(AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true))), ArrayType(ArrayType(StringType))) shouldCast( ArrayType(ArrayType(IntegerType)), - AbstractArrayType(AbstractArrayType(StringTypeWithCollation)), + AbstractArrayType(AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true))), ArrayType(ArrayType(StringType))) shouldCast( ArrayType(ArrayType(StringType)), @@ -1088,16 +1088,16 @@ class AnsiTypeCoercionSuite extends TypeCoercionSuiteBase { // Invalid casts involving casting arrays into non-complex types. shouldNotCast(ArrayType(IntegerType), IntegerType) - shouldNotCast(ArrayType(StringType), StringTypeWithCollation) + shouldNotCast(ArrayType(StringType), StringTypeWithCollation(supportsTrimCollation = true)) shouldNotCast(ArrayType(StringType), IntegerType) - shouldNotCast(ArrayType(IntegerType), StringTypeWithCollation) + shouldNotCast(ArrayType(IntegerType), StringTypeWithCollation(supportsTrimCollation = true)) // Invalid casts involving casting arrays of arrays into arrays of non-complex types. shouldNotCast(ArrayType(ArrayType(IntegerType)), AbstractArrayType(IntegerType)) shouldNotCast(ArrayType(ArrayType(StringType)), - AbstractArrayType(StringTypeWithCollation)) + AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true))) shouldNotCast(ArrayType(ArrayType(StringType)), AbstractArrayType(IntegerType)) shouldNotCast(ArrayType(ArrayType(IntegerType)), - AbstractArrayType(StringTypeWithCollation)) + AbstractArrayType(StringTypeWithCollation(supportsTrimCollation = true))) } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index 0d5c414416d40..7a4777c34fed6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -2143,6 +2143,14 @@ Project [octet_length(collate(utf8_binary#x, utf8_lcase)) AS octet_length(collat +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet +-- !query +select octet_length(utf8_binary collate utf8_lcase_rtrim), octet_length(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query analysis +Project [octet_length(collate(utf8_binary#x, utf8_lcase_rtrim)) AS octet_length(collate(utf8_binary, utf8_lcase_rtrim))#x, octet_length(collate(utf8_lcase#x, utf8_binary_rtrim)) AS octet_length(collate(utf8_lcase, utf8_binary_rtrim))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + -- !query select luhn_check(num) from t9 -- !query analysis @@ -2233,6 +2241,14 @@ Project [is_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS is_valid_utf8(coll +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet +-- !query +select is_valid_utf8(utf8_binary collate utf8_lcase_rtrim), is_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query analysis +Project [is_valid_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS is_valid_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, is_valid_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS is_valid_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + -- !query select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5 -- !query analysis @@ -2249,6 +2265,14 @@ Project [make_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS make_valid_utf8( +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet +-- !query +select make_valid_utf8(utf8_binary collate utf8_lcase_rtrim), make_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query analysis +Project [make_valid_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS make_valid_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, make_valid_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS make_valid_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + -- !query select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5 -- !query analysis @@ -2265,6 +2289,14 @@ Project [validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS validate_utf8(coll +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet +-- !query +select validate_utf8(utf8_binary collate utf8_lcase_rtrim), validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query analysis +Project [validate_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS validate_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, validate_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS validate_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + -- !query select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5 -- !query analysis @@ -2281,6 +2313,14 @@ Project [try_validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS try_validate_u +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet +-- !query +select try_validate_utf8(utf8_binary collate utf8_lcase_rtrim), try_validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query analysis +Project [try_validate_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS try_validate_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, try_validate_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS try_validate_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + -- !query select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5 -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/collations.sql b/sql/core/src/test/resources/sql-tests/inputs/collations.sql index b4d33bb0196c9..df15adf2f8fe4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -328,6 +328,7 @@ select bit_length(utf8_binary), bit_length(utf8_lcase) from t5; select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t5; select octet_length(utf8_binary), octet_length(utf8_lcase) from t5; select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t5; +select octet_length(utf8_binary collate utf8_lcase_rtrim), octet_length(utf8_lcase collate utf8_binary_rtrim) from t5; -- Luhncheck select luhn_check(num) from t9; @@ -344,18 +345,22 @@ select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_ -- IsValidUTF8 select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t5; select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t5; +select is_valid_utf8(utf8_binary collate utf8_lcase_rtrim), is_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; -- MakeValidUTF8 select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5; select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t5; +select make_valid_utf8(utf8_binary collate utf8_lcase_rtrim), make_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; -- ValidateUTF8 select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5; select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t5; +select validate_utf8(utf8_binary collate utf8_lcase_rtrim), validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; -- TryValidateUTF8 select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5; select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t5; +select try_validate_utf8(utf8_binary collate utf8_lcase_rtrim), try_validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; -- Left/Right/Substr select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5; diff --git a/sql/core/src/test/resources/sql-tests/results/collations.sql.out b/sql/core/src/test/resources/sql-tests/results/collations.sql.out index e96549f00d6ec..fbfde3d78c1be 100644 --- a/sql/core/src/test/resources/sql-tests/results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/collations.sql.out @@ -3583,6 +3583,28 @@ struct +-- !query output +23 23 +29 29 +3 3 +3 3 +3 4 +3 4 +4 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + -- !query select luhn_check(num) from t9 -- !query schema @@ -3776,6 +3798,28 @@ true true true true +-- !query +select is_valid_utf8(utf8_binary collate utf8_lcase_rtrim), is_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true + + -- !query select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5 -- !query schema @@ -3820,6 +3864,28 @@ kitten sitTing İo İo +-- !query +select make_valid_utf8(utf8_binary collate utf8_lcase_rtrim), make_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + -- !query select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5 -- !query schema @@ -3864,6 +3930,28 @@ kitten sitTing İo İo +-- !query +select validate_utf8(utf8_binary collate utf8_lcase_rtrim), validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + -- !query select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5 -- !query schema @@ -3908,6 +3996,28 @@ kitten sitTing İo İo +-- !query +select try_validate_utf8(utf8_binary collate utf8_lcase_rtrim), try_validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + -- !query select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5 -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala index 4e91fd721a075..384411a0fd342 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala @@ -39,8 +39,18 @@ class CollationSQLExpressionsSuite with SharedSparkSession with ExpressionEvalHelper { - private val testSuppCollations = Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI") - private val testAdditionalCollations = Seq("UNICODE", "SR", "SR_CI", "SR_AI", "SR_CI_AI") + private val testSuppCollations = + Seq( + "UTF8_BINARY", + "UTF8_BINARY_RTRIM", + "UTF8_LCASE", + "UTF8_LCASE_RTRIM", + "UNICODE", + "UNICODE_RTRIM", + "UNICODE_CI", + "UNICODE_CI_RTRIM") + private val testAdditionalCollations = Seq("UNICODE", + "SR", "SR_RTRIM", "SR_CI", "SR_AI", "SR_CI_AI") private val fullyQualifiedPrefix = s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}." test("Support Md5 hash expression with collation") { @@ -264,11 +274,19 @@ class CollationSQLExpressionsSuite val testCases = Seq( UrlEncodeTestCase("https://spark.apache.org", "UTF8_BINARY", "https%3A%2F%2Fspark.apache.org"), + UrlEncodeTestCase("https://spark.apache.org", "UTF8_BINARY_RTRIM", + "https%3A%2F%2Fspark.apache.org"), UrlEncodeTestCase("https://spark.apache.org", "UTF8_LCASE", "https%3A%2F%2Fspark.apache.org"), + UrlEncodeTestCase("https://spark.apache.org", "UTF8_LCASE_RTRIM", + "https%3A%2F%2Fspark.apache.org"), UrlEncodeTestCase("https://spark.apache.org", "UNICODE", "https%3A%2F%2Fspark.apache.org"), + UrlEncodeTestCase("https://spark.apache.org", "UNICODE_RTRIM", + "https%3A%2F%2Fspark.apache.org"), UrlEncodeTestCase("https://spark.apache.org", "UNICODE_CI", + "https%3A%2F%2Fspark.apache.org"), + UrlEncodeTestCase("https://spark.apache.org", "UNICODE_CI_RTRIM", "https%3A%2F%2Fspark.apache.org") ) @@ -298,11 +316,19 @@ class CollationSQLExpressionsSuite val testCases = Seq( UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_BINARY", "https://spark.apache.org"), + UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_BINARY_RTRIM", + "https://spark.apache.org"), UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_LCASE", "https://spark.apache.org"), + UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_LCASE_RTRIM", + "https://spark.apache.org"), UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE", "https://spark.apache.org"), + UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE_RTRIM", + "https://spark.apache.org"), UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE_CI", + "https://spark.apache.org"), + UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE_CI_RTRIM", "https://spark.apache.org") ) @@ -333,11 +359,19 @@ class CollationSQLExpressionsSuite val testCases = Seq( ParseUrlTestCase("http://spark.apache.org/path?query=1", "UTF8_BINARY", "HOST", "spark.apache.org"), + ParseUrlTestCase("http://spark.apache.org/path?query=1", "UTF8_BINARY_RTRIM", "HOST", + "spark.apache.org"), ParseUrlTestCase("http://spark.apache.org/path?query=2", "UTF8_LCASE", "PATH", "/path"), + ParseUrlTestCase("http://spark.apache.org/path?query=2", "UTF8_LCASE_RTRIM", "PATH", + "/path"), ParseUrlTestCase("http://spark.apache.org/path?query=3", "UNICODE", "QUERY", "query=3"), + ParseUrlTestCase("http://spark.apache.org/path?query=3", "UNICODE_RTRIM", "QUERY", + "query=3"), ParseUrlTestCase("http://spark.apache.org/path?query=4", "UNICODE_CI", "PROTOCOL", + "http"), + ParseUrlTestCase("http://spark.apache.org/path?query=4", "UNICODE_CI_RTRIM", "PROTOCOL", "http") ) @@ -372,11 +406,20 @@ class CollationSQLExpressionsSuite Row(1), Seq( StructField("a", IntegerType, nullable = true) )), + CsvToStructsTestCase("1", "UTF8_BINARY_RTRIM", "'a INT'", "", + Row(1), Seq( + StructField("a", IntegerType, nullable = true) + )), CsvToStructsTestCase("true, 0.8", "UTF8_LCASE", "'A BOOLEAN, B DOUBLE'", "", Row(true, 0.8), Seq( StructField("A", BooleanType, nullable = true), StructField("B", DoubleType, nullable = true) )), + CsvToStructsTestCase("true, 0.8", "UTF8_LCASE_RTRIM", "'A BOOLEAN, B DOUBLE'", "", + Row(true, 0.8), Seq( + StructField("A", BooleanType, nullable = true), + StructField("B", DoubleType, nullable = true) + )), CsvToStructsTestCase("\"Spark\"", "UNICODE", "'a STRING'", "", Row("Spark"), Seq( StructField("a", StringType, nullable = true) @@ -385,6 +428,10 @@ class CollationSQLExpressionsSuite Row("Spark"), Seq( StructField("a", StringType("UNICODE"), nullable = true) )), + CsvToStructsTestCase("\"Spark\"", "UNICODE_RTRIM", "'a STRING COLLATE UNICODE_RTRIM'", "", + Row("Spark"), Seq( + StructField("a", StringType("UNICODE_RTRIM"), nullable = true) + )), CsvToStructsTestCase("26/08/2015", "UTF8_BINARY", "'time Timestamp'", ", map('timestampFormat', 'dd/MM/yyyy')", Row( new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S").parse("2015-08-26 00:00:00.0") @@ -419,10 +466,16 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfCsvTestCase("1", "UTF8_BINARY", "STRUCT<_c0: INT>"), + SchemaOfCsvTestCase("1", "UTF8_BINARY_RTRIM", "STRUCT<_c0: INT>"), SchemaOfCsvTestCase("true,0.8", "UTF8_LCASE", "STRUCT<_c0: BOOLEAN, _c1: DOUBLE>"), + SchemaOfCsvTestCase("true,0.8", "UTF8_LCASE_RTRIM", + "STRUCT<_c0: BOOLEAN, _c1: DOUBLE>"), SchemaOfCsvTestCase("2015-08-26", "UNICODE", "STRUCT<_c0: DATE>"), + SchemaOfCsvTestCase("2015-08-26", "UNICODE_RTRIM", "STRUCT<_c0: DATE>"), SchemaOfCsvTestCase("abc", "UNICODE_CI", + "STRUCT<_c0: STRING>"), + SchemaOfCsvTestCase("abc", "UNICODE_CI_RTRIM", "STRUCT<_c0: STRING>") ) @@ -451,9 +504,14 @@ class CollationSQLExpressionsSuite val testCases = Seq( StructsToCsvTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY", "1,2"), + StructsToCsvTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY_RTRIM", "1,2"), StructsToCsvTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE", "true,2.0"), + StructsToCsvTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE_RTRIM", "true,2.0"), StructsToCsvTestCase("named_struct()", "UNICODE", null), + StructsToCsvTestCase("named_struct()", "UNICODE_RTRIM", null), StructsToCsvTestCase("named_struct('time', to_timestamp('2015-08-26'))", "UNICODE_CI", + "2015-08-26T00:00:00.000-07:00"), + StructsToCsvTestCase("named_struct('time', to_timestamp('2015-08-26'))", "UNICODE_CI_RTRIM", "2015-08-26T00:00:00.000-07:00") ) @@ -484,9 +542,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( ConvTestCase("100", "2", "10", "UTF8_BINARY", "4"), + ConvTestCase("100", "2", "10", "UTF8_BINARY_RTRIM", "4"), ConvTestCase("100", "2", "10", "UTF8_LCASE", "4"), + ConvTestCase("100", "2", "10", "UTF8_LCASE_RTRIM", "4"), ConvTestCase("100", "2", "10", "UNICODE", "4"), - ConvTestCase("100", "2", "10", "UNICODE_CI", "4") + ConvTestCase("100", "2", "10", "UNICODE_RTRIM", "4"), + ConvTestCase("100", "2", "10", "UNICODE_CI", "4"), + ConvTestCase("100", "2", "10", "UNICODE_CI_RTRIM", "4") ) testCases.foreach(t => { val query = @@ -508,9 +570,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( BinTestCase("13", "UTF8_BINARY", "1101"), + BinTestCase("13", "UTF8_BINARY_RTRIM", "1101"), BinTestCase("13", "UTF8_LCASE", "1101"), + BinTestCase("13", "UTF8_LCASE_RTRIM", "1101"), BinTestCase("13", "UNICODE", "1101"), - BinTestCase("13", "UNICODE_CI", "1101") + BinTestCase("13", "UNICODE_RTRIM", "1101"), + BinTestCase("13", "UNICODE_CI", "1101"), + BinTestCase("13", "UNICODE_CI_RTRIM", "1101") ) testCases.foreach(t => { val query = @@ -533,9 +599,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( HexTestCase("13", "UTF8_BINARY", "D"), + HexTestCase("13", "UTF8_BINARY_RTRIM", "D"), HexTestCase("13", "UTF8_LCASE", "D"), + HexTestCase("13", "UTF8_LCASE_RTRIM", "D"), HexTestCase("13", "UNICODE", "D"), - HexTestCase("13", "UNICODE_CI", "D") + HexTestCase("13", "UNICODE_RTRIM", "D"), + HexTestCase("13", "UNICODE_CI", "D"), + HexTestCase("13", "UNICODE_CI_RTRIM", "D") ) testCases.foreach(t => { val query = @@ -558,10 +628,15 @@ class CollationSQLExpressionsSuite val testCases = Seq( HexTestCase("Spark SQL", "UTF8_BINARY", "537061726B2053514C"), + HexTestCase("Spark SQL", "UTF8_BINARY_RTRIM", "537061726B2053514C"), HexTestCase("Spark SQL", "UTF8_LCASE", "537061726B2053514C"), + HexTestCase("Spark SQL", "UTF8_LCASE_RTRIM", "537061726B2053514C"), HexTestCase("Spark SQL", "UNICODE", "537061726B2053514C"), + HexTestCase("Spark SQL", "UNICODE_RTRIM", "537061726B2053514C"), HexTestCase("Spark SQL", "UNICODE_CI", "537061726B2053514C"), - HexTestCase("Spark SQL", "DE_CI_AI", "537061726B2053514C") + HexTestCase("Spark SQL", "UNICODE_CI_RTRIM", "537061726B2053514C"), + HexTestCase("Spark SQL", "DE_CI_AI", "537061726B2053514C"), + HexTestCase("Spark SQL", "DE_CI_AI_RTRIM", "537061726B2053514C") ) testCases.foreach(t => { val query = @@ -582,9 +657,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( UnHexTestCase("537061726B2053514C", "UTF8_BINARY", "Spark SQL"), + UnHexTestCase("537061726B2053514C", "UTF8_BINARY_RTRIM", "Spark SQL"), UnHexTestCase("537061726B2053514C", "UTF8_LCASE", "Spark SQL"), + UnHexTestCase("537061726B2053514C", "UTF8_LCASE_RTRIM", "Spark SQL"), UnHexTestCase("537061726B2053514C", "UNICODE", "Spark SQL"), + UnHexTestCase("537061726B2053514C", "UNICODE_RTRIM", "Spark SQL"), UnHexTestCase("537061726B2053514C", "UNICODE_CI", "Spark SQL"), + UnHexTestCase("537061726B2053514C", "UNICODE_CI_RTRIM", "Spark SQL"), UnHexTestCase("537061726B2053514C", "DE", "Spark SQL") ) testCases.foreach(t => { @@ -613,16 +692,30 @@ class CollationSQLExpressionsSuite "xpath_boolean", "UTF8_BINARY", true, BooleanType), XPathTestCase("12", "sum(A/B)", "xpath_short", "UTF8_BINARY", 3, ShortType), + XPathTestCase("1", "a/b", + "xpath_boolean", "UTF8_BINARY_RTRIM", true, BooleanType), + XPathTestCase("12", "sum(A/B)", + "xpath_short", "UTF8_BINARY_RTRIM", 3, ShortType), XPathTestCase("34", "sum(a/b)", "xpath_int", "UTF8_LCASE", 7, IntegerType), XPathTestCase("56", "sum(A/B)", "xpath_long", "UTF8_LCASE", 11, LongType), + XPathTestCase("34", "sum(a/b)", + "xpath_int", "UTF8_LCASE_RTRIM", 7, IntegerType), + XPathTestCase("56", "sum(A/B)", + "xpath_long", "UTF8_LCASE_RTRIM", 11, LongType), XPathTestCase("78", "sum(a/b)", "xpath_float", "UNICODE", 15.0, FloatType), XPathTestCase("90", "sum(A/B)", "xpath_double", "UNICODE", 9.0, DoubleType), + XPathTestCase("78", "sum(a/b)", + "xpath_float", "UNICODE_RTRIM", 15.0, FloatType), + XPathTestCase("90", "sum(A/B)", + "xpath_double", "UNICODE_RTRIM", 9.0, DoubleType), XPathTestCase("bcc", "a/c", "xpath_string", "UNICODE_CI", "cc", StringType("UNICODE_CI")), + XPathTestCase("bcc ", "a/c", + "xpath_string", "UNICODE_CI_RTRIM", "cc ", StringType("UNICODE_CI_RTRIM")), XPathTestCase("b1b2b3c1c2", "a/b/text()", "xpath", "UNICODE_CI", Array("b1", "b2", "b3"), ArrayType(StringType("UNICODE_CI"))) ) @@ -651,10 +744,15 @@ class CollationSQLExpressionsSuite val testCases = Seq( StringSpaceTestCase(1, "UTF8_BINARY", " "), + StringSpaceTestCase(1, "UTF8_BINARY_RTRIM", " "), StringSpaceTestCase(2, "UTF8_LCASE", " "), + StringSpaceTestCase(2, "UTF8_LCASE_RTRIM", " "), StringSpaceTestCase(3, "UNICODE", " "), + StringSpaceTestCase(3, "UNICODE_RTRIM", " "), StringSpaceTestCase(4, "UNICODE_CI", " "), - StringSpaceTestCase(5, "AF_CI_AI", " ") + StringSpaceTestCase(4, "UNICODE_CI_RTRIM", " "), + StringSpaceTestCase(5, "AF_CI_AI", " "), + StringSpaceTestCase(5, "AF_CI_AI_RTRIM", " ") ) // Supported collations @@ -684,9 +782,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( ToNumberTestCase("123", "UTF8_BINARY", "999", 123, DecimalType(3, 0)), + ToNumberTestCase("123", "UTF8_BINARY_RTRIM", "999", 123, DecimalType(3, 0)), ToNumberTestCase("1", "UTF8_LCASE", "0.00", 1.00, DecimalType(3, 2)), + ToNumberTestCase("1", "UTF8_LCASE_RTRIM", "0.00", 1.00, DecimalType(3, 2)), ToNumberTestCase("99,999", "UNICODE", "99,999", 99999, DecimalType(5, 0)), - ToNumberTestCase("$14.99", "UNICODE_CI", "$99.99", 14.99, DecimalType(4, 2)) + ToNumberTestCase("99,999", "UNICODE_RTRIM", "99,999", 99999, DecimalType(5, 0)), + ToNumberTestCase("$14.99", "UNICODE_CI", "$99.99", 14.99, DecimalType(4, 2)), + ToNumberTestCase("$14.99", "UNICODE_CI_RTRIM", "$99.99", 14.99, DecimalType(4, 2)) ) // Supported collations (ToNumber) @@ -754,9 +856,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( ToCharTestCase(12, "UTF8_BINARY", "999", " 12"), + ToCharTestCase(12, "UTF8_BINARY_RTRIM", "999", " 12"), ToCharTestCase(34, "UTF8_LCASE", "000D00", "034.00"), + ToCharTestCase(34, "UTF8_LCASE_RTRIM", "000D00", "034.00"), ToCharTestCase(56, "UNICODE", "$99.99", "$56.00"), - ToCharTestCase(78, "UNICODE_CI", "99D9S", "78.0+") + ToCharTestCase(56, "UNICODE_RTRIM", "$99.99", "$56.00"), + ToCharTestCase(78, "UNICODE_CI", "99D9S", "78.0+"), + ToCharTestCase(78, "UNICODE_CI_RTRIM", "99D9S", "78.0+") ) // Supported collations @@ -785,9 +891,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( GetJsonObjectTestCase("{\"a\":\"b\"}", "$.a", "UTF8_BINARY", "b"), + GetJsonObjectTestCase("{\"a\":\"b\"}", "$.a", "UTF8_BINARY_RTRIM", "b"), GetJsonObjectTestCase("{\"A\":\"1\"}", "$.A", "UTF8_LCASE", "1"), + GetJsonObjectTestCase("{\"A\":\"1\"}", "$.A", "UTF8_LCASE_RTRIM", "1"), GetJsonObjectTestCase("{\"x\":true}", "$.x", "UNICODE", "true"), - GetJsonObjectTestCase("{\"X\":1}", "$.X", "UNICODE_CI", "1") + GetJsonObjectTestCase("{\"x\":true}", "$.x", "UNICODE_RTRIM", "true"), + GetJsonObjectTestCase("{\"X\":1}", "$.X", "UNICODE_CI", "1"), + GetJsonObjectTestCase("{\"X\":1}", "$.X", "UNICODE_CI_RTRIM", "1") ) // Supported collations @@ -817,10 +927,16 @@ class CollationSQLExpressionsSuite val testCases = Seq( JsonTupleTestCase("{\"a\":1, \"b\":2}", "'a', 'b'", "UTF8_BINARY", Row("1", "2")), + JsonTupleTestCase("{\"a\":1, \"b\":2}", "'a', 'b'", "UTF8_BINARY_RTRIM", + Row("1", "2")), JsonTupleTestCase("{\"A\":\"3\", \"B\":\"4\"}", "'A', 'B'", "UTF8_LCASE", Row("3", "4")), + JsonTupleTestCase("{\"A\":\"3\", \"B\":\"4\"}", "'A', 'B'", "UTF8_LCASE_RTRIM", + Row("3", "4")), JsonTupleTestCase("{\"x\":true, \"y\":false}", "'x', 'y'", "UNICODE", Row("true", "false")), + JsonTupleTestCase("{\"x\":true, \"y\":false}", "'x', 'y'", "UNICODE_RTRIM", + Row("true", "false")), JsonTupleTestCase("{\"X\":null, \"Y\":null}", "'X', 'Y'", "UNICODE_CI", Row(null, null)) ) @@ -852,12 +968,20 @@ class CollationSQLExpressionsSuite val testCases = Seq( JsonToStructsTestCase("{\"a\":1, \"b\":2.0}", "a INT, b DOUBLE", "UTF8_BINARY", Row(Row(1, 2.0))), + JsonToStructsTestCase("{\"a\":1, \"b\":2.0}", "a INT, b DOUBLE", + "UTF8_BINARY_RTRIM", Row(Row(1, 2.0))), JsonToStructsTestCase("{\"A\":\"3\", \"B\":4}", "A STRING COLLATE UTF8_LCASE, B INT", "UTF8_LCASE", Row(Row("3", 4))), + JsonToStructsTestCase("{\"A\":\"3\", \"B\":4}", "A STRING COLLATE UTF8_LCASE, B INT", + "UTF8_LCASE_RTRIM", Row(Row("3", 4))), JsonToStructsTestCase("{\"x\":true, \"y\":null}", "x BOOLEAN, y VOID", "UNICODE", Row(Row(true, null))), + JsonToStructsTestCase("{\"x\":true, \"y\":null}", "x BOOLEAN, y VOID", + "UNICODE_RTRIM", Row(Row(true, null))), + JsonToStructsTestCase("{\"X\":null, \"Y\":false}", "X VOID, Y BOOLEAN", + "UNICODE_CI", Row(Row(null, false))), JsonToStructsTestCase("{\"X\":null, \"Y\":false}", "X VOID, Y BOOLEAN", - "UNICODE_CI", Row(Row(null, false))) + "UNICODE_CI_RTRIM", Row(Row(null, false))) ) // Supported collations @@ -886,12 +1010,20 @@ class CollationSQLExpressionsSuite val testCases = Seq( StructsToJsonTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY", Row("{\"a\":1,\"b\":2}")), + StructsToJsonTestCase("named_struct('a', 1, 'b', 2)", + "UTF8_BINARY_RTRIM", Row("{\"a\":1,\"b\":2}")), StructsToJsonTestCase("array(named_struct('a', 1, 'b', 2))", "UTF8_LCASE", Row("[{\"a\":1,\"b\":2}]")), + StructsToJsonTestCase("array(named_struct('a', 1, 'b', 2))", + "UTF8_LCASE_RTRIM", Row("[{\"a\":1,\"b\":2}]")), StructsToJsonTestCase("map('a', named_struct('b', 1))", "UNICODE", Row("{\"a\":{\"b\":1}}")), + StructsToJsonTestCase("map('a', named_struct('b', 1))", + "UNICODE_RTRIM", Row("{\"a\":{\"b\":1}}")), StructsToJsonTestCase("array(map('a', 1))", - "UNICODE_CI", Row("[{\"a\":1}]")) + "UNICODE_CI", Row("[{\"a\":1}]")), + StructsToJsonTestCase("array(map('a', 1))", + "UNICODE_CI_RTRIM", Row("[{\"a\":1}]")) ) // Supported collations @@ -919,9 +1051,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( LengthOfJsonArrayTestCase("'[1,2,3,4]'", "UTF8_BINARY", Row(4)), + LengthOfJsonArrayTestCase("'[1,2,3,4]'", "UTF8_BINARY_RTRIM", Row(4)), LengthOfJsonArrayTestCase("'[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]'", "UTF8_LCASE", Row(5)), + LengthOfJsonArrayTestCase("'[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]'", "UTF8_LCASE_RTRIM", Row(5)), LengthOfJsonArrayTestCase("'[1,2'", "UNICODE", Row(null)), - LengthOfJsonArrayTestCase("'['", "UNICODE_CI", Row(null)) + LengthOfJsonArrayTestCase("'[1,2'", "UNICODE_RTRIM", Row(null)), + LengthOfJsonArrayTestCase("'['", "UNICODE_CI", Row(null)), + LengthOfJsonArrayTestCase("'['", "UNICODE_CI_RTRIM", Row(null)) ) // Supported collations @@ -949,11 +1085,19 @@ class CollationSQLExpressionsSuite val testCases = Seq( JsonObjectKeysJsonArrayTestCase("{}", "UTF8_BINARY", Row(Seq())), + JsonObjectKeysJsonArrayTestCase("{}", "UTF8_BINARY_RTRIM", + Row(Seq())), JsonObjectKeysJsonArrayTestCase("{\"k\":", "UTF8_LCASE", Row(null)), + JsonObjectKeysJsonArrayTestCase("{\"k\":", "UTF8_LCASE_RTRIM", + Row(null)), JsonObjectKeysJsonArrayTestCase("{\"k1\": \"v1\"}", "UNICODE", Row(Seq("k1"))), + JsonObjectKeysJsonArrayTestCase("{\"k1\": \"v1\"}", "UNICODE_RTRIM", + Row(Seq("k1"))), JsonObjectKeysJsonArrayTestCase("{\"k1\":1,\"k2\":{\"k3\":3, \"k4\":4}}", "UNICODE_CI", + Row(Seq("k1", "k2"))), + JsonObjectKeysJsonArrayTestCase("{\"k1\":1,\"k2\":{\"k3\":3, \"k4\":4}}", "UNICODE_CI_RTRIM", Row(Seq("k1", "k2"))) ) @@ -983,12 +1127,20 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfJsonTestCase("'[{\"col\":0}]'", "UTF8_BINARY", Row("ARRAY>")), + SchemaOfJsonTestCase("'[{\"col\":0}]'", + "UTF8_BINARY_RTRIM", Row("ARRAY>")), SchemaOfJsonTestCase("'[{\"col\":01}]', map('allowNumericLeadingZeros', 'true')", "UTF8_LCASE", Row("ARRAY>")), + SchemaOfJsonTestCase("'[{\"col\":01}]', map('allowNumericLeadingZeros', 'true')", + "UTF8_LCASE_RTRIM", Row("ARRAY>")), SchemaOfJsonTestCase("'[]'", "UNICODE", Row("ARRAY")), + SchemaOfJsonTestCase("'[]'", + "UNICODE_RTRIM", Row("ARRAY")), + SchemaOfJsonTestCase("''", + "UNICODE_CI", Row("STRING")), SchemaOfJsonTestCase("''", - "UNICODE_CI", Row("STRING")) + "UNICODE_CI_RTRIM", Row("STRING")) ) // Supported collations @@ -1029,10 +1181,7 @@ class CollationSQLExpressionsSuite Map("c" -> "1", "č" -> "2", "ć" -> "3")) ) val unsupportedTestCases = Seq( - StringToMapTestCase("a:1,b:2,c:3", "?", "?", "UNICODE_AI", null), - StringToMapTestCase("a:1,b:2,c:3", "?", "?", "UNICODE_RTRIM", null), - StringToMapTestCase("a:1,b:2,c:3", "?", "?", "UTF8_BINARY_RTRIM", null), - StringToMapTestCase("a:1,b:2,c:3", "?", "?", "UTF8_LCASE_RTRIM", null)) + StringToMapTestCase("a:1,b:2,c:3", "?", "?", "UNICODE_AI", null)) testCases.foreach(t => { // Unit test. val text = Literal.create(t.text, StringType(t.collation)) @@ -1079,9 +1228,13 @@ class CollationSQLExpressionsSuite case class RaiseErrorTestCase(errorMessage: String, collationName: String) val testCases = Seq( RaiseErrorTestCase("custom error message 1", "UTF8_BINARY"), + RaiseErrorTestCase("custom error message 1", "UTF8_BINARY_RTRIM"), RaiseErrorTestCase("custom error message 2", "UTF8_LCASE"), + RaiseErrorTestCase("custom error message 2", "UTF8_LCASE_RTRIM"), RaiseErrorTestCase("custom error message 3", "UNICODE"), - RaiseErrorTestCase("custom error message 4", "UNICODE_CI") + RaiseErrorTestCase("custom error message 3", "UNICODE_RTRIM"), + RaiseErrorTestCase("custom error message 4", "UNICODE_CI"), + RaiseErrorTestCase("custom error message 4", "UNICODE_CI_RTRIM") ) testCases.foreach(t => { withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { @@ -1100,7 +1253,13 @@ class CollationSQLExpressionsSuite test("Support CurrentDatabase/Catalog/User expressions with collation") { // Supported collations - Seq("UTF8_LCASE", "UNICODE", "UNICODE_CI", "SR_CI_AI").foreach(collationName => + Seq( + "UTF8_LCASE", + "UTF8_LCASE_RTRIM", + "UNICODE", + "UNICODE_RTRIM", + "UNICODE_CI", + "SR_CI_AI").foreach(collationName => withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { val queryDatabase = sql("SELECT current_schema()") val queryCatalog = sql("SELECT current_catalog()") @@ -1116,7 +1275,14 @@ class CollationSQLExpressionsSuite test("Support Uuid misc expression with collation") { // Supported collations - Seq("UTF8_LCASE", "UNICODE", "UNICODE_CI", "NO_CI_AI").foreach(collationName => + Seq( + "UTF8_LCASE", + "UTF8_LCASE_RTRIM", + "UNICODE", + "UNICODE_RTRIM", + "UNICODE_CI", + "UNICODE_CI_RTRIM", + "NO_CI_AI").foreach(collationName => withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { val query = s"SELECT uuid()" // Result & data type @@ -1291,11 +1457,20 @@ class CollationSQLExpressionsSuite Row(1), Seq( StructField("a", IntegerType, nullable = true) )), + XmlToStructsTestCase("

1

", "UTF8_BINARY_RTRIM", "'a INT'", "", + Row(1), Seq( + StructField("a", IntegerType, nullable = true) + )), XmlToStructsTestCase("

true0.8

", "UTF8_LCASE", "'A BOOLEAN, B DOUBLE'", "", Row(true, 0.8), Seq( StructField("A", BooleanType, nullable = true), StructField("B", DoubleType, nullable = true) )), + XmlToStructsTestCase("

true0.8

", "UTF8_LCASE_RTRIM", + "'A BOOLEAN, B DOUBLE'", "", Row(true, 0.8), Seq( + StructField("A", BooleanType, nullable = true), + StructField("B", DoubleType, nullable = true) + )), XmlToStructsTestCase("

Spark

", "UNICODE", "'s STRING'", "", Row("Spark"), Seq( StructField("s", StringType, nullable = true) @@ -1304,6 +1479,11 @@ class CollationSQLExpressionsSuite Row("Spark"), Seq( StructField("s", StringType("UNICODE"), nullable = true) )), + XmlToStructsTestCase("

Spark

", "UNICODE_RTRIM", + "'s STRING COLLATE UNICODE_RTRIM'", "", + Row("Spark"), Seq( + StructField("s", StringType("UNICODE_RTRIM"), nullable = true) + )), XmlToStructsTestCase("

", "UNICODE_CI", "'time Timestamp'", ", map('timestampFormat', 'dd/MM/yyyy')", Row( new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S").parse("2015-08-26 00:00:00.0") @@ -1337,10 +1517,16 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfXmlTestCase("

1

", "UTF8_BINARY", "STRUCT"), + SchemaOfXmlTestCase("

1

", "UTF8_BINARY_RTRIM", "STRUCT"), SchemaOfXmlTestCase("

true0.8

", "UTF8_LCASE", "STRUCT"), + SchemaOfXmlTestCase("

true0.8

", "UTF8_LCASE_RTRIM", + "STRUCT"), SchemaOfXmlTestCase("

", "UNICODE", "STRUCT<>"), + SchemaOfXmlTestCase("

", "UNICODE_RTRIM", "STRUCT<>"), SchemaOfXmlTestCase("

123

", "UNICODE_CI", + "STRUCT>"), + SchemaOfXmlTestCase("

123

", "UNICODE_CI_RTRIM", "STRUCT>") ) @@ -1373,6 +1559,11 @@ class CollationSQLExpressionsSuite | 1 | 2 |""".stripMargin), + StructsToXmlTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY_RTRIM", + s""" + | 1 + | 2 + |""".stripMargin), StructsToXmlTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE", s""" | true @@ -1383,6 +1574,11 @@ class CollationSQLExpressionsSuite | aa | bb |""".stripMargin), + StructsToXmlTestCase("named_struct('A', 'aa', 'B', 'bb')", "UTF8_LCASE_RTRIM", + s""" + | aa + | bb + |""".stripMargin), StructsToXmlTestCase("named_struct('A', 'aa', 'B', 'bb')", "UTF8_BINARY", s""" | aa @@ -1390,6 +1586,8 @@ class CollationSQLExpressionsSuite |""".stripMargin), StructsToXmlTestCase("named_struct()", "UNICODE", ""), + StructsToXmlTestCase("named_struct()", "UNICODE_RTRIM", + ""), StructsToXmlTestCase("named_struct('time', to_timestamp('2015-08-26'))", "UNICODE_CI", s""" | @@ -1421,9 +1619,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( ParseJsonTestCase("{\"a\":1,\"b\":2}", "UTF8_BINARY", "{\"a\":1,\"b\":2}"), + ParseJsonTestCase("{\"a\":1,\"b\":2}", "UTF8_BINARY_RTRIM", "{\"a\":1,\"b\":2}"), ParseJsonTestCase("{\"A\":3,\"B\":4}", "UTF8_LCASE", "{\"A\":3,\"B\":4}"), + ParseJsonTestCase("{\"A\":3,\"B\":4}", "UTF8_LCASE_RTRIM", "{\"A\":3,\"B\":4}"), ParseJsonTestCase("{\"c\":5,\"d\":6}", "UNICODE", "{\"c\":5,\"d\":6}"), - ParseJsonTestCase("{\"C\":7,\"D\":8}", "UNICODE_CI", "{\"C\":7,\"D\":8}") + ParseJsonTestCase("{\"c\":5,\"d\":6}", "UNICODE_RTRIM", "{\"c\":5,\"d\":6}"), + ParseJsonTestCase("{\"C\":7,\"D\":8}", "UNICODE_CI", "{\"C\":7,\"D\":8}"), + ParseJsonTestCase("{\"C\":7,\"D\":8}", "UNICODE_CI_RTRIM", "{\"C\":7,\"D\":8}") ) // Supported collations (ParseJson) @@ -1493,9 +1695,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( IsVariantNullTestCase("'null'", "UTF8_BINARY", result = true), + IsVariantNullTestCase("'null'", "UTF8_BINARY_RTRIM", result = true), IsVariantNullTestCase("'\"null\"'", "UTF8_LCASE", result = false), + IsVariantNullTestCase("'\"null\"'", "UTF8_LCASE_RTRIM", result = false), IsVariantNullTestCase("'13'", "UNICODE", result = false), - IsVariantNullTestCase("null", "UNICODE_CI", result = false) + IsVariantNullTestCase("'13'", "UNICODE_RTRIM", result = false), + IsVariantNullTestCase("null", "UNICODE_CI", result = false), + IsVariantNullTestCase("null", "UNICODE_CI_RTRIM", result = false) ) // Supported collations @@ -1524,6 +1730,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( VariantGetTestCase("{\"a\": 1}", "$.a", "int", "UTF8_BINARY", 1, IntegerType), + VariantGetTestCase("{\"a\": 1}", "$.a", "int", "UTF8_BINARY_RTRIM", 1, IntegerType), VariantGetTestCase("{\"a\": 1}", "$.b", "int", "UTF8_LCASE", null, IntegerType), VariantGetTestCase("[1, \"2\"]", "$[1]", "string", "UNICODE", "2", StringType), @@ -1610,6 +1817,14 @@ class CollationSQLExpressionsSuite StructField("value", VariantType, nullable = false) ) ), + VariantExplodeTestCase("[\"hello\", \"world\"]", "UTF8_BINARY_RTRIM", + Row(0, "null", "\"hello\"").toString() + Row(1, "null", "\"world\"").toString(), + Seq[StructField]( + StructField("pos", IntegerType, nullable = false), + StructField("key", StringType("UTF8_BINARY_RTRIM")), + StructField("value", VariantType, nullable = false) + ) + ), VariantExplodeTestCase("[\"Spark\", \"SQL\"]", "UTF8_LCASE", Row(0, "null", "\"Spark\"").toString() + Row(1, "null", "\"SQL\"").toString(), Seq[StructField]( @@ -1618,6 +1833,14 @@ class CollationSQLExpressionsSuite StructField("value", VariantType, nullable = false) ) ), + VariantExplodeTestCase("[\"Spark\", \"SQL\"]", "UTF8_LCASE_RTRIM", + Row(0, "null", "\"Spark\"").toString() + Row(1, "null", "\"SQL\"").toString(), + Seq[StructField]( + StructField("pos", IntegerType, nullable = false), + StructField("key", StringType("UTF8_LCASE_RTRIM")), + StructField("value", VariantType, nullable = false) + ) + ), VariantExplodeTestCase("{\"a\": true, \"b\": 3.14}", "UNICODE", Row(0, "a", "true").toString() + Row(1, "b", "3.14").toString(), Seq[StructField]( @@ -1626,6 +1849,14 @@ class CollationSQLExpressionsSuite StructField("value", VariantType, nullable = false) ) ), + VariantExplodeTestCase("{\"a\": true, \"b\": 3.14}", "UNICODE_RTRIM", + Row(0, "a", "true").toString() + Row(1, "b", "3.14").toString(), + Seq[StructField]( + StructField("pos", IntegerType, nullable = false), + StructField("key", StringType("UNICODE_RTRIM")), + StructField("value", VariantType, nullable = false) + ) + ), VariantExplodeTestCase("{\"A\": 9.99, \"B\": false}", "UNICODE_CI", Row(0, "A", "9.99").toString() + Row(1, "B", "false").toString(), Seq[StructField]( @@ -1661,11 +1892,17 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfVariantTestCase("null", "UTF8_BINARY", "VOID"), + SchemaOfVariantTestCase("null", "UTF8_BINARY_RTRIM", "VOID"), SchemaOfVariantTestCase("[]", "UTF8_LCASE", "ARRAY"), + SchemaOfVariantTestCase("[]", "UTF8_LCASE_RTRIM", "ARRAY"), SchemaOfVariantTestCase("[{\"a\":true,\"b\":0}]", "UNICODE", "ARRAY>"), + SchemaOfVariantTestCase("[{\"a\":true,\"b\":0}]", "UNICODE_RTRIM", + "ARRAY>"), SchemaOfVariantTestCase("[{\"A\":\"x\",\"B\":-1.00}]", "UNICODE_CI", - "ARRAY>") + "ARRAY>"), + SchemaOfVariantTestCase("[{\"A\":\"x\",\"B\":-1.00}]", "UNICODE_CI_RTRIM", + "ARRAY>") ) // Supported collations @@ -1692,11 +1929,18 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfVariantAggTestCase("('1'), ('2'), ('3')", "UTF8_BINARY", "BIGINT"), + SchemaOfVariantAggTestCase("('1'), ('2'), ('3')", "UTF8_BINARY_RTRIM", "BIGINT"), SchemaOfVariantAggTestCase("('true'), ('false'), ('true')", "UTF8_LCASE", "BOOLEAN"), + SchemaOfVariantAggTestCase("('true'), ('false'), ('true')", "UTF8_LCASE_RTRIM", "BOOLEAN"), SchemaOfVariantAggTestCase("('{\"a\": 1}'), ('{\"b\": true}'), ('{\"c\": 1.23}')", "UNICODE", "OBJECT"), + SchemaOfVariantAggTestCase("('{\"a\": 1}'), ('{\"b\": true}'), ('{\"c\": 1.23}')", + "UNICODE_RTRIM", "OBJECT"), + SchemaOfVariantAggTestCase("('{\"A\": \"x\"}'), ('{\"B\": 9.99}'), ('{\"C\": 0}')", + "UNICODE_CI", "OBJECT"), SchemaOfVariantAggTestCase("('{\"A\": \"x\"}'), ('{\"B\": 9.99}'), ('{\"C\": 0}')", - "UNICODE_CI", "OBJECT") + "UNICODE_CI_RTRIM", "OBJECT" + ) ) // Supported collations @@ -1716,7 +1960,16 @@ class CollationSQLExpressionsSuite test("Support InputFileName expression with collation") { // Supported collations - Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI", "MT_CI_AI").foreach(collationName => { + Seq( + "UTF8_BINARY", + "UTF8_BINARY_RTRIM", + "UTF8_LCASE", + "UTF8_LCASE_RTRIM", + "UNICODE", + "UNICODE_RTRIM", + "UNICODE_CI", + "UNICODE_CI_RTRIM", + "MT_CI_AI").foreach(collationName => { val query = s""" |select input_file_name() @@ -1735,9 +1988,13 @@ class CollationSQLExpressionsSuite case class DateFormatTestCase[R](date: String, format: String, collation: String, result: R) val testCases = Seq( DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UTF8_BINARY", "2021-01-01"), + DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UTF8_BINARY_RTRIM", "2021-01-01"), DateFormatTestCase("2021-01-01", "yyyy-dd", "UTF8_LCASE", "2021-01"), + DateFormatTestCase("2021-01-01", "yyyy-dd", "UTF8_LCASE_RTRIM", "2021-01"), DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UNICODE", "2021-01-01"), - DateFormatTestCase("2021-01-01", "yyyy", "UNICODE_CI", "2021") + DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UNICODE_RTRIM", "2021-01-01"), + DateFormatTestCase("2021-01-01", "yyyy", "UNICODE_CI", "2021"), + DateFormatTestCase("2021-01-01", "yyyy", "UNICODE_CI_RTRIM", "2021") ) for { @@ -1764,7 +2021,16 @@ class CollationSQLExpressionsSuite } test("Support mode for string expression with collation - Basic Test") { - Seq("utf8_binary", "UTF8_LCASE", "unicode_ci", "unicode", "NL_AI").foreach { collationId => + Seq( + "utf8_binary", + "utf8_binary_rtrim", + "UTF8_LCASE", + "UTF8_LCASE_RTRIM", + "unicode_ci", + "unicode_ci_rtrim", + "unicode", + "unicode_rtrim", + "NL_AI").foreach { collationId => val query = s"SELECT mode(collate('abc', '${collationId}'))" checkAnswer(sql(query), Row("abc")) assert(sql(query).schema.fields.head.dataType.sameType(StringType(collationId))) @@ -1775,9 +2041,13 @@ class CollationSQLExpressionsSuite case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) val testCases = Seq( ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("UTF8_LCASE_RTRIM", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), ModeTestCase("SR", Map("c" -> 3L, "č" -> 2L, "Č" -> 2L), "c") ) testCases.foreach(t => { @@ -1812,9 +2082,14 @@ class CollationSQLExpressionsSuite val testCasesUTF8String = Seq( UTF8StringModeTestCase("utf8_binary", bufferValuesUTF8String, "a"), + UTF8StringModeTestCase("utf8_binary_rtrim", bufferValuesUTF8String, "a"), UTF8StringModeTestCase("UTF8_LCASE", bufferValuesUTF8String, "b"), + UTF8StringModeTestCase("UTF8_LCASE_RTRIM", bufferValuesUTF8String, "b"), UTF8StringModeTestCase("unicode_ci", bufferValuesUTF8String, "b"), - UTF8StringModeTestCase("unicode", bufferValuesUTF8String, "a")) + UTF8StringModeTestCase("unicode_ci_rtrim", bufferValuesUTF8String, "b"), + UTF8StringModeTestCase("unicode", bufferValuesUTF8String, "a"), + UTF8StringModeTestCase("unicode_rtrim", bufferValuesUTF8String, "a") + ) testCasesUTF8String.foreach ( t => { val buffer = new OpenHashMap[AnyRef, Long](5) @@ -1842,9 +2117,13 @@ class CollationSQLExpressionsSuite } val testCasesUTF8String = Seq( UTF8StringModeTestCase("utf8_binary", bufferValuesComplex, "[a,a,a]"), + UTF8StringModeTestCase("utf8_binary_rtrim", bufferValuesComplex, "[a,a,a]"), UTF8StringModeTestCase("UTF8_LCASE", bufferValuesComplex, "[b,b,b]"), + UTF8StringModeTestCase("UTF8_LCASE_rtrim", bufferValuesComplex, "[b,b,b]"), UTF8StringModeTestCase("unicode_ci", bufferValuesComplex, "[b,b,b]"), - UTF8StringModeTestCase("unicode", bufferValuesComplex, "[a,a,a]")) + UTF8StringModeTestCase("unicode_ci_rtrim", bufferValuesComplex, "[b,b,b]"), + UTF8StringModeTestCase("unicode", bufferValuesComplex, "[a,a,a]"), + UTF8StringModeTestCase("unicode_rtrim", bufferValuesComplex, "[a,a,a]")) testCasesUTF8String.foreach { t => val buffer = new OpenHashMap[AnyRef, Long](5) @@ -1862,9 +2141,13 @@ class CollationSQLExpressionsSuite case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) val testCases = Seq( ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("UTF8_LCASE_RTRIM", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") + ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") ) testCases.foreach(t => { val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => @@ -1887,9 +2170,13 @@ class CollationSQLExpressionsSuite case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) val testCases = Seq( ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") + ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") ) testCases.foreach { t => val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => @@ -1912,9 +2199,13 @@ class CollationSQLExpressionsSuite case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) val testCases = Seq( ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") + ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") ) testCases.foreach { t => val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => @@ -1938,9 +2229,13 @@ class CollationSQLExpressionsSuite case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) val testCases = Seq( ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") + ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") ) testCases.foreach { t => val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => @@ -1964,9 +2259,13 @@ class CollationSQLExpressionsSuite case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) val testCases = Seq( ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") + ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), + ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), + ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") ) testCases.foreach { t => val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => @@ -1991,8 +2290,11 @@ class CollationSQLExpressionsSuite case class ModeTestCase(collationId: String, bufferValues: Map[String, Long], result: String) Seq( ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), + ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), + ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), ModeTestCase("utf8_lcase", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{b -> 1}"), + ModeTestCase("utf8_lcase_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{b -> 1}"), ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{b -> 1}") ).foreach { t1 => def getValuesToAdd(t: ModeTestCase): String = { @@ -2023,7 +2325,12 @@ class CollationSQLExpressionsSuite for { collateKey <- Seq(true, false) collateVal <- Seq(true, false) - defaultCollation <- Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE") + defaultCollation <- Seq( + "UTF8_BINARY", + "UTF8_BINARY_RTRIM", + "UTF8_LCASE", + "UTF8_LCASE_RTRIM", + "UNICODE") } { val mapKey = if (collateKey) "'a' collate utf8_lcase" else "'a'" val mapVal = if (collateVal) "'b' collate utf8_lcase" else "'b'" @@ -2420,7 +2727,8 @@ class CollationSQLExpressionsSuite "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", true), ReflectExpressions("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", "A5Cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", false), - + ReflectExpressions("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", + "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary_rtrim", true), ReflectExpressions("A5cf6C42-0C85-418f-af6c-3E4E5b1328f2", "utf8_binary", "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_lcase", true), ReflectExpressions("A5cf6C42-0C85-418f-af6c-3E4E5b1328f2", "utf8_binary", @@ -3166,14 +3474,22 @@ class CollationSQLExpressionsSuite ) val testCases = Seq( - HyperLogLogPlusPlusTestCase("utf8_binary", Seq("a", "a", "A", "z", "zz", "ZZ", "w", "AA", - "aA", "Aa", "aa"), Seq(Row(10))), - HyperLogLogPlusPlusTestCase("utf8_lcase", Seq("a", "a", "A", "z", "zz", "ZZ", "w", "AA", - "aA", "Aa", "aa"), Seq(Row(5))), + HyperLogLogPlusPlusTestCase("utf8_binary", Seq("a", "a", "A", "z", "zz", "ZZ", "w", + "AA", "aA", "Aa", "aa"), Seq(Row(10))), + HyperLogLogPlusPlusTestCase("utf8_binary_rtrim", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", + "w", "AA", "aA", "Aa", "aa"), Seq(Row(10))), + HyperLogLogPlusPlusTestCase("utf8_lcase", Seq("a", "a", "A", "z", "zz", "ZZ", "w", + "AA", "aA", "Aa", "aa"), Seq(Row(5))), + HyperLogLogPlusPlusTestCase("utf8_lcase_rtrim", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", "w", + "AA", "aA", "Aa", "aa"), Seq(Row(5))), HyperLogLogPlusPlusTestCase("UNICODE", Seq("a", "a", "A", "z", "zz", "ZZ", "w", "AA", "aA", "Aa", "aa"), Seq(Row(9))), + HyperLogLogPlusPlusTestCase("UNICODE_RTRIM", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", "w", + "AA", "aA", "Aa", "aa"), Seq(Row(9))), HyperLogLogPlusPlusTestCase("UNICODE_CI", Seq("a", "a", "A", "z", "zz", "ZZ", "w", "AA", - "aA", "Aa", "aa"), Seq(Row(5))) + "aA", "Aa", "aa"), Seq(Row(5))), + HyperLogLogPlusPlusTestCase("UNICODE_CI_RTRIM", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", "w", + "AA", "aA", "Aa", "aa"), Seq(Row(5))) ) testCases.foreach( t => { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index 11f2c4b997a4b..a8fe36c9ba394 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -696,6 +696,11 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { s"IN (COLLATE('aa', 'UTF8_LCASE'))"), Seq(Row("a"), Row("A"))) checkAnswer(sql(s"SELECT c1 FROM $tableName where (c1 || 'a') " + s"IN (COLLATE('aa', 'UTF8_BINARY'))"), Seq(Row("a"))) + checkAnswer(sql(s"SELECT c1 FROM $tableName where c1 || 'a' " + + s"IN (COLLATE('aa', 'UTF8_LCASE_RTRIM'))"), Seq(Row("a"), Row("A"))) + checkAnswer(sql(s"SELECT c1 FROM $tableName where (c1 || 'a') " + + s"IN (COLLATE('aa', 'UTF8_BINARY_RTRIM'))"), Seq(Row("a"))) + // columns have different collation checkError( @@ -806,6 +811,16 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) ) + checkError( + exception = intercept[AnalysisException] { + sql(s"SELECT array('A', 'a' COLLATE UNICODE) == array('b' COLLATE UNICODE_CI_RTRIM)") + }, + condition = "COLLATION_MISMATCH.EXPLICIT", + parameters = Map( + "explicitTypes" -> """"STRING COLLATE UNICODE", "STRING COLLATE UNICODE_CI_RTRIM"""" + ) + ) + checkError( exception = intercept[AnalysisException] { sql("SELECT array_join(array('a', 'b' collate UNICODE), 'c' collate UNICODE_CI)") From 812a9ad3ea92b972fa8d966069ffd394b5c46ac1 Mon Sep 17 00:00:00 2001 From: Kaz Date: Wed, 4 Dec 2024 17:41:07 +0100 Subject: [PATCH 096/438] [MINOR] Fix some typos ### What changes were proposed in this pull request? This PR fixes typos in docstrings and comments (and a few functional bits of code). The typos were found using the typos software: https://github.com/crate-ci/typos I've made a typo-fix-PR before, but haven't seen the result on the website yet. Is there anything else I need to do for that? ### Why are the changes needed? Nice to fix :) ### Does this PR introduce _any_ user-facing change? yes, documentation was updated. ### How was this patch tested? No tests added. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48557 from KazMiddelhoek/master. Lead-authored-by: Kaz Co-authored-by: Kaz <62500382+KazMiddelhoek@users.noreply.github.com> Signed-off-by: Max Gekk --- .github/workflows/update_build_status.yml | 2 +- R/pkg/R/functions.R | 4 ++-- R/pkg/R/serialize.R | 2 +- .../spark/network/shuffle/RemoteBlockPushResolver.java | 6 +++--- connector/connect/docs/client-connection-string.md | 4 ++-- docs/_plugins/include_example.rb | 4 ++-- docs/core-migration-guide.md | 2 +- docs/running-on-yarn.md | 2 +- docs/security.md | 2 +- docs/spark-standalone.md | 2 +- docs/sql-ref-syntax-ddl-declare-variable.md | 2 +- python/docs/source/reference/pyspark.ss/index.rst | 2 +- python/pyspark/ml/connect/io_utils.py | 2 +- python/pyspark/ml/connect/tuning.py | 2 +- python/pyspark/ml/deepspeed/deepspeed_distributor.py | 2 +- python/pyspark/ml/dl_util.py | 2 +- .../pyspark/ml/tests/connect/test_connect_function.py | 2 +- python/pyspark/ml/tests/test_dl_util.py | 2 +- python/pyspark/ml/tests/test_functions.py | 4 ++-- python/pyspark/ml/tests/test_param.py | 4 ++-- python/pyspark/ml/torch/distributor.py | 4 ++-- python/pyspark/pandas/accessors.py | 2 +- python/pyspark/pandas/base.py | 2 +- python/pyspark/pandas/frame.py | 4 ++-- python/pyspark/sql/connect/plan.py | 2 +- python/pyspark/sql/dataframe.py | 4 ++-- python/pyspark/sql/functions/builtin.py | 10 +++++----- python/pyspark/sql/readwriter.py | 2 +- .../pyspark/sql/tests/connect/test_connect_function.py | 2 +- .../tests/streaming/test_streaming_foreach_batch.py | 2 +- python/pyspark/sql/tests/test_udtf.py | 2 +- python/pyspark/sql/udtf.py | 2 +- python/pyspark/streaming/tests/test_dstream.py | 2 +- python/pyspark/worker_util.py | 4 ++-- .../connect/execution/ExecuteGrpcResponseSender.scala | 2 +- 35 files changed, 50 insertions(+), 50 deletions(-) diff --git a/.github/workflows/update_build_status.yml b/.github/workflows/update_build_status.yml index d0a50b2b4aa74..542fa567dea69 100644 --- a/.github/workflows/update_build_status.yml +++ b/.github/workflows/update_build_status.yml @@ -72,7 +72,7 @@ jobs: } catch (error) { console.error(error) // Run not found. This can happen when the PR author removes GitHub Actions runs or - // disalbes GitHub Actions. + // disables GitHub Actions. continue } diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 9c825a99be180..e320981783ecc 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2922,7 +2922,7 @@ setClassUnion("characterOrstructTypeOrColumn", c("character", "structType", "Col #' @details #' \code{from_json}: Parses a column containing a JSON string into a Column of \code{structType} #' with the specified \code{schema} or array of \code{structType} if \code{as.json.array} is set -#' to \code{TRUE}. If the string is unparseable, the Column will contain the value NA. +#' to \code{TRUE}. If the string is unparsable, the Column will contain the value NA. #' #' @rdname column_collection_functions #' @param as.json.array indicating if input string is JSON array of objects or a single object. @@ -3004,7 +3004,7 @@ setMethod("schema_of_json", signature(x = "characterOrColumn"), #' @details #' \code{from_csv}: Parses a column containing a CSV string into a Column of \code{structType} #' with the specified \code{schema}. -#' If the string is unparseable, the Column will contain the value NA. +#' If the string is unparsable, the Column will contain the value NA. #' #' @rdname column_collection_functions #' @aliases from_csv from_csv,Column,characterOrstructTypeOrColumn-method diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 61e174de9ac56..4ccec991bb07b 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -60,7 +60,7 @@ writeObject <- function(con, object, writeType = TRUE) { if (type %in% c("integer", "character", "logical", "double", "numeric")) { if (is.na(object[[1]])) { # Uses the first element for now to keep the behavior same as R before - # 4.2.0. This is wrong because we should differenciate c(NA) from a + # 4.2.0. This is wrong because we should differentiate c(NA) from a # single NA as the former means array(null) and the latter means null # in Spark SQL. However, it requires non-trivial comparison to distinguish # both in R. We should ideally fix this. diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 02a38eac5b409..6e9bd548f5327 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -251,17 +251,17 @@ AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( // Higher shuffleMergeId seen for the shuffle ID meaning new stage attempt is being // run for the shuffle ID. Close and clean up old shuffleMergeId files, // happens in the indeterminate stage retries - AppAttemptShuffleMergeId currrentAppAttemptShuffleMergeId = + AppAttemptShuffleMergeId currentAppAttemptShuffleMergeId = new AppAttemptShuffleMergeId(appShuffleInfo.appId, appShuffleInfo.attemptId, shuffleId, latestShuffleMergeId); logger.info("{}: creating a new shuffle merge metadata since received " + "shuffleMergeId {} is higher than latest shuffleMergeId {}", MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, - currrentAppAttemptShuffleMergeId), + currentAppAttemptShuffleMergeId), MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, shuffleMergeId), MDC.of(LogKeys.LATEST_SHUFFLE_MERGE_ID$.MODULE$, latestShuffleMergeId)); submitCleanupTask(() -> - closeAndDeleteOutdatedPartitions(currrentAppAttemptShuffleMergeId, + closeAndDeleteOutdatedPartitions(currentAppAttemptShuffleMergeId, mergePartitionsInfo.shuffleMergePartitions)); return new AppShuffleMergePartitionsInfo(shuffleMergeId, false); } else { diff --git a/connector/connect/docs/client-connection-string.md b/connector/connect/docs/client-connection-string.md index 37b2956a5c44a..df371c5beaaac 100644 --- a/connector/connect/docs/client-connection-string.md +++ b/connector/connect/docs/client-connection-string.md @@ -2,7 +2,7 @@ From the client perspective, Spark Connect mostly behaves as any other GRPC client and can be configured as such. However, to make it easy to use from -different programming languages and to have a homogenous connection surface +different programming languages and to have a homogeneous connection surface this document proposes what the user surface is for connecting to a Spark Connect endpoint. @@ -136,7 +136,7 @@ server_url = "sc://myhost.com:443/;use_ssl=true;token=ABCDEFG" As mentioned above, Spark Connect uses a regular GRPC client and the server path cannot be configured to remain compatible with the GRPC standard and HTTP. For -example the following examles are invalid. +example the following examples are invalid. ```python server_url = "sc://myhost.com:443/mypathprefix/;token=AAAAAAA" diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb index 7d0e78738095e..6fd14ce31a68c 100644 --- a/docs/_plugins/include_example.rb +++ b/docs/_plugins/include_example.rb @@ -114,8 +114,8 @@ def select_lines(code) range = Range.new(start + 1, endline - 1) trimmed = trim_codeblock(lines[range]) # Filter out possible example tags of overlapped labels. - taggs_filtered = trimmed.select { |l| !l.include? '$example ' } - result += taggs_filtered.join + tags_filtered = trimmed.select { |l| !l.include? '$example ' } + result += tags_filtered.join result += "\n" end result diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 88bad6c5d1b9f..958e442545dcd 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -62,7 +62,7 @@ license: | ## Upgrading from Core 3.3 to 3.4 -- Since Spark 3.4, Spark driver will own `PersistentVolumnClaim`s and try to reuse if they are not assigned to live executors. To restore the behavior before Spark 3.4, you can set `spark.kubernetes.driver.ownPersistentVolumeClaim` to `false` and `spark.kubernetes.driver.reusePersistentVolumeClaim` to `false`. +- Since Spark 3.4, Spark driver will own `PersistentVolumeClaim`s and try to reuse if they are not assigned to live executors. To restore the behavior before Spark 3.4, you can set `spark.kubernetes.driver.ownPersistentVolumeClaim` to `false` and `spark.kubernetes.driver.reusePersistentVolumeClaim` to `false`. - Since Spark 3.4, Spark driver will track shuffle data when dynamic allocation is enabled without shuffle service. To restore the behavior before Spark 3.4, you can set `spark.dynamicAllocation.shuffleTracking.enabled` to `false`. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index aefa979946a6c..b6f847ff533f5 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -673,7 +673,7 @@ To use a custom metrics.properties for the application master and executors, upd
diff --git a/docs/security.md b/docs/security.md index c7d3fd5f8c36f..81173d5f01ce7 100644 --- a/docs/security.md +++ b/docs/security.md @@ -72,7 +72,7 @@ secrets to be secure. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index d828436e77340..4f8e0dc1a3917 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -372,7 +372,7 @@ SPARK_MASTER_OPTS supports the following system properties: diff --git a/docs/sql-ref-syntax-ddl-declare-variable.md b/docs/sql-ref-syntax-ddl-declare-variable.md index ba9857bf1917a..41ecba1364361 100644 --- a/docs/sql-ref-syntax-ddl-declare-variable.md +++ b/docs/sql-ref-syntax-ddl-declare-variable.md @@ -83,7 +83,7 @@ DECLARE OR REPLACE five = 55; -- Explicitly declare the default value of a variable using the keyword `DEFAULT` DECLARE VARIABLE size DEFAULT 6; --- STRING variable initialialized to `NULL` +-- STRING variable initialized to `NULL` DECLARE some_var STRING; ``` diff --git a/python/docs/source/reference/pyspark.ss/index.rst b/python/docs/source/reference/pyspark.ss/index.rst index 2cb0b1216eff9..440228134fac9 100644 --- a/python/docs/source/reference/pyspark.ss/index.rst +++ b/python/docs/source/reference/pyspark.ss/index.rst @@ -20,7 +20,7 @@ Structured Streaming ==================== -This page gives an overview of all public Structed Streaming API. +This page gives an overview of all public Structured Streaming API. .. toctree:: :maxdepth: 2 diff --git a/python/pyspark/ml/connect/io_utils.py b/python/pyspark/ml/connect/io_utils.py index c401e3e76676a..8d93426915d42 100644 --- a/python/pyspark/ml/connect/io_utils.py +++ b/python/pyspark/ml/connect/io_utils.py @@ -74,7 +74,7 @@ class ParamsReadWrite(Params): def _get_extra_metadata(self) -> Any: """ - Returns exta metadata of the instance + Returns extra metadata of the instance """ return None diff --git a/python/pyspark/ml/connect/tuning.py b/python/pyspark/ml/connect/tuning.py index cdb606048a59a..190fc683acf7d 100644 --- a/python/pyspark/ml/connect/tuning.py +++ b/python/pyspark/ml/connect/tuning.py @@ -170,7 +170,7 @@ def _parallelFitTasks( if active_session is None: raise RuntimeError( - "An active SparkSession is required for running cross valiator fit tasks." + "An active SparkSession is required for running cross validator fit tasks." ) def get_single_task(index: int, param_map: Any) -> Callable[[], Tuple[int, float]]: diff --git a/python/pyspark/ml/deepspeed/deepspeed_distributor.py b/python/pyspark/ml/deepspeed/deepspeed_distributor.py index 4ac5ff2fb4207..3fd1d3bb32463 100644 --- a/python/pyspark/ml/deepspeed/deepspeed_distributor.py +++ b/python/pyspark/ml/deepspeed/deepspeed_distributor.py @@ -49,7 +49,7 @@ def __init__( Parameters ---------- numGpus: int - The number of GPUs to use per node (analagous to num_gpus in deepspeed command). + The number of GPUs to use per node (analogous to num_gpus in deepspeed command). nnodes: int The number of nodes that should be used for the run. localMode: bool diff --git a/python/pyspark/ml/dl_util.py b/python/pyspark/ml/dl_util.py index 8ead529d7b729..3b87049ef2777 100644 --- a/python/pyspark/ml/dl_util.py +++ b/python/pyspark/ml/dl_util.py @@ -27,7 +27,7 @@ class FunctionPickler: This class provides a way to pickle a function and its arguments. It also provides a way to create a script that can run a function with arguments if they have them pickled to a file. - It also provides a way of extracting the conents of a pickle file. + It also provides a way of extracting the contents of a pickle file. """ @staticmethod diff --git a/python/pyspark/ml/tests/connect/test_connect_function.py b/python/pyspark/ml/tests/connect/test_connect_function.py index 393d38fdc426a..7d3a115ab0619 100644 --- a/python/pyspark/ml/tests/connect/test_connect_function.py +++ b/python/pyspark/ml/tests/connect/test_connect_function.py @@ -43,7 +43,7 @@ def setUpClass(cls): # Disable the shared namespace so pyspark.sql.functions, etc point the regular # PySpark libraries. os.environ["PYSPARK_NO_NAMESPACE_SHARE"] = "1" - cls.connect = cls.spark # Switch Spark Connect session and regular PySpark sesion. + cls.connect = cls.spark # Switch Spark Connect session and regular PySpark session. cls.spark = PySparkSession._instantiatedSession assert cls.spark is not None diff --git a/python/pyspark/ml/tests/test_dl_util.py b/python/pyspark/ml/tests/test_dl_util.py index e5e2c6bc191d8..c130cf1ff6b9d 100644 --- a/python/pyspark/ml/tests/test_dl_util.py +++ b/python/pyspark/ml/tests/test_dl_util.py @@ -137,7 +137,7 @@ def _are_two_files_identical(self, fpath1: str, fpath2: str) -> bool: "", ), ( - "Check if it creates the correct file with only suffix + boddy", + "Check if it creates the correct file with only suffix + body", "", "print('goodbye')", ), diff --git a/python/pyspark/ml/tests/test_functions.py b/python/pyspark/ml/tests/test_functions.py index e67e46ded67bd..7719b2b27e0ab 100644 --- a/python/pyspark/ml/tests/test_functions.py +++ b/python/pyspark/ml/tests/test_functions.py @@ -265,14 +265,14 @@ def predict(a, b, c): with self.assertRaisesRegex(Exception, "Model expected 3 inputs, but received 4 columns"): preds = self.df.withColumn("preds", sum_cols(*columns)).toPandas() - # muliple scalar columns with one tensor_input_shape => single numpy array + # multiple scalar columns with one tensor_input_shape => single numpy array sum_cols = predict_batch_udf( array_sum_fn, return_type=DoubleType(), batch_size=5, input_tensor_shapes=[[4]] ) preds = self.df.withColumn("preds", sum_cols(struct(*columns))).toPandas() self.assertTrue(np.array_equal(np.sum(self.data, axis=1), preds["preds"].to_numpy())) - # muliple scalar columns with wrong tensor_input_shape => ERROR + # multiple scalar columns with wrong tensor_input_shape => ERROR sum_cols = predict_batch_udf( array_sum_fn, return_type=DoubleType(), batch_size=5, input_tensor_shapes=[[3]] ) diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index 8df50a5963e6b..0aa9827124954 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -368,12 +368,12 @@ def test_default_params_transferred(self): self.assertFalse(binarizer.isSet(binarizer.outputCol)) self.assertEqual(result[0][0], 1.0) - def test_lr_evaluate_invaild_type(self): + def test_lr_evaluate_invalid_type(self): lr = LinearRegressionModel() invalid_type = "" self.assertRaises(TypeError, lr.evaluate, invalid_type) - def test_glr_evaluate_invaild_type(self): + def test_glr_evaluate_invalid_type(self): glr = GeneralizedLinearRegressionModel() invalid_type = "" self.assertRaises(TypeError, glr.evaluate, invalid_type) diff --git a/python/pyspark/ml/torch/distributor.py b/python/pyspark/ml/torch/distributor.py index 62a71c5a96af4..ef86f38b716b7 100644 --- a/python/pyspark/ml/torch/distributor.py +++ b/python/pyspark/ml/torch/distributor.py @@ -232,10 +232,10 @@ def _get_num_tasks(self) -> int: def _validate_input_params(self) -> None: if self.num_processes <= 0: - raise ValueError("num_proccesses has to be a positive integer") + raise ValueError("num_processes has to be a positive integer") def _check_encryption(self) -> None: - """Checks to see if the user requires encrpytion of data. + """Checks to see if the user requires encryption of data. If required, throw an exception since we don't support that. Raises diff --git a/python/pyspark/pandas/accessors.py b/python/pyspark/pandas/accessors.py index 4c36f7976af83..77757e4b60873 100644 --- a/python/pyspark/pandas/accessors.py +++ b/python/pyspark/pandas/accessors.py @@ -936,7 +936,7 @@ def _transform_batch( def pandas_concat(*series: pd.Series) -> pd.DataFrame: # The input can only be a DataFrame for struct from Spark 3.0. - # This works around makeing the input as a frame. See SPARK-27240 + # This works around making the input as a frame. See SPARK-27240 pdf = pd.concat(series, axis=1) pdf.columns = columns return pdf diff --git a/python/pyspark/pandas/base.py b/python/pyspark/pandas/base.py index bc54d8b9b17cb..01e23214d662d 100644 --- a/python/pyspark/pandas/base.py +++ b/python/pyspark/pandas/base.py @@ -1123,7 +1123,7 @@ def shift( Shift Series/Index by desired number of periods. .. note:: the current implementation of shift uses Spark's Window without - specifying partition specification. This leads to moveing all data into + specifying partition specification. This leads to moving all data into a single partition in a single machine and could cause serious performance degradation. Avoid this method with very large datasets. diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index 49aa49f65e35b..f315d59a4fe94 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -7686,7 +7686,7 @@ def _sort( if na_position not in ("first", "last"): raise ValueError("invalid na_position: '{}'".format(na_position)) - # Mapper: Get a spark colum + # Mapper: Get a spark column # n function for (ascending, na_position) combination mapper = { (True, "first"): PySparkColumn.asc_nulls_first, @@ -9808,7 +9808,7 @@ def describe(self, percentiles: Optional[List[float]] = None) -> "DataFrame": if is_all_string_type: # Handling string type columns - # We will retrive the `count`, `unique`, `top` and `freq`. + # We will retrieve the `count`, `unique`, `top` and `freq`. internal = self._internal.resolved_copy exprs_string = [ internal.spark_column_for(psser._column_label) for psser in psser_string diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index b387ca1d4e508..34f11768bcbc0 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -205,7 +205,7 @@ def _parameters_to_print(self, parameters: Mapping[str, Any]) -> Mapping[str, An try: params[name] = getattr(self, "_" + name) except AttributeError: - pass # Simpy ignore + pass # Simply ignore return params def print(self, indent: int = 0) -> str: diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 085a1a629634a..0ea0eef50c0f3 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -2549,7 +2549,7 @@ def join( pyspark.errors.exceptions.captured.AnalysisException: Column name#0 are ambiguous... A better approach is to assign aliases to the dataframes, and then reference - the ouptut columns from the join operation using these aliases: + the output columns from the join operation using these aliases: >>> df.alias("a").join( ... df.alias("b"), sf.col("a.name") == sf.col("b.name"), "outer" @@ -3907,7 +3907,7 @@ def groupingSets( groupingSets : sequence of sequence of columns or str Individual set of columns to group on. cols : :class:`Column` or str - Addional grouping columns specified by users. + Additional grouping columns specified by users. Those columns are shown as the output columns after aggregation. Returns diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 391bc3db7a86f..4b4c164055eaf 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15832,7 +15832,7 @@ def split_part(src: "ColumnOrName", delimiter: "ColumnOrName", partNum: "ColumnO Parameters ---------- src : :class:`~pyspark.sql.Column` or column name - A column of string to be splited. + A column of string to be split. delimiter : :class:`~pyspark.sql.Column` or column name A column of string, the delimiter used for split. partNum : :class:`~pyspark.sql.Column` or column name @@ -19618,7 +19618,7 @@ def from_json( """ Parses a column containing a JSON string into a :class:`MapType` with :class:`StringType` as keys type, :class:`StructType` or :class:`ArrayType` with - the specified schema. Returns `null`, in the case of an unparseable string. + the specified schema. Returns `null`, in the case of an unparsable string. .. versionadded:: 2.1.0 @@ -20230,7 +20230,7 @@ def from_xml( ) -> Column: """ Parses a column containing a XML string to a row with - the specified schema. Returns `null`, in the case of an unparseable string. + the specified schema. Returns `null`, in the case of an unparsable string. .. versionadded:: 4.0.0 @@ -22624,7 +22624,7 @@ def transform_keys(col: "ColumnOrName", f: Callable[[Column, Column], Column]) - Returns ------- :class:`~pyspark.sql.Column` - a new map of enties where new keys were calculated by applying given function to + a new map of entries where new keys were calculated by applying given function to each key value argument. Examples @@ -22664,7 +22664,7 @@ def transform_values(col: "ColumnOrName", f: Callable[[Column, Column], Column]) Returns ------- :class:`~pyspark.sql.Column` - a new map of enties where new values were calculated by applying given function to + a new map of entries where new values were calculated by applying given function to each key value argument. Examples diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 4744bdf861d37..2113f0707f910 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -215,7 +215,7 @@ def options(self, **options: "OptionalPrimitiveType") -> "DataFrameReader": Parameters ---------- **options : dict - The dictionary of string keys and prmitive-type values. + The dictionary of string keys and primitive-type values. Examples -------- diff --git a/python/pyspark/sql/tests/connect/test_connect_function.py b/python/pyspark/sql/tests/connect/test_connect_function.py index e29873173cc3a..b7a02efcd5e2b 100644 --- a/python/pyspark/sql/tests/connect/test_connect_function.py +++ b/python/pyspark/sql/tests/connect/test_connect_function.py @@ -54,7 +54,7 @@ def setUpClass(cls): # Disable the shared namespace so pyspark.sql.functions, etc point the regular # PySpark libraries. os.environ["PYSPARK_NO_NAMESPACE_SHARE"] = "1" - cls.connect = cls.spark # Switch Spark Connect session and regular PySpark sesion. + cls.connect = cls.spark # Switch Spark Connect session and regular PySpark session. cls.spark = PySparkSession._instantiatedSession assert cls.spark is not None diff --git a/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py b/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py index de8f30baebca5..9db66aa252ee6 100644 --- a/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py +++ b/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py @@ -146,7 +146,7 @@ def func(df: DataFrame, batch_id: int): def my_test_function_2(): return 2 - def test_streaming_foreach_batch_fuction_calling(self): + def test_streaming_foreach_batch_function_calling(self): def my_test_function_3(): return 3 diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 206cfd7dc4885..8447edfbbb15d 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -1345,7 +1345,7 @@ def eval(self, a, b): assertSchemaEqual(df.schema, expected_schema) assertDataFrameEqual(df, expected_results) - def test_udtf_with_analyze_arbitary_number_arguments(self): + def test_udtf_with_analyze_arbitrary_number_arguments(self): class TestUDTF: @staticmethod def analyze(*args: AnalyzeArgument) -> AnalyzeResult: diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index 5ce3e2dfd2a9e..cf4f976fd93b3 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -148,7 +148,7 @@ class AnalyzeResult: The schema that the Python UDTF will return. withSinglePartition: bool If true, the UDTF is specifying for Catalyst to repartition all rows of the input TABLE - argument to one collection for consumption by exactly one instance of the correpsonding + argument to one collection for consumption by exactly one instance of the corresponding UDTF class. partitionBy: sequence of :class:`PartitioningColumn` If non-empty, this is a sequence of expressions that the UDTF is specifying for Catalyst to diff --git a/python/pyspark/streaming/tests/test_dstream.py b/python/pyspark/streaming/tests/test_dstream.py index 046247763c0b3..4c9633db311a6 100644 --- a/python/pyspark/streaming/tests/test_dstream.py +++ b/python/pyspark/streaming/tests/test_dstream.py @@ -403,7 +403,7 @@ def failed_func(rdd1, rdd2): self.fail("a failed func should throw an error") - def test_failed_func_with_reseting_failure(self): + def test_failed_func_with_resetting_failure(self): input = [self.sc.parallelize([d], 1) for d in range(4)] input_stream = self.ssc.queueStream(input) diff --git a/python/pyspark/worker_util.py b/python/pyspark/worker_util.py index 81c05ce94eb65..5c758d3f83fe6 100644 --- a/python/pyspark/worker_util.py +++ b/python/pyspark/worker_util.py @@ -107,8 +107,8 @@ def setup_memory_limits(memory_limit_mb: int) -> None: except (resource.error, OSError, ValueError) as e: # not all systems support resource limits, so warn instead of failing - curent = currentframe() - lineno = getframeinfo(curent).lineno + 1 if curent is not None else 0 + current = currentframe() + lineno = getframeinfo(current).lineno + 1 if current is not None else 0 if "__file__" in globals(): print( warnings.formatwarning( diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala index 051093fcad277..44b634af95ca9 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala @@ -291,7 +291,7 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( assert(finished == false) } else { // If it wasn't sent, time deadline must have been reached before stream became available, - // or it was intterupted. Will exit in the next loop iterattion. + // or it was interrupted. Will exit in the next loop iterattion. assert(deadlineLimitReached || interrupted) } } else if (streamFinished) { From f1eecd3578612da510748da8682c199aa6109eac Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 4 Dec 2024 18:53:35 +0100 Subject: [PATCH 097/438] [SPARK-50485][SQL] Unwrap SparkThrowable in (Unchecked)ExecutionException thrown by tableRelationCache ### What changes were proposed in this pull request? This PR unwraps SparkThrowable in (Unchecked)ExecutionException thrown by tableRelationCache ### Why are the changes needed? The guava cache library wraps exceptions thrown by `c: Callable[LogicalPlan]` as `(Unchecked)ExecutionException`s. This makes our code paths of special handling for SparkThrowable inoperative. For example, this kind of error in spark-sql cli is very lengthy. ### Does this PR introduce _any_ user-facing change? Yes, when you visit an invalid plan in the table cache layer, you will get the original spark error instead of getting them under `(Unchecked)ExecutionException`s ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49050 from yaooqinn/SPARK-50485. Authored-by: Kent Yao Signed-off-by: Max Gekk --- .../sql/catalyst/catalog/SessionCatalog.scala | 14 ++++++++++---- .../sql/errors/QueryExecutionErrorsSuite.scala | 16 ++++++++++++++++ 2 files changed, 26 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index a0f7af10fefaf..d87678ac34114 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -19,18 +19,18 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI import java.util.Locale -import java.util.concurrent.Callable -import java.util.concurrent.TimeUnit +import java.util.concurrent.{Callable, ExecutionException, TimeUnit} import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import scala.util.{Failure, Success, Try} import com.google.common.cache.{Cache, CacheBuilder} +import com.google.common.util.concurrent.UncheckedExecutionException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, SparkThrowable} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ @@ -210,7 +210,13 @@ class SessionCatalog( /** This method provides a way to get a cached plan. */ def getCachedPlan(t: QualifiedTableName, c: Callable[LogicalPlan]): LogicalPlan = { - tableRelationCache.get(t, c) + try { + tableRelationCache.get(t, c) + } catch { + case e @ (_: ExecutionException | _: UncheckedExecutionException) + if e.getCause != null && e.getCause.isInstanceOf[SparkThrowable] => + throw e.getCause + } } /** This method provides a way to get a cached plan if the key exists. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 1adb1fdf05032..9abda729c02ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -1258,6 +1258,22 @@ class QueryExecutionErrorsSuite ) ) } + + test("SPARK-50485: Unwrap SparkThrowable in UEE thrown by tableRelationCache") { + withTable("t") { + sql("CREATE TABLE t (a INT)") + checkError( + exception = intercept[SparkUnsupportedOperationException] { + sql("ALTER TABLE t SET LOCATION 'https://mister/spark'") + }, + condition = "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM", + parameters = Map( + "path" -> "https://mister/spark", + "fileSystemClass" -> "org.apache.hadoop.fs.http.HttpsFileSystem", + "method" -> "listStatus")) + sql("ALTER TABLE t SET LOCATION '/mister/spark'") + } + } } class FakeFileSystemSetPermission extends LocalFileSystem { From 424839745a94cd31461639d8bc98927bb3518735 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andrej=20Gobelji=C4=87?= Date: Wed, 4 Dec 2024 20:31:19 +0100 Subject: [PATCH 098/438] [SPARK-49695][SQL] Postgres fix xor push-down MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR fixes the pushdown of ^ operator (XOR operator) for Postgres. Those two databases use this as exponent, rather then bitwise xor. Fix is consisted of overriding the SQLExpressionBuilder to replace the '^' character with '#'. ### Why are the changes needed? Result is incorrect. ### Does this PR introduce _any_ user-facing change? Yes. The user will now have a proper translation of the ^ operator. ### How was this patch tested? ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48144 from andrej-db/SPARK-49695-PostgresXOR. Lead-authored-by: Andrej Gobeljić Co-authored-by: andrej-db Co-authored-by: andrej-gobeljic_data Signed-off-by: Max Gekk --- .../spark/sql/jdbc/v2/PostgresIntegrationSuite.scala | 10 ++++++++++ .../org/apache/spark/sql/jdbc/PostgresDialect.scala | 4 ++++ 2 files changed, 14 insertions(+) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index ef52aebd723a6..eaf2a07ed4594 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -22,6 +22,7 @@ import java.sql.Connection import org.apache.spark.{SparkConf, SparkSQLException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.jdbc.PostgresDatabaseOnDocker import org.apache.spark.sql.types._ @@ -243,6 +244,15 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT } } + test("SPARK-49695: Postgres fix xor push-down") { + val df = spark.sql(s"select dept, name from $catalogName.employee where dept ^ 6 = 0") + val rows = df.collect() + assert(!df.queryExecution.sparkPlan.exists(_.isInstanceOf[FilterExec])) + assert(rows.length == 1) + assert(rows(0).getInt(0) === 6) + assert(rows(0).getString(1) === "jen") + } + override def testDatetime(tbl: String): Unit = { val df1 = sql(s"SELECT name FROM $tbl WHERE " + "dayofyear(date1) > 100 AND dayofmonth(date1) > 10 ") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index bce9c67042782..c1b79f8017419 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -310,6 +310,10 @@ private case class PostgresDialect() case _ => super.visitExtract(field, source) } } + + override def visitBinaryArithmetic(name: String, l: String, r: String): String = { + l + " " + name.replace('^', '#') + " " + r + } } override def compileExpression(expr: Expression): Option[String] = { From fe904e6973b7a8fdadc5e253a6a74e8ccb359287 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 4 Dec 2024 16:11:42 -0400 Subject: [PATCH 099/438] [SPARK-49709][CONNECT][SQL] Support ConfigEntry in the RuntimeConfig interface ### What changes were proposed in this pull request? This PR adds support for ConfigEntry to the RuntimeConfig interface. This was removed in https://github.com/apache/spark/pull/47980. ### Why are the changes needed? This functionality is used a lot by Spark libraries. Removing them caused friction, and adding them does not pollute the RuntimeConfig interface. ### Does this PR introduce _any_ user-facing change? No. This is developer API. ### How was this patch tested? I have added tests cases for Connect and Classic. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49062 from hvanhovell/SPARK-49709. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../spark/internal/config/ConfigBuilder.scala | 4 +- .../spark/internal/config/ConfigEntry.scala | 0 .../internal/config/ConfigProvider.scala | 17 ----- .../spark/internal/config/ConfigReader.scala | 0 .../apache/spark/util/SparkStringUtils.scala | 26 ++++++++ .../sql/internal/ConnectRuntimeConfig.scala | 62 ++++++++++++++----- .../apache/spark/sql/ClientE2ETestSuite.scala | 23 +++++++ .../internal/config/SparkConfigProvider.scala | 35 +++++++++++ .../scala/org/apache/spark/util/Utils.scala | 7 +-- .../org/apache/spark/sql/RuntimeConfig.scala | 25 ++++++++ .../sql/internal/RuntimeConfigImpl.scala | 20 +++++- .../apache/spark/sql/RuntimeConfigSuite.scala | 22 +++++++ 12 files changed, 201 insertions(+), 40 deletions(-) rename {core => common/utils}/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala (99%) rename {core => common/utils}/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala (100%) rename {core => common/utils}/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala (78%) rename {core => common/utils}/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala (100%) create mode 100644 common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala create mode 100644 core/src/main/scala/org/apache/spark/internal/config/SparkConfigProvider.scala diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala rename to common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala index f50cc0f88842a..d3e975d1782f0 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala @@ -24,7 +24,7 @@ import scala.util.matching.Regex import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.network.util.{ByteUnit, JavaUtils} -import org.apache.spark.util.Utils +import org.apache.spark.util.SparkStringUtils private object ConfigHelpers { @@ -47,7 +47,7 @@ private object ConfigHelpers { } def stringToSeq[T](str: String, converter: String => T): Seq[T] = { - Utils.stringToSeq(str).map(converter) + SparkStringUtils.stringToSeq(str).map(converter) } def seqToString[T](v: Seq[T], stringConverter: T => String): String = { diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala rename to common/utils/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala similarity index 78% rename from core/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala rename to common/utils/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala index 392f9d56e7f51..fef019ef1f560 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala @@ -19,8 +19,6 @@ package org.apache.spark.internal.config import java.util.{Map => JMap} -import org.apache.spark.SparkConf - /** * A source of configuration values. */ @@ -47,18 +45,3 @@ private[spark] class MapProvider(conf: JMap[String, String]) extends ConfigProvi override def get(key: String): Option[String] = Option(conf.get(key)) } - -/** - * A config provider that only reads Spark config keys. - */ -private[spark] class SparkConfigProvider(conf: JMap[String, String]) extends ConfigProvider { - - override def get(key: String): Option[String] = { - if (key.startsWith("spark.")) { - Option(conf.get(key)).orElse(SparkConf.getDeprecatedConfig(key, conf)) - } else { - None - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala rename to common/utils/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala new file mode 100644 index 0000000000000..6915f373b84e5 --- /dev/null +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala @@ -0,0 +1,26 @@ +/* + * 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.util + +trait SparkStringUtils { + def stringToSeq(str: String): Seq[String] = { + import org.apache.spark.util.ArrayImplicits._ + str.split(",").map(_.trim()).filter(_.nonEmpty).toImmutableArraySeq + } +} + +object SparkStringUtils extends SparkStringUtils diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/ConnectRuntimeConfig.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/ConnectRuntimeConfig.scala index be1a13cb2fed2..74348e8e015e2 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/ConnectRuntimeConfig.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/ConnectRuntimeConfig.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.internal import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse, KeyValue} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{ConfigEntry, ConfigReader, OptionalConfigEntry} import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.connect.client.SparkConnectClient @@ -28,7 +29,7 @@ import org.apache.spark.sql.connect.client.SparkConnectClient */ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient) extends RuntimeConfig - with Logging { + with Logging { self => /** @inheritdoc */ def set(key: String, value: String): Unit = { @@ -37,6 +38,13 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient) } } + /** @inheritdoc */ + override private[sql] def set[T](entry: ConfigEntry[T], value: T): Unit = { + require(entry != null, "entry cannot be null") + require(value != null, s"value cannot be null for key: ${entry.key}") + set(entry.key, entry.stringConverter(value)) + } + /** @inheritdoc */ @throws[NoSuchElementException]("if the key is not set and there is no default value") def get(key: String): String = getOption(key).getOrElse { @@ -45,11 +53,39 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient) /** @inheritdoc */ def get(key: String, default: String): String = { - executeConfigRequestSingleValue { builder => - builder.getGetWithDefaultBuilder.addPairsBuilder().setKey(key).setValue(default) + val kv = executeConfigRequestSinglePair { builder => + val pairsBuilder = builder.getGetWithDefaultBuilder + .addPairsBuilder() + .setKey(key) + if (default != null) { + pairsBuilder.setValue(default) + } + } + if (kv.hasValue) { + kv.getValue + } else { + default } } + /** @inheritdoc */ + override private[sql] def get[T](entry: ConfigEntry[T]): T = { + require(entry != null, "entry cannot be null") + entry.readFrom(reader) + } + + /** @inheritdoc */ + override private[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T] = { + require(entry != null, "entry cannot be null") + entry.readFrom(reader) + } + + /** @inheritdoc */ + override private[sql] def get[T](entry: ConfigEntry[T], default: T): T = { + require(entry != null, "entry cannot be null") + Option(get(entry.key, null)).map(entry.valueConverter).getOrElse(default) + } + /** @inheritdoc */ def getAll: Map[String, String] = { val response = executeConfigRequest { builder => @@ -65,11 +101,11 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient) /** @inheritdoc */ def getOption(key: String): Option[String] = { - val pair = executeConfigRequestSinglePair { builder => + val kv = executeConfigRequestSinglePair { builder => builder.getGetOptionBuilder.addKeys(key) } - if (pair.hasValue) { - Option(pair.getValue) + if (kv.hasValue) { + Option(kv.getValue) } else { None } @@ -84,17 +120,11 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient) /** @inheritdoc */ def isModifiable(key: String): Boolean = { - val modifiable = executeConfigRequestSingleValue { builder => + val kv = executeConfigRequestSinglePair { builder => builder.getIsModifiableBuilder.addKeys(key) } - java.lang.Boolean.valueOf(modifiable) - } - - private def executeConfigRequestSingleValue( - f: ConfigRequest.Operation.Builder => Unit): String = { - val pair = executeConfigRequestSinglePair(f) - require(pair.hasValue, "The returned pair does not have a value set") - pair.getValue + require(kv.hasValue, "The returned pair does not have a value set") + java.lang.Boolean.valueOf(kv.getValue) } private def executeConfigRequestSinglePair( @@ -113,4 +143,6 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient) } response } + + private val reader = new ConfigReader((key: String) => Option(self.get(key, null))) } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index 92b5808f4d626..c7979b8e033ea 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -33,6 +33,7 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkArithmeticException, SparkException, SparkUpgradeException} import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} +import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, TableAlreadyExistsException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.StringEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema @@ -1006,8 +1007,12 @@ class ClientE2ETestSuite test("RuntimeConfig") { intercept[NoSuchElementException](spark.conf.get("foo.bar")) assert(spark.conf.getOption("foo.bar").isEmpty) + assert(spark.conf.get("foo.bar", "nope") == "nope") + assert(spark.conf.get("foo.bar", null) == null) spark.conf.set("foo.bar", value = true) assert(spark.conf.getOption("foo.bar") === Option("true")) + assert(spark.conf.get("foo.bar", "nope") === "true") + assert(spark.conf.get("foo.bar", null) === "true") spark.conf.set("foo.bar.numBaz", 100L) assert(spark.conf.get("foo.bar.numBaz") === "100") spark.conf.set("foo.bar.name", "donkey") @@ -1020,6 +1025,24 @@ class ClientE2ETestSuite assert(spark.conf.isModifiable("spark.sql.ansi.enabled")) assert(!spark.conf.isModifiable("spark.sql.globalTempDatabase")) intercept[Exception](spark.conf.set("spark.sql.globalTempDatabase", "/dev/null")) + + val entry = ConfigBuilder("my.simple.conf").intConf.createOptional + intercept[NoSuchElementException](spark.conf.get(entry.key)) + assert(spark.conf.get(entry).isEmpty) + assert(spark.conf.get(entry, Option(55)) === Option(55)) + spark.conf.set(entry, Option(33)) + assert(spark.conf.get(entry.key) === "33") + assert(spark.conf.get(entry) === Option(33)) + assert(spark.conf.get(entry, Option(55)) === Option(33)) + + val entryWithDefault = ConfigBuilder("my.important.conf").intConf.createWithDefault(10) + intercept[NoSuchElementException](spark.conf.get(entryWithDefault.key)) + assert(spark.conf.get(entryWithDefault) === 10) + assert(spark.conf.get(entryWithDefault, 11) === 11) + spark.conf.set(entryWithDefault, 12) + assert(spark.conf.get(entryWithDefault.key) === "12") + assert(spark.conf.get(entryWithDefault) === 12) + assert(spark.conf.get(entryWithDefault, 11) === 12) } test("SparkVersion") { diff --git a/core/src/main/scala/org/apache/spark/internal/config/SparkConfigProvider.scala b/core/src/main/scala/org/apache/spark/internal/config/SparkConfigProvider.scala new file mode 100644 index 0000000000000..8739c87a65877 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/SparkConfigProvider.scala @@ -0,0 +1,35 @@ +/* + * 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.internal.config + +import java.util.{Map => JMap} + +import org.apache.spark.SparkConf + +/** + * A config provider that only reads Spark config keys. + */ +private[spark] class SparkConfigProvider(conf: JMap[String, String]) extends ConfigProvider { + + override def get(key: String): Option[String] = { + if (key.startsWith("spark.")) { + Option(conf.get(key)).orElse(SparkConf.getDeprecatedConfig(key, conf)) + } else { + None + } + } +} 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 b2cf99241fdee..9e7ba6d879aa0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -103,7 +103,8 @@ private[spark] object Utils with SparkErrorUtils with SparkFileUtils with SparkSerDeUtils - with SparkStreamUtils { + with SparkStreamUtils + with SparkStringUtils { private val sparkUncaughtExceptionHandler = new SparkUncaughtExceptionHandler @volatile private var cachedLocalDir: String = "" @@ -2799,10 +2800,6 @@ private[spark] object Utils } } - def stringToSeq(str: String): Seq[String] = { - str.split(",").map(_.trim()).filter(_.nonEmpty).toImmutableArraySeq - } - /** * Create instances of extension classes. * diff --git a/sql/api/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/api/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 9e6e0e97f0302..091fbf20a0a7f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql import org.apache.spark.annotation.Stable +import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} /** * Runtime configuration interface for Spark. To access this, use `SparkSession.conf`. @@ -53,6 +54,11 @@ abstract class RuntimeConfig { set(key, value.toString) } + /** + * Sets the given Spark runtime configuration property. + */ + private[sql] def set[T](entry: ConfigEntry[T], value: T): Unit + /** * Returns the value of Spark runtime configuration property for the given key. If the key is * not set yet, return its default value if possible, otherwise `NoSuchElementException` will be @@ -74,6 +80,25 @@ abstract class RuntimeConfig { */ def get(key: String, default: String): String + /** + * Returns the value of Spark runtime configuration property for the given key. If the key is + * not set yet, return `defaultValue` in [[ConfigEntry]]. + */ + @throws[NoSuchElementException]("if the key is not set") + private[sql] def get[T](entry: ConfigEntry[T]): T + + /** + * Returns the value of Spark runtime configuration property for the given key. If the key is + * not set yet, return None. + */ + private[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T] + + /** + * Returns the value of Spark runtime configuration property for the given key. If the key is + * not set yet, return the user given `default`. + */ + private[sql] def get[T](entry: ConfigEntry[T], default: T): T + /** * Returns all properties set in this conf. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/RuntimeConfigImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/RuntimeConfigImpl.scala index 1739b86c8dcb4..b2004215a99f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/RuntimeConfigImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/RuntimeConfigImpl.scala @@ -21,7 +21,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.SPARK_DOC_ROOT import org.apache.spark.annotation.Stable -import org.apache.spark.internal.config.{ConfigEntry, DEFAULT_PARALLELISM} +import org.apache.spark.internal.config.{ConfigEntry, DEFAULT_PARALLELISM, OptionalConfigEntry} import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.errors.QueryCompilationErrors @@ -41,6 +41,12 @@ class RuntimeConfigImpl private[sql](val sqlConf: SQLConf = new SQLConf) extends sqlConf.setConfString(key, value) } + /** @inheritdoc */ + override private[sql] def set[T](entry: ConfigEntry[T], value: T): Unit = { + requireNonStaticConf(entry.key) + sqlConf.setConf(entry, value) + } + /** @inheritdoc */ @throws[NoSuchElementException]("if the key is not set and there is no default value") def get(key: String): String = { @@ -57,6 +63,18 @@ class RuntimeConfigImpl private[sql](val sqlConf: SQLConf = new SQLConf) extends sqlConf.getAllConfs } + /** @inheritdoc */ + override private[sql] def get[T](entry: ConfigEntry[T]): T = + sqlConf.getConf(entry) + + /** @inheritdoc */ + override private[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T] = + sqlConf.getConf(entry) + + /** @inheritdoc */ + override private[sql] def get[T](entry: ConfigEntry[T], default: T): T = + sqlConf.getConf(entry, default) + private[sql] def getAllAsJava: java.util.Map[String, String] = { getAll.asJava } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index c80787c40c487..ce3ac9b8834bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -108,4 +108,26 @@ class RuntimeConfigSuite extends SparkFunSuite { // this set should not fail conf.set(DEFAULT_PARALLELISM.key, "1") } + + test("config entry") { + val conf = newConf() + + val entry = SQLConf.FILES_MAX_PARTITION_NUM + assert(conf.get(entry.key) === null) + assert(conf.get(entry).isEmpty) + assert(conf.get(entry, Option(55)) === Option(55)) + conf.set(entry, Option(33)) + assert(conf.get(entry.key) === "33") + assert(conf.get(entry) === Option(33)) + assert(conf.get(entry, Option(55)) === Option(33)) + + val entryWithDefault = SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD + assert(conf.get(entryWithDefault.key) === "10") + assert(conf.get(entryWithDefault) === 10) + assert(conf.get(entryWithDefault, 11) === 11) + conf.set(entryWithDefault, 12) + assert(conf.get(entryWithDefault.key) === "12") + assert(conf.get(entryWithDefault) === 12) + assert(conf.get(entryWithDefault, 11) === 12) + } } From 7278bc70b126a7b58e0e4544547b26619eaab5db Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 5 Dec 2024 09:41:11 +0900 Subject: [PATCH 100/438] [SPARK-50489][SQL][PYTHON] Fix self-join after `applyInArrow` ### What changes were proposed in this pull request? Fix self-join after `applyInArrow`, the same issue of `applyInPandas` was fixed in https://github.com/apache/spark/pull/31429 ### Why are the changes needed? bug fix before: ``` In [1]: import pyarrow as pa In [2]: df = spark.createDataFrame([(1, 1)], ("k", "v")) In [3]: def arrow_func(key, table): ...: return pa.Table.from_pydict({"x": [2], "y": [2]}) ...: In [4]: df2 = df.groupby("k").applyInArrow(arrow_func, schema="x long, y long") In [5]: df2.show() 24/12/04 17:47:43 WARN CheckAllocator: More than one DefaultAllocationManager on classpath. Choosing first found +---+---+ | x| y| +---+---+ | 2| 2| +---+---+ In [6]: df2.join(df2) ... Failure when resolving conflicting references in Join: 'Join Inner :- FlatMapGroupsInArrow [k#0L], arrow_func(k#0L, v#1L)#2, [x#3L, y#4L] : +- Project [k#0L, k#0L, v#1L] : +- LogicalRDD [k#0L, v#1L], false +- FlatMapGroupsInArrow [k#12L], arrow_func(k#12L, v#13L)#2, [x#3L, y#4L] +- Project [k#12L, k#12L, v#13L] +- LogicalRDD [k#12L, v#13L], false Conflicting attributes: "x", "y". SQLSTATE: XX000 at org.apache.spark.SparkException$.internalError(SparkException.scala:92) at org.apache.spark.SparkException$.internalError(SparkException.scala:79) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis0$2(CheckAnalysis.scala:798) ``` after: ``` In [6]: df2.join(df2) Out[6]: DataFrame[x: bigint, y: bigint, x: bigint, y: bigint] In [7]: df2.join(df2).show() +---+---+---+---+ | x| y| x| y| +---+---+---+---+ | 2| 2| 2| 2| +---+---+---+---+ ``` ### Does this PR introduce _any_ user-facing change? bug fix ### How was this patch tested? added tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49056 from zhengruifeng/fix_arrow_join. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .../pyspark/sql/tests/test_arrow_cogrouped_map.py | 10 ++++++++++ python/pyspark/sql/tests/test_arrow_grouped_map.py | 10 ++++++++++ .../catalyst/analysis/DeduplicateRelations.scala | 14 ++++++++++++++ 3 files changed, 34 insertions(+) diff --git a/python/pyspark/sql/tests/test_arrow_cogrouped_map.py b/python/pyspark/sql/tests/test_arrow_cogrouped_map.py index a90574b7f1928..27a520d2843ed 100644 --- a/python/pyspark/sql/tests/test_arrow_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_arrow_cogrouped_map.py @@ -299,6 +299,16 @@ def summarize(left, right): "+---------+------------+----------+-------------+\n", ) + def test_self_join(self): + df = self.spark.createDataFrame([(1, 1)], ("k", "v")) + + def arrow_func(key, left, right): + return pa.Table.from_pydict({"x": [2], "y": [2]}) + + df2 = df.groupby("k").cogroup(df.groupby("k")).applyInArrow(arrow_func, "x long, y long") + + self.assertEqual(df2.join(df2).count(), 1) + class CogroupedMapInArrowTests(CogroupedMapInArrowTestsMixin, ReusedSQLTestCase): @classmethod diff --git a/python/pyspark/sql/tests/test_arrow_grouped_map.py b/python/pyspark/sql/tests/test_arrow_grouped_map.py index f9947d0788b87..213810e882fd9 100644 --- a/python/pyspark/sql/tests/test_arrow_grouped_map.py +++ b/python/pyspark/sql/tests/test_arrow_grouped_map.py @@ -255,6 +255,16 @@ def foo(_): self.assertEqual(r.a, "hi") self.assertEqual(r.b, 1) + def test_self_join(self): + df = self.spark.createDataFrame([(1, 1)], ("k", "v")) + + def arrow_func(key, table): + return pa.Table.from_pydict({"x": [2], "y": [2]}) + + df2 = df.groupby("k").applyInArrow(arrow_func, schema="x long, y long") + + self.assertEqual(df2.join(df2).count(), 1) + class GroupedMapInArrowTests(GroupedMapInArrowTestsMixin, ReusedSQLTestCase): @classmethod diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index c1535343d7686..52be631d94d85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -132,6 +132,13 @@ object DeduplicateRelations extends Rule[LogicalPlan] { _.output.map(_.exprId.id), newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance()))) + case f: FlatMapGroupsInArrow => + deduplicateAndRenew[FlatMapGroupsInArrow]( + existingRelations, + f, + _.output.map(_.exprId.id), + newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance()))) + case f: FlatMapCoGroupsInPandas => deduplicateAndRenew[FlatMapCoGroupsInPandas]( existingRelations, @@ -139,6 +146,13 @@ object DeduplicateRelations extends Rule[LogicalPlan] { _.output.map(_.exprId.id), newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance()))) + case f: FlatMapCoGroupsInArrow => + deduplicateAndRenew[FlatMapCoGroupsInArrow]( + existingRelations, + f, + _.output.map(_.exprId.id), + newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance()))) + case m: MapInPandas => deduplicateAndRenew[MapInPandas]( existingRelations, From af4f37c94ce47c28e3c324499463e032b68d454d Mon Sep 17 00:00:00 2001 From: WweiL Date: Thu, 5 Dec 2024 12:03:32 +0900 Subject: [PATCH 101/438] [SPARK-50339][SPARK-50360][SS] Enable changelog to store lineage information ### What changes were proposed in this pull request? Break down https://github.com/apache/spark/pull/48355 into smaller PRs. ## Changelog Reader / Writer We purpose to save the lineage to the first line of the changelog files. For changelog reader, there is an abstract function `readLineage` created. In `RocksDBCheckpointManager.getChangelogReader` function, the `readLineage` will be called right after the initialization of the changelog reader to update the file pointer to after the lineage. Subsequent `getNext` function won't be affecter because of this. For changelog writer, there is an abstract function `writeLineage` that writes the lineage. This function will be called before any actual changelog data is written in `RocksDB.load()`. The lineage is stored as json. ### Why are the changes needed? Continue development of SPARK-49374 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #48880 from WweiL/changelog. Authored-by: WweiL Signed-off-by: Jungtaek Lim --- .../resources/error/error-conditions.json | 2 +- .../execution/streaming/state/RocksDB.scala | 2 +- .../streaming/state/RocksDBFileManager.scala | 73 +++-- .../streaming/state/StateStoreChangelog.scala | 243 +++++++++++++- ...ateDataSourceTransformWithStateSuite.scala | 4 +- .../streaming/state/RocksDBSuite.scala | 304 +++++++++++++++++- 6 files changed, 565 insertions(+), 63 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 63c4d18c99de9..b18db93f6291a 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -275,7 +275,7 @@ }, "INVALID_CHANGE_LOG_READER_VERSION" : { "message" : [ - "The change log reader version cannot be ." + "The change log reader version cannot be . The checkpoint probably is from a future Spark version, please upgrade your Spark." ] }, "INVALID_CHANGE_LOG_WRITER_VERSION" : { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index f8e9885cef14e..709197cd56527 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -440,7 +440,7 @@ class RocksDB( log"${MDC(LogKeys.VERSION_NUM, v)}") var changelogReader: StateStoreChangelogReader = null try { - changelogReader = fileManager.getChangelogReader(v, useColumnFamilies) + changelogReader = fileManager.getChangelogReader(v) changelogReader.foreach { case (recordType, key, value) => recordType match { case RecordType.PUT_RECORD => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 6b13ff31c9d50..483e4a32cd85b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -153,58 +153,73 @@ class RocksDBFileManager( @volatile private var rootDirChecked: Boolean = false private val versionToRocksDBFiles = new ConcurrentHashMap[Long, Seq[RocksDBImmutableFile]] - private def getChangelogVersion(useColumnFamilies: Boolean): Short = { - val changelogVersion: Short = if (useColumnFamilies) { - 2 - } else { - 1 + /** + * Get the changelog version based on rocksDB features. + * @return the version of changelog + */ + private def getChangelogWriterVersion( + useColumnFamilies: Boolean, + stateStoreCheckpointIdEnabled: Boolean): Short = { + (useColumnFamilies, stateStoreCheckpointIdEnabled) match { + case (false, false) => 1 + case (true, false) => 2 + case (false, true) => 3 + case _ => 4 } - changelogVersion } def getChangeLogWriter( version: Long, - useColumnFamilies: Boolean = false): StateStoreChangelogWriter = { - val changelogFile = dfsChangelogFile(version) + useColumnFamilies: Boolean = false, + checkpointUniqueId: Option[String] = None, + stateStoreCheckpointIdLineage: Option[Array[LineageItem]] = None + ): StateStoreChangelogWriter = { + val changelogFile = dfsChangelogFile(version, checkpointUniqueId) if (!rootDirChecked) { val rootDir = new Path(dfsRootDir) if (!fm.exists(rootDir)) fm.mkdirs(rootDir) rootDirChecked = true } - val changelogVersion = getChangelogVersion(useColumnFamilies) + val enableStateStoreCheckpointIds = checkpointUniqueId.isDefined + val changelogVersion = getChangelogWriterVersion( + useColumnFamilies, enableStateStoreCheckpointIds) val changelogWriter = changelogVersion match { case 1 => new StateStoreChangelogWriterV1(fm, changelogFile, codec) case 2 => new StateStoreChangelogWriterV2(fm, changelogFile, codec) + case 3 => + assert(enableStateStoreCheckpointIds && stateStoreCheckpointIdLineage.isDefined, + "StateStoreChangelogWriterV3 should only be initialized when " + + "state store checkpoint unique id is enabled") + new StateStoreChangelogWriterV3(fm, changelogFile, codec, stateStoreCheckpointIdLineage.get) + case 4 => + assert(enableStateStoreCheckpointIds && stateStoreCheckpointIdLineage.isDefined, + "StateStoreChangelogWriterV4 should only be initialized when " + + "state store checkpoint unique id is enabled") + new StateStoreChangelogWriterV4(fm, changelogFile, codec, stateStoreCheckpointIdLineage.get) case _ => throw QueryExecutionErrors.invalidChangeLogWriterVersion(changelogVersion) } + + logInfo(log"Loaded change log reader version " + + log"${MDC(LogKeys.FILE_VERSION, changelogWriter.version)}") + changelogWriter } // Get the changelog file at version def getChangelogReader( version: Long, - useColumnFamilies: Boolean = false): StateStoreChangelogReader = { - val changelogFile = dfsChangelogFile(version) - - // Note that ideally we should get the version for the reader from the - // changelog itself. However, since we don't record this for v1, we need to - // rely on external arguments to make this call today. Within the reader, we verify - // for the correctness of the decided/expected version. We might revisit this pattern - // as we add more changelog versions in the future. - val changelogVersion = getChangelogVersion(useColumnFamilies) - val changelogReader = changelogVersion match { - case 1 => - new StateStoreChangelogReaderV1(fm, changelogFile, codec) - case 2 => - new StateStoreChangelogReaderV2(fm, changelogFile, codec) - case _ => - throw QueryExecutionErrors.invalidChangeLogReaderVersion(changelogVersion) - } - changelogReader + checkpointUniqueId: Option[String] = None): StateStoreChangelogReader = { + val changelogFile = dfsChangelogFile(version, checkpointUniqueId) + val reader = new StateStoreChangelogReaderFactory(fm, changelogFile, codec) + .constructChangelogReader() + + logInfo(log"Loaded change log reader version ${MDC(LogKeys.FILE_VERSION, reader.version)}") + + reader } /** @@ -777,7 +792,9 @@ class RocksDBFileManager( private def dfsBatchZipFile(version: Long): Path = new Path(s"$dfsRootDir/$version.zip") // We use changelog suffix intentionally so that we can tell the difference from changelog file of // HDFSBackedStateStore which is named version.delta. - private def dfsChangelogFile(version: Long): Path = new Path(s"$dfsRootDir/$version.changelog") + private def dfsChangelogFile(version: Long, checkpointUniqueId: Option[String] = None): Path = + checkpointUniqueId.map(id => new Path(s"$dfsRootDir/${version}_$id.changelog")) + .getOrElse(new Path(s"$dfsRootDir/$version.changelog")) private def localMetadataFile(parentDir: File): File = new File(parentDir, "metadata") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala index 203af9d10217e..f6787a37bc80d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala @@ -24,6 +24,8 @@ import scala.util.control.NonFatal import com.google.common.io.ByteStreams import org.apache.commons.io.IOUtils import org.apache.hadoop.fs.{FSError, Path} +import org.json4s._ +import org.json4s.jackson.Serialization import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ @@ -78,6 +80,14 @@ object RecordType extends Enumeration { } } +/** + * Class for lineage item for checkpoint format V2. + */ +case class LineageItem( + version: Long, + checkpointUniqueId: String +) + /** * Base class for state store changelog writer * @param fm - checkpoint file manager used to manage streaming query checkpoint @@ -89,18 +99,27 @@ abstract class StateStoreChangelogWriter( file: Path, compressionCodec: CompressionCodec) extends Logging { + implicit val formats: Formats = DefaultFormats + private def compressStream(outputStream: DataOutputStream): DataOutputStream = { val compressed = compressionCodec.compressedOutputStream(outputStream) new DataOutputStream(compressed) } + protected var backingFileStream: CancellableFSDataOutputStream = + fm.createAtomic(file, overwriteIfPossible = true) + protected var compressedStream: DataOutputStream = compressStream(backingFileStream) + protected def writeVersion(): Unit = { compressedStream.writeUTF(s"v${version}") } - protected var backingFileStream: CancellableFSDataOutputStream = - fm.createAtomic(file, overwriteIfPossible = true) - protected var compressedStream: DataOutputStream = compressStream(backingFileStream) + protected def writeLineage(stateStoreCheckpointIdLineage: Array[LineageItem]): Unit = { + assert(version >= 3, + "writeLineage should only be invoked with state store checkpoint id enabled (version >= 3)") + val lineageStr = Serialization.write(stateStoreCheckpointIdLineage) + compressedStream.writeUTF(lineageStr) + } def version: Short @@ -115,9 +134,9 @@ abstract class StateStoreChangelogWriter( if (backingFileStream != null) backingFileStream.cancel() if (compressedStream != null) IOUtils.closeQuietly(compressedStream) } catch { - // Closing the compressedStream causes the stream to write/flush flush data into the + // Closing the compressedStream causes the stream to write/flush data into the // rawStream. Since the rawStream is already closed, there may be errors. - // Usually its an IOException. However, Hadoop's RawLocalFileSystem wraps + // Usually it's an IOException. However, Hadoop's RawLocalFileSystem wraps // IOException into FSError. case e: FSError if e.getCause.isInstanceOf[IOException] => case NonFatal(ex) => @@ -152,15 +171,15 @@ class StateStoreChangelogWriterV1( override def put(key: Array[Byte], value: Array[Byte]): Unit = { assert(compressedStream != null) - compressedStream.writeInt(key.size) + compressedStream.writeInt(key.length) compressedStream.write(key) - compressedStream.writeInt(value.size) + compressedStream.writeInt(value.length) compressedStream.write(value) } override def delete(key: Array[Byte]): Unit = { assert(compressedStream != null) - compressedStream.writeInt(key.size) + compressedStream.writeInt(key.length) compressedStream.write(key) // -1 in the value field means record deletion. compressedStream.writeInt(-1) @@ -206,7 +225,7 @@ class StateStoreChangelogWriterV2( override def version: Short = 2 - // append the version field to the changelog file starting from version 2 + // append the version field to the changelog file writeVersion() override def put(key: Array[Byte], value: Array[Byte]): Unit = { @@ -216,7 +235,7 @@ class StateStoreChangelogWriterV2( override def delete(key: Array[Byte]): Unit = { assert(compressedStream != null) compressedStream.write(RecordType.getRecordTypeAsByte(RecordType.DELETE_RECORD)) - compressedStream.writeInt(key.size) + compressedStream.writeInt(key.length) compressedStream.write(key) // -1 in the value field means record deletion. compressedStream.writeInt(-1) @@ -232,9 +251,9 @@ class StateStoreChangelogWriterV2( assert(recordType == RecordType.PUT_RECORD || recordType == RecordType.MERGE_RECORD) assert(compressedStream != null) compressedStream.write(RecordType.getRecordTypeAsByte(recordType)) - compressedStream.writeInt(key.size) + compressedStream.writeInt(key.length) compressedStream.write(key) - compressedStream.writeInt(value.size) + compressedStream.writeInt(value.length) compressedStream.write(value) } @@ -255,6 +274,127 @@ class StateStoreChangelogWriterV2( } } +/** + * Write changes to the key value state store instance to a changelog file. + * There are 2 types of records, put and delete. + * A put record is written as: | key length | key content | value length | value content | + * A delete record is written as: | key length | key content | -1 | + * Write an Int -1 to signal the end of file. + * The overall changelog format is: | put record | delete record | ... | put record | -1 | + * V3 is a extension of V1 for writing changelogs with version + * in the first line and lineage in the second line. + */ +class StateStoreChangelogWriterV3( + fm: CheckpointFileManager, + file: Path, + compressionCodec: CompressionCodec, + stateStoreCheckpointIdLineage: Array[LineageItem]) + extends StateStoreChangelogWriterV1(fm, file, compressionCodec) { + + override def version: Short = 3 + + // append the version field to the changelog file + writeVersion() + + // Also write lineage information to the changelog, it should appear + // in the second line for v3 because the first line is the version + writeLineage(stateStoreCheckpointIdLineage) +} + +/** + * Write changes to the key value state store instance to a changelog file. + * There are 3 types of data records, put, merge and delete. + * A put record or merge record is written as: | record type | key length + * | key content | value length | value content | -1 | + * A delete record is written as: | record type | key length | key content | -1 + * Write an EOF_RECORD to signal the end of file. + * The overall changelog format is: version | put record | delete record + * | ... | put record | eof record | + * V4 is a extension of V2 for writing changelogs with version + * in the first line and lineage in the second line. + */ +class StateStoreChangelogWriterV4( + fm: CheckpointFileManager, + file: Path, + compressionCodec: CompressionCodec, + stateStoreCheckpointIdLineage: Array[LineageItem]) + extends StateStoreChangelogWriterV2(fm, file, compressionCodec) { + + override def version: Short = 4 + + // Also write lineage information to the changelog, it should appear + // in the second line for v4 because the first line is the version + writeLineage(stateStoreCheckpointIdLineage) +} + +/** + * A factory class for constructing state store readers by reading the first line + * of the change log file, which stores the version. + * Note that for changelog version 1, there is no version written. + * @param fm - checkpoint file manager used to manage streaming query checkpoint + * @param fileToRead - name of file to use to read changelog + * @param compressionCodec - de-compression method using for reading changelog file + */ +class StateStoreChangelogReaderFactory( + fm: CheckpointFileManager, + fileToRead: Path, + compressionCodec: CompressionCodec) extends Logging { + + private def decompressStream(inputStream: DataInputStream): DataInputStream = { + val compressed = compressionCodec.compressedInputStream(inputStream) + new DataInputStream(compressed) + } + + private lazy val sourceStream = try { + fm.open(fileToRead) + } catch { + case f: FileNotFoundException => + throw QueryExecutionErrors.failedToReadStreamingStateFileError(fileToRead, f) + } + protected val input: DataInputStream = decompressStream(sourceStream) + + private def readVersion(): Short = { + try { + val versionStr = input.readUTF() + // Versions in the first line are prefixed with "v", e.g. "v2" + // Since there is no version written for version 1, + // return 1 if first line doesn't start with "v" + if (!versionStr.startsWith("v")) { + 1 + } else { + versionStr.stripPrefix("v").toShort + } + } catch { + // When there is no record being written in the changelog file in V1, + // the file contains a single int -1 meaning EOF, then the above readUTF() + // throws with EOFException and we return version 1. + case _: java.io.EOFException => 1 + } + } + + /** + * Construct the change log reader based on the version stored in changelog file + * @return StateStoreChangelogReader + */ + def constructChangelogReader(): StateStoreChangelogReader = { + var reader: StateStoreChangelogReader = null + try { + reader = readVersion() match { + case 1 => new StateStoreChangelogReaderV1(fm, fileToRead, compressionCodec) + case 2 => new StateStoreChangelogReaderV2(fm, fileToRead, compressionCodec) + case 3 => new StateStoreChangelogReaderV3(fm, fileToRead, compressionCodec) + case 4 => new StateStoreChangelogReaderV4(fm, fileToRead, compressionCodec) + case version => throw QueryExecutionErrors.invalidChangeLogReaderVersion(version) + } + } finally { + if (input != null) { + input.close() + } + } + reader + } +} + /** * Base class for state store changelog reader * @param fm - checkpoint file manager used to manage streaming query checkpoint @@ -267,12 +407,14 @@ abstract class StateStoreChangelogReader( compressionCodec: CompressionCodec) extends NextIterator[(RecordType.Value, Array[Byte], Array[Byte])] with Logging { + implicit val formats: Formats = DefaultFormats + private def decompressStream(inputStream: DataInputStream): DataInputStream = { val compressed = compressionCodec.compressedInputStream(inputStream) new DataInputStream(compressed) } - private val sourceStream = try { + private lazy val sourceStream = try { fm.open(fileToRead) } catch { case f: FileNotFoundException => @@ -280,6 +422,26 @@ abstract class StateStoreChangelogReader( } protected val input: DataInputStream = decompressStream(sourceStream) + // This function is valid only when called upon initialization, + // because version is written in the first line only for version >= 2. + protected def readVersion(): String = input.readUTF() + + protected def verifyVersion(): Unit = { + // ensure that the version read is correct, also updates file position + val changelogVersionStr = readVersion() + assert(changelogVersionStr == s"v${version}", + s"Changelog version mismatch: $changelogVersionStr != v${version}") + } + + private def readLineage(): Array[LineageItem] = { + assert(version >= 3, + "readLineage should only be invoked with state store checkpoint id enabled (version >= 3)") + val lineageStr = input.readUTF() + Serialization.read[Array[LineageItem]](lineageStr) + } + + lazy val lineage: Array[LineageItem] = readLineage() + def version: Short override protected def close(): Unit = { if (input != null) input.close() } @@ -352,10 +514,7 @@ class StateStoreChangelogReaderV2( override def version: Short = 2 - // ensure that the version read is v2 - val changelogVersionStr = input.readUTF() - assert(changelogVersionStr == "v2", - s"Changelog version mismatch: $changelogVersionStr != v2") + verifyVersion() override def getNext(): (RecordType.Value, Array[Byte], Array[Byte]) = { val recordType = RecordType.getRecordTypeFromByte(input.readByte()) @@ -388,6 +547,56 @@ class StateStoreChangelogReaderV2( } } +/** + * Read an iterator of change record from the changelog file. + * A record is represented by tuple(recordType: RecordType.Value, + * key: Array[Byte], value: Array[Byte]) + * A put record is returned as a tuple(recordType, key, value) + * A delete record is return as a tuple(recordType, key, null) + * V3 is a extension of V1 for reading changelogs with version + * in the first line and lineage in the second line. + */ +class StateStoreChangelogReaderV3( + fm: CheckpointFileManager, + fileToRead: Path, + compressionCodec: CompressionCodec) + extends StateStoreChangelogReaderV1(fm, fileToRead, compressionCodec) { + + override def version: Short = 3 + + verifyVersion() + + // If the changelogFile is written when state store checkpoint unique id is enabled + // the first line would be the version and the second line would be the lineage. + // We should update the file position by reading from the lineage during + // the reader initialization. + lineage +} + +/** + * Read an iterator of change record from the changelog file. + * A record is represented by tuple(recordType: RecordType.Value, + * key: Array[Byte], value: Array[Byte]) + * A put or merge record is returned as a tuple(recordType, key, value) + * A delete record is return as a tuple(recordType, key, null) + * V4 is a extension of V2 for reading changelogs with version + * in the first line and lineage in the second line. + */ +class StateStoreChangelogReaderV4( + fm: CheckpointFileManager, + fileToRead: Path, + compressionCodec: CompressionCodec) + extends StateStoreChangelogReaderV2(fm, fileToRead, compressionCodec) { + + override def version: Short = 4 + + // If the changelogFile is written when state store checkpoint unique id is enabled + // the first line would be the version and the second line would be the lineage. + // We should update the file position by reading from the lineage during + // the reader initialization. + lineage +} + /** * Base class representing a iterator that iterates over a range of changelog files in a state * store. In each iteration, it will return a tuple of (changeType: [[RecordType]], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala index af64f563cf7b0..1b63180171be8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala @@ -1075,7 +1075,7 @@ class StateDataSourceTransformWithStateSuite extends StateStoreMetricsTest // Read the changelog for one of the partitions at version 3 and // ensure that we have two entries // For this test - keys 9 and 12 are written at version 3 for partition 4 - val changelogReader = fileManager.getChangelogReader(3, true) + val changelogReader = fileManager.getChangelogReader(3) val entries = changelogReader.toSeq assert(entries.size == 2) val retainEntry = entries.head @@ -1091,7 +1091,7 @@ class StateDataSourceTransformWithStateSuite extends StateStoreMetricsTest // Ensure that we have only one entry in the changelog for version 3 // For this test - key 9 is retained and key 12 is deleted - val changelogReader1 = fileManager.getChangelogReader(3, true) + val changelogReader1 = fileManager.getChangelogReader(3) val entries1 = changelogReader1.toSeq assert(entries1.size == 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 61ca8e7c32f61..2b48bc2e501bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -29,14 +29,17 @@ import scala.util.Random import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FSDataInputStream, Path} import org.rocksdb.CompressionType import org.scalactic.source.Position +import org.scalatest.PrivateMethodTester import org.scalatest.Tag import org.apache.spark.{SparkConf, SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming.{CreateAtomicTestManager, FileSystemBasedCheckpointFileManager} +import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, CreateAtomicTestManager, FileContextBasedCheckpointFileManager, FileSystemBasedCheckpointFileManager} import org.apache.spark.sql.execution.streaming.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS @@ -45,6 +48,7 @@ import org.apache.spark.tags.SlowSQLTest import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.util.ArrayImplicits._ + class NoOverwriteFileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration) extends FileSystemBasedCheckpointFileManager(path, hadoopConf) { @@ -62,6 +66,17 @@ class NoOverwriteFileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Co } } +class TestStateStoreChangelogWriterV101( + fm: CheckpointFileManager, + file: Path, + compressionCodec: CompressionCodec) + extends StateStoreChangelogWriterV1(fm, file, compressionCodec) { + + override def version: Short = 101 + + writeVersion() +} + trait RocksDBStateStoreChangelogCheckpointingTestUtil { val rocksdbChangelogCheckpointingConfKey: String = RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + ".changelogCheckpointing.enabled" @@ -177,14 +192,82 @@ trait AlsoTestWithChangelogCheckpointingEnabled } } +class OpenNumCountedTestInputStream(in: InputStream) extends FSDataInputStream(in) { + import OpenNumCountedTestInputStream._ + + addOpenStreams(this) + + override def close(): Unit = { + removeOpenStream(this) + super.close() + } +} + +class OpenStreamCountedTestFileManager(path: Path, hadoopConf: Configuration) + extends FileContextBasedCheckpointFileManager(path, hadoopConf) { + + override def open(path: Path): FSDataInputStream = { + val stream = new OpenNumCountedTestInputStream(super.open(path)) + stream + } +} + +object OpenNumCountedTestInputStream extends Logging { + private val openStreams = mutable.Map.empty[FSDataInputStream, Throwable] + + def addOpenStreams(stream: FSDataInputStream): Unit = openStreams.synchronized { + openStreams.put(stream, new Throwable()) + } + + def removeOpenStream(stream: FSDataInputStream): Unit = openStreams.synchronized { + openStreams.remove(stream) + } + + def clearOpenStreams(): Unit = openStreams.synchronized { + openStreams.clear() + } + + def assertNoOpenStreams(): Unit = openStreams.synchronized { + val numOpen = openStreams.values.size + if (numOpen > 0) { + for (exc <- openStreams.values) { + logWarning("Leaked filesystem connection created at:") + exc.printStackTrace() + } + throw new IllegalStateException(s"There are $numOpen possibly leaked file streams.", + openStreams.values.head) + } + } +} + @SlowSQLTest -class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with SharedSparkSession { +class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with SharedSparkSession + with PrivateMethodTester { override protected def sparkConf: SparkConf = { super.sparkConf .set(SQLConf.STATE_STORE_PROVIDER_CLASS, classOf[RocksDBStateStoreProvider].getName) } + // In each test we verify opened streams are all closed + private def hadoopConf: Configuration = { + val fmClass = "org.apache.spark.sql.execution.streaming.state." + + "OpenStreamCountedTestFileManager" + val hadoopConf = new Configuration() + hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, fmClass) + hadoopConf + } + + override def beforeEach(): Unit = { + OpenNumCountedTestInputStream.clearOpenStreams() + } + + override def afterEach(): Unit = { + eventually(timeout(10.seconds), interval(2.seconds)) { + OpenNumCountedTestInputStream.assertNoOpenStreams() + } + } + testWithColumnFamilies( "RocksDB: check changelog and snapshot version", TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => @@ -693,7 +776,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared " with Changelog Checkpointing") { val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") val fileManager = new RocksDBFileManager( - dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration) + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) val changelogWriter = fileManager.getChangeLogWriter(1) assert(changelogWriter.version === 1) @@ -742,7 +825,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithChangelogCheckpointingEnabled("RocksDBFileManager: read and write changelog") { val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") val fileManager = new RocksDBFileManager( - dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration) + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) val changelogWriter = fileManager.getChangeLogWriter(1) assert(changelogWriter.version === 1) @@ -761,6 +844,153 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared null, StateStore.DEFAULT_COL_FAMILY_NAME) } + changelogReader.closeIfNeeded() + + assert(entries.size == expectedEntries.size) + entries.zip(expectedEntries).map{ + case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) + } + } + + testWithChangelogCheckpointingEnabled("RocksDBFileManager: StateStoreChangelogReaderFactory " + + "edge case") { + val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") + val fileManager = new RocksDBFileManager( + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) + + val checkpointUniqueId = Some(java.util.UUID.randomUUID.toString) + val lineage: Array[LineageItem] = Array( + LineageItem(1, java.util.UUID.randomUUID.toString), + LineageItem(2, java.util.UUID.randomUUID.toString), + LineageItem(3, java.util.UUID.randomUUID.toString) + ) + + // Create a v1 writer + val changelogWriterV1 = fileManager.getChangeLogWriter(101) + assert(changelogWriterV1.version === 1) + changelogWriterV1.commit() // v1 with empty content + + val changelogReaderV1 = fileManager.getChangelogReader(101) + assert(changelogReaderV1.version === 1) // getChangelogReader should return a v1 reader + changelogReaderV1.closeIfNeeded() + + // Create a v2 writer + val changelogWriterV2 = fileManager.getChangeLogWriter(102, useColumnFamilies = true) + assert(changelogWriterV2.version === 2) + changelogWriterV2.commit() // v2 with empty content + + val changelogReaderV2 = fileManager.getChangelogReader(102) + assert(changelogReaderV2.version === 2) // getChangelogReader should return a v2 reader + changelogReaderV2.closeIfNeeded() + + // Create a v3 writer + val changelogWriterV3 = fileManager.getChangeLogWriter( + 103, useColumnFamilies = false, checkpointUniqueId, Some(lineage)) + assert(changelogWriterV3.version === 3) + changelogWriterV3.commit() // v1 with empty content + + val changelogReaderV3 = fileManager.getChangelogReader( + 103, checkpointUniqueId = checkpointUniqueId) + assert(changelogReaderV3.version === 3) // getChangelogReader should return a v3 reader + assert(changelogReaderV3.lineage sameElements lineage) + changelogReaderV3.closeIfNeeded() + + // Create a v4 writer + val changelogWriterV4 = fileManager.getChangeLogWriter( + 104, useColumnFamilies = true, checkpointUniqueId, Some(lineage)) + assert(changelogWriterV4.version === 4) + changelogWriterV4.commit() // v1 with empty content + + val changelogReaderV4 = fileManager.getChangelogReader( + 104, checkpointUniqueId = checkpointUniqueId) + assert(changelogReaderV4.version === 4) // getChangelogReader should return a v4 reader + assert(changelogReaderV4.lineage sameElements lineage) + changelogReaderV4.closeIfNeeded() + } + + testWithChangelogCheckpointingEnabled("RocksDBFileManager: changelog reader / writer " + + "failure cases") { + val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") + val fileManager = new RocksDBFileManager( + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) + // Failure case 1: reader writer version mismatch + // Create a v1 writer + val changelogWriterV1 = fileManager.getChangeLogWriter(101) + assert(changelogWriterV1.version === 1) + + (1 to 5).foreach(i => changelogWriterV1.put(i.toString, i.toString)) + (2 to 4).foreach(j => changelogWriterV1.delete(j.toString)) + + changelogWriterV1.commit() + // Success case, when reading from the same file, a V1 reader should be constructed. + val changelogReaderV1 = fileManager.getChangelogReader(101) + assert(changelogReaderV1.version === 1) + changelogReaderV1.closeIfNeeded() + + // Failure case, force creating a V3 reader. + val dfsChangelogFile = PrivateMethod[Path](Symbol("dfsChangelogFile")) + val codec = PrivateMethod[CompressionCodec](Symbol("codec")) + var changelogFile = fileManager invokePrivate dfsChangelogFile(101L, None) + val compressionCodec = fileManager invokePrivate codec() + val fm = CheckpointFileManager.create(new Path(dfsRootDir.getAbsolutePath), new Configuration) + val e = intercept[AssertionError] { + new StateStoreChangelogReaderV3(fm, changelogFile, compressionCodec) + } + assert(e.getMessage.contains("Changelog version mismatch")) + + changelogFile = fileManager invokePrivate dfsChangelogFile(1L, None) + // Failure case 2: readerFactory throw when reading from ckpt built in future Spark version + // Create a v101 writer + val changelogWriter = new TestStateStoreChangelogWriterV101( + fm, changelogFile, compressionCodec) + assert(changelogWriter.version === 101) + + changelogWriter.commit() + + // Failure case, force creating a V3 reader. + val ex = intercept[SparkException] { + fileManager.getChangelogReader(1) + } + checkError( + ex, + condition = "CANNOT_LOAD_STATE_STORE.INVALID_CHANGE_LOG_READER_VERSION", + parameters = Map("version" -> 101.toString) + ) + assert(ex.getMessage.contains("please upgrade your Spark")) + } + + testWithChangelogCheckpointingEnabled("RocksDBFileManager: read and write changelog " + + "with state checkpoint id enabled") { + val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") + val fileManager = new RocksDBFileManager( + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) + val checkpointUniqueId = Some(java.util.UUID.randomUUID.toString) + val lineage: Array[LineageItem] = Array( + LineageItem(1, java.util.UUID.randomUUID.toString), + LineageItem(2, java.util.UUID.randomUUID.toString), + LineageItem(3, java.util.UUID.randomUUID.toString) + ) + val changelogWriter = fileManager.getChangeLogWriter( + 3, useColumnFamilies = false, checkpointUniqueId, Some(lineage)) + assert(changelogWriter.version === 3) + + (1 to 5).foreach(i => changelogWriter.put(i.toString, i.toString)) + (2 to 4).foreach(j => changelogWriter.delete(j.toString)) + + changelogWriter.commit() + val changelogReader = fileManager.getChangelogReader(3, checkpointUniqueId) + assert(changelogReader.version === 3) + assert(changelogReader.lineage sameElements lineage) + val entries = changelogReader.toSeq + val expectedEntries = (1 to 5).map { i => + (RecordType.PUT_RECORD, i.toString.getBytes, + i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME) + } ++ (2 to 4).map { j => + (RecordType.DELETE_RECORD, j.toString.getBytes, + null, StateStore.DEFAULT_COL_FAMILY_NAME) + } + changelogReader.closeIfNeeded() + assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) @@ -771,8 +1001,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "RocksDBFileManager: read and write v2 changelog with default col family") { val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") val fileManager = new RocksDBFileManager( - dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration) - val changelogWriter = fileManager.getChangeLogWriter(1, true) + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) + val changelogWriter = fileManager.getChangeLogWriter(1, useColumnFamilies = true) assert(changelogWriter.version === 2) (1 to 5).foreach { i => changelogWriter.put(i.toString, i.toString) @@ -786,7 +1016,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } changelogWriter.commit() - val changelogReader = fileManager.getChangelogReader(1, true) + val changelogReader = fileManager.getChangelogReader(1) assert(changelogReader.version === 2) val entries = changelogReader.toSeq val expectedEntries = (1 to 5).map { i => @@ -796,6 +1026,52 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } ++ (2 to 4).map { j => (RecordType.DELETE_RECORD, j.toString.getBytes, null) } + changelogReader.closeIfNeeded() + + assert(entries.size == expectedEntries.size) + entries.zip(expectedEntries).map{ + case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) + } + } + + testWithChangelogCheckpointingEnabled("RocksDBFileManager: read and write v2 changelog with " + + "default col family and state checkpoint id enabled") { + val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") + val fileManager = new RocksDBFileManager( + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) + val checkpointUniqueId = Some(java.util.UUID.randomUUID.toString) + val lineage: Array[LineageItem] = Array( + LineageItem(1, java.util.UUID.randomUUID.toString), + LineageItem(2, java.util.UUID.randomUUID.toString), + LineageItem(3, java.util.UUID.randomUUID.toString) + ) + val changelogWriter = fileManager.getChangeLogWriter( + 1, useColumnFamilies = true, checkpointUniqueId, Some(lineage)) + assert(changelogWriter.version === 4) + (1 to 5).foreach { i => + changelogWriter.put(i.toString, i.toString) + } + (1 to 5).foreach { i => + changelogWriter.merge(i.toString, i.toString) + } + + (2 to 4).foreach { j => + changelogWriter.delete(j.toString) + } + + changelogWriter.commit() + val changelogReader = fileManager.getChangelogReader(1, checkpointUniqueId) + assert(changelogReader.version === 4) + assert(changelogReader.lineage sameElements lineage) + val entries = changelogReader.toSeq + val expectedEntries = (1 to 5).map { i => + (RecordType.PUT_RECORD, i.toString.getBytes, i.toString.getBytes) + } ++ (1 to 5).map { i => + (RecordType.MERGE_RECORD, i.toString.getBytes, i.toString.getBytes) + } ++ (2 to 4).map { j => + (RecordType.DELETE_RECORD, j.toString.getBytes, null) + } + changelogReader.closeIfNeeded() assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ @@ -810,7 +1086,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared try { val verificationDir = Utils.createTempDir().getAbsolutePath val fileManager = new RocksDBFileManager( - dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration) + dfsRootDir.getAbsolutePath, Utils.createTempDir(), hadoopConf) // Save a version of empty checkpoint files val cpFiles = Seq() generateFiles(verificationDir, cpFiles) @@ -910,10 +1186,10 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // Use 2 file managers here to emulate concurrent execution // that checkpoint the same version of state val fileManager = new RocksDBFileManager( - dfsRootDir, Utils.createTempDir(), new Configuration) + dfsRootDir, Utils.createTempDir(), hadoopConf) val rocksDBFileMapping = new RocksDBFileMapping() val fileManager_ = new RocksDBFileManager( - dfsRootDir, Utils.createTempDir(), new Configuration) + dfsRootDir, Utils.createTempDir(), hadoopConf) val sstDir = s"$dfsRootDir/SSTs" def numRemoteSSTFiles: Int = listFiles(sstDir).length val logDir = s"$dfsRootDir/logs" @@ -994,7 +1270,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared withTempDir { dir => val dfsRootDir = dir.getAbsolutePath val fileManager = new RocksDBFileManager( - dfsRootDir, Utils.createTempDir(), new Configuration) + dfsRootDir, Utils.createTempDir(), hadoopConf) (new File(dfsRootDir, "SSTs")).mkdir() (new File(dfsRootDir, "logs")).mkdir() @@ -1053,7 +1329,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared val dfsRootDir = dir.getAbsolutePath val verificationDir = Utils.createTempDir().getAbsolutePath // local dir to load checkpoints val fileManager = new RocksDBFileManager( - dfsRootDir, Utils.createTempDir(), new Configuration) + dfsRootDir, Utils.createTempDir(), hadoopConf) val sstDir = s"$dfsRootDir/SSTs" def numRemoteSSTFiles: Int = listFiles(sstDir).length val logDir = s"$dfsRootDir/logs" @@ -2446,7 +2722,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared remoteDir: String, version: Int = 0, conf: RocksDBConf = dbConf, - hadoopConf: Configuration = new Configuration(), + hadoopConf: Configuration = hadoopConf, useColumnFamilies: Boolean = false, localDir: File = Utils.createTempDir())( func: RocksDB => T): T = { From 05703908935b7bd1424abd8a87d9a1dbc0212b29 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 5 Dec 2024 07:39:33 -0800 Subject: [PATCH 102/438] [SPARK-50495][K8S][INFRA][DOCS] Upgrade Volcano to 1.10.0 ### What changes were proposed in this pull request? This PR aims to upgrade `Volcano` (from `1.9.0` to `1.10.0`) in K8s integration test document and GA job. ### Why are the changes needed? - https://github.com/volcano-sh/volcano/releases/tag/v1.10.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 #49068 from panbingkun/SPARK-50495. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 2 +- resource-managers/kubernetes/integration-tests/README.md | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index cf49316fafbbf..900c2a7453202 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1226,7 +1226,7 @@ jobs: if [[ "${{ inputs.branch }}" == 'branch-3.5' ]]; then kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.7.0/installer/volcano-development.yaml || true else - kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.9.0/installer/volcano-development.yaml || true + kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.10.0/installer/volcano-development.yaml || true fi eval $(minikube docker-env) build/sbt -Phadoop-3 -Psparkr -Pkubernetes -Pvolcano -Pkubernetes-integration-tests -Dspark.kubernetes.test.volcanoMaxConcurrencyJobNum=1 -Dtest.exclude.tags=local "kubernetes-integration-tests/test" diff --git a/resource-managers/kubernetes/integration-tests/README.md b/resource-managers/kubernetes/integration-tests/README.md index 5b80fe10596c1..36848cba0609e 100644 --- a/resource-managers/kubernetes/integration-tests/README.md +++ b/resource-managers/kubernetes/integration-tests/README.md @@ -330,11 +330,11 @@ You can also specify your specific dockerfile to build JVM/Python/R based image ## Requirements - A minimum of 6 CPUs and 9G of memory is required to complete all Volcano test cases. -- Volcano v1.9.0. +- Volcano v1.10.0. ## Installation - kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.9.0/installer/volcano-development.yaml + kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.10.0/installer/volcano-development.yaml ## Run tests @@ -355,5 +355,5 @@ You can also specify `volcano` tag to only run Volcano test: ## Cleanup Volcano - kubectl delete -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.9.0/installer/volcano-development.yaml + kubectl delete -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.10.0/installer/volcano-development.yaml From 36285956ed2b9b8034d6918a9e951f1a2748f3ce Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 5 Dec 2024 07:47:26 -0800 Subject: [PATCH 103/438] [SPARK-50498][PYTHON] Avoid unnecessary py4j call in `listFunctions` ### What changes were proposed in this pull request? Avoid unnecessary py4j call in `listFunctions` ### Why are the changes needed? ``` iter = self._jcatalog.listFunctions(dbName).toLocalIterator() if pattern is None: iter = self._jcatalog.listFunctions(dbName).toLocalIterator() else: iter = self._jcatalog.listFunctions(dbName, pattern).toLocalIterator() ``` the first `self._jcatalog.listFunctions` is unnecessary ### 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 #49073 from zhengruifeng/avoid_list_funcs. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/catalog.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 8c35aafa7066c..40a0d9346ccc3 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -479,7 +479,6 @@ def listFunctions( """ if dbName is None: dbName = self.currentDatabase() - iter = self._jcatalog.listFunctions(dbName).toLocalIterator() if pattern is None: iter = self._jcatalog.listFunctions(dbName).toLocalIterator() else: From faa2b04726935ab6b347497eedd219e4007df559 Mon Sep 17 00:00:00 2001 From: Terry Wang Date: Thu, 5 Dec 2024 08:25:54 -0800 Subject: [PATCH 104/438] [SPARK-50421][CORE] Fix executor related memory config incorrect when multiple resource profiles worked ### What changes were proposed in this pull request? Reset the executor's env memory related config when resource profile is not as the default resource profile! ### Why are the changes needed? When multiple resource profile exists in the same spark application, now the executor's memory related config is not override by resource profile's memory size, which will cause maxOffHeap in `UnifiedMemoryManager` is not correct. See https://issues.apache.org/jira/browse/SPARK-50421 for more details ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests in our inner spark version and jobs. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48963 from zjuwangg/m_fixConfig. Authored-by: Terry Wang Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/internal/LogKey.scala | 2 ++ .../CoarseGrainedExecutorBackend.scala | 23 +++++++++++++++++++ 2 files changed, 25 insertions(+) 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 c365797cec690..4fe6d96f9aaba 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 @@ -241,6 +241,8 @@ private[spark] object LogKeys { case object EXECUTOR_ID extends LogKey case object EXECUTOR_IDS extends LogKey case object EXECUTOR_LAUNCH_COMMANDS extends LogKey + case object EXECUTOR_MEMORY_OFFHEAP extends LogKey + case object EXECUTOR_MEMORY_OVERHEAD_SIZE extends LogKey case object EXECUTOR_MEMORY_SIZE extends LogKey case object EXECUTOR_RESOURCES extends LogKey case object EXECUTOR_SHUFFLE_INFO extends LogKey diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e880cf8da9ec2..a73380cab690e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -479,6 +479,29 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverConf.set(EXECUTOR_ID, arguments.executorId) cfg.logLevel.foreach(logLevel => Utils.setLogLevelIfNeeded(logLevel)) + // Set executor memory related config here according to resource profile + if (cfg.resourceProfile.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) { + cfg.resourceProfile + .executorResources + .foreach { + case (ResourceProfile.OFFHEAP_MEM, request) => + driverConf.set(MEMORY_OFFHEAP_SIZE.key, request.amount.toString + "m") + logInfo(log"Set executor off-heap memory to " + + log"${MDC(LogKeys.EXECUTOR_MEMORY_OFFHEAP, request)}") + case (ResourceProfile.MEMORY, request) => + driverConf.set(EXECUTOR_MEMORY.key, request.amount.toString + "m") + logInfo(log"Set executor memory to ${MDC(LogKeys.EXECUTOR_MEMORY_SIZE, request)}") + case (ResourceProfile.OVERHEAD_MEM, request) => + // Maybe don't need to set this since it's nearly used by tasks. + driverConf.set(EXECUTOR_MEMORY_OVERHEAD.key, request.amount.toString + "m") + logInfo(log"Set executor memory_overhead to " + + log"${MDC(LogKeys.EXECUTOR_MEMORY_OVERHEAD_SIZE, request)}") + case (ResourceProfile.CORES, request) => + driverConf.set(EXECUTOR_CORES.key, request.amount.toString) + logInfo(log"Set executor cores to ${MDC(LogKeys.NUM_EXECUTOR_CORES, request)}") + case _ => + } + } val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.bindAddress, arguments.hostname, arguments.cores, cfg.ioEncryptionKey, isLocal = false) // Set the application attemptId in the BlockStoreClient if available. From ee8db4ef5d00656b3281104b996e7775cd4b84d7 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 5 Dec 2024 10:36:24 -0800 Subject: [PATCH 105/438] [SPARK-50343][SPARK-50344][SQL] Add SQL pipe syntax for the DROP and AS operators ### What changes were proposed in this pull request? This PR adds SQL pipe syntax support for DROP and AS operators. The DROP operator removes one or more existing column from the input table. The AS operator assigns a new table alias to the preceding relation. These are equivalent to `SELECT * EXCEPT()` and `SELECT ... FROM (...) `, respectively. For example: ``` values (0, 'pqr', 2), (3, 'tuv', 5) as tab(a, b, c) |> as v |> drop v.b |> drop v.c 0 3 ``` This PR also fixes a small bug and adds more testing. ### Why are the changes needed? The SQL pipe operator syntax will let users compose queries in a more flexible fashion. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds a few unit test cases, but mostly relies on golden file test coverage. I did this to make sure the answers are correct as this feature is implemented and also so we can look at the analyzer output plans to ensure they look right as well. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49037 from dtenedor/pipe-syntax-as-drop. Authored-by: Daniel Tenedorio Signed-off-by: Gengliang Wang --- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 + .../sql/catalyst/parser/AstBuilder.scala | 19 +- .../analyzer-results/pipe-operators.sql.out | 1005 ++++++++++++++++- .../sql-tests/inputs/pipe-operators.sql | 592 ++++++++++ .../sql-tests/results/pipe-operators.sql.out | 839 +++++++++++++- .../apache/spark/sql/SQLQueryTestSuite.scala | 24 +- 6 files changed, 2452 insertions(+), 29 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 93cf9974e654c..26edbe15da9fd 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -1510,6 +1510,8 @@ operatorPipeRightSide : selectClause windowClause? | EXTEND extendList=namedExpressionSeq | SET operatorPipeSetAssignmentSeq + | DROP identifierSeq + | AS errorCapturingIdentifier // Note that the WINDOW clause is not allowed in the WHERE pipe operator, but we add it here in // the grammar simply for purposes of catching this invalid syntax and throwing a specific // dedicated error message. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index a3fac7296dcc4..882e895cc7f02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -5968,10 +5968,8 @@ class AstBuilder extends DataTypeAstBuilder // analyzer behave as if we had added the corresponding SQL clause after a table subquery // containing the input plan. def withSubqueryAlias(): LogicalPlan = left match { - case s: SubqueryAlias => - s - case u: UnresolvedRelation => - u + case _: SubqueryAlias | _: UnresolvedRelation | _: Join | _: Filter => + left case _ => SubqueryAlias(SubqueryAlias.generateSubqueryName(), left) } @@ -6006,6 +6004,14 @@ class AstBuilder extends DataTypeAstBuilder Project(projectList, left) }.getOrElse(Option(ctx.SET).map { _ => visitOperatorPipeSet(ctx, left) + }.getOrElse(Option(ctx.DROP).map { _ => + val ids: Seq[String] = visitIdentifierSeq(ctx.identifierSeq()) + val projectList: Seq[NamedExpression] = + Seq(UnresolvedStarExceptOrReplace( + target = None, excepts = ids.map(s => Seq(s)), replacements = None)) + Project(projectList, left) + }.getOrElse(Option(ctx.AS).map { _ => + SubqueryAlias(ctx.errorCapturingIdentifier().getText, left) }.getOrElse(Option(ctx.whereClause).map { c => if (ctx.windowClause() != null) { throw QueryParsingErrors.windowClauseInPipeOperatorWhereClauseNotAllowedError(ctx) @@ -6032,7 +6038,7 @@ class AstBuilder extends DataTypeAstBuilder withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true) }.getOrElse( visitOperatorPipeAggregate(ctx, left) - )))))))))) + )))))))))))) } private def visitOperatorPipeSet( @@ -6119,7 +6125,8 @@ class AstBuilder extends DataTypeAstBuilder Seq("GROUPING", "GROUPING_ID").foreach { name => if (f.nameParts.head.equalsIgnoreCase(name)) error(name) } - case _: WindowSpec => error("window functions") + case _: WindowSpec => error("window functions; please update the query to move " + + "the window functions to a subsequent |> SELECT operator instead") case _ => } e.children.foreach(visit) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index a4cda92fab2e3..1e1ad90946f88 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -934,6 +934,267 @@ org.apache.spark.sql.catalyst.parser.ParseException } +-- !query +table t +|> drop y +-- !query analysis +Project [x#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z, y +-- !query analysis +Project [x#x] ++- Project [1 AS x#x, 2 AS y#x, 3 AS z#x] + +- OneRowRelation + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z +|> drop y +-- !query analysis +Project [x#x] ++- Project [x#x, y#x] + +- Project [1 AS x#x, 2 AS y#x, 3 AS z#x] + +- OneRowRelation + + +-- !query +select x from t +|> drop x +-- !query analysis +Project ++- Project [x#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as `x.y.z` +|> drop `x.y.z` +-- !query analysis +Project [x#x, y#x] ++- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS x.y.z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> drop z +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`z`", + "proposal" : "`x`, `y`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 17, + "fragment" : "table t\n|> drop z" + } ] +} + + +-- !query +table st +|> drop col.i1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +table st +|> drop `col.i1` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`col.i1`", + "proposal" : "`col`, `x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 25, + "fragment" : "table st\n|> drop `col.i1`" + } ] +} + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z, y, z +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "EXCEPT_OVERLAPPING_COLUMNS", + "sqlState" : "42702", + "messageParameters" : { + "columns" : "z, y, z" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "select 1 as x, 2 as y, 3 as z\n|> drop z, y, z" + } ] +} + + +-- !query +table t +|> as u +|> select u.x, u.y +-- !query analysis +Project [x#x, y#x] ++- SubqueryAlias u + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +select 1 as x, 2 as y +|> as u +|> select u.x, u.y +-- !query analysis +Project [x#x, y#x] ++- SubqueryAlias u + +- Project [1 AS x#x, 2 AS y#x] + +- OneRowRelation + + +-- !query +table t +|> as `u.v` +|> select `u.v`.x, `u.v`.y +-- !query analysis +Project [x#x, y#x] ++- SubqueryAlias `u.v` + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> as u +|> as v +|> select v.x, v.y +-- !query analysis +Project [x#x, y#x] ++- SubqueryAlias v + +- SubqueryAlias u + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> as u +|> where u.x = 1 +-- !query analysis +Filter (x#x = 1) ++- SubqueryAlias u + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> as u, v +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "','", + "hint" : "" + } +} + + +-- !query +table t +|> as 1 + 2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : "" + } +} + + +-- !query +table t +|> as u-v +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_IDENTIFIER", + "sqlState" : "42602", + "messageParameters" : { + "ident" : "u-v" + } +} + + +-- !query +table t +|> as u@v +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'@'", + "hint" : "" + } +} + + +-- !query +table t +|> as u#######v +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'#'", + "hint" : "" + } +} + + -- !query table t |> where true @@ -958,10 +1219,9 @@ table t |> where x + length(y) < 3 -- !query analysis Filter ((x#x + length(y#x)) < 3) -+- SubqueryAlias __auto_generated_subquery_name - +- Filter ((x#x + length(y#x)) < 4) - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- Filter ((x#x + length(y#x)) < 4) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -2126,21 +2386,20 @@ table natural_join_test_t1 |> where k = "one" -- !query analysis Filter (k#x = one) -+- SubqueryAlias __auto_generated_subquery_name - +- Project [k#x, v1#x, v2#x] - +- Join Inner, (k#x = k#x) - :- SubqueryAlias natural_join_test_t1 - : +- View (`natural_join_test_t1`, [k#x, v1#x]) - : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] - : +- Project [k#x, v1#x] - : +- SubqueryAlias natural_join_test_t1 - : +- LocalRelation [k#x, v1#x] - +- SubqueryAlias natural_join_test_t2 - +- View (`natural_join_test_t2`, [k#x, v2#x]) - +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] - +- Project [k#x, v2#x] - +- SubqueryAlias natural_join_test_t2 - +- LocalRelation [k#x, v2#x] ++- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias natural_join_test_t1 + : +- View (`natural_join_test_t1`, [k#x, v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias natural_join_test_t1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias natural_join_test_t2 + +- View (`natural_join_test_t2`, [k#x, v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias natural_join_test_t2 + +- LocalRelation [k#x, v2#x] -- !query @@ -3098,7 +3357,7 @@ org.apache.spark.sql.catalyst.parser.ParseException "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", "sqlState" : "0A000", "messageParameters" : { - "case" : "window functions" + "case" : "window functions; please update the query to move the window functions to a subsequent |> SELECT operator instead" }, "queryContext" : [ { "objectType" : "", @@ -3362,6 +3621,712 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +with customer_total_return as +(select + sr_customer_sk as ctr_customer_sk, + sr_store_sk as ctr_store_sk, + sum(sr_return_amt) as ctr_total_return + from store_returns, date_dim + where sr_returned_date_sk = d_date_sk and d_year = 2000 + group by sr_customer_sk, sr_store_sk) +select c_customer_id +from customer_total_return ctr1, store, customer +where ctr1.ctr_total_return > + (select avg(ctr_total_return) * 1.2 + from customer_total_return ctr2 + where ctr1.ctr_store_sk = ctr2.ctr_store_sk) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`store_returns`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 161, + "stopIndex" : 173, + "fragment" : "store_returns" + } ] +} + + +-- !query +with customer_total_return as + (table store_returns + |> join date_dim + |> where sr_returned_date_sk = d_date_sk and d_year = 2000 + |> aggregate sum(sr_return_amt) as ctr_total_return + group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) +table customer_total_return +|> as ctr1 +|> join store +|> join customer +|> where ctr1.ctr_total_return > + (table customer_total_return + |> as ctr2 + |> where ctr1.ctr_store_sk = ctr2.ctr_store_sk + |> aggregate avg(ctr_total_return) * 1.2) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +|> order by c_customer_id +|> limit 100 +|> select c_customer_id +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`store_returns`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 52, + "fragment" : "store_returns" + } ] +} + + +-- !query +with wscs as +( select + sold_date_sk, + sales_price + from (select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + from web_sales) x + union all + (select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + ( select + d_week_seq, + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + from wscs, date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) +select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +from + (select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001) y, + (select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001 + 1) z +where d_week_seq1 = d_week_seq2 - 53 +order by d_week_seq1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`web_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 148, + "stopIndex" : 156, + "fragment" : "web_sales" + } ] +} + + +-- !query +with wscs as + (table web_sales + |> select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + |> as x + |> union all ( + table catalog_sales + |> select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price) + |> select + sold_date_sk, + sales_price), +wswscs as + (table wscs + |> join date_dim + |> where d_date_sk = sold_date_sk + |> aggregate + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + group by d_week_seq) +table wswscs +|> join date_dim +|> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 +|> select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 +|> as y +|> join ( + table wswscs + |> join date_dim + |> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1 + |> select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + |> as z) +|> where d_week_seq1 = d_week_seq2 - 53 +|> order by d_week_seq1 +|> select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`web_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 31, + "fragment" : "web_sales" + } ] +} + + +-- !query +select + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) sum_agg +from date_dim dt, store_sales, item +where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +group by dt.d_year, item.i_brand, item.i_brand_id +order by dt.d_year, sum_agg desc, brand_id +limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`date_dim`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 110, + "stopIndex" : 117, + "fragment" : "date_dim" + } ] +} + + +-- !query +table date_dim +|> as dt +|> join store_sales +|> join item +|> where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +|> aggregate sum(ss_ext_sales_price) sum_agg + group by dt.d_year d_year, item.i_brand_id brand_id, item.i_brand brand +|> order by d_year, sum_agg desc, brand_id +|> limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`date_dim`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 7, + "stopIndex" : 14, + "fragment" : "date_dim" + } ] +} + + +-- !query +select + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) as itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + over + (partition by i_class) as revenueratio +from + web_sales, item, date_dim +where + ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +group by + i_item_id, i_item_desc, i_category, i_class, i_current_price +order by + i_category, i_class, i_item_id, i_item_desc, revenueratio +limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`web_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 227, + "stopIndex" : 235, + "fragment" : "web_sales" + } ] +} + + +-- !query +table web_sales +|> join item +|> join date_dim +|> where ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +|> aggregate sum(ws_ext_sales_price) AS itemrevenue + group by i_item_id, i_item_desc, i_category, i_class, i_current_price +|> extend + itemrevenue * 100 / sum(itemrevenue) + over (partition by i_class) as revenueratio +|> order by i_category, i_class, i_item_id, i_item_desc, revenueratio +|> select i_item_desc, i_category, i_class, i_current_price, itemrevenue, revenueratio +|> limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`web_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 7, + "stopIndex" : 15, + "fragment" : "web_sales" + } ] +} + + +-- !query +select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +from (select * +from (select + item_sk, + rank() + over ( + order by rank_col asc) rnk +from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +from store_sales ss1 +where ss_store_sk = 4 +group by ss_item_sk +having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col +from store_sales +where ss_store_sk = 4 + and ss_addr_sk is null +group by ss_store_sk)) v1) v11 +where rnk < 11) asceding, + (select * + from (select + item_sk, + rank() + over ( + order by rank_col desc) rnk + from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_addr_sk is null + group by ss_store_sk)) v2) v21 + where rnk < 11) descending, + item i1, item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +order by asceding.rnk +limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`store_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 256, + "stopIndex" : 266, + "fragment" : "store_sales" + } ] +} + + +-- !query +table store_sales +|> as ss1 +|> where ss_store_sk = 4 +|> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk +|> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) +|> as v1 +|> select + item_sk, + rank() over ( + order by rank_col asc) rnk +|> as v11 +|> where rnk < 11 +|> as asceding +|> join ( + table store_sales + |> as ss1 + |> where ss_store_sk = 4 + |> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk + |> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) + |> as v2 + |> select + item_sk, + rank() over ( + order by rank_col asc) rnk + |> as v21 + |> where rnk < 11) descending +|> join item i1 +|> join item i2 +|> where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +|> order by asceding.rnk +|> select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`store_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 7, + "stopIndex" : 17, + "fragment" : "store_sales" + } ] +} + + +-- !query +with web_v1 as ( + select + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from web_sales, date_dim + where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + group by ws_item_sk, d_date), + store_v1 as ( + select + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + group by ss_item_sk, d_date) +select * +from (select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +from (select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + ) x) y +where web_cumulative > store_cumulative +order by item_sk, d_date +limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`web_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 222, + "stopIndex" : 230, + "fragment" : "web_sales" + } ] +} + + +-- !query +with web_v1 as ( + table web_sales + |> join date_dim + |> where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + |> aggregate sum(ws_sales_price) as sum_ws_sales_price + group by ws_item_sk as item_sk, d_date + |> extend sum(sum_ws_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales), +store_v1 as ( + table store_sales + |> join date_dim + |> where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + |> aggregate sum(ss_sales_price) as sum_ss_sales_price + group by ss_item_sk as item_sk, d_date + |> extend sum(sum_ss_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales) +table web_v1 +|> as web +|> full outer join store_v1 store + on (web.item_sk = store.item_sk and web.d_date = store.d_date) +|> select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +|> as x +|> select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +|> as y +|> where web_cumulative > store_cumulative +|> order by item_sk, d_date +|> limit 100 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`web_sales`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 34, + "fragment" : "web_sales" + } ] +} + + -- !query drop table t -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql index 6d0e490649d6f..924b42d9d3055 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql @@ -320,6 +320,99 @@ table t select col from st |> set col.i1 = 42; +-- DROP operators: positive tests. +------------------------------------ + +-- Dropping a column. +table t +|> drop y; + +-- Dropping two times. +select 1 as x, 2 as y, 3 as z +|> drop z, y; + +-- Dropping two times in sequence. +select 1 as x, 2 as y, 3 as z +|> drop z +|> drop y; + +-- Dropping all columns in the input relation. +select x from t +|> drop x; + +-- Dropping a backquoted column name with a dot inside. +table t +|> extend 1 as `x.y.z` +|> drop `x.y.z`; + +-- DROP operators: negative tests. +---------------------------------- + +-- Dropping a column that is not present in the input relation. +table t +|> drop z; + +-- Attempting to drop a struct field. +table st +|> drop col.i1; + +table st +|> drop `col.i1`; + +-- Duplicate fields in the drop list. +select 1 as x, 2 as y, 3 as z +|> drop z, y, z; + +-- AS operators: positive tests. +-------------------------------- + +-- Renaming a table. +table t +|> as u +|> select u.x, u.y; + +-- Renaming an input relation that is not a table. +select 1 as x, 2 as y +|> as u +|> select u.x, u.y; + +-- Renaming as a backquoted name including a period. +table t +|> as `u.v` +|> select `u.v`.x, `u.v`.y; + +-- Renaming two times. +table t +|> as u +|> as v +|> select v.x, v.y; + +-- Filtering by referring to the table or table subquery alias. +table t +|> as u +|> where u.x = 1; + +-- AS operators: negative tests. +-------------------------------- + +-- Multiple aliases are not supported. +table t +|> as u, v; + +-- Expressions are not supported. +table t +|> as 1 + 2; + +-- Renaming as an invalid name. +table t +|> as u-v; + +table t +|> as u@v; + +table t +|> as u#######v; + -- WHERE operators: positive tests. ----------------------------------- @@ -1040,6 +1133,505 @@ table windowTestData |> select cate, val, sum(val) over w as sum_val window w as (order by val); +-- Exercise SQL compilation using a subset of TPC-DS table schemas. +------------------------------------------------------------------- + +-- Q1 +with customer_total_return as +(select + sr_customer_sk as ctr_customer_sk, + sr_store_sk as ctr_store_sk, + sum(sr_return_amt) as ctr_total_return + from store_returns, date_dim + where sr_returned_date_sk = d_date_sk and d_year = 2000 + group by sr_customer_sk, sr_store_sk) +select c_customer_id +from customer_total_return ctr1, store, customer +where ctr1.ctr_total_return > + (select avg(ctr_total_return) * 1.2 + from customer_total_return ctr2 + where ctr1.ctr_store_sk = ctr2.ctr_store_sk) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100; + +with customer_total_return as + (table store_returns + |> join date_dim + |> where sr_returned_date_sk = d_date_sk and d_year = 2000 + |> aggregate sum(sr_return_amt) as ctr_total_return + group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) +table customer_total_return +|> as ctr1 +|> join store +|> join customer +|> where ctr1.ctr_total_return > + (table customer_total_return + |> as ctr2 + |> where ctr1.ctr_store_sk = ctr2.ctr_store_sk + |> aggregate avg(ctr_total_return) * 1.2) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +|> order by c_customer_id +|> limit 100 +|> select c_customer_id; + +-- Q2 +with wscs as +( select + sold_date_sk, + sales_price + from (select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + from web_sales) x + union all + (select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + ( select + d_week_seq, + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + from wscs, date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) +select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +from + (select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001) y, + (select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001 + 1) z +where d_week_seq1 = d_week_seq2 - 53 +order by d_week_seq1; + +with wscs as + (table web_sales + |> select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + |> as x + |> union all ( + table catalog_sales + |> select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price) + |> select + sold_date_sk, + sales_price), +wswscs as + (table wscs + |> join date_dim + |> where d_date_sk = sold_date_sk + |> aggregate + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + group by d_week_seq) +table wswscs +|> join date_dim +|> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 +|> select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 +|> as y +|> join ( + table wswscs + |> join date_dim + |> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1 + |> select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + |> as z) +|> where d_week_seq1 = d_week_seq2 - 53 +|> order by d_week_seq1 +|> select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2); + +-- Q3 +select + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) sum_agg +from date_dim dt, store_sales, item +where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +group by dt.d_year, item.i_brand, item.i_brand_id +order by dt.d_year, sum_agg desc, brand_id +limit 100; + +table date_dim +|> as dt +|> join store_sales +|> join item +|> where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +|> aggregate sum(ss_ext_sales_price) sum_agg + group by dt.d_year d_year, item.i_brand_id brand_id, item.i_brand brand +|> order by d_year, sum_agg desc, brand_id +|> limit 100; + +-- Q12 +select + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) as itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + over + (partition by i_class) as revenueratio +from + web_sales, item, date_dim +where + ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +group by + i_item_id, i_item_desc, i_category, i_class, i_current_price +order by + i_category, i_class, i_item_id, i_item_desc, revenueratio +limit 100; + +table web_sales +|> join item +|> join date_dim +|> where ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +|> aggregate sum(ws_ext_sales_price) AS itemrevenue + group by i_item_id, i_item_desc, i_category, i_class, i_current_price +|> extend + itemrevenue * 100 / sum(itemrevenue) + over (partition by i_class) as revenueratio +|> order by i_category, i_class, i_item_id, i_item_desc, revenueratio +|> select i_item_desc, i_category, i_class, i_current_price, itemrevenue, revenueratio +|> limit 100; + +-- Q44 +select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +from (select * +from (select + item_sk, + rank() + over ( + order by rank_col asc) rnk +from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +from store_sales ss1 +where ss_store_sk = 4 +group by ss_item_sk +having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col +from store_sales +where ss_store_sk = 4 + and ss_addr_sk is null +group by ss_store_sk)) v1) v11 +where rnk < 11) asceding, + (select * + from (select + item_sk, + rank() + over ( + order by rank_col desc) rnk + from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_addr_sk is null + group by ss_store_sk)) v2) v21 + where rnk < 11) descending, + item i1, item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +order by asceding.rnk +limit 100; + +table store_sales +|> as ss1 +|> where ss_store_sk = 4 +|> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk +|> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) +|> as v1 +|> select + item_sk, + rank() over ( + order by rank_col asc) rnk +|> as v11 +|> where rnk < 11 +|> as asceding +|> join ( + table store_sales + |> as ss1 + |> where ss_store_sk = 4 + |> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk + |> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) + |> as v2 + |> select + item_sk, + rank() over ( + order by rank_col asc) rnk + |> as v21 + |> where rnk < 11) descending +|> join item i1 +|> join item i2 +|> where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +|> order by asceding.rnk +|> select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing; + +-- Q51 +with web_v1 as ( + select + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from web_sales, date_dim + where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + group by ws_item_sk, d_date), + store_v1 as ( + select + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + group by ss_item_sk, d_date) +select * +from (select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +from (select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + ) x) y +where web_cumulative > store_cumulative +order by item_sk, d_date +limit 100; + +with web_v1 as ( + table web_sales + |> join date_dim + |> where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + |> aggregate sum(ws_sales_price) as sum_ws_sales_price + group by ws_item_sk as item_sk, d_date + |> extend sum(sum_ws_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales), +store_v1 as ( + table store_sales + |> join date_dim + |> where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + |> aggregate sum(ss_sales_price) as sum_ss_sales_price + group by ss_item_sk as item_sk, d_date + |> extend sum(sum_ss_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales) +table web_v1 +|> as web +|> full outer join store_v1 store + on (web.item_sk = store.item_sk and web.d_date = store.d_date) +|> select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +|> as x +|> select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +|> as y +|> where web_cumulative > store_cumulative +|> order by item_sk, d_date +|> limit 100; + -- Cleanup. ----------- drop table t; diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index 85a411f60fe22..570b61f388ead 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -873,6 +873,281 @@ org.apache.spark.sql.catalyst.parser.ParseException } +-- !query +table t +|> drop y +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z, y +-- !query schema +struct +-- !query output +1 + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z +|> drop y +-- !query schema +struct +-- !query output +1 + + +-- !query +select x from t +|> drop x +-- !query schema +struct<> +-- !query output + + + +-- !query +table t +|> extend 1 as `x.y.z` +|> drop `x.y.z` +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> drop z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`z`", + "proposal" : "`x`, `y`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 17, + "fragment" : "table t\n|> drop z" + } ] +} + + +-- !query +table st +|> drop col.i1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +table st +|> drop `col.i1` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`col.i1`", + "proposal" : "`col`, `x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 25, + "fragment" : "table st\n|> drop `col.i1`" + } ] +} + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z, y, z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "EXCEPT_OVERLAPPING_COLUMNS", + "sqlState" : "42702", + "messageParameters" : { + "columns" : "z, y, z" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "select 1 as x, 2 as y, 3 as z\n|> drop z, y, z" + } ] +} + + +-- !query +table t +|> as u +|> select u.x, u.y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +select 1 as x, 2 as y +|> as u +|> select u.x, u.y +-- !query schema +struct +-- !query output +1 2 + + +-- !query +table t +|> as `u.v` +|> select `u.v`.x, `u.v`.y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> as u +|> as v +|> select v.x, v.y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> as u +|> where u.x = 1 +-- !query schema +struct +-- !query output +1 def + + +-- !query +table t +|> as u, v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "','", + "hint" : "" + } +} + + +-- !query +table t +|> as 1 + 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : "" + } +} + + +-- !query +table t +|> as u-v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_IDENTIFIER", + "sqlState" : "42602", + "messageParameters" : { + "ident" : "u-v" + } +} + + +-- !query +table t +|> as u@v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'@'", + "hint" : "" + } +} + + +-- !query +table t +|> as u#######v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'#'", + "hint" : "" + } +} + + -- !query table t |> where true @@ -2771,7 +3046,7 @@ org.apache.spark.sql.catalyst.parser.ParseException "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", "sqlState" : "0A000", "messageParameters" : { - "case" : "window functions" + "case" : "window functions; please update the query to move the window functions to a subsequent |> SELECT operator instead" }, "queryContext" : [ { "objectType" : "", @@ -3041,6 +3316,568 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +with customer_total_return as +(select + sr_customer_sk as ctr_customer_sk, + sr_store_sk as ctr_store_sk, + sum(sr_return_amt) as ctr_total_return + from store_returns, date_dim + where sr_returned_date_sk = d_date_sk and d_year = 2000 + group by sr_customer_sk, sr_store_sk) +select c_customer_id +from customer_total_return ctr1, store, customer +where ctr1.ctr_total_return > + (select avg(ctr_total_return) * 1.2 + from customer_total_return ctr2 + where ctr1.ctr_store_sk = ctr2.ctr_store_sk) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +with customer_total_return as + (table store_returns + |> join date_dim + |> where sr_returned_date_sk = d_date_sk and d_year = 2000 + |> aggregate sum(sr_return_amt) as ctr_total_return + group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) +table customer_total_return +|> as ctr1 +|> join store +|> join customer +|> where ctr1.ctr_total_return > + (table customer_total_return + |> as ctr2 + |> where ctr1.ctr_store_sk = ctr2.ctr_store_sk + |> aggregate avg(ctr_total_return) * 1.2) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +|> order by c_customer_id +|> limit 100 +|> select c_customer_id +-- !query schema +struct +-- !query output + + + +-- !query +with wscs as +( select + sold_date_sk, + sales_price + from (select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + from web_sales) x + union all + (select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + ( select + d_week_seq, + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + from wscs, date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) +select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +from + (select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001) y, + (select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001 + 1) z +where d_week_seq1 = d_week_seq2 - 53 +order by d_week_seq1 +-- !query schema +struct +-- !query output + + + +-- !query +with wscs as + (table web_sales + |> select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + |> as x + |> union all ( + table catalog_sales + |> select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price) + |> select + sold_date_sk, + sales_price), +wswscs as + (table wscs + |> join date_dim + |> where d_date_sk = sold_date_sk + |> aggregate + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + group by d_week_seq) +table wswscs +|> join date_dim +|> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 +|> select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 +|> as y +|> join ( + table wswscs + |> join date_dim + |> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1 + |> select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + |> as z) +|> where d_week_seq1 = d_week_seq2 - 53 +|> order by d_week_seq1 +|> select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +-- !query schema +struct +-- !query output + + + +-- !query +select + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) sum_agg +from date_dim dt, store_sales, item +where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +group by dt.d_year, item.i_brand, item.i_brand_id +order by dt.d_year, sum_agg desc, brand_id +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +table date_dim +|> as dt +|> join store_sales +|> join item +|> where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +|> aggregate sum(ss_ext_sales_price) sum_agg + group by dt.d_year d_year, item.i_brand_id brand_id, item.i_brand brand +|> order by d_year, sum_agg desc, brand_id +|> limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +select + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) as itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + over + (partition by i_class) as revenueratio +from + web_sales, item, date_dim +where + ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +group by + i_item_id, i_item_desc, i_category, i_class, i_current_price +order by + i_category, i_class, i_item_id, i_item_desc, revenueratio +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +table web_sales +|> join item +|> join date_dim +|> where ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +|> aggregate sum(ws_ext_sales_price) AS itemrevenue + group by i_item_id, i_item_desc, i_category, i_class, i_current_price +|> extend + itemrevenue * 100 / sum(itemrevenue) + over (partition by i_class) as revenueratio +|> order by i_category, i_class, i_item_id, i_item_desc, revenueratio +|> select i_item_desc, i_category, i_class, i_current_price, itemrevenue, revenueratio +|> limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +from (select * +from (select + item_sk, + rank() + over ( + order by rank_col asc) rnk +from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +from store_sales ss1 +where ss_store_sk = 4 +group by ss_item_sk +having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col +from store_sales +where ss_store_sk = 4 + and ss_addr_sk is null +group by ss_store_sk)) v1) v11 +where rnk < 11) asceding, + (select * + from (select + item_sk, + rank() + over ( + order by rank_col desc) rnk + from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_addr_sk is null + group by ss_store_sk)) v2) v21 + where rnk < 11) descending, + item i1, item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +order by asceding.rnk +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +table store_sales +|> as ss1 +|> where ss_store_sk = 4 +|> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk +|> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) +|> as v1 +|> select + item_sk, + rank() over ( + order by rank_col asc) rnk +|> as v11 +|> where rnk < 11 +|> as asceding +|> join ( + table store_sales + |> as ss1 + |> where ss_store_sk = 4 + |> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk + |> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) + |> as v2 + |> select + item_sk, + rank() over ( + order by rank_col asc) rnk + |> as v21 + |> where rnk < 11) descending +|> join item i1 +|> join item i2 +|> where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +|> order by asceding.rnk +|> select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +-- !query schema +struct +-- !query output + + + +-- !query +with web_v1 as ( + select + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from web_sales, date_dim + where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + group by ws_item_sk, d_date), + store_v1 as ( + select + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + group by ss_item_sk, d_date) +select * +from (select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +from (select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + ) x) y +where web_cumulative > store_cumulative +order by item_sk, d_date +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +with web_v1 as ( + table web_sales + |> join date_dim + |> where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + |> aggregate sum(ws_sales_price) as sum_ws_sales_price + group by ws_item_sk as item_sk, d_date + |> extend sum(sum_ws_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales), +store_v1 as ( + table store_sales + |> join date_dim + |> where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + |> aggregate sum(ss_sales_price) as sum_ss_sales_price + group by ss_item_sk as item_sk, d_date + |> extend sum(sum_ss_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales) +table web_v1 +|> as web +|> full outer join store_v1 store + on (web.item_sk = store.item_sk and web.d_date = store.d_date) +|> select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +|> as x +|> select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +|> as y +|> where web_cumulative > store_cumulative +|> order by item_sk, d_date +|> limit 100 +-- !query schema +struct +-- !query output + + + -- !query drop table t -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 5c56377f21c20..575a4ae69d1a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -132,7 +132,7 @@ import org.apache.spark.util.Utils // scalastyle:on line.size.limit @ExtendedSQLTest class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper - with SQLQueryTestHelper { + with SQLQueryTestHelper with TPCDSSchema { import IntegratedUDFTestUtils._ @@ -165,13 +165,17 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper protected def ignoreList: Set[String] = Set( "ignored.sql" // Do NOT remove this one. It is here to test the ignore functionality. ) ++ otherIgnoreList + /** List of test cases that require TPCDS table schemas to be loaded. */ + private def requireTPCDSCases: Seq[String] = Seq("pipe-operators.sql") + /** List of TPCDS table names and schemas to load from the [[TPCDSSchema]] base class. */ + private val tpcDSTableNamesToSchemas: Map[String, String] = tableColumns // Create all the test cases. listTestCases.foreach(createScalaTestCase) protected def createScalaTestCase(testCase: TestCase): Unit = { if (ignoreList.exists(t => - testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { + testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { // Create a test case to ignore this case. ignore(testCase.name) { /* Do nothing */ } } else testCase match { @@ -322,6 +326,15 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper setOperations.foreach(localSparkSession.sql) } + // Load TPCDS table schemas for the test case if required. + val lowercaseTestCase = testCase.name.toLowerCase(Locale.ROOT) + if (requireTPCDSCases.contains(lowercaseTestCase)) { + tpcDSTableNamesToSchemas.foreach { case (name: String, schema: String) => + localSparkSession.sql(s"DROP TABLE IF EXISTS $name") + localSparkSession.sql(s"CREATE TABLE `$name` ($schema) USING parquet") + } + } + // Run the SQL queries preparing them for comparison. val outputs: Seq[QueryTestOutput] = queries.map { sql => testCase match { @@ -348,6 +361,13 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper } } + // Drop TPCDS tables after the test case if required. + if (requireTPCDSCases.contains(lowercaseTestCase)) { + tpcDSTableNamesToSchemas.foreach { case (name: String, schema: String) => + localSparkSession.sql(s"DROP TABLE IF EXISTS $name") + } + } + if (regenerateGoldenFiles) { // Again, we are explicitly not using multi-line string due to stripMargin removing "|". val goldenOutput = { From 6add9c89855f9311d5e185774ddddcbf4323beee Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 5 Dec 2024 10:45:02 -0800 Subject: [PATCH 106/438] [SPARK-50501][BUILD] Update cross-spawn to surpress a warning in lint ### What changes were proposed in this pull request? This PR updates cross-spawn from 7.0.3 to 7.0.6. ### Why are the changes needed? Recently, `./dev/lint-js` shows a warning like as follows. ``` 1 high severity vulnerability To address all issues, run: npm audit fix Run `npm audit` for details. ``` We can see the same message in CI. https://github.com/apache/spark/actions/runs/12183892848/job/33986553884#step:24:20 To inspect more, I executed `npm audit` and the result is: ``` cross-spawn 7.0.0 - 7.0.4 Severity: high Regular Expression Denial of Service (ReDoS) in cross-spawn - https://github.com/advisories/GHSA-3xgq-45jj-v275 fix available via `npm audit fix` node_modules/cross-spawn 1 high severity vulnerability ``` `cross-spawn` is a dependency for the linter but the linter reports the issue as `high severity vulnerability` so let's fix it just in case. This change is done by `npm audit fix` as the warning indicates. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./dev/lint-js` doesn't show the warning. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49077 from sarutak/update-cross-spawn. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- dev/package-lock.json | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/dev/package-lock.json b/dev/package-lock.json index f676b9cec0762..e6ec1406a7620 100644 --- a/dev/package-lock.json +++ b/dev/package-lock.json @@ -4,6 +4,7 @@ "requires": true, "packages": { "": { + "name": "dev", "devDependencies": { "ansi-regex": "^5.0.1", "eslint": "^7.25.0", @@ -316,10 +317,11 @@ "dev": true }, "node_modules/cross-spawn": { - "version": "7.0.3", - "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", - "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "version": "7.0.6", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.6.tgz", + "integrity": "sha512-uV2QOWP2nWzsy2aMp8aRibhi9dlzF5Hgh5SHaB9OiTGEyDTiJJyx0uy51QXdyWbtAHNua4XJzUKca3OzKUd3vA==", "dev": true, + "license": "MIT", "dependencies": { "path-key": "^3.1.0", "shebang-command": "^2.0.0", @@ -1469,9 +1471,9 @@ "dev": true }, "cross-spawn": { - "version": "7.0.3", - "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", - "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "version": "7.0.6", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.6.tgz", + "integrity": "sha512-uV2QOWP2nWzsy2aMp8aRibhi9dlzF5Hgh5SHaB9OiTGEyDTiJJyx0uy51QXdyWbtAHNua4XJzUKca3OzKUd3vA==", "dev": true, "requires": { "path-key": "^3.1.0", From 21451fb312fee32188b6d24f406cb4f3a8349414 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 5 Dec 2024 14:43:39 -0800 Subject: [PATCH 107/438] [SPARK-50505][DOCS] Fix `spark.storage.replication.proactive` default value documentation ### What changes were proposed in this pull request? This PR aims to fix `spark.storage.replication.proactive` default value documentation. ### Why are the changes needed? `spark.storage.replication.proactive` has been enabled by default since Apache Spark 3.2.0. https://github.com/apache/spark/blob/6add9c89855f9311d5e185774ddddcbf4323beee/docs/core-migration-guide.md?plain=1#L85 https://github.com/apache/spark/blob/6add9c89855f9311d5e185774ddddcbf4323beee/core/src/main/scala/org/apache/spark/internal/config/package.scala#L494-L502 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49081 from dongjoon-hyun/SPARK-50505. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index e095ae7a61b22..6957ca9a03d23 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2116,7 +2116,7 @@ Apart from these, the following properties are also available, and may be useful - + - + - + @@ -453,7 +453,7 @@ SPARK_WORKER_OPTS supports the following system properties: @@ -493,8 +493,8 @@ SPARK_WORKER_OPTS supports the following system properties: @@ -504,8 +504,8 @@ SPARK_WORKER_OPTS supports the following system properties: @@ -523,11 +523,11 @@ SPARK_WORKER_OPTS supports the following system properties: - + @@ -611,37 +611,37 @@ via http://[host:port]/[version]/submissions/[action] where action is one of the following supported actions.
Thread IDfalse Set to true for applications that have higher security requirements and prefer that their - secret is not saved in the db. The shuffle data of such applications wll not be recovered after + secret is not saved in the db. The shuffle data of such applications will not be recovered after the External Shuffle Service restarts. 3.5.0false Set to true for applications that have higher security requirements and prefer that their - secret is not saved in the db. The shuffle data of such applications wll not be recovered after + secret is not saved in the db. The shuffle data of such applications will not be recovered after the External Shuffle Service restarts. 3.5.0 The pattern for app ID generation based on Java `String.format` method. The default value is `app-%s-%04d` which represents the existing app id string, e.g., - `app-20231031224509-0008`. Plesae be careful to generate unique IDs. + `app-20231031224509-0008`. Please be careful to generate unique IDs. 4.0.0
spark.storage.replication.proactivefalsetrue Enables proactive block replication for RDD blocks. Cached RDD block replicas lost due to executor failures are replenished if there are any existing available replicas. This tries From fc69194fc03212035f4b42701dcbc409e5a36b03 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 6 Dec 2024 09:37:25 +0900 Subject: [PATCH 108/438] [SPARK-50310][CONNECT][PYTHON] Call `with_origin_to_class` when the `Column` initializing ### What changes were proposed in this pull request? This PR followups https://github.com/apache/spark/pull/48964 to fix `with_origin_to_class` to be called when the `Column` actually initializing ### Why are the changes needed? We don't need call `with_origin_to_class` if the Column API is not used. Furthermore, the current way is not working properly from Spark Connect as it causes circular import error as mentioned from https://github.com/apache/spark/pull/48964#discussion_r1868569335 so this is more safer way to support `with_origin_to_class`. ### Does this PR introduce _any_ user-facing change? No API changes, but the Spark Connect Python client no longer issues circular import error when initializing. ### How was this patch tested? Manually tested **Before (Failed initializing Spark Connect Python client)** ``` % ./bin/pyspark --remote local Python 3.9.17 (main, Jul 5 2023, 15:35:09) [Clang 14.0.6 ] :: Anaconda, Inc. on darwin Type "help", "copyright", "credits" or "license" for more information. ... ImportError: cannot import name 'SparkSession' from partially initialized module 'pyspark.sql.connect.session' (most likely due to a circular import) (/Users/haejoon.lee/Desktop/git_repos/spark/python/pyspark/sql/connect/session.py) ``` **After (Successfully initializing Spark Connect Python client)** ``` % ./bin/pyspark --remote local Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /__ / .__/\_,_/_/ /_/\_\ version 4.0.0.dev0 /_/ Using Python version 3.9.17 (main, Jul 5 2023 15:35:09) Client connected to the Spark Connect server at localhost SparkSession available as 'spark'. >>> spark ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49054 from itholic/SPARK-50310-connect. Authored-by: Haejoon Lee Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/classic/column.py | 10 ++++++++-- python/pyspark/sql/connect/column.py | 10 ++++++++-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/classic/column.py b/python/pyspark/sql/classic/column.py index c08eac7f6a049..05fcb21628222 100644 --- a/python/pyspark/sql/classic/column.py +++ b/python/pyspark/sql/classic/column.py @@ -33,7 +33,6 @@ from pyspark.sql.column import Column as ParentColumn from pyspark.errors import PySparkAttributeError, PySparkTypeError, PySparkValueError -from pyspark.errors.utils import with_origin_to_class from pyspark.sql.types import DataType from pyspark.sql.utils import get_active_spark_context, enum_to_value @@ -175,12 +174,19 @@ def _reverse_op( return Column(jc) -@with_origin_to_class class Column(ParentColumn): def __new__( cls, jc: "JavaObject", ) -> "Column": + # We apply `with_origin_to_class` decorator here instead of top of the class definition + # to prevent circular import issue when initializing the SparkSession. + # See https://github.com/apache/spark/pull/49054 for more detail. + from pyspark.errors.utils import with_origin_to_class + + if not hasattr(cls, "_with_origin_applied"): + cls = with_origin_to_class(cls) + cls._with_origin_applied = True self = object.__new__(cls) self.__init__(jc) # type: ignore[misc] return self diff --git a/python/pyspark/sql/connect/column.py b/python/pyspark/sql/connect/column.py index e840081146340..1440c4c2792bf 100644 --- a/python/pyspark/sql/connect/column.py +++ b/python/pyspark/sql/connect/column.py @@ -52,7 +52,6 @@ WithField, DropField, ) -from pyspark.errors.utils import with_origin_to_class if TYPE_CHECKING: @@ -107,12 +106,19 @@ def _to_expr(v: Any) -> Expression: return v._expr if isinstance(v, Column) else LiteralExpression._from_value(v) -@with_origin_to_class(["to_plan"]) class Column(ParentColumn): def __new__( cls, expr: "Expression", ) -> "Column": + # We apply `with_origin_to_class` decorator here instead of top of the class definition + # to prevent circular import issue when initializing the SparkSession. + # See https://github.com/apache/spark/pull/49054 for more detail. + from pyspark.errors.utils import with_origin_to_class + + if not hasattr(cls, "_with_origin_applied"): + cls = with_origin_to_class(["to_plan"])(cls) + cls._with_origin_applied = True self = object.__new__(cls) self.__init__(expr) # type: ignore[misc] return self From a435b2e634bd4fc675dee790961d38423e9bef6a Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 6 Dec 2024 09:46:16 +0900 Subject: [PATCH 109/438] [SPARK-50489][SQL][PYTHON][FOLLOW-UP] Add applyInArrow in `DeduplicateRelations#collectConflictPlans` ### What changes were proposed in this pull request? Add applyInArrow in `DeduplicateRelations#collectConflictPlans` ### Why are the changes needed? In https://github.com/apache/spark/pull/49056, I forgot to add `applyInArrow` in `DeduplicateRelations#collectConflictPlans` ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? tests added in https://github.com/apache/spark/pull/49056 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49069 from zhengruifeng/apply_in_arrow_rule. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/analysis/DeduplicateRelations.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 52be631d94d85..8398fb8d1e830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -392,12 +392,24 @@ object DeduplicateRelations extends Rule[LogicalPlan] { newVersion.copyTagsFrom(oldVersion) Seq((oldVersion, newVersion)) + case oldVersion @ FlatMapGroupsInArrow(_, _, output, _) + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + val newVersion = oldVersion.copy(output = output.map(_.newInstance())) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) + case oldVersion @ FlatMapCoGroupsInPandas(_, _, _, output, _, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => val newVersion = oldVersion.copy(output = output.map(_.newInstance())) newVersion.copyTagsFrom(oldVersion) Seq((oldVersion, newVersion)) + case oldVersion @ FlatMapCoGroupsInArrow(_, _, _, output, _, _) + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + val newVersion = oldVersion.copy(output = output.map(_.newInstance())) + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) + case oldVersion @ MapInPandas(_, output, _, _, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => val newVersion = oldVersion.copy(output = output.map(_.newInstance())) From d67ca731f19cc571e8d69245f4837c0cf28b83ae Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Fri, 6 Dec 2024 10:40:50 +0900 Subject: [PATCH 110/438] [SPARK-50132][SQL][PYTHON] Add DataFrame API for Lateral Joins ### What changes were proposed in this pull request? Adds DataFrame API for Lateral Joins. #### Examples: For the following DataFrames `customers` and `orders`: ```py >>> customers.printSchema() root |-- customer_id: long (nullable = true) |-- name: string (nullable = true) >>> orders.printSchema() root |-- order_id: long (nullable = true) |-- customer_id: long (nullable = true) |-- order_date: string (nullable = true) |-- items: array (nullable = true) | |-- element: struct (containsNull = true) | | |-- product: string (nullable = true) | | |-- quantity: long (nullable = true) ``` ##### Using TVF ```py # select customer_id, name, order_id, order_date, product, quantity # from customers join orders using (customer_id) join lateral (select col.* from explode(items)) # order by customer_id, order_id, product customers.join(orders, "customer_id").lateralJoin( spark.tvf.explode(sf.col("items").outer()).select("col.*") ).select( "customer_id", "name", "order_id", "order_date", "product", "quantity" ).orderBy("customer_id", "order_id", "product").show() ``` ##### Using Subquery ```py # select c.customer_id, name, order_id, order_date # from customers c left join lateral ( # select * from orders o where o.customer_id = c.customer_id order by order_date desc limit 2 # ) # order by customer_id, order_id customers.alias("c").lateralJoin( orders.alias("o") .where(sf.col("o.customer_id") == sf.col("c.customer_id").outer()) .orderBy(sf.col("order_date").desc()) .limit(2), how="left" ).select( "c.customer_id", "name", "order_id", "order_date" ).orderBy("customer_id", "order_id").show() ``` ### Why are the changes needed? Lateral Join APIs are missing in DataFrame API. ### Does this PR introduce _any_ user-facing change? Yes, new DataFrame APIs for lateral join will be available. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49033 from ueshin/issues/SPARK-50132/lateral_join. Lead-authored-by: Takuya Ueshin Co-authored-by: Takuya UESHIN Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/Dataset.scala | 15 + python/pyspark/sql/classic/dataframe.py | 16 + python/pyspark/sql/connect/dataframe.py | 13 + python/pyspark/sql/dataframe.py | 103 ++++++ .../sql/tests/connect/test_parity_tvf.py | 40 +- .../sql/tests/connect/test_parity_udtf.py | 8 + python/pyspark/sql/tests/test_subquery.py | 332 +++++++++++++++++ python/pyspark/sql/tests/test_tvf.py | 349 ++++++++++++++++++ python/pyspark/sql/tests/test_udtf.py | 31 ++ .../org/apache/spark/sql/api/Dataset.scala | 54 +++ .../scala/org/apache/spark/sql/Dataset.scala | 32 ++ .../spark/sql/DataFrameSubquerySuite.scala | 287 ++++++++++++++ .../DataFrameTableValuedFunctionsSuite.scala | 260 +++++++++++++ 13 files changed, 1539 insertions(+), 1 deletion(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 631e9057f8d15..eb166a1e8003b 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -383,6 +383,21 @@ class Dataset[T] private[sql] ( } } + // TODO(SPARK-50134): Support Lateral Join API in Spark Connect + // scalastyle:off not.implemented.error.usage + /** @inheritdoc */ + def lateralJoin(right: DS[_]): DataFrame = ??? + + /** @inheritdoc */ + def lateralJoin(right: DS[_], joinExprs: Column): DataFrame = ??? + + /** @inheritdoc */ + def lateralJoin(right: DS[_], joinType: String): DataFrame = ??? + + /** @inheritdoc */ + def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = ??? + // scalastyle:on not.implemented.error.usage + override protected def sortInternal(global: Boolean, sortCols: Seq[Column]): Dataset[T] = { val sortExprs = sortCols.map { c => ColumnNodeToProtoConverter(c.sortOrder).getSortOrder diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py index 169755c753907..05c19913adf30 100644 --- a/python/pyspark/sql/classic/dataframe.py +++ b/python/pyspark/sql/classic/dataframe.py @@ -715,6 +715,22 @@ def join( jdf = self._jdf.join(other._jdf, on, how) return DataFrame(jdf, self.sparkSession) + def lateralJoin( + self, + other: ParentDataFrame, + on: Optional[Column] = None, + how: Optional[str] = None, + ) -> ParentDataFrame: + if on is None and how is None: + jdf = self._jdf.lateralJoin(other._jdf) + elif on is None: + jdf = self._jdf.lateralJoin(other._jdf, how) + elif how is None: + jdf = self._jdf.lateralJoin(other._jdf, on._jc) + else: + jdf = self._jdf.lateralJoin(other._jdf, on._jc, how) + return DataFrame(jdf, self.sparkSession) + # TODO(SPARK-22947): Fix the DataFrame API. def _joinAsOf( self, diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index e85efeb592dff..124ce5e0d39a5 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -686,6 +686,18 @@ def join( session=self._session, ) + def lateralJoin( + self, + other: ParentDataFrame, + on: Optional[Column] = None, + how: Optional[str] = None, + ) -> ParentDataFrame: + # TODO(SPARK-50134): Implement this method + raise PySparkNotImplementedError( + errorClass="NOT_IMPLEMENTED", + messageParameters={"feature": "lateralJoin()"}, + ) + def _joinAsOf( self, other: ParentDataFrame, @@ -2265,6 +2277,7 @@ def _test() -> None: # TODO(SPARK-50134): Support subquery in connect del pyspark.sql.dataframe.DataFrame.scalar.__doc__ del pyspark.sql.dataframe.DataFrame.exists.__doc__ + del pyspark.sql.dataframe.DataFrame.lateralJoin.__doc__ globs["spark"] = ( PySparkSession.builder.appName("sql.connect.dataframe tests") diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 0ea0eef50c0f3..ccb9806cc76dc 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -2629,6 +2629,109 @@ def join( """ ... + def lateralJoin( + self, + other: "DataFrame", + on: Optional[Column] = None, + how: Optional[str] = None, + ) -> "DataFrame": + """ + Lateral joins with another :class:`DataFrame`, using the given join expression. + + A lateral join (also known as a correlated join) is a type of join where each row from + one DataFrame is used as input to a subquery or a derived table that computes a result + specific to that row. The right side `DataFrame` can reference columns from the current + row of the left side `DataFrame`, allowing for more complex and context-dependent results + than a standard join. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + other : :class:`DataFrame` + Right side of the join + on : :class:`Column`, optional + a join expression (Column). + how : str, optional + default ``inner``. Must be one of: ``inner``, ``cross``, ``left``, ``leftouter``, + and ``left_outer``. + + Returns + ------- + :class:`DataFrame` + Joined DataFrame. + + Examples + -------- + Setup a sample DataFrame. + + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql import Row + >>> customers_data = [ + ... Row(customer_id=1, name="Alice"), Row(customer_id=2, name="Bob"), + ... Row(customer_id=3, name="Charlie"), Row(customer_id=4, name="Diana") + ... ] + >>> customers = spark.createDataFrame(customers_data) + >>> orders_data = [ + ... Row(order_id=101, customer_id=1, order_date="2024-01-10", + ... items=[Row(product="laptop", quantity=5), Row(product="mouse", quantity=12)]), + ... Row(order_id=102, customer_id=1, order_date="2024-02-15", + ... items=[Row(product="phone", quantity=2), Row(product="charger", quantity=15)]), + ... Row(order_id=105, customer_id=1, order_date="2024-03-20", + ... items=[Row(product="tablet", quantity=4)]), + ... Row(order_id=103, customer_id=2, order_date="2024-01-12", + ... items=[Row(product="tablet", quantity=8)]), + ... Row(order_id=104, customer_id=2, order_date="2024-03-05", + ... items=[Row(product="laptop", quantity=7)]), + ... Row(order_id=106, customer_id=3, order_date="2024-04-05", + ... items=[Row(product="monitor", quantity=1)]), + ... ] + >>> orders = spark.createDataFrame(orders_data) + + Example 1 (use TVF): Expanding Items in Each Order into Separate Rows + + >>> customers.join(orders, "customer_id").lateralJoin( + ... spark.tvf.explode(sf.col("items").outer()).select("col.*") + ... ).select( + ... "customer_id", "name", "order_id", "order_date", "product", "quantity" + ... ).orderBy("customer_id", "order_id", "product").show() + +-----------+-------+--------+----------+-------+--------+ + |customer_id| name|order_id|order_date|product|quantity| + +-----------+-------+--------+----------+-------+--------+ + | 1| Alice| 101|2024-01-10| laptop| 5| + | 1| Alice| 101|2024-01-10| mouse| 12| + | 1| Alice| 102|2024-02-15|charger| 15| + | 1| Alice| 102|2024-02-15| phone| 2| + | 1| Alice| 105|2024-03-20| tablet| 4| + | 2| Bob| 103|2024-01-12| tablet| 8| + | 2| Bob| 104|2024-03-05| laptop| 7| + | 3|Charlie| 106|2024-04-05|monitor| 1| + +-----------+-------+--------+----------+-------+--------+ + + Example 2 (use subquery): Finding the Two Most Recent Orders for Customer + + >>> customers.alias("c").lateralJoin( + ... orders.alias("o") + ... .where(sf.col("o.customer_id") == sf.col("c.customer_id").outer()) + ... .orderBy(sf.col("order_date").desc()) + ... .limit(2), + ... how="left" + ... ).select( + ... "c.customer_id", "name", "order_id", "order_date" + ... ).orderBy("customer_id", "order_id").show() + +-----------+-------+--------+----------+ + |customer_id| name|order_id|order_date| + +-----------+-------+--------+----------+ + | 1| Alice| 102|2024-02-15| + | 1| Alice| 105|2024-03-20| + | 2| Bob| 103|2024-01-12| + | 2| Bob| 104|2024-03-05| + | 3|Charlie| 106|2024-04-05| + | 4| Diana| NULL| NULL| + +-----------+-------+--------+----------+ + """ + ... + # TODO(SPARK-22947): Fix the DataFrame API. @dispatch_df_method def _joinAsOf( diff --git a/python/pyspark/sql/tests/connect/test_parity_tvf.py b/python/pyspark/sql/tests/connect/test_parity_tvf.py index 61e3decf562c3..c5edff02810ff 100644 --- a/python/pyspark/sql/tests/connect/test_parity_tvf.py +++ b/python/pyspark/sql/tests/connect/test_parity_tvf.py @@ -21,7 +21,45 @@ class TVFParityTestsMixin(TVFTestsMixin, ReusedConnectTestCase): - pass + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_explode_with_lateral_join(self): + super().test_explode_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_explode_outer_with_lateral_join(self): + super().test_explode_outer_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_inline_with_lateral_join(self): + super().test_inline_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_inline_outer_with_lateral_join(self): + super().test_inline_outer_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_json_tuple_with_lateral_join(self): + super().test_json_tuple_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_posexplode_with_lateral_join(self): + super().test_posexplode_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_posexplode_outer_with_lateral_join(self): + super().test_posexplode_outer_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_stack_with_lateral_join(self): + super().test_stack_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_variant_explode_with_lateral_join(self): + super().test_variant_explode_with_lateral_join() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_variant_explode_outer_with_lateral_join(self): + super().test_variant_explode_outer_with_lateral_join() if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py b/python/pyspark/sql/tests/connect/test_parity_udtf.py index 6955e7377b4c4..29d1718fe3781 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udtf.py +++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py @@ -85,6 +85,14 @@ def _add_archive(self, path): def _add_file(self, path): self.spark.addArtifacts(path, file=True) + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_udtf_with_lateral_join_dataframe(self): + super().test_udtf_with_lateral_join_dataframe() + + @unittest.skip("SPARK-50134: Support Spark Connect") + def test_udtf_with_conditional_return_dataframe(self): + super().test_udtf_with_conditional_return_dataframe() + class ArrowUDTFParityTests(UDTFArrowTestsMixin, UDTFParityTests): @classmethod diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 7cc0360c39421..1b657e075c59e 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -484,6 +484,338 @@ def test_scalar_subquery_with_outer_reference_errors(self): fragment="col", ) + def table1(self): + t1 = self.spark.sql("VALUES (0, 1), (1, 2) AS t1(c1, c2)") + t1.createOrReplaceTempView("t1") + return self.spark.table("t1") + + def table2(self): + t2 = self.spark.sql("VALUES (0, 2), (0, 3) AS t2(c1, c2)") + t2.createOrReplaceTempView("t2") + return self.spark.table("t2") + + def table3(self): + t3 = self.spark.sql( + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) AS t3(c1, c2)" + ) + t3.createOrReplaceTempView("t3") + return self.spark.table("t3") + + def test_lateral_join_with_single_column_select(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin(self.spark.range(1).select(sf.col("c1").outer())), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT c1)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.select(sf.col("t1.c1").outer())), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.c1 FROM t2)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.select(sf.col("t1.c1").outer() + sf.col("t2.c1"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2)"""), + ) + + def test_lateral_join_with_different_join_types(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1).select( + (sf.col("c1").outer() + sf.col("c2").outer()).alias("c3") + ), + sf.col("c2") == sf.col("c3"), + ), + self.spark.sql( + """SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3""" + ), + ) + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1).select( + (sf.col("c1").outer() + sf.col("c2").outer()).alias("c3") + ), + sf.col("c2") == sf.col("c3"), + "left", + ), + self.spark.sql( + """SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3""" + ), + ) + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1).select( + (sf.col("c1").outer() + sf.col("c2").outer()).alias("c3") + ), + how="cross", + ), + self.spark.sql("""SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3)"""), + ) + + def test_lateral_join_with_correlated_predicates(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin( + t2.where(sf.col("t1.c1").outer() == sf.col("t2.c1")).select(sf.col("c2")) + ), + self.spark.sql( + """SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1)""" + ), + ) + assertDataFrameEqual( + t1.lateralJoin( + t2.where(sf.col("t1.c1").outer() < sf.col("t2.c1")).select(sf.col("c2")) + ), + self.spark.sql( + """SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 < t2.c1)""" + ), + ) + + def test_lateral_join_with_aggregation_and_correlated_predicates(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin( + t2.where(sf.col("t1.c2").outer() < sf.col("t2.c2")).select( + sf.max(sf.col("c2")).alias("m") + ) + ), + self.spark.sql( + """ + SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2) + """ + ), + ) + + def test_lateral_join_reference_preceding_from_clause_items(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.join(t2).lateralJoin( + self.spark.range(1).select(sf.col("t1.c2").outer() + sf.col("t2.c2").outer()) + ), + self.spark.sql("""SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2)"""), + ) + + def test_multiple_lateral_joins(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1).select( + (sf.col("c1").outer() + sf.col("c2").outer()).alias("a") + ) + ) + .lateralJoin( + self.spark.range(1).select( + (sf.col("c1").outer() - sf.col("c2").outer()).alias("b") + ) + ) + .lateralJoin( + self.spark.range(1).select( + (sf.col("a").outer() * sf.col("b").outer()).alias("c") + ) + ), + self.spark.sql( + """ + SELECT * FROM t1, + LATERAL (SELECT c1 + c2 AS a), + LATERAL (SELECT c1 - c2 AS b), + LATERAL (SELECT a * b AS c) + """ + ), + ) + + def test_lateral_join_in_between_regular_joins(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin( + t2.where(sf.col("t1.c1").outer() == sf.col("t2.c1")).select(sf.col("c2")), + how="left", + ).join(t1.alias("t3"), sf.col("t2.c2") == sf.col("t3.c2"), how="left"), + self.spark.sql( + """ + SELECT * FROM t1 + LEFT OUTER JOIN LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) s + LEFT OUTER JOIN t1 t3 ON s.c2 = t3.c2 + """ + ), + ) + + def test_nested_lateral_joins(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin(t2.lateralJoin(self.spark.range(1).select(sf.col("c1").outer()))), + self.spark.sql( + """SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT c1))""" + ), + ) + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1) + .select((sf.col("c1").outer() + sf.lit(1)).alias("c1")) + .lateralJoin(self.spark.range(1).select(sf.col("c1").outer())) + ), + self.spark.sql( + """ + SELECT * FROM t1, + LATERAL (SELECT * FROM (SELECT c1 + 1 AS c1), LATERAL (SELECT c1)) + """ + ), + ) + + def test_scalar_subquery_inside_lateral_join(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1).select( + sf.col("c2").outer(), t2.select(sf.min(sf.col("c2"))).scalar() + ) + ), + self.spark.sql( + """SELECT * FROM t1, LATERAL (SELECT c2, (SELECT MIN(c2) FROM t2))""" + ), + ) + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1) + .select(sf.col("c1").outer().alias("a")) + .select( + t2.where(sf.col("c1") == sf.col("a").outer()) + .select(sf.sum(sf.col("c2"))) + .scalar() + ) + ), + self.spark.sql( + """ + SELECT * FROM t1, LATERAL ( + SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = a) FROM (SELECT c1 AS a) + ) + """ + ), + ) + + def test_lateral_join_inside_subquery(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.where( + sf.col("c1") + == ( + t2.lateralJoin(self.spark.range(1).select(sf.col("c1").outer().alias("a"))) + .select(sf.min(sf.col("a"))) + .scalar() + ) + ), + self.spark.sql( + """ + SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a)) + """ + ), + ) + assertDataFrameEqual( + t1.where( + sf.col("c1") + == ( + t2.lateralJoin(self.spark.range(1).select(sf.col("c1").outer().alias("a"))) + .where(sf.col("c1") == sf.col("t1.c1").outer()) + .select(sf.min(sf.col("a"))) + .scalar() + ) + ), + self.spark.sql( + """ + SELECT * FROM t1 + WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a) WHERE c1 = t1.c1) + """ + ), + ) + + def test_lateral_join_with_table_valued_functions(self): + with self.tempView("t1", "t3"): + t1 = self.table1() + t3 = self.table3() + + assertDataFrameEqual( + t1.lateralJoin(self.spark.tvf.range(3)), + self.spark.sql("""SELECT * FROM t1, LATERAL RANGE(3)"""), + ) + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.explode(sf.array(sf.col("c1").outer(), sf.col("c2").outer())) + ).toDF("c1", "c2", "c3"), + self.spark.sql("""SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)"""), + ) + assertDataFrameEqual( + t3.lateralJoin(self.spark.tvf.explode_outer(sf.col("c2").outer())).toDF( + "c1", "c2", "v" + ), + self.spark.sql("""SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)"""), + ) + assertDataFrameEqual( + self.spark.tvf.explode(sf.array(sf.lit(1), sf.lit(2))) + .toDF("v") + .lateralJoin(self.spark.range(1).select((sf.col("v").outer() + 1).alias("v"))), + self.spark.sql( + """SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1 AS v)""" + ), + ) + + def test_lateral_join_with_table_valued_functions_and_join_conditions(self): + with self.tempView("t1", "t3"): + t1 = self.table1() + t3 = self.table3() + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.explode(sf.array(sf.col("c1").outer(), sf.col("c2").outer())), + sf.col("c1") == sf.col("col"), + ).toDF("c1", "c2", "c3"), + self.spark.sql( + """SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3""" + ), + ) + assertDataFrameEqual( + t3.lateralJoin( + self.spark.tvf.explode(sf.col("c2").outer()), + sf.col("c1") == sf.col("col"), + ).toDF("c1", "c2", "c3"), + self.spark.sql("""SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3"""), + ) + assertDataFrameEqual( + t3.lateralJoin( + self.spark.tvf.explode(sf.col("c2").outer()), + sf.col("c1") == sf.col("col"), + "left", + ).toDF("c1", "c2", "c3"), + self.spark.sql( + """SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3""" + ), + ) + class SubqueryTests(SubqueryTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/tests/test_tvf.py b/python/pyspark/sql/tests/test_tvf.py index 5c709437fc4db..ea20cbf9b8f3a 100644 --- a/python/pyspark/sql/tests/test_tvf.py +++ b/python/pyspark/sql/tests/test_tvf.py @@ -52,6 +52,37 @@ def test_explode(self): expected = self.spark.sql("""SELECT * FROM explode(null :: map)""") assertDataFrameEqual(actual=actual, expected=expected) + def test_explode_with_lateral_join(self): + with self.tempView("t1", "t2"): + t1 = self.spark.sql("VALUES (0, 1), (1, 2) AS t1(c1, c2)") + t1.createOrReplaceTempView("t1") + t3 = self.spark.sql( + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) " + "AS t3(c1, c2)" + ) + t3.createOrReplaceTempView("t3") + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.explode(sf.array(sf.col("c1").outer(), sf.col("c2").outer())) + ).toDF("c1", "c2", "c3"), + self.spark.sql("""SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)"""), + ) + assertDataFrameEqual( + t3.lateralJoin(self.spark.tvf.explode(sf.col("c2").outer())).toDF("c1", "c2", "v"), + self.spark.sql("""SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)"""), + ) + assertDataFrameEqual( + self.spark.tvf.explode(sf.array(sf.lit(1), sf.lit(2))) + .toDF("v") + .lateralJoin( + self.spark.range(1).select((sf.col("v").outer() + sf.lit(1)).alias("v2")) + ), + self.spark.sql( + """SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1 AS v2)""" + ), + ) + def test_explode_outer(self): actual = self.spark.tvf.explode_outer(sf.array(sf.lit(1), sf.lit(2))) expected = self.spark.sql("""SELECT * FROM explode_outer(array(1, 2))""") @@ -81,6 +112,43 @@ def test_explode_outer(self): expected = self.spark.sql("""SELECT * FROM explode_outer(null :: map)""") assertDataFrameEqual(actual=actual, expected=expected) + def test_explode_outer_with_lateral_join(self): + with self.tempView("t1", "t2"): + t1 = self.spark.sql("VALUES (0, 1), (1, 2) AS t1(c1, c2)") + t1.createOrReplaceTempView("t1") + t3 = self.spark.sql( + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) " + "AS t3(c1, c2)" + ) + t3.createOrReplaceTempView("t3") + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.explode_outer( + sf.array(sf.col("c1").outer(), sf.col("c2").outer()) + ) + ).toDF("c1", "c2", "c3"), + self.spark.sql("""SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)"""), + ) + assertDataFrameEqual( + t3.lateralJoin(self.spark.tvf.explode_outer(sf.col("c2").outer())).toDF( + "c1", "c2", "v" + ), + self.spark.sql("""SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)"""), + ) + assertDataFrameEqual( + self.spark.tvf.explode_outer(sf.array(sf.lit(1), sf.lit(2))) + .toDF("v") + .lateralJoin( + self.spark.range(1).select((sf.col("v").outer() + sf.lit(1)).alias("v2")) + ), + self.spark.sql( + """ + SELECT * FROM EXPLODE_OUTER(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1 AS v2) + """ + ), + ) + def test_inline(self): actual = self.spark.tvf.inline( sf.array(sf.struct(sf.lit(1), sf.lit("a")), sf.struct(sf.lit(2), sf.lit("b"))) @@ -107,6 +175,35 @@ def test_inline(self): ) assertDataFrameEqual(actual=actual, expected=expected) + def test_inline_with_lateral_join(self): + with self.tempView("array_struct"): + array_struct = self.spark.sql( + """ + VALUES + (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + (2, ARRAY()), + (3, ARRAY(STRUCT(3, 'c'))) AS array_struct(id, arr) + """ + ) + array_struct.createOrReplaceTempView("array_struct") + + assertDataFrameEqual( + array_struct.lateralJoin(self.spark.tvf.inline(sf.col("arr").outer())), + self.spark.sql("""SELECT * FROM array_struct JOIN LATERAL INLINE(arr)"""), + ) + assertDataFrameEqual( + array_struct.lateralJoin( + self.spark.tvf.inline(sf.col("arr").outer()), + sf.col("id") == sf.col("col1"), + "left", + ).toDF("id", "arr", "k", "v"), + self.spark.sql( + """ + SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k + """ + ), + ) + def test_inline_outer(self): actual = self.spark.tvf.inline_outer( sf.array(sf.struct(sf.lit(1), sf.lit("a")), sf.struct(sf.lit(2), sf.lit("b"))) @@ -137,6 +234,35 @@ def test_inline_outer(self): ) assertDataFrameEqual(actual=actual, expected=expected) + def test_inline_outer_with_lateral_join(self): + with self.tempView("array_struct"): + array_struct = self.spark.sql( + """ + VALUES + (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + (2, ARRAY()), + (3, ARRAY(STRUCT(3, 'c'))) AS array_struct(id, arr) + """ + ) + array_struct.createOrReplaceTempView("array_struct") + + assertDataFrameEqual( + array_struct.lateralJoin(self.spark.tvf.inline_outer(sf.col("arr").outer())), + self.spark.sql("""SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr)"""), + ) + assertDataFrameEqual( + array_struct.lateralJoin( + self.spark.tvf.inline_outer(sf.col("arr").outer()), + sf.col("id") == sf.col("col1"), + "left", + ).toDF("id", "arr", "k", "v"), + self.spark.sql( + """ + SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k + """ + ), + ) + def test_json_tuple(self): actual = self.spark.tvf.json_tuple(sf.lit('{"a":1, "b":2}'), sf.lit("a"), sf.lit("b")) expected = self.spark.sql("""SELECT json_tuple('{"a":1, "b":2}', 'a', 'b')""") @@ -151,6 +277,64 @@ def test_json_tuple(self): messageParameters={"item": "field"}, ) + def test_json_tuple_with_lateral_join(self): + with self.tempView("json_table"): + json_table = self.spark.sql( + """ + VALUES + ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + ('4', cast(null as string)), + ('5', '{"f1": null, "f5": ""}'), + ('6', '[invalid JSON string]') AS json_table(key, jstring) + """ + ) + json_table.createOrReplaceTempView("json_table") + + assertDataFrameEqual( + json_table.alias("t1") + .lateralJoin( + self.spark.tvf.json_tuple( + sf.col("jstring").outer(), + sf.lit("f1"), + sf.lit("f2"), + sf.lit("f3"), + sf.lit("f4"), + sf.lit("f5"), + ) + ) + .select("key", "c0", "c1", "c2", "c3", "c4"), + self.spark.sql( + """ + SELECT t1.key, t2.* FROM json_table t1, + LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 + """ + ), + ) + assertDataFrameEqual( + json_table.alias("t1") + .lateralJoin( + self.spark.tvf.json_tuple( + sf.col("jstring").outer(), + sf.lit("f1"), + sf.lit("f2"), + sf.lit("f3"), + sf.lit("f4"), + sf.lit("f5"), + ) + ) + .where(sf.col("c0").isNotNull()) + .select("key", "c0", "c1", "c2", "c3", "c4"), + self.spark.sql( + """ + SELECT t1.key, t2.* FROM json_table t1, + LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 + WHERE t2.c0 IS NOT NULL + """ + ), + ) + def test_posexplode(self): actual = self.spark.tvf.posexplode(sf.array(sf.lit(1), sf.lit(2))) expected = self.spark.sql("""SELECT * FROM posexplode(array(1, 2))""") @@ -180,6 +364,39 @@ def test_posexplode(self): expected = self.spark.sql("""SELECT * FROM posexplode(null :: map)""") assertDataFrameEqual(actual=actual, expected=expected) + def test_posexplode_with_lateral_join(self): + with self.tempView("t1", "t2"): + t1 = self.spark.sql("VALUES (0, 1), (1, 2) AS t1(c1, c2)") + t1.createOrReplaceTempView("t1") + t3 = self.spark.sql( + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) " + "AS t3(c1, c2)" + ) + t3.createOrReplaceTempView("t3") + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.posexplode(sf.array(sf.col("c1").outer(), sf.col("c2").outer())) + ), + self.spark.sql("""SELECT * FROM t1, LATERAL POSEXPLODE(ARRAY(c1, c2))"""), + ) + assertDataFrameEqual( + t3.lateralJoin(self.spark.tvf.posexplode(sf.col("c2").outer())), + self.spark.sql("""SELECT * FROM t3, LATERAL POSEXPLODE(c2)"""), + ) + assertDataFrameEqual( + self.spark.tvf.posexplode(sf.array(sf.lit(1), sf.lit(2))) + .toDF("p", "v") + .lateralJoin( + self.spark.range(1).select((sf.col("v").outer() + sf.lit(1)).alias("v2")) + ), + self.spark.sql( + """ + SELECT * FROM POSEXPLODE(ARRAY(1, 2)) t(p, v), LATERAL (SELECT v + 1 AS v2) + """ + ), + ) + def test_posexplode_outer(self): actual = self.spark.tvf.posexplode_outer(sf.array(sf.lit(1), sf.lit(2))) expected = self.spark.sql("""SELECT * FROM posexplode_outer(array(1, 2))""") @@ -209,11 +426,93 @@ def test_posexplode_outer(self): expected = self.spark.sql("""SELECT * FROM posexplode_outer(null :: map)""") assertDataFrameEqual(actual=actual, expected=expected) + def test_posexplode_outer_with_lateral_join(self): + with self.tempView("t1", "t2"): + t1 = self.spark.sql("VALUES (0, 1), (1, 2) AS t1(c1, c2)") + t1.createOrReplaceTempView("t1") + t3 = self.spark.sql( + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) " + "AS t3(c1, c2)" + ) + t3.createOrReplaceTempView("t3") + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.posexplode_outer( + sf.array(sf.col("c1").outer(), sf.col("c2").outer()) + ) + ), + self.spark.sql("""SELECT * FROM t1, LATERAL POSEXPLODE_OUTER(ARRAY(c1, c2))"""), + ) + assertDataFrameEqual( + t3.lateralJoin(self.spark.tvf.posexplode_outer(sf.col("c2").outer())), + self.spark.sql("""SELECT * FROM t3, LATERAL POSEXPLODE_OUTER(c2)"""), + ) + assertDataFrameEqual( + self.spark.tvf.posexplode_outer(sf.array(sf.lit(1), sf.lit(2))) + .toDF("p", "v") + .lateralJoin( + self.spark.range(1).select((sf.col("v").outer() + sf.lit(1)).alias("v2")) + ), + self.spark.sql( + """ + SELECT * FROM POSEXPLODE_OUTER(ARRAY(1, 2)) t(p, v), + LATERAL (SELECT v + 1 AS v2) + """ + ), + ) + def test_stack(self): actual = self.spark.tvf.stack(sf.lit(2), sf.lit(1), sf.lit(2), sf.lit(3)) expected = self.spark.sql("""SELECT * FROM stack(2, 1, 2, 3)""") assertDataFrameEqual(actual=actual, expected=expected) + def test_stack_with_lateral_join(self): + with self.tempView("t1", "t3"): + t1 = self.spark.sql("VALUES (0, 1), (1, 2) AS t1(c1, c2)") + t1.createOrReplaceTempView("t1") + t3 = self.spark.sql( + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) " + "AS t3(c1, c2)" + ) + t3.createOrReplaceTempView("t3") + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.stack( + sf.lit(2), + sf.lit("Key"), + sf.col("c1").outer(), + sf.lit("Value"), + sf.col("c2").outer(), + ) + ).select("col0", "col1"), + self.spark.sql( + """SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t""" + ), + ) + assertDataFrameEqual( + t1.lateralJoin( + self.spark.tvf.stack(sf.lit(1), sf.col("c1").outer(), sf.col("c2").outer()) + ).select("col0", "col1"), + self.spark.sql("""SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t"""), + ) + assertDataFrameEqual( + t1.join(t3, sf.col("t1.c1") == sf.col("t3.c1")) + .lateralJoin( + self.spark.tvf.stack( + sf.lit(1), sf.col("t1.c2").outer(), sf.col("t3.c2").outer() + ) + ) + .select("col0", "col1"), + self.spark.sql( + """ + SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 + JOIN LATERAL stack(1, t1.c2, t3.c2) t + """ + ), + ) + def test_collations(self): actual = self.spark.tvf.collations() expected = self.spark.sql("""SELECT * FROM collations()""") @@ -256,6 +555,31 @@ def test_variant_explode(self): expected = self.spark.sql("""SELECT * FROM variant_explode(parse_json('1'))""") assertDataFrameEqual(actual=actual, expected=expected) + def test_variant_explode_with_lateral_join(self): + with self.tempView("variant_table"): + variant_table = self.spark.sql( + """ + SELECT id, parse_json(v) AS v FROM VALUES + (0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + (2, '[]'), (3, '{}'), + (4, NULL), (5, '1') + AS t(id, v) + """ + ) + variant_table.createOrReplaceTempView("variant_table") + + assertDataFrameEqual( + variant_table.alias("t1") + .lateralJoin(self.spark.tvf.variant_explode(sf.col("v").outer())) + .select("id", "pos", "key", "value"), + self.spark.sql( + """ + SELECT t1.id, t.* FROM variant_table AS t1, + LATERAL variant_explode(v) AS t + """ + ), + ) + def test_variant_explode_outer(self): actual = self.spark.tvf.variant_explode_outer(sf.parse_json(sf.lit('["hello", "world"]'))) expected = self.spark.sql( @@ -290,6 +614,31 @@ def test_variant_explode_outer(self): expected = self.spark.sql("""SELECT * FROM variant_explode_outer(parse_json('1'))""") assertDataFrameEqual(actual=actual, expected=expected) + def test_variant_explode_outer_with_lateral_join(self): + with self.tempView("variant_table"): + variant_table = self.spark.sql( + """ + SELECT id, parse_json(v) AS v FROM VALUES + (0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + (2, '[]'), (3, '{}'), + (4, NULL), (5, '1') + AS t(id, v) + """ + ) + variant_table.createOrReplaceTempView("variant_table") + + assertDataFrameEqual( + variant_table.alias("t1") + .lateralJoin(self.spark.tvf.variant_explode_outer(sf.col("v").outer())) + .select("id", "pos", "key", "value"), + self.spark.sql( + """ + SELECT t1.id, t.* FROM variant_table AS t1, + LATERAL variant_explode_outer(v) AS t + """ + ), + ) + class TVFTests(TVFTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 8447edfbbb15d..31cd4c80370ee 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -31,6 +31,7 @@ from pyspark.util import PythonEvalType from pyspark.sql.functions import ( array, + col, create_map, array, lit, @@ -155,6 +156,22 @@ def eval(self, a: int, b: int) -> Iterator: ) assertDataFrameEqual(df, expected) + def test_udtf_with_lateral_join_dataframe(self): + @udtf(returnType="a: int, b: int, c: int") + class TestUDTF: + def eval(self, a: int, b: int) -> Iterator: + yield a, b, a + b + yield a, b, a - b + + self.spark.udtf.register("testUDTF", TestUDTF) + + assertDataFrameEqual( + self.spark.sql("values (0, 1), (1, 2) t(a, b)").lateralJoin( + TestUDTF(col("a").outer(), col("b").outer()) + ), + self.spark.sql("SELECT * FROM values (0, 1), (1, 2) t(a, b), LATERAL testUDTF(a, b)"), + ) + def test_udtf_eval_with_return_stmt(self): class TestUDTF: def eval(self, a: int, b: int): @@ -239,6 +256,20 @@ def eval(self, a: int): [Row(id=6, a=6), Row(id=7, a=7)], ) + def test_udtf_with_conditional_return_dataframe(self): + @udtf(returnType="a: int") + class TestUDTF: + def eval(self, a: int): + if a > 5: + yield a, + + self.spark.udtf.register("test_udtf", TestUDTF) + + assertDataFrameEqual( + self.spark.range(8).lateralJoin(TestUDTF(col("id").outer())), + self.spark.sql("SELECT * FROM range(0, 8) JOIN LATERAL test_udtf(id)"), + ) + def test_udtf_with_empty_yield(self): @udtf(returnType="a: int") class TestUDTF: diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala index 9d41998f11dc6..20c181e7b9cf6 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala @@ -859,6 +859,60 @@ abstract class Dataset[T] extends Serializable { joinWith(other, condition, "inner") } + /** + * Lateral join with another `DataFrame`. + * + * Behaves as an JOIN LATERAL. + * + * @param right + * Right side of the join operation. + * @group untypedrel + * @since 4.0.0 + */ + def lateralJoin(right: DS[_]): Dataset[Row] + + /** + * Lateral join with another `DataFrame`. + * + * Behaves as an JOIN LATERAL. + * + * @param right + * Right side of the join operation. + * @param joinExprs + * Join expression. + * @group untypedrel + * @since 4.0.0 + */ + def lateralJoin(right: DS[_], joinExprs: Column): Dataset[Row] + + /** + * Lateral join with another `DataFrame`. + * + * @param right + * Right side of the join operation. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `left`, + * `leftouter`, `left_outer`. + * @group untypedrel + * @since 4.0.0 + */ + def lateralJoin(right: DS[_], joinType: String): Dataset[Row] + + /** + * Lateral join with another `DataFrame`. + * + * @param right + * Right side of the join operation. + * @param joinExprs + * Join expression. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `left`, + * `leftouter`, `left_outer`. + * @group untypedrel + * @since 4.0.0 + */ + def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): Dataset[Row] + protected def sortInternal(global: Boolean, sortExprs: Seq[Column]): Dataset[T] /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 846d97b257869..8726ee268a477 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -709,6 +709,38 @@ class Dataset[T] private[sql]( new Dataset(sparkSession, joinWith, joinEncoder) } + private[sql] def lateralJoin( + right: DS[_], joinExprs: Option[Column], joinType: JoinType): DataFrame = { + withPlan { + LateralJoin( + logicalPlan, + LateralSubquery(right.logicalPlan), + joinType, + joinExprs.map(_.expr) + ) + } + } + + /** @inheritdoc */ + def lateralJoin(right: DS[_]): DataFrame = { + lateralJoin(right, None, Inner) + } + + /** @inheritdoc */ + def lateralJoin(right: DS[_], joinExprs: Column): DataFrame = { + lateralJoin(right, Some(joinExprs), Inner) + } + + /** @inheritdoc */ + def lateralJoin(right: DS[_], joinType: String): DataFrame = { + lateralJoin(right, None, JoinType(joinType)) + } + + /** @inheritdoc */ + def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = { + lateralJoin(right, Some(joinExprs), JoinType(joinType)) + } + // TODO(SPARK-22947): Fix the DataFrame API. private[sql] def joinAsOf( other: Dataset[_], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index 2420ad34d9bab..cd425162fb016 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -377,4 +377,291 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { Array(ExpectedContext(fragment = "$", callSitePattern = getCurrentClassCallSitePattern)) ) } + + private def table1() = { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + spark.table("t1") + } + + private def table2() = { + sql("CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3)") + spark.table("t2") + } + + private def table3() = { + sql("CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + spark.table("t3") + } + + test("lateral join with single column select") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c1".outer())), + sql("SELECT * FROM t1, LATERAL (SELECT c1)") + ) + checkAnswer( + t1.lateralJoin(t2.select($"c1")), + sql("SELECT * FROM t1, LATERAL (SELECT c1 FROM t2)") + ) + checkAnswer( + t1.lateralJoin(t2.select($"t1.c1".outer())), + sql("SELECT * FROM t1, LATERAL (SELECT t1.c1 FROM t2)") + ) + checkAnswer( + t1.lateralJoin(t2.select($"t1.c1".outer() + $"t2.c1")), + sql("SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2)") + ) + } + } + + test("lateral join with different join types") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), + $"c2" === $"c3"), + sql("SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3") + ) + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), + $"c2" === $"c3", + "left"), + sql("SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3") + ) + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), + "cross"), + sql("SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3)") + ) + } + } + + test("lateral join with correlated equality / non-equality predicates") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2")), + sql("SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1)") + ) + checkAnswer( + t1.lateralJoin(t2.where($"t1.c1".outer() < $"t2.c1").select($"c2")), + sql("SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 < t2.c1)") + ) + } + } + + test("lateral join with aggregation and correlated non-equality predicates") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.where($"t1.c2".outer() < $"t2.c2").select(max($"c2").as("m"))), + sql("SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2)") + ) + } + } + + test("lateral join can reference preceding FROM clause items") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.join(t2).lateralJoin( + spark.range(1).select($"t1.c2".outer() + $"t2.c2".outer()) + ), + sql("SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2)") + ) + } + } + + test("multiple lateral joins") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("a")) + ).lateralJoin( + spark.range(1).select(($"c1".outer() - $"c2".outer()).as("b")) + ).lateralJoin( + spark.range(1).select(($"a".outer() * $"b".outer()).as("c")) + ), + sql( + """ + |SELECT * FROM t1, + |LATERAL (SELECT c1 + c2 AS a), + |LATERAL (SELECT c1 - c2 AS b), + |LATERAL (SELECT a * b AS c) + |""".stripMargin) + ) + } + } + + test("lateral join in between regular joins") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin( + t2.where($"t1.c1".outer() === $"t2.c1").select($"c2"), "left" + ).join(t1.as("t3"), $"t2.c2" === $"t3.c2", "left"), + sql( + """ + |SELECT * FROM t1 + |LEFT OUTER JOIN LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) s + |LEFT OUTER JOIN t1 t3 ON s.c2 = t3.c2 + |""".stripMargin) + ) + } + } + + test("nested lateral joins") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin( + t2.lateralJoin(spark.range(1).select($"c1".outer())) + ), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT c1))") + ) + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + lit(1)).as("c1")) + .lateralJoin(spark.range(1).select($"c1".outer())) + ), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1 + 1 AS c1), LATERAL (SELECT c1))") + ) + } + } + + test("scalar subquery inside lateral join") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + // uncorrelated + checkAnswer( + t1.lateralJoin( + spark.range(1).select( + $"c2".outer(), + t2.select(min($"c2")).scalar() + ) + ), + sql("SELECT * FROM t1, LATERAL (SELECT c2, (SELECT MIN(c2) FROM t2))") + ) + + // correlated + checkAnswer( + t1.lateralJoin( + spark.range(1).select($"c1".outer().as("a")) + .select(t2.where($"c1" === $"a".outer()).select(sum($"c2")).scalar()) + ), + sql( + """ + |SELECT * FROM t1, LATERAL ( + | SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = a) FROM (SELECT c1 AS a) + |) + |""".stripMargin) + ) + } + } + + test("lateral join inside subquery") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + // uncorrelated + checkAnswer( + t1.where( + $"c1" === t2.lateralJoin( + spark.range(1).select($"c1".outer().as("a"))).select(min($"a") + ).scalar() + ), + sql("SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a))") + ) + // correlated + checkAnswer( + t1.where( + $"c1" === t2.lateralJoin( + spark.range(1).select($"c1".outer().as("a"))) + .where($"c1" === $"t1.c1".outer()) + .select(min($"a")) + .scalar() + ), + sql("SELECT * FROM t1 " + + "WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a) WHERE c1 = t1.c1)") + ) + } + } + + test("lateral join with table-valued functions") { + withView("t1", "t3") { + val t1 = table1() + val t3 = table3() + + checkAnswer( + t1.lateralJoin(spark.tvf.range(3)), + sql("SELECT * FROM t1, LATERAL RANGE(3)") + ) + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)") + ) + checkAnswer( + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)") + ) + checkAnswer( + spark.tvf.explode(array(lit(1), lit(2))).toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + 1)), + sql("SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)") + ) + } + } + + test("lateral join with table-valued functions and join conditions") { + withView("t1", "t3") { + val t1 = table1() + val t3 = table3() + + checkAnswer( + t1.lateralJoin( + spark.tvf.explode(array($"c1".outer(), $"c2".outer())), + $"c1" === $"col" + ), + sql("SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3") + ) + checkAnswer( + t3.lateralJoin( + spark.tvf.explode($"c2".outer()), + $"c1" === $"col" + ), + sql("SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3") + ) + checkAnswer( + t3.lateralJoin( + spark.tvf.explode($"c2".outer()), + $"c1" === $"col", + "left" + ), + sql("SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3") + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala index c2f53ff56d1aa..4f2cd275ffdfd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSession { + import testImplicits._ test("explode") { val actual1 = spark.tvf.explode(array(lit(1), lit(2))) @@ -50,6 +51,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer(actual6, expected6) } + test("explode - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql("CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)") + ) + checkAnswer( + t3.lateralJoin(spark.tvf.explode($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)") + ) + checkAnswer( + spark.tvf.explode(array(lit(1), lit(2))).toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)") + ) + } + } + test("explode_outer") { val actual1 = spark.tvf.explode_outer(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM explode_outer(array(1, 2))") @@ -78,6 +103,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer(actual6, expected6) } + test("explode_outer - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql("CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)") + ) + checkAnswer( + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)") + ) + checkAnswer( + spark.tvf.explode_outer(array(lit(1), lit(2))).toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM EXPLODE_OUTER(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)") + ) + } + } + test("inline") { val actual1 = spark.tvf.inline(array(struct(lit(1), lit("a")), struct(lit(2), lit("b")))) val expected1 = spark.sql("SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b')))") @@ -98,6 +147,32 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer(actual3, expected3) } + test("inline - lateral join") { + withView("array_struct") { + sql( + """ + |CREATE VIEW array_struct(id, arr) AS VALUES + | (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + | (2, ARRAY()), + | (3, ARRAY(STRUCT(3, 'c'))) + |""".stripMargin) + val arrayStruct = spark.table("array_struct") + + checkAnswer( + arrayStruct.lateralJoin(spark.tvf.inline($"arr".outer())), + sql("SELECT * FROM array_struct JOIN LATERAL INLINE(arr)") + ) + checkAnswer( + arrayStruct.lateralJoin( + spark.tvf.inline($"arr".outer()), + $"id" === $"col1", + "left" + ), + sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k") + ) + } + } + test("inline_outer") { val actual1 = spark.tvf.inline_outer(array(struct(lit(1), lit("a")), struct(lit(2), lit("b")))) val expected1 = spark.sql("SELECT * FROM inline_outer(array(struct(1, 'a'), struct(2, 'b')))") @@ -118,6 +193,32 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer(actual3, expected3) } + test("inline_outer - lateral join") { + withView("array_struct") { + sql( + """ + |CREATE VIEW array_struct(id, arr) AS VALUES + | (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + | (2, ARRAY()), + | (3, ARRAY(STRUCT(3, 'c'))) + |""".stripMargin) + val arrayStruct = spark.table("array_struct") + + checkAnswer( + arrayStruct.lateralJoin(spark.tvf.inline_outer($"arr".outer())), + sql("SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr)") + ) + checkAnswer( + arrayStruct.lateralJoin( + spark.tvf.inline_outer($"arr".outer()), + $"id" === $"col1", + "left" + ), + sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k") + ) + } + } + test("json_tuple") { val actual = spark.tvf.json_tuple(lit("""{"a":1,"b":2}"""), lit("a"), lit("b")) val expected = spark.sql("""SELECT * FROM json_tuple('{"a":1,"b":2}', 'a', 'b')""") @@ -130,6 +231,43 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi assert(ex.messageParameters("functionName") == "`json_tuple`") } + test("json_tuple - lateral join") { + withView("json_table") { + sql( + """ + |CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES + | ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + | ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + | ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + | ('4', cast(null as string)), + | ('5', '{"f1": null, "f5": ""}'), + | ('6', '[invalid JSON string]') + |""".stripMargin) + val jsonTable = spark.table("json_table") + + checkAnswer( + jsonTable.as("t1").lateralJoin( + spark.tvf.json_tuple( + $"t1.jstring".outer(), + lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) + ).select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + sql("SELECT t1.key, t2.* FROM json_table t1, " + + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2") + ) + checkAnswer( + jsonTable.as("t1").lateralJoin( + spark.tvf.json_tuple( + $"jstring".outer(), + lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) + ).where($"c0".isNotNull) + .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + sql("SELECT t1.key, t2.* FROM json_table t1, " + + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 " + + "WHERE t2.c0 IS NOT NULL") + ) + } + } + test("posexplode") { val actual1 = spark.tvf.posexplode(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM posexplode(array(1, 2))") @@ -158,6 +296,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer(actual6, expected6) } + test("posexplode - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql("CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.posexplode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL POSEXPLODE(ARRAY(c1, c2))") + ) + checkAnswer( + t3.lateralJoin(spark.tvf.posexplode($"c2".outer())), + sql("SELECT * FROM t3, LATERAL POSEXPLODE(c2)") + ) + checkAnswer( + spark.tvf.posexplode(array(lit(1), lit(2))).toDF("p", "v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM POSEXPLODE(ARRAY(1, 2)) t(p, v), LATERAL (SELECT v + 1)") + ) + } + } + test("posexplode_outer") { val actual1 = spark.tvf.posexplode_outer(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM posexplode_outer(array(1, 2))") @@ -186,12 +348,66 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer(actual6, expected6) } + test("posexplode_outer - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql("CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.posexplode_outer(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL POSEXPLODE_OUTER(ARRAY(c1, c2))") + ) + checkAnswer( + t3.lateralJoin(spark.tvf.posexplode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL POSEXPLODE_OUTER(c2)") + ) + checkAnswer( + spark.tvf.posexplode_outer(array(lit(1), lit(2))).toDF("p", "v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM POSEXPLODE_OUTER(ARRAY(1, 2)) t(p, v), LATERAL (SELECT v + 1)") + ) + } + } + test("stack") { val actual = spark.tvf.stack(lit(2), lit(1), lit(2), lit(3)) val expected = spark.sql("SELECT * FROM stack(2, 1, 2, 3)") checkAnswer(actual, expected) } + test("stack - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql("CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin( + spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer()) + ).select($"col0", $"col1"), + sql("SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t") + ) + checkAnswer( + t1.lateralJoin( + spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer()) + ).select($"col0".as("x"), $"col1".as("y")), + sql("SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)") + ) + checkAnswer( + t1.join(t3, $"t1.c1" === $"t3.c1") + .lateralJoin( + spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer()) + ).select($"col0", $"col1"), + sql("SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t") + ) + } + } + test("collations") { val actual = spark.tvf.collations() val expected = spark.sql("SELECT * FROM collations()") @@ -235,6 +451,28 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer(actual6, expected6) } + test("variant_explode - lateral join") { + withView("variant_table") { + sql( + """ + |CREATE VIEW variant_table(id, v) AS + |SELECT id, parse_json(v) AS v FROM VALUES + |(0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + |(2, '[]'), (3, '{}'), + |(4, NULL), (5, '1') + |AS t(id, v) + |""".stripMargin) + val variantTable = spark.table("variant_table") + + checkAnswer( + variantTable.as("t1").lateralJoin( + spark.tvf.variant_explode($"v".outer()) + ).select($"id", $"pos", $"key", $"value"), + sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode(v) AS t") + ) + } + } + test("variant_explode_outer") { val actual1 = spark.tvf.variant_explode_outer(parse_json(lit("""["hello", "world"]"""))) val expected1 = spark.sql( @@ -265,4 +503,26 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi val expected6 = spark.sql("SELECT * FROM variant_explode_outer(parse_json('1'))") checkAnswer(actual6, expected6) } + + test("variant_explode_outer - lateral join") { + withView("variant_table") { + sql( + """ + |CREATE VIEW variant_table(id, v) AS + |SELECT id, parse_json(v) AS v FROM VALUES + |(0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + |(2, '[]'), (3, '{}'), + |(4, NULL), (5, '1') + |AS t(id, v) + |""".stripMargin) + val variantTable = spark.table("variant_table") + + checkAnswer( + variantTable.as("t1").lateralJoin( + spark.tvf.variant_explode_outer($"v".outer()) + ).select($"id", $"pos", $"key", $"value"), + sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode_outer(v) AS t") + ) + } + } } From ecd1911a59b2d628627df9a0a76e61be9531715b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 5 Dec 2024 17:48:09 -0800 Subject: [PATCH 111/438] [SPARK-50506][DOCS] Codify Spark Standalone documentation consistently ### What changes were proposed in this pull request? This PR aims to codify `Spark Standalone` documentation more consistently. ### Why are the changes needed? - https://spark.apache.org/docs/4.0.0-preview2/spark-standalone.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49082 from dongjoon-hyun/SPARK-50506. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/internal/config/Deploy.scala | 2 +- docs/spark-standalone.md | 54 +++++++++---------- 2 files changed, 28 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala index 0c2db21905d1f..1ed53868992ac 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala @@ -45,7 +45,7 @@ private[spark] object Deploy { val RECOVERY_TIMEOUT = ConfigBuilder("spark.deploy.recoveryTimeout") .doc("Configures the timeout for recovery process. The default value is the same " + - "with ${WORKER_TIMEOUT.key}.") + s"with ${Worker.WORKER_TIMEOUT.key}.") .version("4.0.0") .timeConf(TimeUnit.SECONDS) .checkValue(_ > 0, "spark.deploy.recoveryTimeout must be positive.") diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 4f8e0dc1a3917..51f6ca977c991 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -351,28 +351,28 @@ SPARK_MASTER_OPTS supports the following system properties: spark.deploy.appNumberModulo (None) - The modulo for app number. By default, the next of `app-yyyyMMddHHmmss-9999` is - `app-yyyyMMddHHmmss-10000`. If we have 10000 as modulo, it will be `app-yyyyMMddHHmmss-0000`. - In most cases, the prefix `app-yyyyMMddHHmmss` is increased already during creating 10000 applications. + The modulo for app number. By default, the next of app-yyyyMMddHHmmss-9999 is + app-yyyyMMddHHmmss-10000. If we have 10000 as modulo, it will be app-yyyyMMddHHmmss-0000. + In most cases, the prefix app-yyyyMMddHHmmss is increased already during creating 10000 applications. 4.0.0
spark.deploy.driverIdPatterndriver-%s-%04ddriver-%s-%04d - The pattern for driver ID generation based on Java `String.format` method. - The default value is `driver-%s-%04d` which represents the existing driver id string, e.g., `driver-20231031224459-0019`. Please be careful to generate unique IDs. + The pattern for driver ID generation based on Java String.format method. + The default value is driver-%s-%04d which represents the existing driver id string, e.g., driver-20231031224459-0019. Please be careful to generate unique IDs. 4.0.0
spark.deploy.appIdPatternapp-%s-%04dapp-%s-%04d - The pattern for app ID generation based on Java `String.format` method. - The default value is `app-%s-%04d` which represents the existing app id string, e.g., - `app-20231031224509-0008`. Please be careful to generate unique IDs. + The pattern for app ID generation based on Java String.format method. + The default value is app-%s-%04d which represents the existing app id string, e.g., + app-20231031224509-0008. Please be careful to generate unique IDs. 4.0.0
Enable periodic cleanup of worker / application directories. Note that this only affects standalone mode, as YARN works differently. Only the directories of stopped applications are cleaned up. - This should be enabled if spark.shuffle.service.db.enabled is "true" + This should be enabled if spark.shuffle.service.db.enabled is "true" 1.0.0
ROCKSDB When spark.shuffle.service.db.enabled is true, user can use this to specify the kind of disk-based - store used in shuffle service state store. This supports `ROCKSDB` and `LEVELDB` (deprecated) now and `ROCKSDB` as default value. - The original data store in `RocksDB/LevelDB` will not be automatically convert to another kind of storage now. + store used in shuffle service state store. This supports ROCKSDB and LEVELDB (deprecated) now and ROCKSDB as default value. + The original data store in RocksDB/LevelDB will not be automatically convert to another kind of storage now. 3.4.0
Enable cleanup non-shuffle files(such as temp. shuffle blocks, cached RDD/broadcast blocks, spill files, etc) of worker directories following executor exits. Note that this doesn't - overlap with `spark.worker.cleanup.enabled`, as this enables cleanup of non-shuffle files in - local directories of a dead executor, while `spark.worker.cleanup.enabled` enables cleanup of + overlap with spark.worker.cleanup.enabled, as this enables cleanup of non-shuffle files in + local directories of a dead executor, while spark.worker.cleanup.enabled enables cleanup of all files/subdirectories of a stopped and timeout application. This only affects Standalone mode, support of other cluster managers can be added in the future.
spark.worker.idPatternworker-%s-%s-%dworker-%s-%s-%d - The pattern for worker ID generation based on Java `String.format` method. - The default value is `worker-%s-%s-%d` which represents the existing worker id string, e.g., - `worker-20231109183042-[fe80::1%lo0]-39729`. Please be careful to generate unique IDs + The pattern for worker ID generation based on Java String.format method. + The default value is worker-%s-%s-%d which represents the existing worker id string, e.g., + worker-20231109183042-[fe80::1%lo0]-39729. Please be careful to generate unique IDs 4.0.0
- + + - - + - + - + - +
CommandDescriptionHTTP METHODSince Version
CommandHTTP METHODDescriptionSince Version
createPOST Create a Spark driver via cluster mode. Since 4.0.0, Spark master supports server-side variable replacements for the values of Spark properties and environment variables. POST 1.3.0
killKill a single Spark driver. POSTKill a single Spark driver. 1.3.0
killallKill all running Spark drivers. POSTKill all running Spark drivers. 4.0.0
statusCheck the status of a Spark job. GETCheck the status of a Spark job. 1.3.0
clearClear the completed drivers and applications. POSTClear the completed drivers and applications. 4.0.0
@@ -864,13 +864,13 @@ In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spa spark.deploy.zookeeper.url None - When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this configuration is used to set the zookeeper URL to connect to. + When spark.deploy.recoveryMode is set to ZOOKEEPER, this configuration is used to set the zookeeper URL to connect to. 0.8.1 spark.deploy.zookeeper.dir None - When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this configuration is used to set the zookeeper directory to store recovery state. + When spark.deploy.recoveryMode is set to ZOOKEEPER, this configuration is used to set the zookeeper directory to store recovery state. 0.8.1 From b1c118f48ea4dfe20eab0c269d670f3e0ddff07d Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 5 Dec 2024 17:50:00 -0800 Subject: [PATCH 112/438] [SPARK-50507][PYTHON][TESTS] Group pandas function related tests ### What changes were proposed in this pull request? Group pandas function related tests ### Why are the changes needed? these tests were grouped in classic side, also group the parity tests for consistency ### Does this PR introduce _any_ user-facing change? no, test-only ### How was this patch tested? updated ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #49074 from zhengruifeng/group_connect_test_pandas. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- dev/sparktestsupport/modules.py | 18 +++++++++--------- .../sql/tests/connect/pandas/__init__.py | 16 ++++++++++++++++ .../test_parity_pandas_cogrouped_map.py | 2 +- .../test_parity_pandas_grouped_map.py | 2 +- ...est_parity_pandas_grouped_map_with_state.py | 2 +- .../{ => pandas}/test_parity_pandas_map.py | 2 +- .../{ => pandas}/test_parity_pandas_udf.py | 2 +- .../test_parity_pandas_udf_grouped_agg.py | 2 +- .../test_parity_pandas_udf_scalar.py | 2 +- .../test_parity_pandas_udf_window.py | 2 +- .../{ => pandas}/test_pandas_sqlmetrics.py | 2 +- 11 files changed, 34 insertions(+), 18 deletions(-) create mode 100644 python/pyspark/sql/tests/connect/pandas/__init__.py rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_cogrouped_map.py (93%) rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_grouped_map.py (94%) rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_grouped_map_with_state.py (92%) rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_map.py (93%) rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_udf.py (93%) rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_udf_grouped_agg.py (93%) rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_udf_scalar.py (93%) rename python/pyspark/sql/tests/connect/{ => pandas}/test_parity_pandas_udf_window.py (93%) rename python/pyspark/sql/tests/{ => pandas}/test_pandas_sqlmetrics.py (96%) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index b8702113a26c7..e21a0c85de066 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -534,8 +534,8 @@ def __hash__(self): "pyspark.sql.tests.pandas.test_pandas_udf_typehints", "pyspark.sql.tests.pandas.test_pandas_udf_typehints_with_future_annotations", "pyspark.sql.tests.pandas.test_pandas_udf_window", + "pyspark.sql.tests.pandas.test_pandas_sqlmetrics", "pyspark.sql.tests.pandas.test_converter", - "pyspark.sql.tests.test_pandas_sqlmetrics", "pyspark.sql.tests.test_python_datasource", "pyspark.sql.tests.test_python_streaming_datasource", "pyspark.sql.tests.test_readwriter", @@ -1054,11 +1054,7 @@ def __hash__(self): "pyspark.sql.tests.connect.test_parity_memory_profiler", "pyspark.sql.tests.connect.test_parity_udtf", "pyspark.sql.tests.connect.test_parity_tvf", - "pyspark.sql.tests.connect.test_parity_pandas_udf", - "pyspark.sql.tests.connect.test_parity_pandas_map", "pyspark.sql.tests.connect.test_parity_arrow_map", - "pyspark.sql.tests.connect.test_parity_pandas_grouped_map", - "pyspark.sql.tests.connect.test_parity_pandas_cogrouped_map", "pyspark.sql.tests.connect.test_parity_arrow_grouped_map", "pyspark.sql.tests.connect.test_parity_arrow_cogrouped_map", "pyspark.sql.tests.connect.test_parity_python_datasource", @@ -1074,13 +1070,17 @@ def __hash__(self): "pyspark.sql.tests.connect.streaming.test_parity_listener", "pyspark.sql.tests.connect.streaming.test_parity_foreach", "pyspark.sql.tests.connect.streaming.test_parity_foreach_batch", - "pyspark.sql.tests.connect.test_parity_pandas_grouped_map_with_state", - "pyspark.sql.tests.connect.test_parity_pandas_udf_scalar", - "pyspark.sql.tests.connect.test_parity_pandas_udf_grouped_agg", - "pyspark.sql.tests.connect.test_parity_pandas_udf_window", "pyspark.sql.tests.connect.test_resources", "pyspark.sql.tests.connect.shell.test_progress", "pyspark.sql.tests.connect.test_df_debug", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_map", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map_with_state", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_scalar", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_grouped_agg", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_window", ], excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy, pandas, and pyarrow and diff --git a/python/pyspark/sql/tests/connect/pandas/__init__.py b/python/pyspark/sql/tests/connect/pandas/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/python/pyspark/sql/tests/connect/pandas/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_cogrouped_map.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_cogrouped_map.py similarity index 93% rename from python/pyspark/sql/tests/connect/test_parity_pandas_cogrouped_map.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_cogrouped_map.py index 00d71bda2d938..a71e6369f5e2f 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_cogrouped_map.py @@ -28,7 +28,7 @@ class CogroupedApplyInPandasTests( if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_pandas_cogrouped_map import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_cogrouped_map import * # noqa: F401 try: import xmlrunner diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_grouped_map.py similarity index 94% rename from python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_grouped_map.py index 8c76313c5c96b..52110718808be 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_grouped_map.py @@ -28,7 +28,7 @@ def test_supported_types(self): if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_pandas_grouped_map import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map import * # noqa: F401 try: import xmlrunner diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_grouped_map_with_state.py similarity index 92% rename from python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_grouped_map_with_state.py index 67d42a7c86138..2da8b4aa3be8a 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_grouped_map_with_state.py @@ -29,7 +29,7 @@ class GroupedApplyInPandasWithStateTests( if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_pandas_grouped_map_with_state import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map_with_state import * # noqa: F401,E501 try: import xmlrunner diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_map.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_map.py similarity index 93% rename from python/pyspark/sql/tests/connect/test_parity_pandas_map.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_map.py index 999afd24c6528..965ef5dcf8949 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_map.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_map.py @@ -28,7 +28,7 @@ class MapInPandasParityTests( if __name__ == "__main__": import unittest - from pyspark.sql.tests.connect.test_parity_pandas_map import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_map import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_udf.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf.py similarity index 93% rename from python/pyspark/sql/tests/connect/test_parity_pandas_udf.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf.py index 364e41716474b..aa2b4748ff3f7 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_udf.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf.py @@ -25,7 +25,7 @@ class PandasUDFParityTests(PandasUDFTestsMixin, ReusedConnectTestCase): if __name__ == "__main__": import unittest - from pyspark.sql.tests.connect.test_parity_pandas_udf import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_udf import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_grouped_agg.py similarity index 93% rename from python/pyspark/sql/tests/connect/test_parity_pandas_udf_grouped_agg.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_grouped_agg.py index fdb81bffbce12..dfcb2b94c1bcc 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_grouped_agg.py @@ -28,7 +28,7 @@ class PandasUDFGroupedAggParityTests( if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_pandas_udf_grouped_agg import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_grouped_agg import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_udf_scalar.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_scalar.py similarity index 93% rename from python/pyspark/sql/tests/connect/test_parity_pandas_udf_scalar.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_scalar.py index 451f0f68d6ee5..9cab05f569d46 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_scalar.py @@ -24,7 +24,7 @@ class PandasUDFScalarParityTests(ScalarPandasUDFTestsMixin, ReusedConnectTestCas if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_pandas_udf_scalar import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_scalar import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_udf_window.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_window.py similarity index 93% rename from python/pyspark/sql/tests/connect/test_parity_pandas_udf_window.py rename to python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_window.py index b2288c9d949e5..08da1b4648b22 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_udf_window.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_udf_window.py @@ -28,7 +28,7 @@ class PandasUDFWindowParityTests( if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_pandas_udf_window import * # noqa: F401 + from pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_window import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/test_pandas_sqlmetrics.py b/python/pyspark/sql/tests/pandas/test_pandas_sqlmetrics.py similarity index 96% rename from python/pyspark/sql/tests/test_pandas_sqlmetrics.py rename to python/pyspark/sql/tests/pandas/test_pandas_sqlmetrics.py index 22a0e92e818db..cb2f2ff285684 100644 --- a/python/pyspark/sql/tests/test_pandas_sqlmetrics.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_sqlmetrics.py @@ -57,7 +57,7 @@ def test_pandas(col1): if __name__ == "__main__": - from pyspark.sql.tests.test_pandas_sqlmetrics import * # noqa: F401 + from pyspark.sql.tests.pandas.test_pandas_sqlmetrics import * # noqa: F401 try: import xmlrunner From 851f5f2ff905636388ff31f349c6fc5064875172 Mon Sep 17 00:00:00 2001 From: Livia Zhu Date: Fri, 6 Dec 2024 11:28:55 +0900 Subject: [PATCH 113/438] [SPARK-50492][SS] Fix java.util.NoSuchElementException when event time column is dropped after dropDuplicatesWithinWatermark ### What changes were proposed in this pull request? Update `DeduplicateWithinWatermark` references to include all attributes that could be the watermarking column. ### Why are the changes needed? Fix `java.util.NoSuchElementException` due to ColumnPruning. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit test ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49065 from liviazhu-db/liviazhu-db/dedup-watermark-fix. Authored-by: Livia Zhu Signed-off-by: Jungtaek Lim --- .../plans/logical/basicLogicalOperators.scala | 3 +++ ...treamingDeduplicationWithinWatermarkSuite.scala | 14 ++++++++++++++ 2 files changed, 17 insertions(+) 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 0cb04064a6178..f2a46b57b478f 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 @@ -2014,6 +2014,9 @@ case class Deduplicate( } case class DeduplicateWithinWatermark(keys: Seq[Attribute], child: LogicalPlan) extends UnaryNode { + // Ensure that references include event time columns so they are not pruned away. + override def references: AttributeSet = AttributeSet(keys) ++ + AttributeSet(child.output.filter(_.metadata.contains(EventTimeWatermark.delayKey))) override def maxRows: Option[Long] = child.maxRows override def output: Seq[Attribute] = child.output final override val nodePatterns: Seq[TreePattern] = Seq(DISTINCT_LIKE) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala index 9a02ab3df7dd4..af86e6ec88996 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala @@ -220,4 +220,18 @@ class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest { ) } } + + test("SPARK-50492: drop event time column after dropDuplicatesWithinWatermark") { + val inputData = MemoryStream[(Int, Int)] + val result = inputData.toDS() + .withColumn("first", timestamp_seconds($"_1")) + .withWatermark("first", "10 seconds") + .dropDuplicatesWithinWatermark("_2") + .select("_2") + + testStream(result, Append)( + AddData(inputData, (1, 2)), + CheckAnswer(2) + ) + } } From c149942f0f9692ad83eb683abc0a8bcaa1ef70ff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Du=C5=A1an=20Ti=C5=A1ma?= Date: Fri, 6 Dec 2024 12:30:57 +0800 Subject: [PATCH 114/438] [SPARK-50449][SQL] Fix SQL Scripting grammar allowing empty bodies for loops, IF and CASE MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Before this PR, SQL Scripting grammar allowed for loops, IF and CASE to have empty bodies. Example: `WHILE 1 = 1 DO END WHILE;` If they have an empty body, an internal error is thrown during execution. This PR changes the grammar so that loops, IF and CASE must have at least one statement in their bodies. Note that this does not completely fix the internal error issue. It is still possible to have something like ``` WHILE 1 = 1 DO BEGIN END; END WHILE; ``` where the same error is still thrown, except this construct is correct grammar wise. This issue will be fixed by a separate PR, as non-trivial interpreter logic changes are required. ### Why are the changes needed? The existing grammar was wrong. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests that make sure parsing loops, IF and CASE with empty bodies throws an error. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48989 from dusantism-db/scripting-empty-bodies-fix. Authored-by: Dušan Tišma Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 6 +- .../sql/catalyst/parser/AstBuilder.scala | 13 +- .../parser/SqlScriptingParserSuite.scala | 137 +++++++++++++++++- 3 files changed, 145 insertions(+), 11 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 26edbe15da9fd..a0f447dba798e 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -48,15 +48,15 @@ compoundOrSingleStatement ; singleCompoundStatement - : BEGIN compoundBody END SEMICOLON? EOF + : BEGIN compoundBody? END SEMICOLON? EOF ; beginEndCompoundBlock - : beginLabel? BEGIN compoundBody END endLabel? + : beginLabel? BEGIN compoundBody? END endLabel? ; compoundBody - : (compoundStatements+=compoundStatement SEMICOLON)* + : (compoundStatements+=compoundStatement SEMICOLON)+ ; compoundStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 882e895cc7f02..fad4fcefc1d1d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -144,7 +144,9 @@ class AstBuilder extends DataTypeAstBuilder override def visitSingleCompoundStatement(ctx: SingleCompoundStatementContext): CompoundBody = { val labelCtx = new SqlScriptingLabelContext() - visitCompoundBodyImpl(ctx.compoundBody(), None, allowVarDeclare = true, labelCtx) + Option(ctx.compoundBody()) + .map(visitCompoundBodyImpl(_, None, allowVarDeclare = true, labelCtx)) + .getOrElse(CompoundBody(Seq.empty, None)) } private def visitCompoundBodyImpl( @@ -191,12 +193,9 @@ class AstBuilder extends DataTypeAstBuilder labelCtx: SqlScriptingLabelContext): CompoundBody = { val labelText = labelCtx.enterLabeledScope(Option(ctx.beginLabel()), Option(ctx.endLabel())) - val body = visitCompoundBodyImpl( - ctx.compoundBody(), - Some(labelText), - allowVarDeclare = true, - labelCtx - ) + val body = Option(ctx.compoundBody()) + .map(visitCompoundBodyImpl(_, Some(labelText), allowVarDeclare = true, labelCtx)) + .getOrElse(CompoundBody(Seq.empty, Some(labelText))) labelCtx.exitLabeledScope(Option(ctx.beginLabel())) body } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index ab647f83b42a4..c9e2f42e164f9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -82,7 +82,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { } } - test("empty BEGIN END block") { + test("empty singleCompoundStatement") { val sqlScriptText = """ |BEGIN @@ -91,6 +91,20 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(tree.collection.isEmpty) } + test("empty beginEndCompoundBlock") { + val sqlScriptText = + """ + |BEGIN + | BEGIN + | END; + |END""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CompoundBody]) + val innerBody = tree.collection.head.asInstanceOf[CompoundBody] + assert(innerBody.collection.isEmpty) + } + test("multiple ; in row - should fail") { val sqlScriptText = """ @@ -439,6 +453,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(ifStmt.conditions.head.getText == "1=1") } + test("if with empty body") { + val sqlScriptText = + """BEGIN + | IF 1 = 1 THEN + | END IF; + |END + """.stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'IF'", "hint" -> "")) + } + test("if else") { val sqlScriptText = """BEGIN @@ -623,6 +652,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(whileStmt.label.contains("lbl")) } + test("while with empty body") { + val sqlScriptText = + """BEGIN + | WHILE 1 = 1 DO + | END WHILE; + |END + """.stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'WHILE'", "hint" -> "")) + } + test("while with complex condition") { val sqlScriptText = """ @@ -1067,6 +1111,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(repeatStmt.label.contains("lbl")) } + test("repeat with empty body") { + val sqlScriptText = + """BEGIN + | REPEAT UNTIL 1 = 1 + | END REPEAT; + |END + """.stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'1'", "hint" -> "")) + } + test("repeat with complex condition") { val sqlScriptText = """ @@ -1197,6 +1256,22 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(caseStmt.conditions.head.getText == "1 = 1") } + test("searched case statement with empty body") { + val sqlScriptText = + """BEGIN + | CASE + | WHEN 1 = 1 THEN + | END CASE; + |END + """.stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'CASE'", "hint" -> "")) + } + test("searched case statement - multi when") { val sqlScriptText = """ @@ -1335,6 +1410,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { checkSimpleCaseStatementCondition(caseStmt.conditions.head, _ == Literal(1), _ == Literal(1)) } + test("simple case statement with empty body") { + val sqlScriptText = + """BEGIN + | CASE 1 + | WHEN 1 THEN + | END CASE; + |END + """.stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'CASE'", "hint" -> "")) + } test("simple case statement - multi when") { val sqlScriptText = @@ -1482,6 +1572,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(whileStmt.label.contains("lbl")) } + test("loop with empty body") { + val sqlScriptText = + """BEGIN + | LOOP + | END LOOP; + |END + """.stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'LOOP'", "hint" -> "")) + } + test("loop with if else block") { val sqlScriptText = """BEGIN @@ -1960,6 +2065,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(forStmt.label.contains("lbl")) } + test("for statement - empty body") { + val sqlScriptText = + """ + |BEGIN + | lbl: FOR x AS SELECT 5 DO + | END FOR; + |END""".stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'FOR'", "hint" -> "")) + } + test("for statement - no label") { val sqlScriptText = """ @@ -2076,6 +2196,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(forStmt.label.contains("lbl")) } + test("for statement - no variable - empty body") { + val sqlScriptText = + """ + |BEGIN + | lbl: FOR SELECT 5 DO + | END FOR; + |END""".stripMargin + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'FOR'", "hint" -> "")) + } + test("for statement - no variable - no label") { val sqlScriptText = """ From ede9cfc92f0fe6fa735cd20e6e2da79735fd35ff Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 6 Dec 2024 14:17:39 +0800 Subject: [PATCH 115/438] [SPARK-50494][INFRA] Add a separate docker file for python 3.10 daily build ### What changes were proposed in this pull request? Add a separate docker file for python 3.10 daily build ### Why are the changes needed? To isolate the envs ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? test PR builder with envs: ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-310", "PYTHON_TO_TEST": "python3.10"}' ``` https://github.com/zhengruifeng/spark/runs/33973040152 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49067 from zhengruifeng/py_image_310. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../workflows/build_infra_images_cache.yml | 14 ++++ .github/workflows/build_python_3.10.yml | 1 + dev/spark-test-image/python-310/Dockerfile | 77 +++++++++++++++++++ 3 files changed, 92 insertions(+) create mode 100644 dev/spark-test-image/python-310/Dockerfile diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index 3d5a8306aca80..031a09af69541 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -31,6 +31,7 @@ on: - 'dev/spark-test-image/lint/Dockerfile' - 'dev/spark-test-image/sparkr/Dockerfile' - 'dev/spark-test-image/python-309/Dockerfile' + - 'dev/spark-test-image/python-310/Dockerfile' - '.github/workflows/build_infra_images_cache.yml' # Create infra image when cutting down branches/tags create: @@ -116,3 +117,16 @@ jobs: - name: Image digest (PySpark with Python 3.9) if: hashFiles('dev/spark-test-image/python-309/Dockerfile') != '' run: echo ${{ steps.docker_build_pyspark_python_309.outputs.digest }} + - name: Build and push (PySpark with Python 3.10) + if: hashFiles('dev/spark-test-image/python-310/Dockerfile') != '' + id: docker_build_pyspark_python_310 + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/python-310/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-310-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-310-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-310-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark with Python 3.10) + if: hashFiles('dev/spark-test-image/python-310/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_python_310.outputs.digest }} diff --git a/.github/workflows/build_python_3.10.yml b/.github/workflows/build_python_3.10.yml index 5ae37fbc9120e..131f22935f111 100644 --- a/.github/workflows/build_python_3.10.yml +++ b/.github/workflows/build_python_3.10.yml @@ -36,6 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-310", "PYTHON_TO_TEST": "python3.10" } jobs: >- diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile new file mode 100644 index 0000000000000..b86ce88b9807f --- /dev/null +++ b/dev/spark-test-image/python-310/Dockerfile @@ -0,0 +1,77 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark with Python 3.10" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE 20241205 + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pkg-config \ + python3.10 \ + python3-psutil \ + qpdf \ + tzdata \ + wget \ + zlib1g-dev \ + && apt-get autoremove --purge -y \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +# Python deps for Spark Connect +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" + +# Install Python 3.10 packages +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 +RUN python3.10 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.10 -m pip install --ignore-installed 'six==1.16.0' # Avoid `python3-six` installation +RUN python3.10 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ + python3.10 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ + python3.10 -m pip install deepspeed torcheval && \ + python3.10 -m pip cache purge From 1d6932c30b7319f8d2a17e8a91274c044ebc8589 Mon Sep 17 00:00:00 2001 From: Jovan Markovic Date: Fri, 6 Dec 2024 11:12:12 +0100 Subject: [PATCH 116/438] [SPARK-50478][SQL] Fix StringType matching ### What changes were proposed in this pull request? In `canUpCast` method in `UpCastRule.scala` we match against `StringType`, which does not match collated string, as opposed to matching `_: StringType`. Similarly, in `needsTimeZone` method in `Cast.scala` it is matched against non collated StringType. ### Why are the changes needed? Upcasting to collated strings was disabled even when upcasting to non collated strings was not. Likewise, casting from timestamp to collated string did not need time zone. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Test added in `DataTypeWriteCompatibilitySuite.scala` and `CastSuiteBase.scala`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49043 from jovanm-db/upcast_fix. Lead-authored-by: Jovan Markovic Co-authored-by: Maxim Gekk Signed-off-by: Max Gekk --- .../scala/org/apache/spark/sql/types/UpCastRule.scala | 4 ++-- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 2 +- .../spark/sql/catalyst/expressions/CastSuiteBase.scala | 8 ++++++++ .../spark/sql/types/DataTypeWriteCompatibilitySuite.scala | 5 +++++ 4 files changed, 16 insertions(+), 3 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala index 4993e249b3059..8f2bdb2a595bf 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala @@ -40,8 +40,8 @@ private[sql] object UpCastRule { case (DateType, TimestampNTZType) => true case (TimestampNTZType, TimestampType) => true case (TimestampType, TimestampNTZType) => true - case (_: AtomicType, StringType) => true - case (_: CalendarIntervalType, StringType) => true + case (_: AtomicType, _: StringType) => true + case (_: CalendarIntervalType, _: StringType) => true case (NullType, _) => true // Spark supports casting between long and timestamp, please see `longToTimestamp` and diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 154199d37c46d..4ef49cc97f6e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -281,7 +281,7 @@ object Cast extends QueryErrorsBase { def needsTimeZone(from: DataType, to: DataType): Boolean = (from, to) match { case (VariantType, _) => true case (_: StringType, TimestampType) => true - case (TimestampType, StringType) => true + case (TimestampType, _: StringType) => true case (DateType, TimestampType) => true case (TimestampType, DateType) => true case (TimestampType, TimestampNTZType) => true diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index e87b54339821f..f6173e019cdeb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -729,6 +729,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { assert(Cast.canUpCast(DateType, TimestampNTZType)) assert(Cast.canUpCast(TimestampType, TimestampNTZType)) assert(Cast.canUpCast(TimestampNTZType, TimestampType)) + assert(Cast.canUpCast(IntegerType, StringType("UTF8_LCASE"))) + assert(Cast.canUpCast(CalendarIntervalType, StringType("UTF8_LCASE"))) assert(!Cast.canUpCast(TimestampType, DateType)) assert(!Cast.canUpCast(TimestampNTZType, DateType)) } @@ -1409,4 +1411,10 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { assert(!Cast(timestampLiteral, TimestampNTZType).resolved) assert(!Cast(timestampNTZLiteral, TimestampType).resolved) } + + test("Casting between TimestampType and StringType requires timezone") { + val timestampLiteral = Literal.create(1L, TimestampType) + assert(!Cast(timestampLiteral, StringType).resolved) + assert(!Cast(timestampLiteral, StringType("UTF8_LCASE")).resolved) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala index f07ee8b35bbb2..ba3eaf46a5597 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala @@ -685,6 +685,11 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { ) } + test("Check string types: cast allowed regardless of collation") { + assertAllowed(StringType, StringType("UTF8_LCASE"), + "date time types", "Should allow writing string to collated string") + } + // Helper functions def assertAllowed( From 934a3878f733bc823d021e15cc0b3a7093da79d6 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 6 Dec 2024 12:58:37 -0800 Subject: [PATCH 117/438] [SPARK-50483][CORE][SQL] BlockMissingException should be thrown even if ignoreCorruptFiles is enabled ### What changes were proposed in this pull request? `BlockMissingException` extends from `IOException`. When `BlockMissingException` occurs and ignoreCorruptFiles is enabled, the current task may not get any data and will be marked as successful([code](https://github.com/apache/spark/blob/0d045db8d15d0aeb0f54a1557fd360363e77ed42/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L271-L273)). This will cause data quality issues. Generally speaking, `BlockMissingException` is a system issue, not a file corruption issue. Therefore, `BlockMissingException` should be thrown even if ignoreCorruptFiles is enabled. Related error message: ``` 24/11/29 01:56:00 WARN FileScanRDD: Skipped the rest of the content in the corrupted file: path: viewfs://hadoop-cluster/path/to/data/part-00320-7915e327-3214-4585-a44e-f9c58e362b43.c000.snappy.parquet, range: 191727616-281354675, partition values: [empty row] org.apache.hadoop.hdfs.BlockMissingException: Could not obtain block: BP-169998034-10.210.23.11-1507067630530:blk_83565156183_82548880660 file/path/to/data/part-00320-7915e327-3214-4585-a44e-f9c58e362b43.c000.snappy.parquet No live nodes contain current block Block locations: DatanodeInfoWithStorage[10.209.145.174:50010,DS-c7c0a172-5ffa-4f90-bfb5-717fb1e9ecf2,DISK] DatanodeInfoWithStorage[10.3.22.142:50010,DS-a1ba9ac9-dc92-4131-a2c2-9f7d03b97caf,DISK] DatanodeInfoWithStorage[10.209.146.156:50010,DS-71d8ae97-15d3-454e-a715-d9490e184989,DISK] Dead nodes: DatanodeInfoWithStorage[10.209.146.156:50010,DS-71d8ae97-15d3-454e-a715-d9490e184989,DISK] DatanodeInfoWithStorage[10.209.145.174:50010,DS-c7c0a172-5ffa-4f90-bfb5-717fb1e9ecf2,DISK] DatanodeInfoWithStorage[10.3.22.142:50010,DS-a1ba9ac9-dc92-4131-a2c2-9f7d03b97caf,DISK] ``` ![image](https://github.com/user-attachments/assets/e040ce9d-1a0e-44eb-bd03-4cd7a9fff80f) ### Why are the changes needed? Avoid data issue if ignoreCorruptFiles is enabled when `BlockMissingException` occurred. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Manual test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49089 from wangyum/SPARK-50483. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 3 +++ core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 3 +++ .../org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala | 5 +++++ .../org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala | 3 +++ .../spark/sql/execution/datasources/FileScanRDD.scala | 2 ++ .../sql/execution/datasources/v2/FilePartitionReader.scala | 6 ++++++ .../spark/sql/execution/datasources/xml/XmlDataSource.scala | 2 ++ 7 files changed, 24 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 545eafe7a4449..7f12d3833b638 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -25,6 +25,7 @@ import java.util.{Date, Locale} import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.hdfs.BlockMissingException import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.mapred._ import org.apache.hadoop.mapred.lib.CombineFileSplit @@ -319,6 +320,7 @@ class HadoopRDD[K, V]( null // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e: BlockMissingException => throw e case e: IOException if ignoreCorruptFiles => logWarning(log"Skipped the rest content in the corrupted file: " + log"${MDC(PATH, split.inputSplit)}", e) @@ -345,6 +347,7 @@ class HadoopRDD[K, V]( finished = true // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e: BlockMissingException => throw e case e: IOException if ignoreCorruptFiles => logWarning(log"Skipped the rest content in the corrupted file: " + log"${MDC(PATH, split.inputSplit)}", e) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2b6f322d1805d..bedd0aa3a67aa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -26,6 +26,7 @@ import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.hdfs.BlockMissingException import org.apache.hadoop.io.Writable import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.mapred.JobConf @@ -255,6 +256,7 @@ class NewHadoopRDD[K, V]( null // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e: BlockMissingException => throw e case e: IOException if ignoreCorruptFiles => logWarning( log"Skipped the rest content in the corrupted file: " + @@ -284,6 +286,7 @@ class NewHadoopRDD[K, V]( finished = true // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e: BlockMissingException => throw e case e: IOException if ignoreCorruptFiles => logWarning( log"Skipped the rest content in the corrupted file: " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala index 9a0528468842c..a615ae61bd674 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala @@ -32,6 +32,7 @@ import scala.util.control.NonFatal import scala.xml.SAXException import org.apache.commons.lang3.exception.ExceptionUtils +import org.apache.hadoop.hdfs.BlockMissingException import org.apache.spark.{SparkIllegalArgumentException, SparkUpgradeException} import org.apache.spark.internal.Logging @@ -655,6 +656,10 @@ class XmlTokenizer( e) case NonFatal(e) => ExceptionUtils.getRootCause(e) match { + case _: BlockMissingException => + reader.close() + reader = null + throw e case _: RuntimeException | _: IOException if options.ignoreCorruptFiles => logWarning( "Skipping the rest of" + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala index 848e6ff45c5a2..00bba00584fdd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala @@ -30,6 +30,8 @@ import scala.util.control.Exception._ import scala.util.control.NonFatal import scala.xml.SAXException +import org.apache.hadoop.hdfs.BlockMissingException + import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -157,6 +159,7 @@ class XmlInferSchema(options: XmlOptions, caseSensitive: Boolean) logWarning("Skipped missing file", e) Some(StructType(Nil)) case e: FileNotFoundException if !options.ignoreMissingFiles => throw e + case e: BlockMissingException => throw e case e @ (_: IOException | _: RuntimeException) if options.ignoreCorruptFiles => logWarning("Skipped the rest of the content in the corrupted file", e) Some(StructType(Nil)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 9bcdbadf7c5c0..b9b34238719c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -21,6 +21,7 @@ import java.io.{Closeable, FileNotFoundException, IOException} import java.net.URI import org.apache.hadoop.fs.Path +import org.apache.hadoop.hdfs.BlockMissingException import org.apache.spark.{Partition => RDDPartition, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil @@ -266,6 +267,7 @@ class FileScanRDD( null // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e: BlockMissingException => throw e case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning(log"Skipped the rest of the content in the corrupted file: " + log"${MDC(PATH, currentFile)}", e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala index 2679f14144569..2aaca708b04bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.execution.datasources.v2 import java.io.{FileNotFoundException, IOException} +import org.apache.hadoop.hdfs.BlockMissingException + import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.{CURRENT_FILE, PARTITIONED_FILE_READER} import org.apache.spark.rdd.InputFileBlockHolder @@ -48,6 +50,8 @@ class FilePartitionReader[T]( case e: FileNotFoundException if ignoreMissingFiles => logWarning(s"Skipped missing file.", e) currentReader = null + case e: BlockMissingException => + throw FileDataSourceV2.attachFilePath(file.urlEncodedPath, e) case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning( s"Skipped the rest of the content in the corrupted file.", e) @@ -64,6 +68,8 @@ class FilePartitionReader[T]( val hasNext = try { currentReader != null && currentReader.next() } catch { + case e: BlockMissingException => + throw FileDataSourceV2.attachFilePath(currentReader.file.urlEncodedPath, e) case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning(log"Skipped the rest of the content in the corrupted file: " + log"${MDC(PARTITIONED_FILE_READER, currentReader)}", e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala index 71f285e381745..f0aefcb9b2a1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala @@ -25,6 +25,7 @@ import scala.util.control.NonFatal import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hdfs.BlockMissingException import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.mapreduce.lib.input.FileInputFormat @@ -190,6 +191,7 @@ object MultiLineXmlDataSource extends XmlDataSource { Iterator.empty[String] case NonFatal(e) => ExceptionUtils.getRootCause(e) match { + case e: BlockMissingException => throw e case _: RuntimeException | _: IOException if parsedOptions.ignoreCorruptFiles => logWarning("Skipped the rest of the content in the corrupted file", e) Iterator.empty[String] From be0780b8f931abda193ee69f65caf625f7118cb4 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 6 Dec 2024 13:10:07 -0800 Subject: [PATCH 118/438] [SPARK-50350][SQL] Avro: add new function `schema_of_avro` (`scala` side) ### What changes were proposed in this pull request? The pr aims to add new function `schema_of_avro` for `avro`. ### Why are the changes needed? - The schema format of Avro is different from that of Spark when presented to end users. In order to facilitate the intuitive understanding of Avro's schema by end users. - Similar functions exist in other formats of data, such as `csv`, `json` and `xml`, https://github.com/apache/spark/blob/87a5b37ec3c4b383a5938144612c07187d597ff8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L872-L875 https://github.com/apache/spark/blob/87a5b37ec3c4b383a5938144612c07187d597ff8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L836-L839 https://github.com/apache/spark/blob/87a5b37ec3c4b383a5938144612c07187d597ff8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L877-L880 ### Does this PR introduce _any_ user-facing change? Yes, end-users will be able to clearly know what `Avro's schema` format should look like in `Spark` through the function `schema_of_avro`. ### How was this patch tested? - Add new UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48889 from panbingkun/SPARK-50350. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- .../sql/avro/AvroExpressionEvalUtils.scala | 45 +++++++++ .../apache/spark/sql/avro/SchemaOfAvro.scala | 71 ++++++++++++++ .../spark/sql/avro/AvroFunctionsSuite.scala | 36 +++++++ .../org/apache/spark/sql/avro/functions.scala | 28 ++++++ .../catalyst/analysis/FunctionRegistry.scala | 1 + ...Functions.scala => avroSqlFunctions.scala} | 93 +++++++++++++++++++ .../sql/CollationExpressionWalkerSuite.scala | 1 + .../spark/sql/ExpressionsSchemaSuite.scala | 3 +- .../sql/expressions/ExpressionInfoSuite.scala | 1 + 9 files changed, 278 insertions(+), 1 deletion(-) create mode 100644 connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroExpressionEvalUtils.scala create mode 100644 connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{toFromAvroSqlFunctions.scala => avroSqlFunctions.scala} (69%) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroExpressionEvalUtils.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroExpressionEvalUtils.scala new file mode 100644 index 0000000000000..1a9a3609c8a5e --- /dev/null +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroExpressionEvalUtils.scala @@ -0,0 +1,45 @@ +/* + * 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.avro + +import org.apache.avro.Schema + +import org.apache.spark.sql.catalyst.util.{ParseMode, PermissiveMode} +import org.apache.spark.unsafe.types.UTF8String + +object AvroExpressionEvalUtils { + + def schemaOfAvro( + avroOptions: AvroOptions, + parseMode: ParseMode, + expectedSchema: Schema): UTF8String = { + val dt = SchemaConverters.toSqlType( + expectedSchema, + avroOptions.useStableIdForUnionType, + avroOptions.stableIdPrefixForUnionType, + avroOptions.recursiveFieldMaxDepth).dataType + val schema = parseMode match { + // With PermissiveMode, the output Catalyst row might contain columns of null values for + // corrupt records, even if some of the columns are not nullable in the user-provided schema. + // Therefore we force the schema to be all nullable here. + case PermissiveMode => dt.asNullable + case _ => dt + } + UTF8String.fromString(schema.sql) + } +} diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala new file mode 100644 index 0000000000000..094fd4254e16a --- /dev/null +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala @@ -0,0 +1,71 @@ +/* + * 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.avro + +import org.apache.avro.Schema + +import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression, Literal, RuntimeReplaceable} +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke +import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, ObjectType} + +private[sql] case class SchemaOfAvro( + jsonFormatSchema: String, + options: Map[String, String]) + extends LeafExpression with RuntimeReplaceable { + + override def dataType: DataType = SQLConf.get.defaultStringType + + override def nullable: Boolean = false + + @transient private lazy val avroOptions = AvroOptions(options) + + @transient private lazy val actualSchema = + new Schema.Parser().setValidateDefaults(false).parse(jsonFormatSchema) + + @transient private lazy val expectedSchema = avroOptions.schema.getOrElse(actualSchema) + + @transient private lazy val parseMode: ParseMode = { + val mode = avroOptions.parseMode + if (mode != PermissiveMode && mode != FailFastMode) { + throw QueryCompilationErrors.parseModeUnsupportedError( + prettyName, mode + ) + } + mode + } + + override def prettyName: String = "schema_of_avro" + + @transient private lazy val avroOptionsObjectType = ObjectType(classOf[AvroOptions]) + @transient private lazy val parseModeObjectType = ObjectType(classOf[ParseMode]) + @transient private lazy val schemaObjectType = ObjectType(classOf[Schema]) + + override def replacement: Expression = StaticInvoke( + AvroExpressionEvalUtils.getClass, + dataType, + "schemaOfAvro", + Seq( + Literal(avroOptions, avroOptionsObjectType), + Literal(parseMode, parseModeObjectType), + Literal(expectedSchema, schemaObjectType)), + Seq(avroOptionsObjectType, parseModeObjectType, schemaObjectType) + ) +} diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala index 096cdfe0b9ee4..8c128d4c7ea65 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala @@ -629,4 +629,40 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { assert(readbackPerson2.get(2).toString === person2.get(2)) } } + + test("schema_of_avro") { + val df = spark.range(1) + val avroIntType = s""" + |{ + | "type": "int", + | "name": "id" + |}""".stripMargin + checkAnswer(df.select(functions.schema_of_avro(avroIntType)), Row("INT")) + + val avroStructType = + """ + |{ + | "type": "record", + | "name": "person", + | "fields": [ + | {"name": "name", "type": "string"}, + | {"name": "age", "type": "int"}, + | {"name": "country", "type": "string"} + | ] + |}""".stripMargin + checkAnswer(df.select(functions.schema_of_avro(avroStructType)), + Row("STRUCT")) + + val avroMultiType = + """ + |{ + | "type": "record", + | "name": "person", + | "fields": [ + | {"name": "u", "type": ["int", "string"]} + | ] + |}""".stripMargin + checkAnswer(df.select(functions.schema_of_avro(avroMultiType)), + Row("STRUCT NOT NULL>")) + } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/avro/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/avro/functions.scala index fffad557aca5e..e30a9e7c2ba01 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/avro/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/avro/functions.scala @@ -94,4 +94,32 @@ object functions { def to_avro(data: Column, jsonFormatSchema: String): Column = { Column.fn("to_avro", data, lit(jsonFormatSchema)) } + + /** + * Returns schema in the DDL format of the avro schema in JSON string format. + * + * @param jsonFormatSchema + * the avro schema in JSON string format. + * + * @since 4.0.0 + */ + @Experimental + def schema_of_avro(jsonFormatSchema: String): Column = { + Column.fn("schema_of_avro", lit(jsonFormatSchema)) + } + + /** + * Returns schema in the DDL format of the avro schema in JSON string format. + * + * @param jsonFormatSchema + * the avro schema in JSON string format. + * @param options + * options to control how the Avro record is parsed. + * + * @since 4.0.0 + */ + @Experimental + def schema_of_avro(jsonFormatSchema: String, options: java.util.Map[String, String]): Column = { + Column.fnWithOptions("schema_of_avro", options.asScala.iterator, lit(jsonFormatSchema)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index d9e9f49ce065e..54f6820d2091f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -884,6 +884,7 @@ object FunctionRegistry { // Avro expression[FromAvro]("from_avro"), expression[ToAvro]("to_avro"), + expression[SchemaOfAvro]("schema_of_avro"), // Protobuf expression[FromProtobuf]("from_protobuf"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromAvroSqlFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/avroSqlFunctions.scala similarity index 69% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromAvroSqlFunctions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/avroSqlFunctions.scala index 457f469e0f687..6693ee83fd4af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromAvroSqlFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/avroSqlFunctions.scala @@ -200,3 +200,96 @@ case class ToAvro(child: Expression, jsonFormatSchema: Expression) override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("to_avro") } + +/** + * Returns schema in the DDL format of the avro schema in JSON string format. + * This is a thin wrapper over the [[SchemaOfAvro]] class to create a SQL function. + * + * @param jsonFormatSchema the Avro schema in JSON string format. + * @param options the options to use when performing the conversion. + * + * @since 4.0.0 + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(jsonFormatSchema, options) - Returns schema in the DDL format of the avro schema in JSON string format. + """, + examples = """ + Examples: + > SELECT _FUNC_('{"type": "record", "name": "struct", "fields": [{"name": "u", "type": ["int", "string"]}]}', map()); + STRUCT NOT NULL> + """, + group = "misc_funcs", + since = "4.0.0" +) +// scalastyle:on line.size.limit +case class SchemaOfAvro(jsonFormatSchema: Expression, options: Expression) + extends BinaryExpression with RuntimeReplaceable { + + override def left: Expression = jsonFormatSchema + override def right: Expression = options + + override protected def withNewChildrenInternal( + newLeft: Expression, newRight: Expression): Expression = + copy(jsonFormatSchema = newLeft, options = newRight) + + def this(jsonFormatSchema: Expression) = + this(jsonFormatSchema, Literal.create(null)) + + override def checkInputDataTypes(): TypeCheckResult = { + val schemaCheck = jsonFormatSchema.dataType match { + case _: StringType | + _: NullType + if jsonFormatSchema.foldable => + None + case _ => + Some(TypeCheckResult.TypeCheckFailure("The first argument of the SCHEMA_OF_AVRO SQL " + + "function must be a constant string containing the JSON representation of the schema " + + "to use for converting the value from AVRO format")) + } + val optionsCheck = options.dataType match { + case MapType(StringType, StringType, _) | + MapType(NullType, NullType, _) | + _: NullType + if options.foldable => + None + case _ => + Some(TypeCheckResult.TypeCheckFailure("The second argument of the SCHEMA_OF_AVRO SQL " + + "function must be a constant map of strings to strings containing the options to use " + + "for converting the value from AVRO format")) + } + schemaCheck.getOrElse( + optionsCheck.getOrElse( + TypeCheckResult.TypeCheckSuccess)) + } + + override lazy val replacement: Expression = { + val schemaValue: String = jsonFormatSchema.eval() match { + case s: UTF8String => + s.toString + case null => + "" + } + val optionsValue: Map[String, String] = options.eval() match { + case a: ArrayBasedMapData if a.keyArray.array.nonEmpty => + val keys: Array[String] = a.keyArray.array.map(_.toString) + val values: Array[String] = a.valueArray.array.map(_.toString) + keys.zip(values).toMap + case _ => + Map.empty + } + val constructor = try { + Utils.classForName("org.apache.spark.sql.avro.SchemaOfAvro").getConstructors.head + } catch { + case _: java.lang.ClassNotFoundException => + throw QueryCompilationErrors.avroNotLoadedSqlFunctionsUnusable( + functionName = "SCHEMA_OF_AVRO") + } + val expr = constructor.newInstance(schemaValue, optionsValue) + expr.asInstanceOf[Expression] + } + + override def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("schema_of_avro") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala index e3622c3101854..d0581621148a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala @@ -729,6 +729,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi // other functions which are not yet supported "to_avro", "from_avro", + "schema_of_avro", "to_protobuf", "from_protobuf" ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala index 8c0231fddf39f..0468ceb9f967c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala @@ -118,7 +118,8 @@ class ExpressionsSchemaSuite extends QueryTest with SharedSparkSession { // SET spark.sql.parser.escapedStringLiterals=true example.split(" > ").tail.filterNot(_.trim.startsWith("SET")).take(1).foreach { case _ if funcName == "from_avro" || funcName == "to_avro" || - funcName == "from_protobuf" || funcName == "to_protobuf" => + funcName == "schema_of_avro" || funcName == "from_protobuf" || + funcName == "to_protobuf" => // Skip running the example queries for the from_avro, to_avro, from_protobuf and // to_protobuf functions because these functions dynamically load the // AvroDataToCatalyst or CatalystDataToAvro classes which are not available in this diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index a6fc43aa087da..c00f00ceaa355 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -229,6 +229,7 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { // Requires dynamic class loading not available in this test suite. "org.apache.spark.sql.catalyst.expressions.FromAvro", "org.apache.spark.sql.catalyst.expressions.ToAvro", + "org.apache.spark.sql.catalyst.expressions.SchemaOfAvro", "org.apache.spark.sql.catalyst.expressions.FromProtobuf", "org.apache.spark.sql.catalyst.expressions.ToProtobuf", classOf[CurrentUser].getName, From 28766d4120f4f5bb13f474d53e83e05f38a31475 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 7 Dec 2024 10:22:15 +0800 Subject: [PATCH 119/438] [SPARK-50514][DOCS] Add `IDENTIFIER clause` page to `menu-sql.yaml` ### What changes were proposed in this pull request? This PR aims to add `IDENTIFIER clause` page to `menu-sql.yaml` for Apache Spark 3.5.4. ### Why are the changes needed? This was missed at SPARK-43205 (Apache Spark 3.5.0). - #42506 ### Does this PR introduce _any_ user-facing change? **BEFORE** ![Screenshot 2024-12-06 at 11 35 52](https://github.com/user-attachments/assets/c3c8dc56-b8d4-4f8d-bb9e-31bccb1f5d42) **AFTER** ![Screenshot 2024-12-06 at 11 36 14](https://github.com/user-attachments/assets/bd1606d2-eb3f-4640-92ef-b0079847c3a3) ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49097 from dongjoon-hyun/SPARK-50514. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- docs/_data/menu-sql.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 5fc1f3bcf9b5a..b1688aec57f01 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -93,6 +93,8 @@ url: sql-ref-functions.html - text: Identifiers url: sql-ref-identifier.html + - text: IDENTIFIER clause + url: sql-ref-identifier-clause.html - text: Literals url: sql-ref-literals.html - text: Null Semantics From c1267d697490614d5a6e1162a0eb7cfbe14302ae Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 6 Dec 2024 18:45:56 -0800 Subject: [PATCH 120/438] [SPARK-50512][SQL][DOCS] Fix `CREATE TABLE` syntax in `sql-pipe-syntax.md` ### What changes were proposed in this pull request? This PR aims to fix invalid `CREATE TABLE` syntax in `sql-pipe-syntax.md`. It seems that we missed during documentation. It causes `ParseException: Operation not allowed` . - #48852 ### Why are the changes needed? The current documentation is here. - https://apache.github.io/spark/sql-pipe-syntax.html#from-or-table **BEFORE** ``` spark-sql (default)> CREATE TABLE t(col INT) AS VALUES (0), (1); Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement. == SQL (line 1, position 1) == CREATE TABLE t(col INT) AS VALUES (0), (1) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ``` **AFTER** ``` spark-sql (default)> CREATE TABLE t AS VALUES (0), (1); Time taken: 0.142 seconds spark-sql (default)> TABLE t; 1 0 Time taken: 0.287 seconds, Fetched 2 row(s) ``` ### Does this PR introduce _any_ user-facing change? No, this is a documentation-only change of the unreleased SQL Pipe syntax. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49096 from dongjoon-hyun/SPARK-50512. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/sql-pipe-syntax.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-pipe-syntax.md b/docs/sql-pipe-syntax.md index 2c7db9f456475..829aa0e607bde 100644 --- a/docs/sql-pipe-syntax.md +++ b/docs/sql-pipe-syntax.md @@ -179,7 +179,7 @@ Returns all the output rows from the source table unmodified. For example: ```sql -CREATE TABLE t(a INT, b INT) AS VALUES (1, 2), (3, 4); +CREATE TABLE t AS VALUES (1, 2), (3, 4) AS t(a, b); TABLE t; +---+---+ @@ -207,7 +207,7 @@ provided. You may provide the window specification in the `WINDOW` clause. For example: ```sql -CREATE TABLE t(col INT) AS VALUES (0), (1); +CREATE TABLE t AS VALUES (0), (1) AS t(col); FROM t |> SELECT col * 2 AS result; From 2fea84e1334c1afc4e6f9c46eeed13365068c242 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 6 Dec 2024 18:47:07 -0800 Subject: [PATCH 121/438] [SPARK-48426][DOCS][FOLLOWUP] Add `Operators` page to `sql-ref.md` ### What changes were proposed in this pull request? This is a follow-up of SPARK-48426 to add the newly added page to `SQL Reference` page, too. - #46757 ### Why are the changes needed? **BEFORE** ![Screenshot 2024-12-06 at 11 42 07](https://github.com/user-attachments/assets/47cc4625-9897-4f73-9bac-535b638aa2a2) **AFTER** ![Screenshot 2024-12-06 at 11 43 07](https://github.com/user-attachments/assets/dbc38d97-8c65-4983-a3cb-0ea3612c069c) ### Does this PR introduce _any_ user-facing change? No, this is a new documentation page of Spark 4. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49098 from dongjoon-hyun/SPARK-48426. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/sql-ref.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sql-ref.md b/docs/sql-ref.md index 6eb2bf77c6ac0..6d557caaca3d6 100644 --- a/docs/sql-ref.md +++ b/docs/sql-ref.md @@ -26,6 +26,7 @@ Spark SQL is Apache Spark's module for working with structured data. This guide * [Data Types](sql-ref-datatypes.html) * [Datetime Pattern](sql-ref-datetime-pattern.html) * [Number Pattern](sql-ref-number-pattern.html) + * [Operators](sql-ref-operators.html) * [Functions](sql-ref-functions.html) * [Built-in Functions](sql-ref-functions-builtin.html) * [Scalar User-Defined Functions (UDFs)](sql-ref-functions-udf-scalar.html) From fff679307f56d7a6c7e04fb5705584e8d36a71b6 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Sat, 7 Dec 2024 21:25:36 +0900 Subject: [PATCH 122/438] [SPARK-50516][SS][MINOR] Fix the init state related test to use StreamManualClock ### What changes were proposed in this pull request? Fix the init state related test to use StreamManualClock ### Why are the changes needed? Fix flakiness on CI runs ``` [info] Run completed in 2 minutes, 43 seconds. [info] Total number of tests run: 22 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 22, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Test only change ### Was this patch authored or co-authored using generative AI tooling? No Closes #49102 from anishshri-db/task/SPARK-50516. Authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim --- .../TransformWithStateInitialStateSuite.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala index 806d2f19f6f5c..29f40df83f24a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala @@ -379,6 +379,8 @@ class TransformWithStateInitialStateSuite extends StateStoreMetricsTest withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName) { + val clock = new StreamManualClock + val inputData = MemoryStream[InitInputRow] val kvDataSet = inputData.toDS() .groupByKey(x => x.key) @@ -390,10 +392,12 @@ class TransformWithStateInitialStateSuite extends StateStoreMetricsTest TimeMode.None(), OutputMode.Append(), initStateDf) testStream(query, OutputMode.Update())( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), // non-exist key test AddData(inputData, InitInputRow("k1", "update", 37.0)), AddData(inputData, InitInputRow("k2", "update", 40.0)), AddData(inputData, InitInputRow("non-exist", "getOption", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("non-exist", "getOption", -1.0)), Execute { q => assert(q.lastProgress @@ -402,59 +406,80 @@ class TransformWithStateInitialStateSuite extends StateStoreMetricsTest AddData(inputData, InitInputRow("k1", "appendList", 37.0)), AddData(inputData, InitInputRow("k2", "appendList", 40.0)), AddData(inputData, InitInputRow("non-exist", "getList", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(), AddData(inputData, InitInputRow("k1", "incCount", 37.0)), AddData(inputData, InitInputRow("k2", "incCount", 40.0)), AddData(inputData, InitInputRow("non-exist", "getCount", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("non-exist", "getCount", 0.0)), + AddData(inputData, InitInputRow("k2", "incCount", 40.0)), AddData(inputData, InitInputRow("k2", "getCount", 40.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("k2", "getCount", 2.0)), // test every row in initial State is processed AddData(inputData, InitInputRow("init_1", "getOption", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getOption", 40.0)), + AddData(inputData, InitInputRow("init_2", "getOption", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_2", "getOption", 100.0)), AddData(inputData, InitInputRow("init_1", "getList", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getList", 40.0)), + AddData(inputData, InitInputRow("init_2", "getList", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_2", "getList", 100.0)), AddData(inputData, InitInputRow("init_1", "getCount", 40.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getCount", 1.0)), + AddData(inputData, InitInputRow("init_2", "getCount", 100.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_2", "getCount", 1.0)), // Update row with key in initial row will work AddData(inputData, InitInputRow("init_1", "update", 50.0)), AddData(inputData, InitInputRow("init_1", "getOption", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getOption", 50.0)), + AddData(inputData, InitInputRow("init_1", "remove", -1.0)), AddData(inputData, InitInputRow("init_1", "getOption", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getOption", -1.0)), AddData(inputData, InitInputRow("init_1", "appendList", 50.0)), AddData(inputData, InitInputRow("init_1", "getList", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getList", 50.0), ("init_1", "getList", 40.0)), AddData(inputData, InitInputRow("init_1", "incCount", 40.0)), AddData(inputData, InitInputRow("init_1", "getCount", 40.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getCount", 2.0)), // test remove AddData(inputData, InitInputRow("k1", "remove", -1.0)), AddData(inputData, InitInputRow("k1", "getOption", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("k1", "getOption", -1.0)), AddData(inputData, InitInputRow("init_1", "clearCount", -1.0)), AddData(inputData, InitInputRow("init_1", "getCount", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer(("init_1", "getCount", 0.0)), AddData(inputData, InitInputRow("init_1", "clearList", -1.0)), AddData(inputData, InitInputRow("init_1", "getList", -1.0)), + AdvanceManualClock(1 * 1000), CheckNewAnswer() ) } From bb17665955ad536d8c81605da9a59fb94b6e0162 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 8 Dec 2024 01:32:18 +0800 Subject: [PATCH 123/438] [SPARK-49249][SQL][FOLLOWUP] Rename `spark.sql.artifact.isolation.(always.apply.classloader -> alwaysApplyClassloader)` ### What changes were proposed in this pull request? This is a follow-up to rename a new configuration to comply Apache Spark config namespace. - #48120 ### Why are the changes needed? Currently, `spark.sql.artifact.isolation.always.apply.classloader` config name introduces redundant namespace `spark.sql.artifact.isolation.always.*` and `spark.sql.artifact.isolation.always.apply.*`. ``` - spark.sql.artifact.isolation.always.apply.classloader + spark.sql.artifact.isolation.alwaysApplyClassloader ``` Since we have `spark.sql.artifact.isolation.enabled` already, we had better keep the above in the same namespace `spark.sql.artifact.isolation.*`. ### Does this PR introduce _any_ user-facing change? No, this is a newly added configuration at Spark 4.0.0 ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49101 from dongjoon-hyun/SPARK-49249. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- python/pyspark/sql/connect/session.py | 2 +- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 2d544f3f0eca7..7c40f1dd54a2b 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -1044,7 +1044,7 @@ def _start_connect_server(master: str, opts: Dict[str, Any]) -> None: default_conf = { "spark.plugins": "org.apache.spark.sql.connect.SparkConnectPlugin", "spark.sql.artifact.isolation.enabled": "true", - "spark.sql.artifact.isolation.always.apply.classloader": "true", + "spark.sql.artifact.isolation.alwaysApplyClassloader": "true", } if "SPARK_TESTING" in os.environ: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c0d35fa0ce2b4..47b670b730ad5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3999,7 +3999,7 @@ object SQLConf { .createWithDefault(true) val ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER = - buildConf("spark.sql.artifact.isolation.always.apply.classloader") + buildConf("spark.sql.artifact.isolation.alwaysApplyClassloader") .internal() .doc("When enabled, the classloader holding per-session artifacts will always be applied " + "during SQL executions (useful for Spark Connect). When disabled, the classloader will " + From 6c2e87aae659039b063e8f9054535c971d8dad0d Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 9 Dec 2024 14:23:26 +0900 Subject: [PATCH 124/438] [SPARK-50507][PYTHON][TESTS][FOLLOW-UP] Add refactored package into pure Python test ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/49074 that adds refactored package into pure Python test ### Why are the changes needed? In order to fix the pure Python build https://github.com/apache/spark/actions/runs/12215379954/job/34077255570. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Will monitor the build. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49106 from HyukjinKwon/SPARK-50507-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/packaging/connect/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index de76d51d0cfdc..916a6da79de05 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -101,6 +101,7 @@ "pyspark.pandas.tests.connect.groupby", "pyspark.pandas.tests.connect.indexes", "pyspark.pandas.tests.connect.io", + "pyspark.pandas.tests.connect.pandas", "pyspark.pandas.tests.connect.plot", "pyspark.pandas.tests.connect.resample", "pyspark.pandas.tests.connect.reshape", From 85d92d7c3a6a38b1b6cfc667caac9176fab5813b Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 9 Dec 2024 08:04:37 -0800 Subject: [PATCH 125/438] [SPARK-50517][PYTHON][TESTS] Group arrow function related tests ### What changes were proposed in this pull request? Group arrow function related tests ### Why are the changes needed? test clean up ### Does this PR introduce _any_ user-facing change? no, test only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #49104 from zhengruifeng/group_connect_test_arrow. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- dev/sparktestsupport/modules.py | 20 +++++++++---------- python/pyspark/sql/tests/arrow/__init__.py | 16 +++++++++++++++ .../sql/tests/{ => arrow}/test_arrow.py | 2 +- .../{ => arrow}/test_arrow_cogrouped_map.py | 2 +- .../{ => arrow}/test_arrow_grouped_map.py | 2 +- .../sql/tests/{ => arrow}/test_arrow_map.py | 2 +- .../{ => arrow}/test_arrow_python_udf.py | 2 +- .../sql/tests/connect/arrow/__init__.py | 16 +++++++++++++++ .../connect/{ => arrow}/test_parity_arrow.py | 4 ++-- .../test_parity_arrow_cogrouped_map.py | 4 ++-- .../test_parity_arrow_grouped_map.py | 4 ++-- .../{ => arrow}/test_parity_arrow_map.py | 4 ++-- .../test_parity_arrow_python_udf.py | 4 ++-- 13 files changed, 57 insertions(+), 25 deletions(-) create mode 100644 python/pyspark/sql/tests/arrow/__init__.py rename python/pyspark/sql/tests/{ => arrow}/test_arrow.py (99%) rename python/pyspark/sql/tests/{ => arrow}/test_arrow_cogrouped_map.py (99%) rename python/pyspark/sql/tests/{ => arrow}/test_arrow_grouped_map.py (99%) rename python/pyspark/sql/tests/{ => arrow}/test_arrow_map.py (98%) rename python/pyspark/sql/tests/{ => arrow}/test_arrow_python_udf.py (99%) create mode 100644 python/pyspark/sql/tests/connect/arrow/__init__.py rename python/pyspark/sql/tests/connect/{ => arrow}/test_parity_arrow.py (97%) rename python/pyspark/sql/tests/connect/{ => arrow}/test_parity_arrow_cogrouped_map.py (86%) rename python/pyspark/sql/tests/connect/{ => arrow}/test_parity_arrow_grouped_map.py (87%) rename python/pyspark/sql/tests/connect/{ => arrow}/test_parity_arrow_map.py (88%) rename python/pyspark/sql/tests/connect/{ => arrow}/test_parity_arrow_python_udf.py (90%) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index e21a0c85de066..2a19439bd92f9 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -502,10 +502,6 @@ def __hash__(self): "pyspark.sql.observation", "pyspark.sql.tvf", # unittests - "pyspark.sql.tests.test_arrow", - "pyspark.sql.tests.test_arrow_cogrouped_map", - "pyspark.sql.tests.test_arrow_grouped_map", - "pyspark.sql.tests.test_arrow_python_udf", "pyspark.sql.tests.test_catalog", "pyspark.sql.tests.test_column", "pyspark.sql.tests.test_conf", @@ -522,12 +518,16 @@ def __hash__(self): "pyspark.sql.tests.test_functions", "pyspark.sql.tests.test_group", "pyspark.sql.tests.test_sql", + "pyspark.sql.tests.arrow.test_arrow", + "pyspark.sql.tests.arrow.test_arrow_map", + "pyspark.sql.tests.arrow.test_arrow_cogrouped_map", + "pyspark.sql.tests.arrow.test_arrow_grouped_map", + "pyspark.sql.tests.arrow.test_arrow_python_udf", "pyspark.sql.tests.pandas.test_pandas_cogrouped_map", "pyspark.sql.tests.pandas.test_pandas_grouped_map", "pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state", "pyspark.sql.tests.pandas.test_pandas_map", "pyspark.sql.tests.pandas.test_pandas_transform_with_state", - "pyspark.sql.tests.test_arrow_map", "pyspark.sql.tests.pandas.test_pandas_udf", "pyspark.sql.tests.pandas.test_pandas_udf_grouped_agg", "pyspark.sql.tests.pandas.test_pandas_udf_scalar", @@ -1029,8 +1029,6 @@ def __hash__(self): "pyspark.sql.tests.connect.test_connect_readwriter", "pyspark.sql.tests.connect.test_connect_session", "pyspark.sql.tests.connect.test_connect_stat", - "pyspark.sql.tests.connect.test_parity_arrow", - "pyspark.sql.tests.connect.test_parity_arrow_python_udf", "pyspark.sql.tests.connect.test_parity_datasources", "pyspark.sql.tests.connect.test_parity_errors", "pyspark.sql.tests.connect.test_parity_catalog", @@ -1054,9 +1052,6 @@ def __hash__(self): "pyspark.sql.tests.connect.test_parity_memory_profiler", "pyspark.sql.tests.connect.test_parity_udtf", "pyspark.sql.tests.connect.test_parity_tvf", - "pyspark.sql.tests.connect.test_parity_arrow_map", - "pyspark.sql.tests.connect.test_parity_arrow_grouped_map", - "pyspark.sql.tests.connect.test_parity_arrow_cogrouped_map", "pyspark.sql.tests.connect.test_parity_python_datasource", "pyspark.sql.tests.connect.test_parity_python_streaming_datasource", "pyspark.sql.tests.connect.test_parity_frame_plot", @@ -1073,6 +1068,11 @@ def __hash__(self): "pyspark.sql.tests.connect.test_resources", "pyspark.sql.tests.connect.shell.test_progress", "pyspark.sql.tests.connect.test_df_debug", + "pyspark.sql.tests.connect.arrow.test_parity_arrow", + "pyspark.sql.tests.connect.arrow.test_parity_arrow_map", + "pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map", + "pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map", + "pyspark.sql.tests.connect.arrow.test_parity_arrow_python_udf", "pyspark.sql.tests.connect.pandas.test_parity_pandas_map", "pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map", "pyspark.sql.tests.connect.pandas.test_parity_pandas_grouped_map_with_state", diff --git a/python/pyspark/sql/tests/arrow/__init__.py b/python/pyspark/sql/tests/arrow/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/python/pyspark/sql/tests/arrow/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/arrow/test_arrow.py similarity index 99% rename from python/pyspark/sql/tests/test_arrow.py rename to python/pyspark/sql/tests/arrow/test_arrow.py index b71bdb1eece28..a2ee113b6386e 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/arrow/test_arrow.py @@ -1778,7 +1778,7 @@ def conf(cls): if __name__ == "__main__": - from pyspark.sql.tests.test_arrow import * # noqa: F401 + from pyspark.sql.tests.arrow.test_arrow import * # noqa: F401 try: import xmlrunner # type: ignore diff --git a/python/pyspark/sql/tests/test_arrow_cogrouped_map.py b/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py similarity index 99% rename from python/pyspark/sql/tests/test_arrow_cogrouped_map.py rename to python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py index 27a520d2843ed..80b12d3a7798b 100644 --- a/python/pyspark/sql/tests/test_arrow_cogrouped_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_cogrouped_map.py @@ -334,7 +334,7 @@ def tearDownClass(cls): if __name__ == "__main__": - from pyspark.sql.tests.test_arrow_cogrouped_map import * # noqa: F401 + from pyspark.sql.tests.arrow.test_arrow_cogrouped_map import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/test_arrow_grouped_map.py b/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py similarity index 99% rename from python/pyspark/sql/tests/test_arrow_grouped_map.py rename to python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py index 213810e882fd9..c9ad602edfd27 100644 --- a/python/pyspark/sql/tests/test_arrow_grouped_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_grouped_map.py @@ -290,7 +290,7 @@ def tearDownClass(cls): if __name__ == "__main__": - from pyspark.sql.tests.test_arrow_grouped_map import * # noqa: F401 + from pyspark.sql.tests.arrow.test_arrow_grouped_map import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/test_arrow_map.py b/python/pyspark/sql/tests/arrow/test_arrow_map.py similarity index 98% rename from python/pyspark/sql/tests/test_arrow_map.py rename to python/pyspark/sql/tests/arrow/test_arrow_map.py index 2e82869230db4..71bb36a902e3e 100644 --- a/python/pyspark/sql/tests/test_arrow_map.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_map.py @@ -195,7 +195,7 @@ def tearDownClass(cls): if __name__ == "__main__": - from pyspark.sql.tests.test_arrow_map import * # noqa: F401 + from pyspark.sql.tests.arrow.test_arrow_map import * # noqa: F401 try: import xmlrunner diff --git a/python/pyspark/sql/tests/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py similarity index 99% rename from python/pyspark/sql/tests/test_arrow_python_udf.py rename to python/pyspark/sql/tests/arrow/test_arrow_python_udf.py index a3fd8c01992ad..1f430d1ab00d9 100644 --- a/python/pyspark/sql/tests/test_arrow_python_udf.py +++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py @@ -253,7 +253,7 @@ def tearDownClass(cls): if __name__ == "__main__": - from pyspark.sql.tests.test_arrow_python_udf import * # noqa: F401 + from pyspark.sql.tests.arrow.test_arrow_python_udf import * # noqa: F401 try: import xmlrunner diff --git a/python/pyspark/sql/tests/connect/arrow/__init__.py b/python/pyspark/sql/tests/connect/arrow/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/python/pyspark/sql/tests/connect/arrow/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow.py similarity index 97% rename from python/pyspark/sql/tests/connect/test_parity_arrow.py rename to python/pyspark/sql/tests/connect/arrow/test_parity_arrow.py index 885b3001b1db1..fa8cf286b9bd6 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow.py @@ -17,7 +17,7 @@ import unittest -from pyspark.sql.tests.test_arrow import ArrowTestsMixin +from pyspark.sql.tests.arrow.test_arrow import ArrowTestsMixin from pyspark.testing.connectutils import ReusedConnectTestCase from pyspark.testing.pandasutils import PandasOnSparkTestUtils @@ -139,7 +139,7 @@ def test_create_dataframe_namedtuples(self): if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_arrow import * # noqa: F401 + from pyspark.sql.tests.connect.arrow.test_parity_arrow import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow_cogrouped_map.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_cogrouped_map.py similarity index 86% rename from python/pyspark/sql/tests/connect/test_parity_arrow_cogrouped_map.py rename to python/pyspark/sql/tests/connect/arrow/test_parity_arrow_cogrouped_map.py index 90c5f2c9b0613..c14c69b5ed4ec 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow_cogrouped_map.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_cogrouped_map.py @@ -17,7 +17,7 @@ import unittest -from pyspark.sql.tests.test_arrow_cogrouped_map import CogroupedMapInArrowTestsMixin +from pyspark.sql.tests.arrow.test_arrow_cogrouped_map import CogroupedMapInArrowTestsMixin from pyspark.testing.connectutils import ReusedConnectTestCase @@ -26,7 +26,7 @@ class CogroupedMapInArrowParityTests(CogroupedMapInArrowTestsMixin, ReusedConnec if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_arrow_cogrouped_map import * # noqa: F401 + from pyspark.sql.tests.connect.arrow.test_parity_arrow_cogrouped_map import * # noqa: F401 try: import xmlrunner diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow_grouped_map.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_grouped_map.py similarity index 87% rename from python/pyspark/sql/tests/connect/test_parity_arrow_grouped_map.py rename to python/pyspark/sql/tests/connect/arrow/test_parity_arrow_grouped_map.py index 0fb96ba13b838..ca12a8b06fdbf 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow_grouped_map.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_grouped_map.py @@ -17,7 +17,7 @@ import unittest -from pyspark.sql.tests.test_arrow_grouped_map import GroupedMapInArrowTestsMixin +from pyspark.sql.tests.arrow.test_arrow_grouped_map import GroupedMapInArrowTestsMixin from pyspark.testing.connectutils import ReusedConnectTestCase @@ -26,7 +26,7 @@ class GroupedApplyInArrowParityTests(GroupedMapInArrowTestsMixin, ReusedConnectT if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_arrow_grouped_map import * # noqa: F401 + from pyspark.sql.tests.connect.arrow.test_parity_arrow_grouped_map import * # noqa: F401 try: import xmlrunner diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow_map.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_map.py similarity index 88% rename from python/pyspark/sql/tests/connect/test_parity_arrow_map.py rename to python/pyspark/sql/tests/connect/arrow/test_parity_arrow_map.py index ed51d0d3d1996..1da356f524f31 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow_map.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_map.py @@ -17,7 +17,7 @@ import unittest -from pyspark.sql.tests.test_arrow_map import MapInArrowTestsMixin +from pyspark.sql.tests.arrow.test_arrow_map import MapInArrowTestsMixin from pyspark.testing.connectutils import ReusedConnectTestCase @@ -26,7 +26,7 @@ class ArrowMapParityTests(MapInArrowTestsMixin, ReusedConnectTestCase): if __name__ == "__main__": - from pyspark.sql.tests.connect.test_parity_arrow_map import * # noqa: F401 + from pyspark.sql.tests.connect.arrow.test_parity_arrow_map import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow_python_udf.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py similarity index 90% rename from python/pyspark/sql/tests/connect/test_parity_arrow_python_udf.py rename to python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py index 732008eb05a35..fe81513f005f9 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow_python_udf.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py @@ -16,7 +16,7 @@ # from pyspark.sql.tests.connect.test_parity_udf import UDFParityTests -from pyspark.sql.tests.test_arrow_python_udf import PythonUDFArrowTestsMixin +from pyspark.sql.tests.arrow.test_arrow_python_udf import PythonUDFArrowTestsMixin class ArrowPythonUDFParityTests(UDFParityTests, PythonUDFArrowTestsMixin): @@ -35,7 +35,7 @@ def tearDownClass(cls): if __name__ == "__main__": import unittest - from pyspark.sql.tests.connect.test_parity_arrow_python_udf import * # noqa: F401 + from pyspark.sql.tests.connect.arrow.test_parity_arrow_python_udf import * # noqa: F401 try: import xmlrunner # type: ignore[import] From 88102d35d1f8839a9763e68586932f6cba892dad Mon Sep 17 00:00:00 2001 From: Mihailo Milosevic Date: Mon, 9 Dec 2024 12:08:11 -0800 Subject: [PATCH 126/438] [SPARK-50497][SQL] Fail queries with proper message if MultiAlias contains non-generator function ### What changes were proposed in this pull request? MultiAlias fails if the underlying function is not a generator. ### Why are the changes needed? Syntax for multi aliasing should return proper error and not fail in check analysis with unresolved. ### Does this PR introduce _any_ user-facing change? Yes, error message changes ### How was this patch tested? Added test. ### Was this patch authored or co-authored using generative AI tooling? Yes. Closes #49072 from mihailom-db/fixGenerators. Authored-by: Mihailo Milosevic Signed-off-by: Wenchen Fan --- .../utils/src/main/resources/error/error-conditions.json | 6 ++++++ .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++++ .../spark/sql/catalyst/analysis/AnalysisSuite.scala | 8 ++++++++ 3 files changed, 18 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index b18db93f6291a..3acecf6c7e338 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3574,6 +3574,12 @@ ], "sqlState" : "42710" }, + "MULTI_ALIAS_WITHOUT_GENERATOR" : { + "message" : [ + "Multi part aliasing () is not supported with as it is not a generator function." + ], + "sqlState" : "42K0E" + }, "MULTI_SOURCES_UNSUPPORTED_FOR_EXPRESSION" : { "message" : [ "The expression does not support more than one source." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index d4b97ff037f36..d4370e3421c46 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -482,6 +482,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB errorClass = "UNBOUND_SQL_PARAMETER", messageParameters = Map("name" -> p.name)) + case ma @ MultiAlias(child, names) if child.resolved && !child.isInstanceOf[Generator] => + ma.failAnalysis( + errorClass = "MULTI_ALIAS_WITHOUT_GENERATOR", + messageParameters = Map("expr" -> toSQLExpr(child), "names" -> names.mkString(", "))) case _ => }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 939801e3f07af..ae27985a3ba64 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -777,6 +777,14 @@ class AnalysisSuite extends AnalysisTest with Matchers { PosExplode($"list"), Seq("first_pos", "first_val")), Seq("second_pos", "second_val")))) } + test("SPARK-50497 Non-generator function with multiple aliases") { + assertAnalysisErrorCondition(parsePlan("SELECT 'length' (a)"), + "MULTI_ALIAS_WITHOUT_GENERATOR", + Map("expr" -> "\"length\"", "names" -> "a"), + Array(ExpectedContext("SELECT 'length' (a)", 0, 18)) + ) + } + test("SPARK-24151: CURRENT_DATE, CURRENT_TIMESTAMP should be case insensitive") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { val input = Project(Seq( From 88c38138f1553bc2d0a218e8a3ce53011f17c38b Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Mon, 9 Dec 2024 13:40:55 -0800 Subject: [PATCH 127/438] [SPARK-50503][SQL] Prohibit partitioning by Variant data ### What changes were proposed in this pull request? Prior to this PR, repartition by Variant producing expressions wasn't blocked during analysis. It should be blocked because Variant equality is not defined. It is similar to [this PR](https://github.com/apache/spark/pull/48909) which blocked Variant from Set operations. ### Why are the changes needed? Variant equality is not defined yet and therefore shouldn't be allowed in repartitioning. ### Does this PR introduce _any_ user-facing change? Yes, prior to this PR, Variants repartition did not throw a well defined error. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49080 from harshmotw-db/harsh-motwani_data/variant_repartition. Authored-by: Harsh Motwani Signed-off-by: Herman van Hovell --- .../resources/error/error-conditions.json | 5 ++ .../sql/catalyst/analysis/CheckAnalysis.scala | 15 +++++ .../org/apache/spark/sql/DataFrameSuite.scala | 63 +++++++++++++++++++ 3 files changed, 83 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 3acecf6c7e338..cbd85335679d6 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5309,6 +5309,11 @@ "Parameter markers are not allowed in ." ] }, + "PARTITION_BY_VARIANT" : { + "message" : [ + "Cannot use VARIANT producing expressions to partition a DataFrame, but the type of expression is ." + ] + }, "PARTITION_WITH_NESTED_COLUMN_IS_UNSUPPORTED" : { "message" : [ "Invalid partitioning: is missing or is in a map or array." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index d4370e3421c46..484777c454fea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -96,6 +96,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case _ => None } + protected def variantExprInPartitionExpression(plan: LogicalPlan): Option[Expression] = + plan match { + case r: RepartitionByExpression => + r.partitionExpressions.find(e => hasVariantType(e.dataType)) + case _ => None + } + private def checkLimitLikeClause(name: String, limitExpr: Expression): Unit = { limitExpr match { case e if !e.foldable => limitExpr.failAnalysis( @@ -853,6 +860,14 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB "colName" -> toSQLId(variantCol.name), "dataType" -> toSQLType(variantCol.dataType))) + case o if variantExprInPartitionExpression(o).isDefined => + val variantExpr = variantExprInPartitionExpression(o).get + o.failAnalysis( + errorClass = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", + messageParameters = Map( + "expr" -> variantExpr.sql, + "dataType" -> toSQLType(variantExpr.dataType))) + case o if o.expressions.exists(!_.deterministic) && !operatorAllowsNonDeterministicExpressions(o) && !o.isInstanceOf[Project] && diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index c1d977dad82d2..0972a63a2495c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -308,6 +308,69 @@ class DataFrameSuite extends QueryTest testData.select("key").collect().toSeq) } + test("SPARK-50503 - cannot partition by variant columns") { + val df = sql("select parse_json(case when id = 0 then 'null' else '1' end)" + + " as v, id % 5 as id, named_struct('v', parse_json(id::string)) s from range(0, 100, 1, 5)") + // variant column + checkError( + exception = intercept[AnalysisException](df.repartition(5, col("v"))), + condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", + parameters = Map( + "expr" -> "v", + "dataType" -> "\"VARIANT\"") + ) + // nested variant column + checkError( + exception = intercept[AnalysisException](df.repartition(5, col("s"))), + condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", + parameters = Map( + "expr" -> "s", + "dataType" -> "\"STRUCT\"") + ) + // variant producing expression + checkError( + exception = + intercept[AnalysisException](df.repartition(5, parse_json(col("id").cast("string")))), + condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", + parameters = Map( + "expr" -> "parse_json(CAST(id AS STRING))", + "dataType" -> "\"VARIANT\"") + ) + // Partitioning by non-variant column works + try { + df.repartition(5, col("id")).collect() + } catch { + case e: Exception => + fail(s"Expected no exception to be thrown but an exception was thrown: ${e.getMessage}") + } + // SQL + withTempView("tv") { + df.createOrReplaceTempView("tv") + checkError( + exception = intercept[AnalysisException](sql("SELECT * FROM tv DISTRIBUTE BY v")), + condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", + parameters = Map( + "expr" -> "tv.v", + "dataType" -> "\"VARIANT\""), + context = ExpectedContext( + fragment = "DISTRIBUTE BY v", + start = 17, + stop = 31) + ) + checkError( + exception = intercept[AnalysisException](sql("SELECT * FROM tv DISTRIBUTE BY s")), + condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", + parameters = Map( + "expr" -> "tv.s", + "dataType" -> "\"STRUCT\""), + context = ExpectedContext( + fragment = "DISTRIBUTE BY s", + start = 17, + stop = 31) + ) + } + } + test("repartition with SortOrder") { // passing SortOrder expressions to .repartition() should result in an informative error From 32431cf35975549e979b0c0d63c01ef2c1a8915a Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Mon, 9 Dec 2024 13:53:18 -0800 Subject: [PATCH 128/438] [SPARK-50063][SQL][CONNECT] Add support for Variant in the Spark Connect Scala client ### What changes were proposed in this pull request? This PR adds support for Variant in the Spark Connect Scala client by adding cases for `VariantEncoder` in `ArrowSerializer.scala` and `ArrowDeserializer.scala`. ### Why are the changes needed? The Spark Connect Scala client does not currently support Variants. This PR enables this support. ### Does this PR introduce _any_ user-facing change? Yes, users using the Spark Connect Scala client will now be able to interact with Variants. ### How was this patch tested? Unit tests to round trip Arrow encoding and decoding of Variants. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49057 from harshmotw-db/harsh-motwani_data/variant_connect. Authored-by: Harsh Motwani Signed-off-by: Herman van Hovell --- .../DataFrameTableValuedFunctionsSuite.scala | 6 +-- .../spark/sql/SQLExpressionsSuite.scala | 47 +++++++++++++++++++ .../client/arrow/ArrowEncoderSuite.scala | 47 +++++++++++++++++++ .../client/arrow/ArrowDeserializer.scala | 31 +++++++++++- .../client/arrow/ArrowSerializer.scala | 19 +++++++- 5 files changed, 144 insertions(+), 6 deletions(-) create mode 100644 connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLExpressionsSuite.scala diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala index 4c0357a3ed984..f4a0d1bcf5d6f 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala @@ -205,8 +205,7 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual, expected) } - // TODO(SPARK-50063): Support VARIANT in Spark Connect Scala client - ignore("variant_explode") { + test("variant_explode") { val actual1 = spark.tvf.variant_explode(parse_json(lit("""["hello", "world"]"""))) val expected1 = spark.sql("""SELECT * FROM variant_explode(parse_json('["hello", "world"]'))""") @@ -237,8 +236,7 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } - // TODO(SPARK-50063): Support VARIANT in Spark Connect Scala client - ignore("variant_explode_outer") { + test("variant_explode_outer") { val actual1 = spark.tvf.variant_explode_outer(parse_json(lit("""["hello", "world"]"""))) val expected1 = spark.sql("""SELECT * FROM variant_explode_outer(parse_json('["hello", "world"]'))""") diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLExpressionsSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLExpressionsSuite.scala new file mode 100644 index 0000000000000..fcd2b3a388042 --- /dev/null +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLExpressionsSuite.scala @@ -0,0 +1,47 @@ +/* + * 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 + +import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} +import org.apache.spark.unsafe.types.VariantVal + +class SQLExpressionsSuite extends QueryTest with RemoteSparkSession { + + test("variants") { + val topLevelVariants = spark.sql("select parse_json(id::string) from range(10)") + checkAnswer( + topLevelVariants, + (0 until 10) + .map(i => Row(new VariantVal(Array[Byte](12, i.toByte), Array[Byte](1, 0, 0))))) + val structsOfVariants = spark.sql("select struct(parse_json(id::string)) from range(10)") + checkAnswer( + structsOfVariants, + (0 until 10) + .map(i => Row(Row(new VariantVal(Array[Byte](12, i.toByte), Array[Byte](1, 0, 0)))))) + val arraysOfVariants = spark.sql("select array(parse_json(id::string)) from range(10)") + checkAnswer( + arraysOfVariants, + (0 until 10) + .map(i => Row(Seq(new VariantVal(Array[Byte](12, i.toByte), Array[Byte](1, 0, 0)))))) + val mapsOfVariants = spark.sql("select map(id, parse_json(id::string)) from range(10)") + checkAnswer( + mapsOfVariants, + (0 until 10) + .map(i => Row(Map((i, new VariantVal(Array[Byte](12, i.toByte), Array[Byte](1, 0, 0))))))) + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala index 10e4c11c406fe..d0468c8d57b58 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala @@ -45,6 +45,7 @@ import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.connect.client.arrow.FooEnum.FooEnum import org.apache.spark.sql.test.ConnectFunSuite import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, IntegerType, Metadata, SQLUserDefinedType, StringType, StructType, UserDefinedType, YearMonthIntervalType} +import org.apache.spark.unsafe.types.VariantVal /** * Tests for encoding external data to and from arrow. @@ -264,6 +265,52 @@ class ArrowEncoderSuite extends ConnectFunSuite with BeforeAndAfterAll { assert(inspector.numBatches == 1) } + test("variant round trip") { + val variantEncoder = toRowEncoder(new StructType().add("v", "variant")) + roundTripAndCheckIdentical(variantEncoder) { () => + val maybeNull = MaybeNull(7) + Iterator.tabulate(101)(i => + Row(maybeNull(new VariantVal(Array[Byte](12, i.toByte), Array[Byte](1, 0, 0))))) + } + + val nestedVariantEncoder = toRowEncoder( + new StructType() + .add( + "s", + new StructType() + .add("i1", "int") + .add("v1", "variant") + .add("i2", "int") + .add("v2", "variant")) + .add("a", "array") + .add("m", "map")) + + roundTripAndCheckIdentical(nestedVariantEncoder) { () => + val maybeNull5 = MaybeNull(5) + val maybeNull7 = MaybeNull(7) + val maybeNull11 = MaybeNull(11) + val maybeNull13 = MaybeNull(13) + val maybeNull17 = MaybeNull(17) + Iterator.tabulate(100)(i => + Row( + maybeNull5( + Row( + i, + maybeNull7(new VariantVal(Array[Byte](12, i.toByte), Array[Byte](1, 0, 0))), + i + 1, + maybeNull11( + new VariantVal(Array[Byte](12, (i + 1).toByte), Array[Byte](1, 0, 0))))), + maybeNull7((0 until 10).map(j => + new VariantVal(Array[Byte](12, (i + j).toByte), Array[Byte](1, 0, 0)))), + maybeNull13( + Map( + ( + i.toString, + maybeNull17( + new VariantVal(Array[Byte](12, (i + 2).toByte), Array[Byte](1, 0, 0)))))))) + } + } + test("multiple batches - split by record count") { val inspector = new CountingBatchInspector roundTripAndCheckIdentical( diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index f3abaddb0110b..4618c7e24d4ac 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.errors.{CompilationErrors, ExecutionErrors} import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.VariantVal /** * Helper class for converting arrow batches into user objects. @@ -336,6 +337,34 @@ object ArrowDeserializers { } } + case (VariantEncoder, StructVectors(struct, vectors)) => + assert(vectors.exists(_.getName == "value")) + assert( + vectors.exists(field => + field.getName == "metadata" && field.getField.getMetadata + .containsKey("variant") && field.getField.getMetadata.get("variant") == "true")) + val valueDecoder = + deserializerFor( + BinaryEncoder, + vectors + .find(_.getName == "value") + .getOrElse(throw CompilationErrors.columnNotFoundError("value")), + timeZoneId) + val metadataDecoder = + deserializerFor( + BinaryEncoder, + vectors + .find(_.getName == "metadata") + .getOrElse(throw CompilationErrors.columnNotFoundError("metadata")), + timeZoneId) + new StructFieldSerializer[VariantVal](struct) { + def value(i: Int): VariantVal = { + new VariantVal( + valueDecoder.get(i).asInstanceOf[Array[Byte]], + metadataDecoder.get(i).asInstanceOf[Array[Byte]]) + } + } + case (JavaBeanEncoder(tag, fields), StructVectors(struct, vectors)) => val constructor = methodLookup.findConstructor(tag.runtimeClass, MethodType.methodType(classOf[Unit])) @@ -366,7 +395,7 @@ object ArrowDeserializers { override def get(i: Int): Any = codec.decode(deserializer.get(i)) } - case (CalendarIntervalEncoder | VariantEncoder | _: UDTEncoder[_], _) => + case (CalendarIntervalEncoder | _: UDTEncoder[_], _) => throw ExecutionErrors.unsupportedDataTypeError(encoder.dataType) case _ => diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala index f8a5c63ac3abe..c01390bf07857 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala @@ -42,6 +42,7 @@ import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.errors.ExecutionErrors import org.apache.spark.sql.types.Decimal import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.unsafe.types.VariantVal /** * Helper class for converting user objects into arrow batches. @@ -433,6 +434,22 @@ object ArrowSerializer { case (RowEncoder(fields), StructVectors(struct, vectors)) => structSerializerFor(fields, struct, vectors) { (_, i) => r => r.asInstanceOf[Row].get(i) } + case (VariantEncoder, StructVectors(struct, vectors)) => + assert(vectors.exists(_.getName == "value")) + assert( + vectors.exists(field => + field.getName == "metadata" && field.getField.getMetadata + .containsKey("variant") && field.getField.getMetadata.get("variant") == "true")) + new StructSerializer( + struct, + Seq( + new StructFieldSerializer( + extractor = (v: Any) => v.asInstanceOf[VariantVal].getValue, + serializerFor(BinaryEncoder, struct.getChild("value"))), + new StructFieldSerializer( + extractor = (v: Any) => v.asInstanceOf[VariantVal].getMetadata, + serializerFor(BinaryEncoder, struct.getChild("metadata"))))) + case (JavaBeanEncoder(tag, fields), StructVectors(struct, vectors)) => structSerializerFor(fields, struct, vectors) { (field, _) => val getter = methodLookup.findVirtual( @@ -450,7 +467,7 @@ object ArrowSerializer { delegate.write(index, codec.encode(value)) } - case (CalendarIntervalEncoder | VariantEncoder | _: UDTEncoder[_], _) => + case (CalendarIntervalEncoder | _: UDTEncoder[_], _) => throw ExecutionErrors.unsupportedDataTypeError(encoder.dataType) case _ => From 74eeceda08e02650f7a3c91df23e10a07d1e0ad4 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Tue, 10 Dec 2024 00:57:05 +0300 Subject: [PATCH 129/438] [SPARK-50523][SQL] Fix casts on complex types in collation coercion ### What changes were proposed in this pull request? Add proper support and tests for finding the collation context of cast expressions, especially on complex types. ### Why are the changes needed? #48936 introduced a way for handling collation type coercion in complex types, however I didn't think about casting, which should be special cased when searching for the collation context of an expression. This means that currently collation context could return the type which is not the target type of the cast, and that is just incorrect. Also we should be able to use casts to change collation of complex types (arrays, maps, structs), and since we can't just use `collate` clause on complex types we should assign IMPLICIT strength to user defined casts on complex types. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49075 from stefankandic/fixComplexTypesCast. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../analysis/CollationTypeCoercion.scala | 31 +++++++--- .../CollationTypePrecedenceSuite.scala | 62 +++++++++++++++++++ 2 files changed, 83 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index 02640aba2d283..c6a2213a496e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -149,16 +149,18 @@ object CollationTypeCoercion { * should be that of the childType. */ private def shouldRemoveCast(cast: Cast): Boolean = { - val isUserDefined = cast.getTagValue(Cast.USER_SPECIFIED_CAST).isDefined val isChildTypeCollatedString = cast.child.dataType match { case st: StringType => !st.isUTF8BinaryCollation case _ => false } val targetType = cast.dataType - isUserDefined && isChildTypeCollatedString && targetType == StringType + isUserDefined(cast) && isChildTypeCollatedString && targetType == StringType } + private def isUserDefined(cast: Cast): Boolean = + cast.getTagValue(Cast.USER_SPECIFIED_CAST).isDefined + /** * Changes the data type of the expression to the given `newType`. */ @@ -167,19 +169,12 @@ object CollationTypeCoercion { case Some(newDataType) if newDataType != expr.dataType => assert(!newDataType.existsRecursively(_.isInstanceOf[StringTypeWithContext])) - val exprWithNewType = expr match { + expr match { case lit: Literal => lit.copy(dataType = newDataType) case cast: Cast => cast.copy(dataType = newDataType) case _ => Cast(expr, newDataType) } - // also copy the collation context tag - if (hasCollationContextTag(expr)) { - exprWithNewType.setTagValue( - COLLATION_CONTEXT_TAG, expr.getTagValue(COLLATION_CONTEXT_TAG).get) - } - exprWithNewType - case _ => expr } @@ -362,6 +357,15 @@ object CollationTypeCoercion { case collate: Collate => Some(addContextToStringType(collate.dataType, Explicit)) + case cast: Cast => + if (isUserDefined(cast) && isComplexType(cast.dataType)) { + // since we can't use collate clause with complex types + // user defined casts should be treated as implicit + Some(addContextToStringType(cast.dataType, Implicit)) + } else { + Some(addContextToStringType(cast.dataType, Default)) + } + case expr @ (_: Alias | _: SubqueryExpression | _: AttributeReference | _: VariableReference) => Some(addContextToStringType(expr.dataType, Implicit)) @@ -435,6 +439,13 @@ object CollationTypeCoercion { else right } } + + private def isComplexType(dataType: DataType): Boolean = { + dataType match { + case _: ArrayType | _: MapType | _: StructType => true + case _ => false + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index 23d0d4ad8c21c..6f33633cc11b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -438,6 +438,32 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { sql(s"SELECT map('key1', 'val1' collate utf8_lcase, 'key2', 'val2' collate unicode)")) } + test("user defined cast on maps") { + checkAnswer( + sql(s""" + |SELECT map_contains_key( + | map('a' collate utf8_lcase, 'b'), + | 'A' collate utf8_lcase) + |""".stripMargin), + Seq(Row(true))) + + checkAnswer( + sql(s""" + |SELECT map_contains_key( + | CAST(map('a' collate utf8_lcase, 'b') AS MAP), + | 'A') + |""".stripMargin), + Seq(Row(false))) + + checkAnswer( + sql(s""" + |SELECT map_contains_key( + | CAST(map('a' collate utf8_lcase, 'b') AS MAP), + | 'A' COLLATE UNICODE) + |""".stripMargin), + Seq(Row(false))) + } + test("maps of structs") { assertQuerySchema( sql(s"SELECT map('key1', struct(1, 'a' collate unicode), 'key2', struct(2, 'b'))"), @@ -482,6 +508,42 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { sql(s"SELECT array('a', 'b' collate unicode) = array('A' collate utf8_lcase, 'B')")) } + test("user defined cast on arrays") { + checkAnswer( + sql(s""" + |SELECT array_contains( + | array('a', 'b' collate utf8_lcase), + | 'A') + |""".stripMargin), + Seq(Row(true))) + + // should be false because ARRAY should take precedence + // over UTF8_LCASE in array creation + checkAnswer( + sql(s""" + |SELECT array_contains( + | CAST(array('a', 'b' collate utf8_lcase) AS ARRAY), + | 'A') + |""".stripMargin), + Seq(Row(false))) + + checkAnswer( + sql(s""" + |SELECT array_contains( + | CAST(array('a', 'b' collate utf8_lcase) AS ARRAY), + | 'A') + |""".stripMargin), + Seq(Row(false))) + + checkAnswer( + sql(s""" + |SELECT array_contains( + | CAST(array('a', 'b' collate utf8_lcase) AS ARRAY), + | 'A' collate unicode) + |""".stripMargin), + Seq(Row(false))) + } + test("array of structs") { assertQuerySchema( sql(s"SELECT array(struct(1, 'a' collate unicode), struct(2, 'b'))[0]"), From 290b4b31bae2e02b648d2c5ef61183f337b18f8f Mon Sep 17 00:00:00 2001 From: WweiL Date: Tue, 10 Dec 2024 07:48:03 +0900 Subject: [PATCH 130/438] [SPARK-49461][SS][TESTS][FOLLOWUP] Add compatibility test for new commit log ### What changes were proposed in this pull request? Per comment from another PR: https://github.com/apache/spark/pull/48355#discussion_r1868853316 Add a new compatibility test. This test verifies the following scenario: A Spark running under version 3.5 trying to read a commit log generated with the new changelog change but under V1. Then the commit log would look like: ``` {"nextBatchWatermarkMs":1,"stateUniqueIds":{}} ``` But in Spark 3.5 and before, there is no such `stateUniqueIds` field, so we need to make sure queries won't throw error in such cases. In the new test, I create a `CommitMetadataLegacy` that only has `nextBatchWatermarkMs` and no `stateUniqueIds`, to read from a commit log as above. This simulates the scenario stated above, and the test passed. ### Why are the changes needed? More testing ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Test only addition ### Was this patch authored or co-authored using generative AI tooling? No Closes #49063 from WweiL/commitLog-followup. Authored-by: WweiL Signed-off-by: Jungtaek Lim --- .../spark/sql/streaming/CommitLogSuite.scala | 62 ++++++++++++++++++- 1 file changed, 60 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala index 92bea82b35874..bdf4163c72068 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala @@ -17,11 +17,18 @@ package org.apache.spark.sql.streaming -import java.io.{ByteArrayInputStream, FileInputStream, FileOutputStream} +import java.io.{ByteArrayInputStream, FileInputStream, FileOutputStream, InputStream, OutputStream} +import java.nio.charset.StandardCharsets.UTF_8 import java.nio.file.Path +import scala.io.{Source => IOSource} + +import org.json4s.{Formats, NoTypeHints} +import org.json4s.jackson.Serialization + import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.streaming.{CommitLog, CommitMetadata} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.{CommitLog, CommitMetadata, HDFSMetadataLog} import org.apache.spark.sql.test.SharedSparkSession class CommitLogSuite extends SparkFunSuite with SharedSparkSession { @@ -108,6 +115,57 @@ class CommitLogSuite extends SparkFunSuite with SharedSparkSession { assert(commitMetadata.nextBatchWatermarkMs === 233) assert(commitMetadata.stateUniqueIds === Map.empty) } + + // Test an old version of Spark can ser-de the new version of commit log, + // but running under V1 (i.e. no stateUniqueIds) + test("v1 Serde backward compatibility") { + // This is the json created by a V1 commit log + val commitLogV1WithStateUniqueId = """v1 + |{"nextBatchWatermarkMs":1,"stateUniqueIds":{}}""".stripMargin + val inputStream: ByteArrayInputStream = + new ByteArrayInputStream(commitLogV1WithStateUniqueId.getBytes("UTF-8")) + val commitMetadata: CommitMetadataLegacy = new CommitLogLegacy( + spark, testCommitLogV1FilePath.toString).deserialize(inputStream) + assert(commitMetadata.nextBatchWatermarkMs === 1) + } +} + +// DO-NOT-MODIFY-THE-CODE-BELOW +// Below are the legacy commit log code carbon copied from Spark branch-3.5, except +// adding a "Legacy" to the class names. +case class CommitMetadataLegacy(nextBatchWatermarkMs: Long = 0) { + def json: String = Serialization.write(this)(CommitMetadataLegacy.format) +} + +object CommitMetadataLegacy { + implicit val format: Formats = Serialization.formats(NoTypeHints) + + def apply(json: String): CommitMetadataLegacy = Serialization.read[CommitMetadataLegacy](json) } +class CommitLogLegacy(sparkSession: SparkSession, path: String) + extends HDFSMetadataLog[CommitMetadataLegacy](sparkSession, path) { + + private val VERSION = 1 + private val EMPTY_JSON = "{}" + override def deserialize(in: InputStream): CommitMetadataLegacy = { + // called inside a try-finally where the underlying stream is closed in the caller + val lines = IOSource.fromInputStream(in, UTF_8.name()).getLines() + if (!lines.hasNext) { + throw new IllegalStateException("Incomplete log file in the offset commit log") + } + validateVersion(lines.next().trim, VERSION) + val metadataJson = if (lines.hasNext) lines.next() else EMPTY_JSON + CommitMetadataLegacy(metadataJson) + } + + override def serialize(metadata: CommitMetadataLegacy, out: OutputStream): Unit = { + // called inside a try-finally where the underlying stream is closed in the caller + out.write(s"v${VERSION}".getBytes(UTF_8)) + out.write('\n') + + // write metadata + out.write(metadata.json.getBytes(UTF_8)) + } +} From 0830a190b993ee630b58547651aa9fa529718df4 Mon Sep 17 00:00:00 2001 From: Haoyu Qi Date: Mon, 9 Dec 2024 16:15:12 -0800 Subject: [PATCH 131/438] [SPARK-50329][SQL] fix InSet$toString ### What changes were proposed in this pull request? Fix InSet$toString for unresolved plan node ### Why are the changes needed? InSet$toString should always work even for unresolved node ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? end to end test by running TPCDS benchmark suite with planChangeLog enabled. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48865 from averyqi-db/patch-1. Authored-by: Haoyu Qi Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/predicates.scala | 3 +++ .../catalyst/optimizer/OptimizerLoggingSuite.scala | 11 +++++++++++ 2 files changed, 14 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 986bc63363d5d..5e402fa2b6caa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -609,6 +609,9 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with require(hset != null, "hset could not be null") override def toString: String = { + if (!child.resolved) { + return s"$child INSET (values with unresolved data types)" + } val listString = hset.toSeq .map(elem => Literal(elem, child.dataType).toString) // Sort elements for deterministic behaviours diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala index ac10fbfa3a3ee..95b55797b294c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala @@ -21,6 +21,7 @@ import org.apache.logging.log4j.Level import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.InSet import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -152,4 +153,14 @@ class OptimizerLoggingSuite extends PlanTest { verifyLog(Level.INFO, Seq("Batch Has No Effect")) } } + + test("SPARK-50329: toString for InSet should be valid for unresolved plan") { + val input = LocalRelation($"a".int, $"b".string, $"c".double) + val inSetPredicate = InSet($"a", Set(1, 2)) + val query = input.select($"a", $"b").where(inSetPredicate) + val analyzed = query.analyze + + assert(query.toString.contains("'a INSET (values with unresolved data types)")) + assert(analyzed.toString.contains("INSET 1, 2")) + } } From 2cb818350de6b406f7e47cf353cadebcb04c07ab Mon Sep 17 00:00:00 2001 From: Jovan Markovic Date: Mon, 9 Dec 2024 16:16:18 -0800 Subject: [PATCH 132/438] [SPARK-50480][SQL] Extend CharType and VarcharType from StringType ### What changes were proposed in this pull request? Change type hierarchy so that CharType and VarcharType extend StringType. ### Why are the changes needed? This change is a step towards to properly supporting char and varchar types. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests in `DataTypeSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48883 from jovanm-db/char_varchar. Authored-by: Jovan Markovic Signed-off-by: Wenchen Fan --- .../sql/catalyst/encoders/RowEncoder.scala | 4 + .../org/apache/spark/sql/types/CharType.scala | 7 +- .../org/apache/spark/sql/types/DataType.scala | 2 +- .../apache/spark/sql/types/StringType.scala | 25 +- .../apache/spark/sql/types/UpCastRule.scala | 3 + .../apache/spark/sql/types/VarcharType.scala | 7 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 4 +- .../spark/sql/catalyst/expressions/Cast.scala | 11 +- .../sql/catalyst/expressions/literals.scala | 2 + .../variant/variantExpressions.scala | 1 + .../apache/spark/sql/util/SchemaUtils.scala | 4 +- .../catalyst/expressions/CastSuiteBase.scala | 11 + .../spark/sql/types/DataTypeSuite.scala | 325 +++++++++++++++++- .../common/DataTypeProtoConverter.scala | 21 +- .../spark/sql/connector/AlterTableTests.scala | 35 ++ 15 files changed, 434 insertions(+), 28 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 8b6da805a6e87..3f384235ff32d 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -80,6 +80,10 @@ object RowEncoder extends DataTypeErrorsBase { case DoubleType => BoxedDoubleEncoder case dt: DecimalType => JavaDecimalEncoder(dt, lenientSerialization = true) case BinaryType => BinaryEncoder + case CharType(_) | VarcharType(_) => + throw new AnalysisException( + errorClass = "UNSUPPORTED_DATA_TYPE_FOR_ENCODER", + messageParameters = Map("dataType" -> toSQLType(dataType))) case _: StringType => StringEncoder case TimestampType if SqlApiConf.get.datetimeJava8ApiEnabled => InstantEncoder(lenient) case TimestampType => TimestampEncoder(lenient) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/CharType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/CharType.scala index 5e30ff6e52a14..68dad6c87c01e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/CharType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/CharType.scala @@ -17,14 +17,19 @@ package org.apache.spark.sql.types +import org.json4s.JsonAST.{JString, JValue} + import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.util.CollationFactory @Experimental -case class CharType(length: Int) extends AtomicType { +case class CharType(length: Int) + extends StringType(CollationFactory.UTF8_BINARY_COLLATION_ID, FixedLength(length)) { require(length >= 0, "The length of char type cannot be negative.") override def defaultSize: Int = length override def typeName: String = s"char($length)" + override def jsonValue: JValue = JString(typeName) override def toString: String = s"CharType($length)" private[spark] override def asNullable: CharType = this } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala index 12cfed5b58685..db7e7c0ae1885 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -447,7 +447,7 @@ object DataType { private[sql] def equalsIgnoreCompatibleCollation(from: DataType, to: DataType): Boolean = { (from, to) match { // String types with possibly different collations are compatible. - case (_: StringType, _: StringType) => true + case (a: StringType, b: StringType) => a.constraint == b.constraint case (fromDataType, toDataType) => fromDataType == toDataType } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index b2cf502f8bdc1..735df2bd3fb19 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -30,7 +30,11 @@ import org.apache.spark.sql.catalyst.util.CollationFactory * The id of collation for this StringType. */ @Stable -class StringType private[sql] (val collationId: Int) extends AtomicType with Serializable { +class StringType private[sql] ( + val collationId: Int, + val constraint: StringConstraint = NoConstraint) + extends AtomicType + with Serializable { /** * Support for Binary Equality implies that strings are considered equal only if they are byte @@ -89,8 +93,12 @@ class StringType private[sql] (val collationId: Int) extends AtomicType with Ser // the collation information is written to struct field metadata override def jsonValue: JValue = JString("string") - override def equals(obj: Any): Boolean = - obj.isInstanceOf[StringType] && obj.asInstanceOf[StringType].collationId == collationId + override def equals(obj: Any): Boolean = { + obj match { + case s: StringType => s.collationId == collationId && s.constraint == constraint + case _ => false + } + } override def hashCode(): Int = collationId.hashCode() @@ -108,7 +116,8 @@ class StringType private[sql] (val collationId: Int) extends AtomicType with Ser * @since 1.3.0 */ @Stable -case object StringType extends StringType(0) { +case object StringType + extends StringType(CollationFactory.UTF8_BINARY_COLLATION_ID, NoConstraint) { private[spark] def apply(collationId: Int): StringType = new StringType(collationId) def apply(collation: String): StringType = { @@ -116,3 +125,11 @@ case object StringType extends StringType(0) { new StringType(collationId) } } + +sealed trait StringConstraint + +case object NoConstraint extends StringConstraint + +case class FixedLength(length: Int) extends StringConstraint + +case class MaxLength(length: Int) extends StringConstraint diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala index 8f2bdb2a595bf..1462559625967 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala @@ -40,6 +40,9 @@ private[sql] object UpCastRule { case (DateType, TimestampNTZType) => true case (TimestampNTZType, TimestampType) => true case (TimestampType, TimestampNTZType) => true + + case (_: AtomicType, CharType(_) | VarcharType(_)) => false + case (_: CalendarIntervalType, CharType(_) | VarcharType(_)) => false case (_: AtomicType, _: StringType) => true case (_: CalendarIntervalType, _: StringType) => true case (NullType, _) => true diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/VarcharType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/VarcharType.scala index 3d21e2e65804e..22f7947b25037 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/VarcharType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/VarcharType.scala @@ -16,14 +16,19 @@ */ package org.apache.spark.sql.types +import org.json4s.JsonAST.{JString, JValue} + import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.util.CollationFactory @Experimental -case class VarcharType(length: Int) extends AtomicType { +case class VarcharType(length: Int) + extends StringType(CollationFactory.UTF8_BINARY_COLLATION_ID, MaxLength(length)) { require(length >= 0, "The length of varchar type cannot be negative.") override def defaultSize: Int = length override def typeName: String = s"varchar($length)" + override def jsonValue: JValue = JString(typeName) override def toString: String = s"VarcharType($length)" private[spark] override def asNullable: VarcharType = this } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 484777c454fea..d34ffcb02d0a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1634,9 +1634,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case (CharType(l1), CharType(l2)) => l1 == l2 case (CharType(l1), VarcharType(l2)) => l1 <= l2 case (VarcharType(l1), VarcharType(l2)) => l1 <= l2 - case _ => - Cast.canUpCast(from, to) || - DataType.equalsIgnoreCompatibleCollation(field.dataType, newDataType) + case _ => Cast.canUpCast(from, to) } if (!canAlterColumnType(field.dataType, newDataType)) { alter.failAnalysis( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 4ef49cc97f6e6..d4ebdf10ef11f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -93,7 +93,9 @@ object Cast extends QueryErrorsBase { case (NullType, _) => true + case (_, CharType(_) | VarcharType(_)) => false case (_, _: StringType) => true + case (CharType(_) | VarcharType(_), _) => false case (_: StringType, _: BinaryType) => true @@ -198,7 +200,9 @@ object Cast extends QueryErrorsBase { case (NullType, _) => true + case (_, CharType(_) | VarcharType(_)) => false case (_, _: StringType) => true + case (CharType(_) | VarcharType(_), _) => false case (_: StringType, BinaryType) => true case (_: IntegralType, BinaryType) => true @@ -314,6 +318,8 @@ object Cast extends QueryErrorsBase { case _ if from == to => true case (NullType, _) => true case (_: NumericType, _: NumericType) => true + case (_: AtomicType, CharType(_) | VarcharType(_)) => false + case (_: CalendarIntervalType, CharType(_) | VarcharType(_)) => false case (_: AtomicType, _: StringType) => true case (_: CalendarIntervalType, _: StringType) => true case (_: DatetimeType, _: DatetimeType) => true @@ -355,9 +361,10 @@ object Cast extends QueryErrorsBase { case (_, _) if from == to => false case (VariantType, _) => true + case (CharType(_) | VarcharType(_), BinaryType | _: StringType) => false case (_: StringType, BinaryType | _: StringType) => false - case (_: StringType, _) => true - case (_, _: StringType) => false + case (st: StringType, _) if st.constraint == NoConstraint => true + case (_, st: StringType) if st.constraint == NoConstraint => false case (TimestampType, ByteType | ShortType | IntegerType) => true case (FloatType | DoubleType, TimestampType) => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 4cffc7f0b53a3..b874cb53cb31f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -196,6 +196,8 @@ object Literal { case TimestampNTZType => create(0L, TimestampNTZType) case it: DayTimeIntervalType => create(0L, it) case it: YearMonthIntervalType => create(0, it) + case CharType(_) | VarcharType(_) => + throw QueryExecutionErrors.noDefaultForDataTypeError(dataType) case st: StringType => Literal(UTF8String.fromString(""), st) case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8)) case CalendarIntervalType => Literal(new CalendarInterval(0, 0, 0)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala index 1639a161df4cc..2fa0ce0f570c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala @@ -330,6 +330,7 @@ case object VariantGet { */ def checkDataType(dataType: DataType, allowStructsAndMaps: Boolean = true): Boolean = dataType match { + case CharType(_) | VarcharType(_) => false case _: NumericType | BooleanType | _: StringType | BinaryType | _: DatetimeType | VariantType => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 0aadd3cd3a449..1d9172e2ba0d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, NoConstraint, StringType, StructField, StructType} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SparkSchemaUtils @@ -328,7 +328,7 @@ private[spark] object SchemaUtils { StructType(fields.map { field => field.copy(dataType = replaceCollatedStringWithString(field.dataType)) }) - case _: StringType => StringType + case st: StringType if st.constraint == NoConstraint => StringType case _ => dt } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index f6173e019cdeb..d7c6c94158b18 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -1015,6 +1015,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } } + test("disallow type conversions between calendar interval type and char/varchar types") { + Seq(CharType(10), VarcharType(10)) + .foreach { typ => + verifyCastFailure( + cast(Literal.default(CalendarIntervalType), typ), + DataTypeMismatch( + "CAST_WITHOUT_SUGGESTION", + Map("srcType" -> "\"INTERVAL\"", "targetType" -> toSQLType(typ)))) + } + } + test("SPARK-35720: cast string to timestamp without timezone") { specialTs.foreach { s => val expectedTs = LocalDateTime.parse(s) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index 7250b6e2b90e6..397241be76eb1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -366,6 +366,8 @@ class DataTypeSuite extends SparkFunSuite { checkDefaultSize(TimestampType, 8) checkDefaultSize(TimestampNTZType, 8) checkDefaultSize(StringType, 20) + checkDefaultSize(CharType(20), 20) + checkDefaultSize(VarcharType(20), 20) checkDefaultSize(BinaryType, 100) checkDefaultSize(ArrayType(DoubleType, true), 8) checkDefaultSize(ArrayType(StringType, false), 20) @@ -410,6 +412,14 @@ class DataTypeSuite extends SparkFunSuite { from = ArrayType(DoubleType, containsNull = false), to = ArrayType(StringType, containsNull = false), expected = false) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(CharType(5), containsNull = false), + to = ArrayType(StringType, containsNull = false), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(VarcharType(5), containsNull = false), + to = ArrayType(StringType, containsNull = false), + expected = false) checkEqualsIgnoreCompatibleNullability( from = MapType(StringType, DoubleType, valueContainsNull = true), @@ -423,6 +433,14 @@ class DataTypeSuite extends SparkFunSuite { from = MapType(StringType, DoubleType, valueContainsNull = false), to = MapType(StringType, DoubleType, valueContainsNull = true), expected = true) + checkEqualsIgnoreCompatibleNullability( + from = MapType(CharType(5), DoubleType, valueContainsNull = false), + to = MapType(StringType, DoubleType, valueContainsNull = true), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = MapType(VarcharType(5), DoubleType, valueContainsNull = false), + to = MapType(StringType, DoubleType, valueContainsNull = true), + expected = false) checkEqualsIgnoreCompatibleNullability( from = MapType(StringType, DoubleType, valueContainsNull = true), to = MapType(StringType, DoubleType, valueContainsNull = false), @@ -441,10 +459,26 @@ class DataTypeSuite extends SparkFunSuite { from = StructType(StructField("a", StringType, nullable = true) :: Nil), to = StructType(StructField("a", StringType, nullable = true) :: Nil), expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", CharType(5), nullable = true) :: Nil), + to = StructType(StructField("a", StringType, nullable = true) :: Nil), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", VarcharType(5), nullable = true) :: Nil), + to = StructType(StructField("a", StringType, nullable = true) :: Nil), + expected = false) checkEqualsIgnoreCompatibleNullability( from = StructType(StructField("a", StringType, nullable = false) :: Nil), to = StructType(StructField("a", StringType, nullable = false) :: Nil), expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", CharType(5), nullable = false) :: Nil), + to = StructType(StructField("a", StringType, nullable = false) :: Nil), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", VarcharType(5), nullable = false) :: Nil), + to = StructType(StructField("a", StringType, nullable = false) :: Nil), + expected = false) checkEqualsIgnoreCompatibleNullability( from = StructType(StructField("a", StringType, nullable = false) :: Nil), to = StructType(StructField("a", StringType, nullable = true) :: Nil), @@ -483,6 +517,8 @@ class DataTypeSuite extends SparkFunSuite { checkCatalogString(DecimalType(10, 5)) checkCatalogString(BinaryType) checkCatalogString(StringType) + checkCatalogString(CharType(5)) + checkCatalogString(VarcharType(10)) checkCatalogString(DateType) checkCatalogString(TimestampType) checkCatalogString(createStruct(4)) @@ -507,8 +543,18 @@ class DataTypeSuite extends SparkFunSuite { checkEqualsStructurally(BooleanType, BooleanType, true) checkEqualsStructurally(IntegerType, IntegerType, true) checkEqualsStructurally(IntegerType, LongType, false) + checkEqualsStructurally(CharType(5), CharType(5), true) + checkEqualsStructurally(CharType(5), CharType(10), false) + checkEqualsStructurally(CharType(5), VarcharType(5), false) + checkEqualsStructurally(VarcharType(5), VarcharType(5), true) + checkEqualsStructurally(VarcharType(5), VarcharType(10), false) + checkEqualsStructurally(VarcharType(5), CharType(5), false) checkEqualsStructurally(ArrayType(IntegerType, true), ArrayType(IntegerType, true), true) checkEqualsStructurally(ArrayType(IntegerType, true), ArrayType(IntegerType, false), false) + checkEqualsStructurally(ArrayType(CharType(5), true), ArrayType(CharType(5), true), true) + checkEqualsStructurally(ArrayType(CharType(5), true), ArrayType(CharType(5), false), false) + checkEqualsStructurally(ArrayType(VarcharType(5), true), ArrayType(VarcharType(5), true), true) + checkEqualsStructurally(ArrayType(VarcharType(5), true), ArrayType(VarcharType(5), false), false) checkEqualsStructurally( new StructType().add("f1", IntegerType), @@ -519,6 +565,15 @@ class DataTypeSuite extends SparkFunSuite { new StructType().add("f2", IntegerType, false), false) + checkEqualsStructurally( + new StructType().add("f1", CharType(5)), + new StructType().add("f2", StringType), + false) + checkEqualsStructurally( + new StructType().add("f1", VarcharType(5)), + new StructType().add("f2", StringType), + false) + checkEqualsStructurally( new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", StringType)), new StructType().add("f2", IntegerType).add("g", new StructType().add("f1", StringType)), @@ -538,6 +593,14 @@ class DataTypeSuite extends SparkFunSuite { .add("g", new StructType().add("f1", StringType)), true, ignoreNullability = true) + checkEqualsStructurally( + new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", CharType(5))), + new StructType().add("f2", IntegerType).add("g", new StructType().add("f1", StringType)), + false) + checkEqualsStructurally( + new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", VarcharType(5))), + new StructType().add("f2", IntegerType).add("g", new StructType().add("f1", StringType)), + false) checkEqualsStructurally( ArrayType( @@ -579,6 +642,22 @@ class DataTypeSuite extends SparkFunSuite { true, ignoreNullability = true) + checkEqualsStructurally( + ArrayType( + ArrayType(CharType(5), true), true), + ArrayType( + ArrayType(StringType, true), true), + false, + ignoreNullability = false) + + checkEqualsStructurally( + ArrayType( + ArrayType(VarcharType(5), true), true), + ArrayType( + ArrayType(StringType, true), true), + false, + ignoreNullability = false) + checkEqualsStructurally( MapType( ArrayType(IntegerType, true), ArrayType(IntegerType, true), true), @@ -627,6 +706,22 @@ class DataTypeSuite extends SparkFunSuite { true, ignoreNullability = true) + checkEqualsStructurally( + MapType( + ArrayType(IntegerType, true), ArrayType(CharType(5), true), true), + MapType( + ArrayType(IntegerType, true), ArrayType(StringType, true), true), + false, + ignoreNullability = false) + + checkEqualsStructurally( + MapType( + ArrayType(IntegerType, true), ArrayType(VarcharType(5), true), true), + MapType( + ArrayType(IntegerType, true), ArrayType(StringType, true), true), + false, + ignoreNullability = false) + def checkEqualsStructurallyByName( from: DataType, to: DataType, @@ -657,6 +752,10 @@ class DataTypeSuite extends SparkFunSuite { checkEqualsStructurallyByName(BooleanType, BooleanType, true) checkEqualsStructurallyByName(BooleanType, IntegerType, true) checkEqualsStructurallyByName(IntegerType, LongType, true) + checkEqualsStructurallyByName(StringType, CharType(5), true) + checkEqualsStructurallyByName(StringType, VarcharType(5), true) + checkEqualsStructurallyByName(CharType(5), StringType, true) + checkEqualsStructurallyByName(VarcharType(5), StringType, true) checkEqualsStructurallyByName( new StructType().add("f1", IntegerType).add("f2", IntegerType), @@ -665,6 +764,16 @@ class DataTypeSuite extends SparkFunSuite { checkEqualsStructurallyByName( new StructType().add("f1", IntegerType).add("f2", IntegerType), + new StructType().add("f1", CharType(5)).add("f2", StringType), + true) + + checkEqualsStructurallyByName( + new StructType().add("f1", IntegerType).add("f2", IntegerType), + new StructType().add("f2", LongType).add("f1", StringType), + false) + + checkEqualsStructurallyByName( + new StructType().add("f1", IntegerType).add("f2", VarcharType(5)), new StructType().add("f2", LongType).add("f1", StringType), false) @@ -673,23 +782,45 @@ class DataTypeSuite extends SparkFunSuite { new StructType().add("f1", LongType).add("f", new StructType().add("f2", BooleanType)), true) + checkEqualsStructurallyByName( + new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", StringType)), + new StructType().add("f1", LongType).add("f", new StructType().add("f2", VarcharType(5))), + true) + checkEqualsStructurallyByName( new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", StringType)), new StructType().add("f", new StructType().add("f2", BooleanType)).add("f1", LongType), false) + checkEqualsStructurallyByName( + new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", StringType)), + new StructType().add("f", new StructType().add("f2", CharType(5))).add("f1", LongType), + false) + checkEqualsStructurallyByName( new StructType().add("f1", IntegerType).add("f2", IntegerType), new StructType().add("F1", LongType).add("F2", StringType), true, caseSensitive = false) + checkEqualsStructurallyByName( + new StructType().add("f1", IntegerType).add("f2", IntegerType), + new StructType().add("F1", LongType).add("F2", CharType(5)), + true, + caseSensitive = false) + checkEqualsStructurallyByName( new StructType().add("f1", IntegerType).add("f2", IntegerType), new StructType().add("F1", LongType).add("F2", StringType), false, caseSensitive = true) + checkEqualsStructurallyByName( + new StructType().add("f1", IntegerType).add("f2", IntegerType), + new StructType().add("F1", LongType).add("F2", VarcharType(5)), + false, + caseSensitive = true) + def checkEqualsIgnoreCompatibleCollation( from: DataType, to: DataType, @@ -703,19 +834,45 @@ class DataTypeSuite extends SparkFunSuite { // Simple types. checkEqualsIgnoreCompatibleCollation(IntegerType, IntegerType, expected = true) - checkEqualsIgnoreCompatibleCollation(BooleanType, BooleanType, expected = true) - checkEqualsIgnoreCompatibleCollation(StringType, StringType, expected = true) checkEqualsIgnoreCompatibleCollation(IntegerType, BooleanType, expected = false) + checkEqualsIgnoreCompatibleCollation(IntegerType, StringType, expected = false) + checkEqualsIgnoreCompatibleCollation(IntegerType, CharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation(IntegerType, VarcharType(5), expected = false) checkEqualsIgnoreCompatibleCollation(BooleanType, IntegerType, expected = false) - checkEqualsIgnoreCompatibleCollation(StringType, BooleanType, expected = false) + checkEqualsIgnoreCompatibleCollation(BooleanType, BooleanType, expected = true) checkEqualsIgnoreCompatibleCollation(BooleanType, StringType, expected = false) + checkEqualsIgnoreCompatibleCollation(BooleanType, CharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation(BooleanType, VarcharType(5), expected = false) checkEqualsIgnoreCompatibleCollation(StringType, IntegerType, expected = false) - checkEqualsIgnoreCompatibleCollation(IntegerType, StringType, expected = false) + checkEqualsIgnoreCompatibleCollation(StringType, BooleanType, expected = false) + checkEqualsIgnoreCompatibleCollation(StringType, StringType, expected = true) + checkEqualsIgnoreCompatibleCollation(StringType, CharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation(StringType, VarcharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation(CharType(5), IntegerType, expected = false) + checkEqualsIgnoreCompatibleCollation(CharType(5), BooleanType, expected = false) + checkEqualsIgnoreCompatibleCollation(CharType(5), StringType, expected = false) + checkEqualsIgnoreCompatibleCollation(CharType(5), CharType(5), expected = true) + checkEqualsIgnoreCompatibleCollation(CharType(5), CharType(10), expected = false) + checkEqualsIgnoreCompatibleCollation(CharType(5), VarcharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation(VarcharType(5), IntegerType, expected = false) + checkEqualsIgnoreCompatibleCollation(VarcharType(5), BooleanType, expected = false) + checkEqualsIgnoreCompatibleCollation(VarcharType(5), StringType, expected = false) + checkEqualsIgnoreCompatibleCollation(VarcharType(5), CharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation(VarcharType(5), VarcharType(5), expected = true) + checkEqualsIgnoreCompatibleCollation(VarcharType(5), VarcharType(10), expected = false) // Collated `StringType`. checkEqualsIgnoreCompatibleCollation(StringType, StringType("UTF8_LCASE"), expected = true) checkEqualsIgnoreCompatibleCollation( StringType("UTF8_LCASE"), StringType("UTF8_BINARY"), expected = true) + checkEqualsIgnoreCompatibleCollation( + StringType("UTF8_LCASE"), CharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation( + CharType(5), StringType("UTF8_LCASE"), expected = false) + checkEqualsIgnoreCompatibleCollation( + StringType("UTF8_LCASE"), VarcharType(5), expected = false) + checkEqualsIgnoreCompatibleCollation( + VarcharType(5), StringType("UTF8_LCASE"), expected = false) // Complex types. checkEqualsIgnoreCompatibleCollation( ArrayType(StringType), @@ -732,6 +889,26 @@ class DataTypeSuite extends SparkFunSuite { ArrayType(ArrayType(StringType("UTF8_LCASE"))), expected = false ) + checkEqualsIgnoreCompatibleCollation( + ArrayType(ArrayType(StringType)), + ArrayType(ArrayType(CharType(5))), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + ArrayType(ArrayType(StringType("UTF8_LCASE"))), + ArrayType(ArrayType(CharType(5))), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + ArrayType(ArrayType(StringType)), + ArrayType(ArrayType(VarcharType(5))), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + ArrayType(ArrayType(StringType("UTF8_LCASE"))), + ArrayType(ArrayType(VarcharType(5))), + expected = false + ) checkEqualsIgnoreCompatibleCollation( MapType(StringType, StringType), MapType(StringType, StringType("UTF8_LCASE")), @@ -742,9 +919,49 @@ class DataTypeSuite extends SparkFunSuite { MapType(StringType, StringType), expected = false ) + checkEqualsIgnoreCompatibleCollation( + MapType(StringType("UTF8_LCASE"), StringType), + MapType(CharType(5), StringType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(StringType, StringType), + MapType(CharType(5), StringType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(StringType("UTF8_LCASE"), StringType), + MapType(VarcharType(5), StringType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(StringType, StringType), + MapType(VarcharType(5), StringType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(StringType("UTF8_LCASE"), ArrayType(StringType)), + MapType(StringType("UTF8_LCASE"), ArrayType(StringType("UTF8_LCASE"))), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(StringType("UTF8_LCASE"), ArrayType(StringType)), + MapType(StringType("UTF8_LCASE"), ArrayType(CharType(5))), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(StringType("UTF8_LCASE"), ArrayType(StringType("UTF8_LCASE"))), + MapType(StringType("UTF8_LCASE"), ArrayType(CharType(5))), + expected = false + ) checkEqualsIgnoreCompatibleCollation( MapType(StringType("UTF8_LCASE"), ArrayType(StringType)), + MapType(StringType("UTF8_LCASE"), ArrayType(VarcharType(5))), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( MapType(StringType("UTF8_LCASE"), ArrayType(StringType("UTF8_LCASE"))), + MapType(StringType("UTF8_LCASE"), ArrayType(VarcharType(5))), expected = false ) checkEqualsIgnoreCompatibleCollation( @@ -757,19 +974,99 @@ class DataTypeSuite extends SparkFunSuite { MapType(ArrayType(StringType("UTF8_LCASE")), IntegerType), expected = true ) + checkEqualsIgnoreCompatibleCollation( + MapType(ArrayType(StringType), IntegerType), + MapType(ArrayType(CharType(5)), IntegerType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(ArrayType(StringType("UTF8_LCASE")), IntegerType), + MapType(ArrayType(CharType(5)), IntegerType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(ArrayType(StringType), IntegerType), + MapType(ArrayType(VarcharType(5)), IntegerType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + MapType(ArrayType(StringType("UTF8_LCASE")), IntegerType), + MapType(ArrayType(VarcharType(5)), IntegerType), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", StringType) :: Nil), + StructType(StructField("a", StringType("UTF8_LCASE")) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", StringType) :: Nil), + StructType(StructField("a", CharType(5)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", StringType("UTF8_LCASE")) :: Nil), + StructType(StructField("a", CharType(5)) :: Nil), + expected = false + ) checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", StringType) :: Nil), + StructType(StructField("a", VarcharType(5)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", StringType("UTF8_LCASE")) :: Nil), + StructType(StructField("a", VarcharType(5)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", ArrayType(StringType)) :: Nil), + StructType(StructField("a", ArrayType(StringType("UTF8_LCASE"))) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", ArrayType(StringType)) :: Nil), + StructType(StructField("a", ArrayType(CharType(5))) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", ArrayType(StringType("UTF8_LCASE"))) :: Nil), + StructType(StructField("a", ArrayType(CharType(5))) :: Nil), expected = false ) checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", ArrayType(StringType)) :: Nil), + StructType(StructField("a", ArrayType(VarcharType(5))) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", ArrayType(StringType("UTF8_LCASE"))) :: Nil), + StructType(StructField("a", ArrayType(VarcharType(5))) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", MapType(StringType, IntegerType)) :: Nil), + StructType(StructField("a", MapType(StringType("UTF8_LCASE"), IntegerType)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", MapType(StringType, IntegerType)) :: Nil), + StructType(StructField("a", MapType(CharType(5), IntegerType)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", MapType(StringType("UTF8_LCASE"), IntegerType)) :: Nil), + StructType(StructField("a", MapType(CharType(5), IntegerType)) :: Nil), expected = false ) checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", MapType(StringType, IntegerType)) :: Nil), + StructType(StructField("a", MapType(VarcharType(5), IntegerType)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", MapType(StringType("UTF8_LCASE"), IntegerType)) :: Nil), + StructType(StructField("a", MapType(VarcharType(5), IntegerType)) :: Nil), expected = false ) checkEqualsIgnoreCompatibleCollation( @@ -777,6 +1074,26 @@ class DataTypeSuite extends SparkFunSuite { StructType(StructField("b", StringType("UTF8_LCASE")) :: Nil), expected = false ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", StringType) :: Nil), + StructType(StructField("b", CharType(5)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", StringType("UTF8_LCASE")) :: Nil), + StructType(StructField("b", CharType(5)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", StringType) :: Nil), + StructType(StructField("b", VarcharType(5)) :: Nil), + expected = false + ) + checkEqualsIgnoreCompatibleCollation( + StructType(StructField("a", StringType("UTF8_LCASE")) :: Nil), + StructType(StructField("b", VarcharType(5)) :: Nil), + expected = false + ) // Null compatibility checks. checkEqualsIgnoreCompatibleCollation( ArrayType(StringType, containsNull = true), diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala index f63692717947a..3577ca228b03e 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala @@ -175,16 +175,6 @@ object DataTypeProtoConverter { proto.DataType.Decimal.newBuilder().setPrecision(precision).setScale(scale).build()) .build() - case s: StringType => - proto.DataType - .newBuilder() - .setString( - proto.DataType.String - .newBuilder() - .setCollation(CollationFactory.fetchCollation(s.collationId).collationName) - .build()) - .build() - case CharType(length) => proto.DataType .newBuilder() @@ -197,6 +187,17 @@ object DataTypeProtoConverter { .setVarChar(proto.DataType.VarChar.newBuilder().setLength(length).build()) .build() + // StringType must be matched after CharType and VarcharType + case s: StringType => + proto.DataType + .newBuilder() + .setString( + proto.DataType.String + .newBuilder() + .setCollation(CollationFactory.fetchCollation(s.collationId).collationName) + .build()) + .build() + case DateType => ProtoDataTypes.DateType case TimestampType => ProtoDataTypes.TimestampType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 21aa57cc1eace..00e1f2f93fdcb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -1396,4 +1396,39 @@ trait AlterTableTests extends SharedSparkSession with QueryErrorsBase { parameters = Map("columnName" -> "`data`")) } } + + test("Alter column type between string and char/varchar") { + val types = Seq( + ("STRING", "\"STRING\""), + ("STRING COLLATE UTF8_LCASE", "\"STRING COLLATE UTF8_LCASE\""), + ("CHAR(5)", "\"CHAR\\(5\\)\""), + ("VARCHAR(5)", "\"VARCHAR\\(5\\)\"")) + types.flatMap { a => types.map { b => (a, b) } } + .filter { case (a, b) => a != b } + .filter { case ((a, _), (b, _)) => !a.startsWith("STRING") || !b.startsWith("STRING") } + .foreach { case ((from, originType), (to, newType)) => + val t = "table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id $from) USING PARQUET") + val sql1 = s"ALTER TABLE $t ALTER COLUMN id TYPE $to" + checkErrorMatchPVals( + exception = intercept[AnalysisException] { + sql(sql1) + }, + condition = "NOT_SUPPORTED_CHANGE_COLUMN", + sqlState = None, + parameters = Map( + "originType" -> originType, + "newType" -> newType, + "newName" -> "`id`", + "originName" -> "`id`", + "table" -> ".*table_name.*"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = sql1.length - 1) + ) + } + } + } } From 156cf1608273fb36ceb3e1254e7d31d88c7acc2e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 10 Dec 2024 09:37:47 +0900 Subject: [PATCH 133/438] [SPARK-50524][SQL] Lower `RowBasedKeyValueBatch.spill` warning message to debug level ### What changes were proposed in this pull request? This PR aims to lower `RowBasedKeyValueBatch.spill` warning message to debug level. ```java public final long spill(long size, MemoryConsumer trigger) throws IOException { - logger.warn("Calling spill() on RowBasedKeyValueBatch. Will not spill but return 0."); + logger.debug("Calling spill() on RowBasedKeyValueBatch. Will not spill but return 0."); return 0; } ``` ### Why are the changes needed? Although Apache Spark has been showing a warning message since 2.1.0, there is nothing for a user to be able to do further. This is more like a dev-side debug message. So, we had better lower the level to `DEBUG` from `WARN`. - #14349 ### Does this PR introduce _any_ user-facing change? No behavior change. This is a log message. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49116 from dongjoon-hyun/SPARK-50524. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- .../spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java index c057c36ca8204..5074348a1fd6a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java @@ -174,7 +174,7 @@ public final UnsafeRow getValueRow(int rowId) { */ @Override public final long spill(long size, MemoryConsumer trigger) throws IOException { - logger.warn("Calling spill() on RowBasedKeyValueBatch. Will not spill but return 0."); + logger.debug("Calling spill() on RowBasedKeyValueBatch. Will not spill but return 0."); return 0; } From a89fcfca29f661f15fecfe13882378616fe740fe Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 10 Dec 2024 09:55:12 +0900 Subject: [PATCH 134/438] [SPARK-50507][PYTHON][TESTS][FOLLOW-UP] Add refactored package into pure Python test ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/49074 that adds refactored package into pure Python test ### Why are the changes needed? In order to fix the pure Python build https://github.com/apache/spark/actions/runs/12215379954/job/34077255570. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Will monitor the build. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49119 from HyukjinKwon/SPARK-50507-followup2. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/packaging/connect/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 916a6da79de05..cf8ca7d3dbd84 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -75,6 +75,7 @@ "pyspark.sql.tests.connect", "pyspark.sql.tests.connect.streaming", "pyspark.sql.tests.connect.client", + "pyspark.sql.tests.connect.pandas", "pyspark.sql.tests.connect.shell", "pyspark.sql.tests.pandas", "pyspark.sql.tests.plot", @@ -101,7 +102,6 @@ "pyspark.pandas.tests.connect.groupby", "pyspark.pandas.tests.connect.indexes", "pyspark.pandas.tests.connect.io", - "pyspark.pandas.tests.connect.pandas", "pyspark.pandas.tests.connect.plot", "pyspark.pandas.tests.connect.resample", "pyspark.pandas.tests.connect.reshape", From 5f34af975f3545fc28cc16f18fc4e94223bd4123 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 10 Dec 2024 13:27:55 +0800 Subject: [PATCH 135/438] [SPARK-50477][INFRA][FOLLOW-UP] Python 3.9 testing image clean up ### What changes were proposed in this pull request? 1, `Python 3.9` was installed twice; 2, add `apt-get autoremove` and `apt-get clean`; 3, explicitly install `tzdata` which is needed for timezone related test (it was installed with python 3.9) ### Why are the changes needed? clean up ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-309", "PYTHON_TO_TEST": "python3.9"}' ``` https://github.com/zhengruifeng/spark/runs/34168664848 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49123 from zhengruifeng/py_image_309_followup. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- dev/spark-test-image/python-309/Dockerfile | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/dev/spark-test-image/python-309/Dockerfile b/dev/spark-test-image/python-309/Dockerfile index dbab99c1441bc..c2104dca95fc1 100644 --- a/dev/spark-test-image/python-309/Dockerfile +++ b/dev/spark-test-image/python-309/Dockerfile @@ -24,7 +24,7 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark wi # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241119 +ENV FULL_REFRESH_DATE 20241205 ENV DEBIAN_FRONTEND noninteractive ENV DEBCONF_NONINTERACTIVE_SEEN true @@ -51,29 +51,27 @@ RUN apt-get update && apt-get install -y \ libtiff5-dev \ libxml2-dev \ openjdk-17-jdk-headless \ - pandoc \ pkg-config \ qpdf \ + tzdata \ software-properties-common \ wget \ - zlib1g-dev \ - && rm -rf /var/lib/apt/lists/* + zlib1g-dev # Install Python 3.9 RUN add-apt-repository ppa:deadsnakes/ppa RUN apt-get update && apt-get install -y \ - python3.9 python3.9-distutils \ + python3.9 \ + python3.9-distutils \ + && apt-get autoremove --purge -y \ + && apt-get clean \ && rm -rf /var/lib/apt/lists/* ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" -# Install Python 3.9 -RUN add-apt-repository ppa:deadsnakes/ppa -RUN apt-get update && apt-get install -y \ - python3.9 python3.9-distutils \ - && rm -rf /var/lib/apt/lists/* +# Install Python 3.9 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 RUN python3.9 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this RUN python3.9 -m pip install --force $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ From 02ebf128247b60986ec4dc21a22efd7c1a284d15 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Tue, 10 Dec 2024 15:27:52 +0900 Subject: [PATCH 136/438] [SPARK-50460][PYTHON][CONNECT] Generalize and simplify Connect exception handling ### What changes were proposed in this pull request? This PR introduces a generalized and streamlined approach to handling exceptions in the Spark Connect Python client. This PR includes: 1. Replaces the large if-else block for exception type matching with a more extensible class mapping (`EXCEPTION_CLASS_MAPPING`). 2. Maintains compatibility for existing exception types while making it easier to add new ones in the future. 3. Keeps the unique handling logic for specific exceptions like `PythonException` intact. ### Why are the changes needed? The previous exception handling logic relied on a large `if-else` block, which was not scalable or maintainable as more exception types were added. Generalizing the exception handling mechanism reduces the complexity of the code, making it easier to extend when adding new exception types now requires minimal changes to the `EXCEPTION_CLASS_MAPPING`, avoiding duplication of logic. ### Does this PR introduce _any_ user-facing change? No API changes, but it enhances the internal implementation of exception handling. ### How was this patch tested? Added new UTs to verify that all existing exception handling scenarios are still supported. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49041 from itholic/SPARK-50460. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- python/pyspark/errors/exceptions/connect.py | 261 +++++------------- .../tests/test_connect_errors_conversion.py | 169 ++++++++++++ 2 files changed, 240 insertions(+), 190 deletions(-) create mode 100644 python/pyspark/errors/tests/test_connect_errors_conversion.py diff --git a/python/pyspark/errors/exceptions/connect.py b/python/pyspark/errors/exceptions/connect.py index c24b25af01631..ef90f8559b425 100644 --- a/python/pyspark/errors/exceptions/connect.py +++ b/python/pyspark/errors/exceptions/connect.py @@ -54,206 +54,67 @@ def convert_exception( resp: Optional[pb2.FetchErrorDetailsResponse], display_server_stacktrace: bool = False, ) -> SparkConnectException: - classes = [] - sql_state = None - errorClass = None - messageParameters = None - contexts: Optional[List[BaseQueryContext]] = None - - if "classes" in info.metadata: - classes = json.loads(info.metadata["classes"]) - - if "sqlState" in info.metadata: - sql_state = info.metadata["sqlState"] - - if "errorClass" in info.metadata: - errorClass = info.metadata["errorClass"] - - if "messageParameters" in info.metadata: - messageParameters = json.loads(info.metadata["messageParameters"]) - + raw_classes = info.metadata.get("classes") + classes: List[str] = json.loads(raw_classes) if raw_classes else [] + sql_state = info.metadata.get("sqlState") + error_class = info.metadata.get("errorClass") + raw_message_parameters = info.metadata.get("messageParameters") + message_parameters: Dict[str, str] = ( + json.loads(raw_message_parameters) if raw_message_parameters else {} + ) stacktrace: Optional[str] = None + if resp is not None and resp.HasField("root_error_idx"): message = resp.errors[resp.root_error_idx].message stacktrace = _extract_jvm_stacktrace(resp) else: message = truncated_message - stacktrace = info.metadata["stackTrace"] if "stackTrace" in info.metadata else None - display_server_stacktrace = display_server_stacktrace if stacktrace is not None else False - - if ( - resp is not None - and resp.errors - and hasattr(resp.errors[resp.root_error_idx], "spark_throwable") - ): - messageParameters = dict( - resp.errors[resp.root_error_idx].spark_throwable.message_parameters - ) - contexts = [] - for context in resp.errors[resp.root_error_idx].spark_throwable.query_contexts: - if context.context_type == pb2.FetchErrorDetailsResponse.QueryContext.SQL: - contexts.append(SQLQueryContext(context)) - else: - contexts.append(DataFrameQueryContext(context)) - - if "org.apache.spark.sql.catalyst.parser.ParseException" in classes: - return ParseException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - # Order matters. ParseException inherits AnalysisException. - elif "org.apache.spark.sql.AnalysisException" in classes: - return AnalysisException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "org.apache.spark.sql.streaming.StreamingQueryException" in classes: - return StreamingQueryException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "org.apache.spark.sql.execution.QueryExecutionException" in classes: - return QueryExecutionException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - # Order matters. NumberFormatException inherits IllegalArgumentException. - elif "java.lang.NumberFormatException" in classes: - return NumberFormatException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "java.lang.IllegalArgumentException" in classes: - return IllegalArgumentException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "java.lang.ArithmeticException" in classes: - return ArithmeticException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "java.lang.UnsupportedOperationException" in classes: - return UnsupportedOperationException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "java.lang.ArrayIndexOutOfBoundsException" in classes: - return ArrayIndexOutOfBoundsException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "java.time.DateTimeException" in classes: - return DateTimeException( - message, - errorClass=errorClass, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "org.apache.spark.SparkRuntimeException" in classes: - return SparkRuntimeException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "org.apache.spark.SparkUpgradeException" in classes: - return SparkUpgradeException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - elif "org.apache.spark.api.python.PythonException" in classes: + stacktrace = info.metadata.get("stackTrace") + display_server_stacktrace = display_server_stacktrace if stacktrace else False + + contexts = None + if resp and resp.HasField("root_error_idx"): + root_error = resp.errors[resp.root_error_idx] + if hasattr(root_error, "spark_throwable"): + message_parameters = dict(root_error.spark_throwable.message_parameters) + contexts = [ + SQLQueryContext(c) + if c.context_type == pb2.FetchErrorDetailsResponse.QueryContext.SQL + else DataFrameQueryContext(c) + for c in root_error.spark_throwable.query_contexts + ] + + if "org.apache.spark.api.python.PythonException" in classes: return PythonException( "\n An exception was thrown from the Python worker. " "Please see the stack trace below.\n%s" % message ) - elif "org.apache.spark.SparkNoSuchElementException" in classes: - return SparkNoSuchElementException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - # Make sure that the generic SparkException is handled last. - elif "org.apache.spark.SparkException" in classes: - return SparkException( - message, - errorClass=errorClass, - messageParameters=messageParameters, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) - else: - return SparkConnectGrpcException( - message, - reason=info.reason, - messageParameters=messageParameters, - errorClass=errorClass, - sql_state=sql_state, - server_stacktrace=stacktrace, - display_server_stacktrace=display_server_stacktrace, - contexts=contexts, - ) + + # Return exception based on class mapping + for error_class_name in classes: + ExceptionClass = EXCEPTION_CLASS_MAPPING.get(error_class_name) + if ExceptionClass: + return ExceptionClass( + message, + errorClass=error_class, + messageParameters=message_parameters, + sql_state=sql_state, + server_stacktrace=stacktrace, + display_server_stacktrace=display_server_stacktrace, + contexts=contexts, + ) + + # Return SparkConnectGrpcException if there is no matched exception class + return SparkConnectGrpcException( + message, + reason=info.reason, + messageParameters=message_parameters, + errorClass=error_class, + sql_state=sql_state, + server_stacktrace=stacktrace, + display_server_stacktrace=display_server_stacktrace, + contexts=contexts, + ) def _extract_jvm_stacktrace(resp: pb2.FetchErrorDetailsResponse) -> str: @@ -434,6 +295,26 @@ class SparkNoSuchElementException(SparkConnectGrpcException, BaseNoSuchElementEx """ +# Update EXCEPTION_CLASS_MAPPING here when adding a new exception +EXCEPTION_CLASS_MAPPING = { + "org.apache.spark.sql.catalyst.parser.ParseException": ParseException, + "org.apache.spark.sql.AnalysisException": AnalysisException, + "org.apache.spark.sql.streaming.StreamingQueryException": StreamingQueryException, + "org.apache.spark.sql.execution.QueryExecutionException": QueryExecutionException, + "java.lang.NumberFormatException": NumberFormatException, + "java.lang.IllegalArgumentException": IllegalArgumentException, + "java.lang.ArithmeticException": ArithmeticException, + "java.lang.UnsupportedOperationException": UnsupportedOperationException, + "java.lang.ArrayIndexOutOfBoundsException": ArrayIndexOutOfBoundsException, + "java.time.DateTimeException": DateTimeException, + "org.apache.spark.SparkRuntimeException": SparkRuntimeException, + "org.apache.spark.SparkUpgradeException": SparkUpgradeException, + "org.apache.spark.api.python.PythonException": PythonException, + "org.apache.spark.SparkNoSuchElementException": SparkNoSuchElementException, + "org.apache.spark.SparkException": SparkException, +} + + class SQLQueryContext(BaseQueryContext): def __init__(self, q: pb2.FetchErrorDetailsResponse.QueryContext): self._q = q diff --git a/python/pyspark/errors/tests/test_connect_errors_conversion.py b/python/pyspark/errors/tests/test_connect_errors_conversion.py new file mode 100644 index 0000000000000..a6ed5e7d391ee --- /dev/null +++ b/python/pyspark/errors/tests/test_connect_errors_conversion.py @@ -0,0 +1,169 @@ +# -*- encoding: utf-8 -*- +# +# 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. +# + +import unittest +from pyspark.errors.exceptions.connect import ( + convert_exception, + EXCEPTION_CLASS_MAPPING, + SparkConnectGrpcException, + PythonException, + AnalysisException, +) +from pyspark.sql.connect.proto import FetchErrorDetailsResponse as pb2 +from google.rpc.error_details_pb2 import ErrorInfo + + +class ConnectErrorsTest(unittest.TestCase): + def test_convert_exception_known_class(self): + # Mock ErrorInfo with a known error class + info = { + "reason": "org.apache.spark.sql.AnalysisException", + "metadata": { + "classes": '["org.apache.spark.sql.AnalysisException"]', + "sqlState": "42000", + "errorClass": "ANALYSIS.ERROR", + "messageParameters": '{"param1": "value1"}', + }, + } + truncated_message = "Analysis error occurred" + exception = convert_exception( + info=ErrorInfo(**info), truncated_message=truncated_message, resp=None + ) + + self.assertIsInstance(exception, AnalysisException) + self.assertEqual(exception.getSqlState(), "42000") + self.assertEqual(exception._errorClass, "ANALYSIS.ERROR") + self.assertEqual(exception._messageParameters, {"param1": "value1"}) + + def test_convert_exception_python_exception(self): + # Mock ErrorInfo for PythonException + info = { + "reason": "org.apache.spark.api.python.PythonException", + "metadata": { + "classes": '["org.apache.spark.api.python.PythonException"]', + }, + } + truncated_message = "Python worker error occurred" + exception = convert_exception( + info=ErrorInfo(**info), truncated_message=truncated_message, resp=None + ) + + self.assertIsInstance(exception, PythonException) + self.assertIn("An exception was thrown from the Python worker", exception.getMessage()) + + def test_convert_exception_unknown_class(self): + # Mock ErrorInfo with an unknown error class + info = { + "reason": "org.apache.spark.UnknownException", + "metadata": {"classes": '["org.apache.spark.UnknownException"]'}, + } + truncated_message = "Unknown error occurred" + exception = convert_exception( + info=ErrorInfo(**info), truncated_message=truncated_message, resp=None + ) + + self.assertIsInstance(exception, SparkConnectGrpcException) + self.assertEqual( + exception.getMessage(), "(org.apache.spark.UnknownException) Unknown error occurred" + ) + + def test_exception_class_mapping(self): + # Ensure that all keys in EXCEPTION_CLASS_MAPPING are valid + for error_class_name, exception_class in EXCEPTION_CLASS_MAPPING.items(): + self.assertTrue( + hasattr(exception_class, "__name__"), + f"{exception_class} in EXCEPTION_CLASS_MAPPING is not a valid class", + ) + + def test_convert_exception_with_stacktrace(self): + # Mock FetchErrorDetailsResponse with stacktrace + resp = pb2( + root_error_idx=0, + errors=[ + pb2.Error( + message="Root error message", + error_type_hierarchy=["org.apache.spark.SparkException"], + stack_trace=[ + pb2.StackTraceElement( + declaring_class="org.apache.spark.Main", + method_name="main", + file_name="Main.scala", + line_number=42, + ), + ], + cause_idx=1, + ), + pb2.Error( + message="Cause error message", + error_type_hierarchy=["java.lang.RuntimeException"], + stack_trace=[ + pb2.StackTraceElement( + declaring_class="org.apache.utils.Helper", + method_name="help", + file_name="Helper.java", + line_number=10, + ), + ], + ), + ], + ) + + info = { + "reason": "org.apache.spark.SparkException", + "metadata": { + "classes": '["org.apache.spark.SparkException"]', + "sqlState": "42000", + }, + } + truncated_message = "Root error message" + exception = convert_exception( + info=ErrorInfo(**info), truncated_message=truncated_message, resp=resp + ) + + self.assertIsInstance(exception, SparkConnectGrpcException) + self.assertIn("Root error message", exception.getMessage()) + self.assertIn("Caused by", exception.getMessage()) + + def test_convert_exception_fallback(self): + # Mock ErrorInfo with missing class information + info = { + "reason": "org.apache.spark.UnknownReason", + "metadata": {}, + } + truncated_message = "Fallback error occurred" + exception = convert_exception( + info=ErrorInfo(**info), truncated_message=truncated_message, resp=None + ) + + self.assertIsInstance(exception, SparkConnectGrpcException) + self.assertEqual( + exception.getMessage(), "(org.apache.spark.UnknownReason) Fallback error occurred" + ) + + +if __name__ == "__main__": + import unittest + from pyspark.errors.tests.test_errors import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) From 03c57995beb15bcf196a5aa12c4e82944ab4f1f5 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 10 Dec 2024 17:17:56 +0900 Subject: [PATCH 137/438] [SPARK-50513][SS][SQL] Split EncoderImplicits from SQLImplicits and provide helper object within StatefulProcessor to access underlying SQL Encoder related implicit functions ### What changes were proposed in this pull request? Split EncoderImplicits from SQLImplicits and provide helper object within StatefulProcessor to access underlying SQL Encoder related implicit functions ### Why are the changes needed? Without this, we cannot handle the implicit encoder APIs on the executor. We would run into a NPE since the spark session is not available on the executors. One option is to pass the `SparkSession` or `SQLImplicits` directly to the `StatefulProcessor`. However, this risks exposing some methods relying on `SparkSession` again. ``` Caused by: Task 11 in stage 4.0 failed 4 times, most recent failure: Lost task 11.3 in stage 4.0 (TID 57) (10.68.181.194 executor 2): java.lang.NullPointerException: Cannot invoke "org.apache.spark.sql.SparkSession.implicits()" because the return value of "$line94e51c7e8a51415d827a613fcad00cc19.$read$$iw$$iw.spark()" is null at $line94e51c7e8a51415d827a613fcad00cc152.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$FruitCountStatefulProcessor.init(command-412580759966543:18) at org.apache.spark.sql.execution.streaming.TransformWithStateExec.processDataWithInitialState(TransformWithStateExec.scala:732) at org.apache.spark.sql.execution.streaming.TransformWithStateExec.$anonfun$doExecute$2(TransformWithStateExec.scala:625) ``` Hence, we do 2 things: - split the encoder related functions that don't rely on `SparkSession` into a separate trait - expose a helper `implicits` object within the `StatefulProcessor` interface that users can import directly, providing access to only the necessary functions ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Added unit tests ``` [info] Run completed in 9 minutes, 30 seconds. [info] Total number of tests run: 128 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 128, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #49099 from anishshri-db/task/SPARK-50513. Authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim --- .../apache/spark/sql/api/SQLImplicits.scala | 53 +++-- .../sql/streaming/StatefulProcessor.scala | 9 + .../TransformWithStateClusterSuite.scala | 211 ++++++++++++++++++ .../TransformWithStateInitialStateSuite.scala | 12 +- .../streaming/TransformWithStateSuite.scala | 191 ++++++++-------- 5 files changed, 357 insertions(+), 119 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/SQLImplicits.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/SQLImplicits.scala index 5e022570d3ca7..200e913b5412e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/SQLImplicits.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/SQLImplicits.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, DE * * @since 1.6.0 */ -abstract class SQLImplicits extends LowPrioritySQLImplicits with Serializable { +abstract class SQLImplicits extends EncoderImplicits with Serializable { type DS[U] <: Dataset[U] protected def session: SparkSession @@ -51,8 +51,35 @@ abstract class SQLImplicits extends LowPrioritySQLImplicits with Serializable { } } - // Primitives + /** + * Creates a [[Dataset]] from a local Seq. + * @since 1.6.0 + */ + implicit def localSeqToDatasetHolder[T: Encoder](s: Seq[T]): DatasetHolder[T, DS] = { + new DatasetHolder(session.createDataset(s).asInstanceOf[DS[T]]) + } + + /** + * Creates a [[Dataset]] from an RDD. + * + * @since 1.6.0 + */ + implicit def rddToDatasetHolder[T: Encoder](rdd: RDD[T]): DatasetHolder[T, DS] = + new DatasetHolder(session.createDataset(rdd).asInstanceOf[DS[T]]) + + /** + * An implicit conversion that turns a Scala `Symbol` into a [[org.apache.spark.sql.Column]]. + * @since 1.3.0 + */ + implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name) +} +/** + * EncoderImplicits used to implicitly generate SQL Encoders. Note that these functions don't rely + * on or expose `SparkSession`. + */ +trait EncoderImplicits extends LowPrioritySQLImplicits with Serializable { + // Primitives /** @since 1.6.0 */ implicit def newIntEncoder: Encoder[Int] = Encoders.scalaInt @@ -270,28 +297,6 @@ abstract class SQLImplicits extends LowPrioritySQLImplicits with Serializable { /** @since 1.6.1 */ implicit def newProductArrayEncoder[A <: Product: TypeTag]: Encoder[Array[A]] = newArrayEncoder(ScalaReflection.encoderFor[A]) - - /** - * Creates a [[Dataset]] from a local Seq. - * @since 1.6.0 - */ - implicit def localSeqToDatasetHolder[T: Encoder](s: Seq[T]): DatasetHolder[T, DS] = { - new DatasetHolder(session.createDataset(s).asInstanceOf[DS[T]]) - } - - /** - * Creates a [[Dataset]] from an RDD. - * - * @since 1.6.0 - */ - implicit def rddToDatasetHolder[T: Encoder](rdd: RDD[T]): DatasetHolder[T, DS] = - new DatasetHolder(session.createDataset(rdd).asInstanceOf[DS[T]]) - - /** - * An implicit conversion that turns a Scala `Symbol` into a [[org.apache.spark.sql.Column]]. - * @since 1.3.0 - */ - implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name) } /** diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala index 55477b4dda0c9..b47629cb54396 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala @@ -20,16 +20,25 @@ package org.apache.spark.sql.streaming import java.io.Serializable import org.apache.spark.annotation.{Evolving, Experimental} +import org.apache.spark.sql.api.EncoderImplicits import org.apache.spark.sql.errors.ExecutionErrors /** * Represents the arbitrary stateful logic that needs to be provided by the user to perform * stateful manipulations on keyed streams. + * + * Users can also explicitly use `import implicits._` to access the EncoderImplicits and use the + * state variable APIs relying on implicit encoders. */ @Experimental @Evolving private[sql] abstract class StatefulProcessor[K, I, O] extends Serializable { + // scalastyle:off + // Disable style checker so "implicits" object can start with lowercase i + object implicits extends EncoderImplicits + // scalastyle:on + /** * Handle to the stateful processor that provides access to the state store and other stateful * processing related APIs. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala new file mode 100644 index 0000000000000..3e2899f7c6ee7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala @@ -0,0 +1,211 @@ +/* + * 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.streaming + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.sql.{Dataset, Encoders, Row, SparkSession} +import org.apache.spark.sql.LocalSparkSession.withSparkSession +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider +import org.apache.spark.sql.internal.SQLConf + +case class FruitState( + name: String, + count: Long, + family: String +) + +class FruitCountStatefulProcessor(useImplicits: Boolean) + extends StatefulProcessor[String, String, (String, Long, String)] { + import implicits._ + + @transient protected var _fruitState: ValueState[FruitState] = _ + + override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = { + if (useImplicits) { + _fruitState = getHandle.getValueState[FruitState]("fruitState", TTLConfig.NONE) + } else { + _fruitState = getHandle.getValueState("fruitState", Encoders.product[FruitState], + TTLConfig.NONE) + } + } + + private def getFamily(fruitName: String): String = { + if (fruitName == "orange" || fruitName == "lemon" || fruitName == "lime") { + "citrus" + } else { + "non-citrus" + } + } + + override def handleInputRows(key: String, inputRows: Iterator[String], timerValues: TimerValues): + Iterator[(String, Long, String)] = { + val new_cnt = _fruitState.getOption().map(x => x.count).getOrElse(0L) + inputRows.size + val family = getFamily(key) + _fruitState.update(FruitState(key, new_cnt, family)) + Iterator.single((key, new_cnt, family)) + } +} + +class FruitCountStatefulProcessorWithInitialState(useImplicits: Boolean) + extends StatefulProcessorWithInitialState[String, String, (String, Long, String), String] { + import implicits._ + + @transient protected var _fruitState: ValueState[FruitState] = _ + + override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = { + if (useImplicits) { + _fruitState = getHandle.getValueState[FruitState]("fruitState", TTLConfig.NONE) + } else { + _fruitState = getHandle.getValueState("fruitState", Encoders.product[FruitState], + TTLConfig.NONE) + } + } + + private def getFamily(fruitName: String): String = { + if (fruitName == "orange" || fruitName == "lemon" || fruitName == "lime") { + "citrus" + } else { + "non-citrus" + } + } + + override def handleInitialState(key: String, initialState: String, + timerValues: TimerValues): Unit = { + val new_cnt = _fruitState.getOption().map(x => x.count).getOrElse(0L) + 1 + val family = getFamily(key) + _fruitState.update(FruitState(key, new_cnt, family)) + } + + override def handleInputRows(key: String, inputRows: Iterator[String], timerValues: TimerValues): + Iterator[(String, Long, String)] = { + val new_cnt = _fruitState.getOption().map(x => x.count).getOrElse(0L) + inputRows.size + val family = getFamily(key) + _fruitState.update(FruitState(key, new_cnt, family)) + Iterator.single((key, new_cnt, family)) + } +} + +trait TransformWithStateClusterSuiteBase extends SparkFunSuite { + def getSparkConf(): SparkConf = { + val conf = new SparkConf() + .setMaster("local-cluster[2, 2, 1024]") + .set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, + classOf[RocksDBStateStoreProvider].getCanonicalName) + .set(SQLConf.SHUFFLE_PARTITIONS.key, + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) + .set(SQLConf.STREAMING_STOP_TIMEOUT, 5000L) + conf + } + + // Start a new test with cluster containing two executors and streaming stop timeout set to 5s + val testSparkConf = getSparkConf() + + protected def testWithAndWithoutImplicitEncoders(name: String) + (func: (SparkSession, Boolean) => Any): Unit = { + Seq(false, true).foreach { useImplicits => + test(s"$name - useImplicits = $useImplicits") { + withSparkSession(SparkSession.builder().config(testSparkConf).getOrCreate()) { spark => + func(spark, useImplicits) + } + } + } + } +} + +/** + * Test suite spawning local cluster with multiple executors to test serde of stateful + * processors along with use of implicit encoders, if applicable in transformWithState operator. + */ +class TransformWithStateClusterSuite extends StreamTest with TransformWithStateClusterSuiteBase { + testWithAndWithoutImplicitEncoders("streaming with transformWithState - " + + "without initial state") { (spark, useImplicits) => + import spark.implicits._ + val input = MemoryStream(Encoders.STRING, spark.sqlContext) + val agg = input.toDS() + .groupByKey(x => x) + .transformWithState(new FruitCountStatefulProcessor(useImplicits), + TimeMode.None(), + OutputMode.Update() + ) + + val query = agg.writeStream + .format("memory") + .outputMode("update") + .queryName("output") + .start() + + input.addData("apple", "apple", "orange", "orange", "orange") + query.processAllAvailable() + + checkAnswer(spark.sql("select * from output"), + Seq(Row("apple", 2, "non-citrus"), + Row("orange", 3, "citrus"))) + + input.addData("lemon", "lime") + query.processAllAvailable() + checkAnswer(spark.sql("select * from output"), + Seq(Row("apple", 2, "non-citrus"), + Row("orange", 3, "citrus"), + Row("lemon", 1, "citrus"), + Row("lime", 1, "citrus"))) + + query.stop() + } + + testWithAndWithoutImplicitEncoders("streaming with transformWithState - " + + "with initial state") { (spark, useImplicits) => + import spark.implicits._ + + val fruitCountInitialDS: Dataset[String] = Seq( + "apple", "apple", "orange", "orange", "orange").toDS() + + val fruitCountInitial = fruitCountInitialDS + .groupByKey(x => x) + + val input = MemoryStream(Encoders.STRING, spark.sqlContext) + val agg = input.toDS() + .groupByKey(x => x) + .transformWithState(new FruitCountStatefulProcessorWithInitialState(useImplicits), + TimeMode.None(), + OutputMode.Update(), fruitCountInitial) + + val query = agg.writeStream + .format("memory") + .outputMode("update") + .queryName("output") + .start() + + input.addData("apple", "apple", "orange", "orange", "orange") + query.processAllAvailable() + + checkAnswer(spark.sql("select * from output"), + Seq(Row("apple", 4, "non-citrus"), + Row("orange", 6, "citrus"))) + + input.addData("lemon", "lime") + query.processAllAvailable() + checkAnswer(spark.sql("select * from output"), + Seq(Row("apple", 4, "non-citrus"), + Row("orange", 6, "citrus"), + Row("lemon", 1, "citrus"), + Row("lime", 1, "citrus"))) + + query.stop() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala index 29f40df83f24a..c7ad8536ebd46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala @@ -47,6 +47,8 @@ case class UnionUnflattenInitialStateRow( abstract class StatefulProcessorWithInitialStateTestClass[V] extends StatefulProcessorWithInitialState[ String, InitInputRow, (String, String, Double), V] { + import implicits._ + @transient var _valState: ValueState[Double] = _ @transient var _listState: ListState[Double] = _ @transient var _mapState: MapState[Double, Int] = _ @@ -54,13 +56,9 @@ abstract class StatefulProcessorWithInitialStateTestClass[V] override def init( outputMode: OutputMode, timeMode: TimeMode): Unit = { - _valState = getHandle.getValueState[Double]("testValueInit", Encoders.scalaDouble, - TTLConfig.NONE) - _listState = getHandle.getListState[Double]("testListInit", Encoders.scalaDouble, - TTLConfig.NONE) - _mapState = getHandle.getMapState[Double, Int]( - "testMapInit", Encoders.scalaDouble, Encoders.scalaInt, - TTLConfig.NONE) + _valState = getHandle.getValueState[Double]("testValueInit", TTLConfig.NONE) + _listState = getHandle.getListState[Double]("testListInit", TTLConfig.NONE) + _mapState = getHandle.getMapState[Double, Int]("testMapInit", TTLConfig.NONE) } override def handleInputRows( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 91a47645f4179..d4c5a735ce6fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -45,13 +45,13 @@ object TransformWithStateSuiteUtils { class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (String, String)] with Logging { + import implicits._ @transient protected var _countState: ValueState[Long] = _ override def init( outputMode: OutputMode, timeMode: TimeMode): Unit = { - _countState = getHandle.getValueState[Long]("countState", - Encoders.scalaLong, TTLConfig.NONE) + _countState = getHandle.getValueState[Long]("countState", TTLConfig.NONE) } override def handleInputRows( @@ -72,12 +72,13 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S class RunningCountStatefulProcessorWithTTL extends StatefulProcessor[String, String, (String, String)] with Logging { + import implicits._ @transient protected var _countState: ValueState[Long] = _ override def init( outputMode: OutputMode, timeMode: TimeMode): Unit = { - _countState = getHandle.getValueState[Long]("countState", Encoders.scalaLong, + _countState = getHandle.getValueState[Long]("countState", TTLConfig(Duration.ofMillis(1000))) } @@ -384,20 +385,32 @@ class RunningCountStatefulProcessorWithError extends RunningCountStatefulProcess } // class for verify state schema is correctly written for all state var types -class StatefulProcessorWithCompositeTypes extends RunningCountStatefulProcessor { +class StatefulProcessorWithCompositeTypes(useImplicits: Boolean) + extends RunningCountStatefulProcessor { + import implicits._ @transient private var _listState: ListState[TestClass] = _ @transient private var _mapState: MapState[POJOTestClass, String] = _ override def init( outputMode: OutputMode, timeMode: TimeMode): Unit = { - _countState = getHandle.getValueState[Long]("countState", Encoders.scalaLong, - TTLConfig.NONE) - _listState = getHandle.getListState[TestClass]( - "listState", Encoders.product[TestClass], TTLConfig.NONE) - _mapState = getHandle.getMapState[POJOTestClass, String]( - "mapState", Encoders.bean(classOf[POJOTestClass]), Encoders.STRING, - TTLConfig.NONE) + + if (useImplicits) { + _countState = getHandle.getValueState[Long]("countState", TTLConfig.NONE) + _listState = getHandle.getListState[TestClass]( + "listState", TTLConfig.NONE) + _mapState = getHandle.getMapState[POJOTestClass, String]( + "mapState", Encoders.bean(classOf[POJOTestClass]), Encoders.STRING, + TTLConfig.NONE) + } else { + _countState = getHandle.getValueState[Long]("countState", Encoders.scalaLong, + TTLConfig.NONE) + _listState = getHandle.getListState[TestClass]( + "listState", Encoders.product[TestClass], TTLConfig.NONE) + _mapState = getHandle.getMapState[POJOTestClass, String]( + "mapState", Encoders.bean(classOf[POJOTestClass]), Encoders.STRING, + TTLConfig.NONE) + } } } @@ -1037,85 +1050,87 @@ class TransformWithStateSuite extends StateStoreMetricsTest } } - test("transformWithState - verify StateSchemaV3 writes " + - "correct SQL schema of key/value") { - withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> - classOf[RocksDBStateStoreProvider].getName, - SQLConf.SHUFFLE_PARTITIONS.key -> - TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { - withTempDir { checkpointDir => - val metadataPathPostfix = "state/0/_stateSchema/default" - val stateSchemaPath = new Path(checkpointDir.toString, - s"$metadataPathPostfix") - val hadoopConf = spark.sessionState.newHadoopConf() - val fm = CheckpointFileManager.create(stateSchemaPath, hadoopConf) - - val keySchema = new StructType().add("value", StringType) - val schema0 = StateStoreColFamilySchema( - "countState", - keySchema, - new StructType().add("value", LongType, false), - Some(NoPrefixKeyStateEncoderSpec(keySchema)), - None - ) - val schema1 = StateStoreColFamilySchema( - "listState", - keySchema, - new StructType() + Seq(false, true).foreach { useImplicits => + test("transformWithState - verify StateSchemaV3 writes " + + s"correct SQL schema of key/value with useImplicits=$useImplicits") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { checkpointDir => + val metadataPathPostfix = "state/0/_stateSchema/default" + val stateSchemaPath = new Path(checkpointDir.toString, + s"$metadataPathPostfix") + val hadoopConf = spark.sessionState.newHadoopConf() + val fm = CheckpointFileManager.create(stateSchemaPath, hadoopConf) + + val keySchema = new StructType().add("value", StringType) + val schema0 = StateStoreColFamilySchema( + "countState", + keySchema, + new StructType().add("value", LongType, false), + Some(NoPrefixKeyStateEncoderSpec(keySchema)), + None + ) + val schema1 = StateStoreColFamilySchema( + "listState", + keySchema, + new StructType() .add("id", LongType, false) .add("name", StringType), - Some(NoPrefixKeyStateEncoderSpec(keySchema)), - None - ) - - val userKeySchema = new StructType() - .add("id", IntegerType, false) - .add("name", StringType) - val compositeKeySchema = new StructType() - .add("key", new StructType().add("value", StringType)) - .add("userKey", userKeySchema) - val schema2 = StateStoreColFamilySchema( - "mapState", - compositeKeySchema, - new StructType().add("value", StringType), - Some(PrefixKeyScanStateEncoderSpec(compositeKeySchema, 1)), - Option(userKeySchema) - ) - - val inputData = MemoryStream[String] - val result = inputData.toDS() - .groupByKey(x => x) - .transformWithState(new StatefulProcessorWithCompositeTypes(), - TimeMode.None(), - OutputMode.Update()) - - testStream(result, OutputMode.Update())( - StartStream(checkpointLocation = checkpointDir.getCanonicalPath), - AddData(inputData, "a", "b"), - CheckNewAnswer(("a", "1"), ("b", "1")), - Execute { q => - q.lastProgress.runId - val schemaFilePath = fm.list(stateSchemaPath).toSeq.head.getPath - val providerId = StateStoreProviderId(StateStoreId( - checkpointDir.getCanonicalPath, 0, 0), q.lastProgress.runId) - val checker = new StateSchemaCompatibilityChecker(providerId, - hadoopConf, Some(schemaFilePath)) - val colFamilySeq = checker.readSchemaFile() - - assert(TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS == - q.lastProgress.stateOperators.head.customMetrics.get("numValueStateVars").toInt) - assert(TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS == - q.lastProgress.stateOperators.head.customMetrics.get("numListStateVars").toInt) - assert(TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS == - q.lastProgress.stateOperators.head.customMetrics.get("numMapStateVars").toInt) - - assert(colFamilySeq.length == 3) - assert(colFamilySeq.map(_.toString).toSet == Set( - schema0, schema1, schema2 - ).map(_.toString)) - }, - StopStream - ) + Some(NoPrefixKeyStateEncoderSpec(keySchema)), + None + ) + + val userKeySchema = new StructType() + .add("id", IntegerType, false) + .add("name", StringType) + val compositeKeySchema = new StructType() + .add("key", new StructType().add("value", StringType)) + .add("userKey", userKeySchema) + val schema2 = StateStoreColFamilySchema( + "mapState", + compositeKeySchema, + new StructType().add("value", StringType), + Some(PrefixKeyScanStateEncoderSpec(compositeKeySchema, 1)), + Option(userKeySchema) + ) + + val inputData = MemoryStream[String] + val result = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new StatefulProcessorWithCompositeTypes(useImplicits), + TimeMode.None(), + OutputMode.Update()) + + testStream(result, OutputMode.Update())( + StartStream(checkpointLocation = checkpointDir.getCanonicalPath), + AddData(inputData, "a", "b"), + CheckNewAnswer(("a", "1"), ("b", "1")), + Execute { q => + q.lastProgress.runId + val schemaFilePath = fm.list(stateSchemaPath).toSeq.head.getPath + val providerId = StateStoreProviderId(StateStoreId( + checkpointDir.getCanonicalPath, 0, 0), q.lastProgress.runId) + val checker = new StateSchemaCompatibilityChecker(providerId, + hadoopConf, Some(schemaFilePath)) + val colFamilySeq = checker.readSchemaFile() + + assert(TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS == + q.lastProgress.stateOperators.head.customMetrics.get("numValueStateVars").toInt) + assert(TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS == + q.lastProgress.stateOperators.head.customMetrics.get("numListStateVars").toInt) + assert(TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS == + q.lastProgress.stateOperators.head.customMetrics.get("numMapStateVars").toInt) + + assert(colFamilySeq.length == 3) + assert(colFamilySeq.map(_.toString).toSet == Set( + schema0, schema1, schema2 + ).map(_.toString)) + }, + StopStream + ) + } } } } From bac386dcfc6b5765d0f035c4ec7facc4501363bb Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 10 Dec 2024 17:24:18 +0800 Subject: [PATCH 138/438] [SPARK-50528][CONNECT] Move `InvalidCommandInput` to common module ### What changes were proposed in this pull request? Move InvalidCommandInput to common module ### Why are the changes needed? 1, for reusing; 2, to be consistent with `InvalidPlanInput` ### 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 #49125 from zhengruifeng/check_cmd. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../connect/common/InvalidCommandInput.scala | 25 +++++++++++++++++++ .../connect/planner/SparkConnectPlanner.scala | 7 +----- .../planner/SparkConnectProtoSuite.scala | 2 +- 3 files changed, 27 insertions(+), 7 deletions(-) create mode 100644 sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/InvalidCommandInput.scala diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/InvalidCommandInput.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/InvalidCommandInput.scala new file mode 100644 index 0000000000000..313fe7262a10b --- /dev/null +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/InvalidCommandInput.scala @@ -0,0 +1,25 @@ +/* + * 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.connect.common + +/** + * Error thrown when a connect command is not valid. + */ +final case class InvalidCommandInput( + private val message: String = "", + private val cause: Throwable = null) + extends Exception(message, cause) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 6ecf3ce110389..4aa567f19e16c 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -59,7 +59,7 @@ import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.classic.ClassicConversions._ -import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidCommandInput, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry import org.apache.spark.sql.connect.service.{ExecuteHolder, SessionHolder, SparkConnectService} @@ -85,11 +85,6 @@ import org.apache.spark.storage.CacheId import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils -final case class InvalidCommandInput( - private val message: String = "", - private val cause: Throwable = null) - extends Exception(message, cause) - class SparkConnectPlanner( val sessionHolder: SessionHolder, val executeHolderOpt: Option[ExecuteHolder] = None) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala index cad7fe6370827..1a86ced3a2ac9 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericInt import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftAnti, LeftOuter, LeftSemi, PlanTest, RightOuter} import org.apache.spark.sql.catalyst.plans.logical.{CollectMetrics, Distinct, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.connect.common.InvalidPlanInput +import org.apache.spark.sql.connect.common.{InvalidCommandInput, InvalidPlanInput} import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto import org.apache.spark.sql.connect.dsl.MockRemoteSession import org.apache.spark.sql.connect.dsl.commands._ From 12967fe02e8ae723c62f3138174483cad4defac2 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 10 Dec 2024 14:27:17 +0300 Subject: [PATCH 139/438] [SPARK-49349][SQL] Improve error message for LCA with Generate ### What changes were proposed in this pull request? This PR add a new check in `CheckAnalysis` to improve error message for LCA with `Generate`. ### Why are the changes needed? Improve error message. Before this PR: ``` [INTERNAL_ERROR] Invalid call to dataType on unresolved object SQLSTATE: XX000 org.apache.spark.sql.catalyst.analysis.UnresolvedException: [INTERNAL_ERROR] Invalid call to dataType on unresolved object SQLSTATE: XX000 at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:292) at org.apache.spark.sql.catalyst.expressions.LateralColumnAliasReference.dataType(namedExpressions.scala:472) ``` After this PR: ``` [UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GENERATOR] The feature is not supported: Referencing a lateral column alias `new_name` in generator expression "unresolvedalias(lateralAliasReference(new_name) LIKE a%)". SQLSTATE: 0A000; 'Project [explode(split(name#21, ,, -1)) AS new_name#19, unresolvedalias(lateralAliasReference(new_name) LIKE a%)] +- SubqueryAlias spark_catalog.default.employee +- Relation spark_catalog.default.employee[dept#20,name#21,salary#22,bonus#23,properties#24] orc ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48915 from wangyum/SPARK-49349. Authored-by: Yuming Wang Signed-off-by: Max Gekk --- .../resources/error/error-conditions.json | 5 +++ .../sql/catalyst/analysis/CheckAnalysis.scala | 17 +++++++++ .../spark/sql/LateralColumnAliasSuite.scala | 37 +++++++++++++++++++ 3 files changed, 59 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index cbd85335679d6..ae75992da2f6d 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5259,6 +5259,11 @@ "Referencing lateral column alias in the aggregate query both with window expressions and with having clause. Please rewrite the aggregate query by removing the having clause or removing lateral alias reference in the SELECT list." ] }, + "LATERAL_COLUMN_ALIAS_IN_GENERATOR" : { + "message" : [ + "Referencing a lateral column alias in generator expression ." + ] + }, "LATERAL_COLUMN_ALIAS_IN_GROUP_BY" : { "message" : [ "Referencing a lateral column alias via GROUP BY alias/ALL is not supported yet." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index d34ffcb02d0a9..aaae763991f10 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -192,6 +192,15 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB ) } + private def isContainsUnsupportedLCA(e: Expression, operator: LogicalPlan): Boolean = { + e.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE) && operator.expressions.exists { + case a: Alias + if e.collect { case l: LateralColumnAliasReference => l.nameParts.head }.contains(a.name) => + a.exists(_.isInstanceOf[Generator]) + case _ => false + } + } + /** * Checks for errors in a `SELECT` clause, such as a trailing comma or an empty select list. * @@ -359,6 +368,14 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB // surrounded with single quotes, or there is a typo in the attribute name. case GetMapValue(map, key: Attribute) if isMapWithStringKey(map) && !key.resolved => failUnresolvedAttribute(operator, key, "UNRESOLVED_MAP_KEY") + + case e: Expression if isContainsUnsupportedLCA(e, operator) => + val lcaRefNames = + e.collect { case lcaRef: LateralColumnAliasReference => lcaRef.name }.distinct + failAnalysis( + errorClass = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GENERATOR", + messageParameters = + Map("lca" -> toSQLId(lcaRefNames), "generatorExpr" -> toSQLExpr(e))) } // Fail if we still have an unresolved all in group by. This needs to run before the diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala index d7177e19a6177..3def42cd7ee55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala @@ -1365,4 +1365,41 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { // the states are cleared - a subsequent correct query should succeed sql("select 1 as a, a").queryExecution.assertAnalyzed() } + + test("SPARK-49349: Improve error message for LCA with Generate") { + checkError( + exception = intercept[AnalysisException] { + sql( + s""" + |SELECT + | explode(split(name , ',')) AS new_name, + | new_name like 'a%' + |FROM $testTable + |""".stripMargin) + }, + condition = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GENERATOR", + sqlState = "0A000", + parameters = Map( + "lca" -> "`new_name`", + "generatorExpr" -> "\"unresolvedalias(lateralAliasReference(new_name) LIKE a%)\"")) + + checkError( + exception = intercept[AnalysisException] { + sql( + s""" + |SELECT + | explode_outer(from_json(name,'array>')) as newName, + | size(from_json(newName.values,'array')) + + | size(array(from_json(newName.values,'map'))) as size + |FROM $testTable + |""".stripMargin) + }, + condition = "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GENERATOR", + sqlState = "0A000", + parameters = Map( + "lca" -> "`newName.values`", + "generatorExpr" -> ("\"(size(from_json(lateralAliasReference(newName.values), " + + "array)) + size(array(from_json(lateralAliasReference(newName.values), " + + "map)))) AS size\""))) + } } From faef3fa9b2c34343eafa3a53a7216509e707aa9f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 10 Dec 2024 19:39:10 +0800 Subject: [PATCH 140/438] [SPARK-50527][INFRA] Add a separate docker file for python 3.12 daily build ### What changes were proposed in this pull request? Add a separate docker file for python 3.12 daily build ### Why are the changes needed? to isolate the testing environment ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with `env`: ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-312", "PYTHON_TO_TEST": "python3.12"}' ``` https://github.com/zhengruifeng/spark/runs/34169304629 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49122 from zhengruifeng/py_image_312. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../workflows/build_infra_images_cache.yml | 14 ++++ .github/workflows/build_python_3.12.yml | 1 + dev/spark-test-image/python-312/Dockerfile | 83 +++++++++++++++++++ 3 files changed, 98 insertions(+) create mode 100644 dev/spark-test-image/python-312/Dockerfile diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index 031a09af69541..b4e7a2cbd0b37 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -32,6 +32,7 @@ on: - 'dev/spark-test-image/sparkr/Dockerfile' - 'dev/spark-test-image/python-309/Dockerfile' - 'dev/spark-test-image/python-310/Dockerfile' + - 'dev/spark-test-image/python-312/Dockerfile' - '.github/workflows/build_infra_images_cache.yml' # Create infra image when cutting down branches/tags create: @@ -130,3 +131,16 @@ jobs: - name: Image digest (PySpark with Python 3.10) if: hashFiles('dev/spark-test-image/python-310/Dockerfile') != '' run: echo ${{ steps.docker_build_pyspark_python_310.outputs.digest }} + - name: Build and push (PySpark with Python 3.12) + if: hashFiles('dev/spark-test-image/python-312/Dockerfile') != '' + id: docker_build_pyspark_python_312 + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/python-312/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-312-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-312-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-312-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark with Python 3.12) + if: hashFiles('dev/spark-test-image/python-312/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_python_312.outputs.digest }} diff --git a/.github/workflows/build_python_3.12.yml b/.github/workflows/build_python_3.12.yml index e1fd45a7d8838..2503a2f158357 100644 --- a/.github/workflows/build_python_3.12.yml +++ b/.github/workflows/build_python_3.12.yml @@ -36,6 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-312", "PYTHON_TO_TEST": "python3.12" } jobs: >- diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile new file mode 100644 index 0000000000000..ecfb1ab07123c --- /dev/null +++ b/dev/spark-test-image/python-312/Dockerfile @@ -0,0 +1,83 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark with Python 3.12" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE 20241206 + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pkg-config \ + qpdf \ + tzdata \ + software-properties-common \ + wget \ + zlib1g-dev + +# Install Python 3.12 +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt-get update && apt-get install -y \ + python3.12 \ + && apt-get autoremove --purge -y \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +# Python deps for Spark Connect +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" + +# Install Python 3.12 at the last stage to avoid breaking the existing Python installations +RUN apt-get update && apt-get install -y \ + python3.12 \ + && rm -rf /var/lib/apt/lists/* +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 +RUN python3.12 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.12 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS lxml && \ + python3.12 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ + python3.12 -m pip install torcheval && \ + python3.12 -m pip cache purge \ From 559fda794d1d9ff05f502054ce164f4eff1e2a23 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Tue, 10 Dec 2024 21:48:21 +0300 Subject: [PATCH 141/438] [SPARK-50530][SQL] Fix bad implicit string type context calculation ### What changes were proposed in this pull request? Catch more cases where string type context should have implicit strength (lateral column aliases, outer references etc) ### Why are the changes needed? Before we would only assign implicit strength when we encountered `AttributeReference` and `Alias`. However, this should be the case for all `NamedExpressions` and not just these two. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49114 from stefankandic/implicitStrengthImprovement. Lead-authored-by: Stefan Kandic Co-authored-by: Stefan Kandic <154237371+stefankandic@users.noreply.github.com> Signed-off-by: Max Gekk --- .../analysis/CollationTypeCoercion.scala | 6 +-- .../CollationTypePrecedenceSuite.scala | 48 +++++++++++++++++++ 2 files changed, 51 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index c6a2213a496e2..fa5bb2c071527 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -357,6 +357,9 @@ object CollationTypeCoercion { case collate: Collate => Some(addContextToStringType(collate.dataType, Explicit)) + case expr @ (_: NamedExpression | _: SubqueryExpression | _: VariableReference) => + Some(addContextToStringType(expr.dataType, Implicit)) + case cast: Cast => if (isUserDefined(cast) && isComplexType(cast.dataType)) { // since we can't use collate clause with complex types @@ -366,9 +369,6 @@ object CollationTypeCoercion { Some(addContextToStringType(cast.dataType, Default)) } - case expr @ (_: Alias | _: SubqueryExpression | _: AttributeReference | _: VariableReference) => - Some(addContextToStringType(expr.dataType, Implicit)) - case lit: Literal => Some(addContextToStringType(lit.dataType, Default)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index 6f33633cc11b3..c0e61de57cd9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -110,6 +110,54 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { } } + test("lateral alias has implicit strength") { + checkAnswer( + sql(""" + |SELECT + | a collate unicode as col1, + | COLLATION(col1 || 'b') + |FROM VALUES ('a') AS t(a) + |""".stripMargin), + Row("a", UNICODE_COLLATION_NAME)) + + assertImplicitMismatch( + sql(""" + |SELECT + | a collate unicode as col1, + | a collate utf8_lcase as col2, + | col1 = col2 + |FROM VALUES ('a') AS t(a) + |""".stripMargin)) + + checkAnswer( + sql(""" + |SELECT + | a collate unicode as col1, + | COLLATION(col1 || 'b' collate UTF8_LCASE) + |FROM VALUES ('a') AS t(a) + |""".stripMargin), + Row("a", UTF8_LCASE_COLLATION_NAME)) + } + + test("outer reference has implicit strength") { + val tableName = "outer_ref_tbl" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (c STRING COLLATE UNICODE_CI, c1 STRING) USING $dataSource") + sql(s"INSERT INTO $tableName VALUES ('a', 'a'), ('A', 'A')") + + checkAnswer( + sql(s"SELECT DISTINCT (SELECT COLLATION(c || 'a')) FROM $tableName"), + Seq(Row(UNICODE_CI_COLLATION_NAME))) + + assertImplicitMismatch( + sql(s"SELECT DISTINCT (SELECT COLLATION(c || c1)) FROM $tableName")) + + checkAnswer( + sql(s"SELECT DISTINCT (SELECT COLLATION(c || 'a' collate utf8_lcase)) FROM $tableName"), + Seq(Row(UTF8_LCASE_COLLATION_NAME))) + } + } + test("variables have implicit collation") { val v1Collation = UTF8_BINARY_COLLATION_NAME val v2Collation = UTF8_LCASE_COLLATION_NAME From e70f8abacec44f255cdd54d0154d4f3b0aef730c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Du=C5=A1an=20Ti=C5=A1ma?= Date: Tue, 10 Dec 2024 11:55:22 -0800 Subject: [PATCH 142/438] [SPARK-50491][SQL] Fix bug where empty BEGIN END blocks throw an error MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR depends on https://github.com/apache/spark/pull/48989 ### What changes were proposed in this pull request? There is a bug in SQL scripting which causes empty compound statements to throw an error if their body consists solely of empty BEGIN END blocks. Examples: ``` WHILE 1 = 1 DO BEGIN END; END WHILE; ``` ``` BEGIN BEGIN BEGIN END; END; END; ``` This PR fixes this by introducing a NO-OP statement for SQL scripting, which empty BEGIN END blocks will return. ### Why are the changes needed? Currenty, compound bodies declare they have the next element even if their body is consisted only of empty blocks. This is because it only checks for existence of statements in the body, not whether there is at least one statement which is not an empty block. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests were added to existing suites. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49064 from dusantism-db/scripting-noop-statement. Authored-by: Dušan Tišma Signed-off-by: Wenchen Fan --- .../scripting/SqlScriptingExecutionNode.scala | 8 + .../scripting/SqlScriptingInterpreter.scala | 11 +- .../SqlScriptingInterpreterSuite.scala | 257 +++++++++++++++++- 3 files changed, 272 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index e3559e8f18ae2..99719ce19e3de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -163,6 +163,14 @@ class SingleStatementExec( override def reset(): Unit = isExecuted = false } +/** + * NO-OP leaf node, which does nothing when returned to the iterator. + * It is emitted by empty BEGIN END blocks. + */ +class NoOpStatementExec extends LeafStatementExec { + override def reset(): Unit = () +} + /** * Executable node for CompoundBody. * @param statements diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala index a3dc3d4599314..5d3edeefc532b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala @@ -86,9 +86,14 @@ case class SqlScriptingInterpreter(session: SparkSession) { .map(varName => DropVariable(varName, ifExists = true)) .map(new SingleStatementExec(_, Origin(), args, isInternal = true)) .reverse - new CompoundBodyExec( - collection.map(st => transformTreeIntoExecutable(st, args)) ++ dropVariables, - label) + + val statements = + collection.map(st => transformTreeIntoExecutable(st, args)) ++ dropVariables match { + case Nil => Seq(new NoOpStatementExec) + case s => s + } + + new CompoundBodyExec(statements, label) case IfElseStatement(conditions, conditionalBodies, elseBody) => val conditionsExec = conditions.map(condition => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 71556c5502225..2ec42c4554e09 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -110,6 +110,61 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { } } + test("empty begin end block") { + val sqlScript = + """ + |BEGIN + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(sqlScript, expected) + } + + test("empty begin end blocks") { + val sqlScript = + """ + |BEGIN + | BEGIN + | END; + | BEGIN + | END; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(sqlScript, expected) + } + + test("empty begin end blocks with single statement") { + val sqlScript = + """ + |BEGIN + | BEGIN + | END; + | SELECT 1; + | BEGIN + | END; + |END + |""".stripMargin + val expected = Seq(Seq(Row(1))) + verifySqlScriptResult(sqlScript, expected) + } + + test("empty begin end blocks - nested") { + val sqlScript = + """ + |BEGIN + | BEGIN + | BEGIN + | END; + | BEGIN + | END; + | END; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(sqlScript, expected) + } + test("session vars - set and read (SET VAR)") { val sqlScript = """ @@ -240,6 +295,40 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { verifySqlScriptResult(commands, expected) } + test("if - empty body") { + val commands = + """ + |BEGIN + | IF 1=1 THEN + | BEGIN + | END; + | END IF; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + + test("if - nested empty body") { + val commands = + """ + |BEGIN + | IF 1=1 THEN + | BEGIN + | BEGIN + | END; + | END; + | BEGIN + | BEGIN + | END; + | END; + | END IF; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + test("if nested") { val commands = """ @@ -389,6 +478,42 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { verifySqlScriptResult(commands, expected) } + test("searched case - empty body") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = 1 THEN + | BEGIN + | END; + | END CASE; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + + test("searched case - nested empty body") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = 1 THEN + | BEGIN + | BEGIN + | END; + | END; + | BEGIN + | BEGIN + | END; + | END; + | END CASE; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + test("searched case nested") { val commands = """ @@ -589,6 +714,42 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { verifySqlScriptResult(commands, expected) } + test("simple case - empty body") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | BEGIN + | END; + | END CASE; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + + test("simple case - nested empty body") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | BEGIN + | BEGIN + | END; + | END; + | BEGIN + | BEGIN + | END; + | END; + | END CASE; + |END + |""".stripMargin + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + test("simple case nested") { val commands = """ @@ -985,6 +1146,42 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { verifySqlScriptResult(commands, expected) } + test("repeat - empty body") { + val commands = + """ + |BEGIN + | REPEAT + | BEGIN + | END; + | UNTIL 1 = 1 + | END REPEAT; + |END + |""".stripMargin + + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + + test("repeat - nested empty body") { + val commands = + """ + |BEGIN + | REPEAT + | BEGIN + | BEGIN + | END; + | END; + | BEGIN + | END; + | UNTIL 1 = 1 + | END REPEAT; + |END + |""".stripMargin + + val expected = Seq.empty[Seq[Row]] + verifySqlScriptResult(commands, expected) + } + test("nested repeat") { val commands = """ @@ -1795,7 +1992,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { } } - test("for statement empty result") { + test("for statement - empty result") { withTable("t") { val sqlScript = """ @@ -1814,6 +2011,64 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { } } + test("for statement - empty body") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING, doubleCol DOUBLE) using parquet; + | INSERT INTO t VALUES (1, 'first', 1.0); + | FOR row AS SELECT * FROM t DO + | BEGIN + | END; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + + test("for statement - nested empty body") { + withTable("t") { + val sqlScript = + """ + |BEGIN + | CREATE TABLE t (intCol INT, stringCol STRING, doubleCol DOUBLE) using parquet; + | INSERT INTO t VALUES (1, 'first', 1.0); + | FOR row AS SELECT * FROM t DO + | BEGIN + | BEGIN + | END; + | END; + | BEGIN + | BEGIN + | END; + | END; + | END FOR; + |END + |""".stripMargin + + val expected = Seq( + Seq.empty[Row], // create table + Seq.empty[Row], // insert + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row], // drop local var + Seq.empty[Row] // drop local var + ) + verifySqlScriptResult(sqlScript, expected) + } + } + test("for statement iterate") { withTable("t") { val sqlScript = From 18f0e231335ae46876bf9704bd8a9cc66eb1b6f2 Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Tue, 10 Dec 2024 13:14:28 -0800 Subject: [PATCH 143/438] [SPARK-45891][SQL] Rebuild variant binary from shredded data ### What changes were proposed in this pull request? It implements the variant rebuild functionality according to the current shredding spec in https://github.com/apache/parquet-format/pull/461, and allows the Parquet reader will be able to read shredded variant data. ### Why are the changes needed? It gives Spark the basic ability to read shredded variant data. It can be improved in the future to read only requested fields. ### Does this PR introduce _any_ user-facing change? Yes, the Parquet reader will be able to read shredded variant data. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48851 from chenhao-db/rebuild_variant. Authored-by: Chenhao Li Signed-off-by: Wenchen Fan --- .../spark/types/variant/ShreddingUtils.java | 170 ++++++++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 9 + .../parquet/ParquetColumnVector.java | 40 ++++- .../datasources/parquet/ParquetColumn.scala | 5 +- .../parquet/ParquetRowConverter.scala | 50 +++++- .../parquet/ParquetSchemaConverter.scala | 7 +- .../parquet/SparkShreddingUtils.scala | 27 ++- .../spark/sql/VariantShreddingSuite.scala | 166 +++++++++++++++++ .../org/apache/spark/sql/VariantSuite.scala | 11 +- 9 files changed, 475 insertions(+), 10 deletions(-) create mode 100644 common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala diff --git a/common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java b/common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java new file mode 100644 index 0000000000000..59e16b77ab01d --- /dev/null +++ b/common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java @@ -0,0 +1,170 @@ +/* + * 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.types.variant; + +import java.math.BigDecimal; +import java.util.ArrayList; + +import static org.apache.spark.types.variant.VariantUtil.*; + +public class ShreddingUtils { + // Interface to read from a shredded result. It essentially has the same interface and semantics + // as Spark's `SpecializedGetters`, but we need a new interface to avoid the dependency. + public interface ShreddedRow { + boolean isNullAt(int ordinal); + boolean getBoolean(int ordinal); + byte getByte(int ordinal); + short getShort(int ordinal); + int getInt(int ordinal); + long getLong(int ordinal); + float getFloat(int ordinal); + double getDouble(int ordinal); + BigDecimal getDecimal(int ordinal, int precision, int scale); + String getString(int ordinal); + byte[] getBinary(int ordinal); + ShreddedRow getStruct(int ordinal, int numFields); + ShreddedRow getArray(int ordinal); + int numElements(); + } + + // This `rebuild` function should only be called on the top-level schema, and that other private + // implementation will be called on any recursively shredded sub-schema. + public static Variant rebuild(ShreddedRow row, VariantSchema schema) { + if (schema.topLevelMetadataIdx < 0 || row.isNullAt(schema.topLevelMetadataIdx)) { + throw malformedVariant(); + } + byte[] metadata = row.getBinary(schema.topLevelMetadataIdx); + if (schema.variantIdx >= 0 && schema.typedIdx < 0) { + // The variant is unshredded. We are not required to do anything special, but we can have an + // optimization to avoid `rebuild`. + if (row.isNullAt(schema.variantIdx)) { + throw malformedVariant(); + } + return new Variant(row.getBinary(schema.variantIdx), metadata); + } + VariantBuilder builder = new VariantBuilder(false); + rebuild(row, metadata, schema, builder); + return builder.result(); + } + + // Rebuild a variant value from the shredded data according to the reconstruction algorithm in + // https://github.com/apache/parquet-format/blob/master/VariantShredding.md. + // Append the result to `builder`. + private static void rebuild(ShreddedRow row, byte[] metadata, VariantSchema schema, + VariantBuilder builder) { + int typedIdx = schema.typedIdx; + int variantIdx = schema.variantIdx; + if (typedIdx >= 0 && !row.isNullAt(typedIdx)) { + if (schema.scalarSchema != null) { + VariantSchema.ScalarType scalar = schema.scalarSchema; + if (scalar instanceof VariantSchema.StringType) { + builder.appendString(row.getString(typedIdx)); + } else if (scalar instanceof VariantSchema.IntegralType) { + VariantSchema.IntegralType it = (VariantSchema.IntegralType) scalar; + long value = 0; + switch (it.size) { + case BYTE: + value = row.getByte(typedIdx); + break; + case SHORT: + value = row.getShort(typedIdx); + break; + case INT: + value = row.getInt(typedIdx); + break; + case LONG: + value = row.getLong(typedIdx); + break; + } + builder.appendLong(value); + } else if (scalar instanceof VariantSchema.FloatType) { + builder.appendFloat(row.getFloat(typedIdx)); + } else if (scalar instanceof VariantSchema.DoubleType) { + builder.appendDouble(row.getDouble(typedIdx)); + } else if (scalar instanceof VariantSchema.BooleanType) { + builder.appendBoolean(row.getBoolean(typedIdx)); + } else if (scalar instanceof VariantSchema.BinaryType) { + builder.appendBinary(row.getBinary(typedIdx)); + } else if (scalar instanceof VariantSchema.DecimalType) { + VariantSchema.DecimalType dt = (VariantSchema.DecimalType) scalar; + builder.appendDecimal(row.getDecimal(typedIdx, dt.precision, dt.scale)); + } else if (scalar instanceof VariantSchema.DateType) { + builder.appendDate(row.getInt(typedIdx)); + } else if (scalar instanceof VariantSchema.TimestampType) { + builder.appendTimestamp(row.getLong(typedIdx)); + } else { + assert scalar instanceof VariantSchema.TimestampNTZType; + builder.appendTimestampNtz(row.getLong(typedIdx)); + } + } else if (schema.arraySchema != null) { + VariantSchema elementSchema = schema.arraySchema; + ShreddedRow array = row.getArray(typedIdx); + int start = builder.getWritePos(); + ArrayList offsets = new ArrayList<>(array.numElements()); + for (int i = 0; i < array.numElements(); i++) { + offsets.add(builder.getWritePos() - start); + rebuild(array.getStruct(i, elementSchema.numFields), metadata, elementSchema, builder); + } + builder.finishWritingArray(start, offsets); + } else { + ShreddedRow object = row.getStruct(typedIdx, schema.objectSchema.length); + ArrayList fields = new ArrayList<>(); + int start = builder.getWritePos(); + for (int fieldIdx = 0; fieldIdx < schema.objectSchema.length; ++fieldIdx) { + // Shredded field must not be null. + if (object.isNullAt(fieldIdx)) { + throw malformedVariant(); + } + String fieldName = schema.objectSchema[fieldIdx].fieldName; + VariantSchema fieldSchema = schema.objectSchema[fieldIdx].schema; + ShreddedRow fieldValue = object.getStruct(fieldIdx, fieldSchema.numFields); + // If the field doesn't have non-null `typed_value` or `value`, it is missing. + if ((fieldSchema.typedIdx >= 0 && !fieldValue.isNullAt(fieldSchema.typedIdx)) || + (fieldSchema.variantIdx >= 0 && !fieldValue.isNullAt(fieldSchema.variantIdx))) { + int id = builder.addKey(fieldName); + fields.add(new VariantBuilder.FieldEntry(fieldName, id, builder.getWritePos() - start)); + rebuild(fieldValue, metadata, fieldSchema, builder); + } + } + if (variantIdx >= 0 && !row.isNullAt(variantIdx)) { + // Add the leftover fields in the variant binary. + Variant v = new Variant(row.getBinary(variantIdx), metadata); + if (v.getType() != VariantUtil.Type.OBJECT) throw malformedVariant(); + for (int i = 0; i < v.objectSize(); ++i) { + Variant.ObjectField field = v.getFieldAtIndex(i); + // `value` must not contain any shredded field. + if (schema.objectSchemaMap.containsKey(field.key)) { + throw malformedVariant(); + } + int id = builder.addKey(field.key); + fields.add(new VariantBuilder.FieldEntry(field.key, id, builder.getWritePos() - start)); + builder.appendVariant(field.value); + } + } + builder.finishWritingObject(start, fields); + } + } else if (variantIdx >= 0 && !row.isNullAt(variantIdx)) { + // `typed_value` doesn't exist or is null. Read from `value`. + builder.appendVariant(new Variant(row.getBinary(variantIdx), metadata)); + } else { + // This means the variant is missing in a context where it must present, so the input data is + // invalid. + throw malformedVariant(); + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 47b670b730ad5..6b430f167ca8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4546,6 +4546,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val VARIANT_ALLOW_READING_SHREDDED = + buildConf("spark.sql.variant.allowReadingShredded") + .internal() + .doc("When true, the Parquet reader is allowed to read shredded or unshredded variant. " + + "When false, it only reads unshredded variant.") + .version("4.0.0") + .booleanConf + .createWithDefault(true) + val LEGACY_CSV_ENABLE_DATE_TIME_PARSING_FALLBACK = buildConf("spark.sql.legacy.csv.enableDateTimeParsingFallback") .internal() diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java index 7f5b69a09e90c..0b9a25fc46a0f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java @@ -34,6 +34,8 @@ import org.apache.spark.sql.types.MapType; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.VariantType; +import org.apache.spark.types.variant.VariantSchema; +import org.apache.spark.unsafe.types.VariantVal; /** * Contains necessary information representing a Parquet column, either of primitive or nested type. @@ -43,6 +45,11 @@ final class ParquetColumnVector { private final List children; private final WritableColumnVector vector; + // Describes the file schema of the Parquet variant column. When it is not null, `children` + // contains only one child that reads the underlying file content. This `ParquetColumnVector` + // should assemble Spark variant values from the file content. + private VariantSchema variantSchema; + /** * Repetition & Definition levels * These are allocated only for leaf columns; for non-leaf columns, they simply maintain @@ -101,7 +108,18 @@ final class ParquetColumnVector { } } - if (isPrimitive) { + if (column.variantFileType().isDefined()) { + ParquetColumn fileContentCol = column.variantFileType().get(); + WritableColumnVector fileContent = memoryMode == MemoryMode.OFF_HEAP + ? new OffHeapColumnVector(capacity, fileContentCol.sparkType()) + : new OnHeapColumnVector(capacity, fileContentCol.sparkType()); + ParquetColumnVector contentVector = new ParquetColumnVector(fileContentCol, + fileContent, capacity, memoryMode, missingColumns, false, null); + children.add(contentVector); + variantSchema = SparkShreddingUtils.buildVariantSchema(fileContentCol.sparkType()); + repetitionLevels = contentVector.repetitionLevels; + definitionLevels = contentVector.definitionLevels; + } else if (isPrimitive) { if (column.repetitionLevel() > 0) { repetitionLevels = allocateLevelsVector(capacity, memoryMode); } @@ -167,6 +185,26 @@ private static void getLeavesHelper(ParquetColumnVector vector, List - new ParquetVariantConverter(parquetType.asGroupType(), updater) + if (SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED)) { + // Infer a Spark type from `parquetType`. This piece of code is copied from + // `ParquetArrayConverter`. + val messageType = Types.buildMessage().addField(parquetType).named("foo") + val column = new ColumnIOFactory().getColumnIO(messageType) + val parquetSparkType = schemaConverter.convertField(column.getChild(0)).sparkType + new ParquetVariantConverter(parquetType.asGroupType(), parquetSparkType, updater) + } else { + new ParquetUnshreddedVariantConverter(parquetType.asGroupType(), updater) + } case t => throw QueryExecutionErrors.cannotCreateParquetConverterForDataTypeError( @@ -845,8 +854,8 @@ private[parquet] class ParquetRowConverter( } } - /** Parquet converter for Variant */ - private final class ParquetVariantConverter( + /** Parquet converter for unshredded Variant */ + private final class ParquetUnshreddedVariantConverter( parquetType: GroupType, updater: ParentContainerUpdater) extends ParquetGroupConverter(updater) { @@ -898,6 +907,41 @@ private[parquet] class ParquetRowConverter( } } + /** Parquet converter for Variant (shredded or unshredded) */ + private final class ParquetVariantConverter( + parquetType: GroupType, + parquetSparkType: DataType, + updater: ParentContainerUpdater) + extends ParquetGroupConverter(updater) { + + private[this] var currentRow: Any = _ + private[this] val variantSchema = SparkShreddingUtils.buildVariantSchema(parquetSparkType) + // A struct converter that reads the underlying file data. + private[this] val fileConverter = new ParquetRowConverter( + schemaConverter, + parquetType, + parquetSparkType.asInstanceOf[StructType], + convertTz, + datetimeRebaseSpec, + int96RebaseSpec, + new ParentContainerUpdater { + override def set(value: Any): Unit = currentRow = value + }) + + override def getConverter(fieldIndex: Int): Converter = fileConverter.getConverter(fieldIndex) + + override def end(): Unit = { + fileConverter.end() + val v = SparkShreddingUtils.rebuild(currentRow.asInstanceOf[InternalRow], variantSchema) + updater.set(v) + } + + override def start(): Unit = { + fileConverter.start() + currentRow = null + } + } + private trait RepeatedConverter { private[this] val currentArray = ArrayBuffer.empty[Any] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 350d42c8efd76..7f1b49e737900 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -179,7 +179,12 @@ class ParquetToSparkSchemaConverter( field match { case primitiveColumn: PrimitiveColumnIO => convertPrimitiveField(primitiveColumn, targetType) case groupColumn: GroupColumnIO if targetType.contains(VariantType) => - convertVariantField(groupColumn) + if (SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED)) { + val col = convertGroupField(groupColumn) + col.copy(sparkType = VariantType, variantFileType = Some(col)) + } else { + convertVariantField(groupColumn) + } case groupColumn: GroupColumnIO => convertGroupField(groupColumn, targetType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index 2b81668b88b87..41244e20c369f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -19,12 +19,32 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types._ import org.apache.spark.types.variant._ import org.apache.spark.unsafe.types._ +case class SparkShreddedRow(row: SpecializedGetters) extends ShreddingUtils.ShreddedRow { + override def isNullAt(ordinal: Int): Boolean = row.isNullAt(ordinal) + override def getBoolean(ordinal: Int): Boolean = row.getBoolean(ordinal) + override def getByte(ordinal: Int): Byte = row.getByte(ordinal) + override def getShort(ordinal: Int): Short = row.getShort(ordinal) + override def getInt(ordinal: Int): Int = row.getInt(ordinal) + override def getLong(ordinal: Int): Long = row.getLong(ordinal) + override def getFloat(ordinal: Int): Float = row.getFloat(ordinal) + override def getDouble(ordinal: Int): Double = row.getDouble(ordinal) + override def getDecimal(ordinal: Int, precision: Int, scale: Int): java.math.BigDecimal = + row.getDecimal(ordinal, precision, scale).toJavaBigDecimal + override def getString(ordinal: Int): String = row.getUTF8String(ordinal).toString + override def getBinary(ordinal: Int): Array[Byte] = row.getBinary(ordinal) + override def getStruct(ordinal: Int, numFields: Int): SparkShreddedRow = + SparkShreddedRow(row.getStruct(ordinal, numFields)) + override def getArray(ordinal: Int): SparkShreddedRow = + SparkShreddedRow(row.getArray(ordinal)) + override def numElements(): Int = row.asInstanceOf[ArrayData].numElements() +} + case object SparkShreddingUtils { val VariantValueFieldName = "value"; val TypedValueFieldName = "typed_value"; @@ -217,4 +237,9 @@ case object SparkShreddingUtils { .asInstanceOf[SparkShreddedResult] .row } + + def rebuild(row: InternalRow, schema: VariantSchema): VariantVal = { + val v = ShreddingUtils.rebuild(SparkShreddedRow(row), schema) + new VariantVal(v.getValue, v.getMetadata) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala new file mode 100644 index 0000000000000..4ff346b957aa0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala @@ -0,0 +1,166 @@ +/* + * 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 + +import java.io.File +import java.sql.{Date, Timestamp} +import java.time.LocalDateTime + +import org.apache.spark.SparkThrowable +import org.apache.spark.sql.execution.datasources.parquet.{ParquetTest, SparkShreddingUtils} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ +import org.apache.spark.types.variant._ + +class VariantShreddingSuite extends QueryTest with SharedSparkSession with ParquetTest { + // Make a variant value binary by parsing a JSON string. + def value(s: String): Array[Byte] = VariantBuilder.parseJson(s, false).getValue + + // Make a variant metadata binary that includes a set of keys. + def metadata(keys: Seq[String]): Array[Byte] = { + val builder = new VariantBuilder(false) + keys.foreach(builder.addKey) + builder.result().getMetadata + } + + // Build a shredded variant value binary. Its IDs refer to the metadata built from `metadataKeys`, + // which can include more keys than the JSON string contains. + def shreddedValue(s: String, metadataKeys: Seq[String]): Array[Byte] = { + val builder = new VariantBuilder(false) + metadataKeys.foreach(builder.addKey) + builder.appendVariant(VariantBuilder.parseJson(s, false)) + builder.result().getValue + } + + // Given an expected schema of a Variant value, return a write schema with a single column `v` + // with the corresponding shredding schema. + def writeSchema(schema: DataType): StructType = + StructType(Array(StructField("v", SparkShreddingUtils.variantShreddingSchema(schema)))) + + def testWithTempPath(name: String)(block: File => Unit): Unit = test(name) { + withTempPath { path => + block(path) + } + } + + def writeRows(path: File, schema: StructType, rows: Row*): Unit = + spark.createDataFrame(spark.sparkContext.parallelize(rows.map(Row(_)), numSlices = 1), schema) + .write.mode("overwrite").parquet(path.getAbsolutePath) + + def read(path: File): DataFrame = + spark.read.schema("v variant").parquet(path.getAbsolutePath) + + def checkExpr(path: File, expr: String, expected: Any*): Unit = withAllParquetReaders { + checkAnswer(read(path).selectExpr(expr), expected.map(Row(_))) + } + + def checkException(path: File, expr: String, msg: String): Unit = withAllParquetReaders { + val ex = intercept[Exception with SparkThrowable] { + read(path).selectExpr(expr).collect() + } + // When reading with the parquet-mr reader, the expected message can be nested in + // `ex.getCause.getCause`. + assert(ex.getMessage.contains(msg) || ex.getCause.getMessage.contains(msg) + || ex.getCause.getCause.getMessage.contains(msg)) + } + + testWithTempPath("scalar types rebuild") { path => + val scalarTypes = Array( + BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, + TimestampType, TimestampNTZType, DateType, + StringType, BinaryType, + DecimalType(9, 3), DecimalType(18, 6), DecimalType(22, 9)) + val schema = StructType(scalarTypes.zipWithIndex.map { case (t, i) => + StructField(i.toString, t) + }) + + val values = Seq[Any]( + true, 1.toByte, 2.toShort, 3, 4L, 5.5F, 6.6, + new Timestamp(7), LocalDateTime.of(1, 1, 1, 0, 0, 8, 0), new Date(9), + "str10", Array[Byte](11), + Decimal("12.12"), Decimal("13.13"), Decimal("14.14")).map(Row(null, _)) + val row = Row(metadata(scalarTypes.indices.map(_.toString)), null, Row.fromSeq(values)) + + writeRows(path, writeSchema(schema), row) + for (tz <- Seq("Etc/UTC", "America/Los_Angeles")) { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + val timestamp = if (tz == "Etc/UTC") { + "1970-01-01 00:00:00.007+00:00" + } else { + "1969-12-31 16:00:00.007-08:00" + } + checkExpr(path, "to_json(v)", + """{"0":true,"1":1,"10":"str10","11":"Cw==","12":12.12,"13":13.13,"14":14.14,""" + + s""""2":2,"3":3,"4":4,"5":5.5,"6":6.6,"7":"$timestamp",""" + + """"8":"0001-01-01 00:00:08","9":"1969-12-31"}""") + checkExpr(path, "variant_get(v, '$.0', 'int')", 1) + checkExpr(path, "variant_get(v, '$.2', 'boolean')", true) + checkExpr(path, "variant_get(v, '$.6', 'float')", 6.6F) + checkExpr(path, "variant_get(v, '$.11', 'string')", new String(Array[Byte](11))) + checkExpr(path, "variant_get(v, '$.14', 'decimal(9, 1)')", BigDecimal("14.1")) + } + } + } + + testWithTempPath("object rebuild") { path => + writeRows(path, writeSchema(StructType.fromDDL("b int, d int")), + Row(metadata(Seq("b", "d")), null, Row(Row(null, 1), Row(null, null))), + Row(metadata(Seq("b", "d")), null, Row(Row(null, 1), Row(value("null"), null))), + Row(metadata(Seq("a", "b", "c", "d")), + shreddedValue("""{"a": 1, "c": 3}""", Seq("a", "b", "c", "d")), + Row(Row(null, 2), Row(value("4"), null))), + Row(metadata(Nil), value("null"), null), + null) + checkExpr(path, "to_json(v)", """{"b":1}""", """{"b":1,"d":null}""", + """{"a":1,"b":2,"c":3,"d":4}""", "null", null) + checkExpr(path, "variant_get(v, '$.b', 'string')", "1", "1", "2", null, null) + checkExpr(path, "variant_get(v, '$.d', 'string')", null, null, "4", null, null) + } + + testWithTempPath("array rebuild") { path => + writeRows(path, writeSchema(ArrayType(IntegerType)), + Row(metadata(Nil), null, Array(Row(null, 1), Row(null, 2), Row(value("3"), null))), + Row(metadata(Seq("a", "b")), null, Array( + Row(shreddedValue("""{"a": 1}""", Seq("a", "b")), null), + Row(shreddedValue("""{"b": 2}""", Seq("a", "b")), null))), + Row(metadata(Seq("a", "b")), value("""{"a": 1, "b": 2}"""), null)) + checkExpr(path, "to_json(v)", """[1,2,3]""", """[{"a":1},{"b":2}]""", """{"a":1,"b":2}""") + checkExpr(path, "variant_get(v, '$[2]', 'int')", 3, null, null) + checkExpr(path, "variant_get(v, '$[1].b', 'int')", null, 2, null) + checkExpr(path, "variant_get(v, '$.a', 'long')", null, null, 1L) + } + + testWithTempPath("malformed input") { path => + // Top-level variant must not be missing. + writeRows(path, writeSchema(IntegerType), Row(metadata(Nil), null, null)) + checkException(path, "v", "MALFORMED_VARIANT") + // Array-element variant must not be missing. + writeRows(path, writeSchema(ArrayType(IntegerType)), + Row(metadata(Nil), null, Array(Row(null, null)))) + checkException(path, "v", "MALFORMED_VARIANT") + // Shredded field must not be null. + writeRows(path, writeSchema(StructType.fromDDL("a int")), + Row(metadata(Seq("a")), null, Row(null))) + checkException(path, "v", "MALFORMED_VARIANT") + // `value` must not contain any shredded field. + writeRows(path, writeSchema(StructType.fromDDL("a int")), + Row(metadata(Seq("a")), value("""{"a": 1}"""), Row(Row(null, null)))) + checkException(path, "v", "MALFORMED_VARIANT") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala index 5d59a3e0f8256..f196795989025 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ +import org.apache.spark.types.variant.VariantUtil._ import org.apache.spark.unsafe.types.{UTF8String, VariantVal} import org.apache.spark.util.ArrayImplicits._ @@ -117,7 +118,8 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval rand.nextBytes(value) val metadata = new Array[Byte](rand.nextInt(50)) rand.nextBytes(metadata) - new VariantVal(value, metadata) + // Generate a valid metadata, otherwise the shredded reader will fail. + new VariantVal(value, Array[Byte](VERSION, 0, 0) ++ metadata) } } @@ -151,7 +153,8 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval val metadata = new Array[Byte](rand.nextInt(50)) rand.nextBytes(metadata) val numElements = 3 // rand.nextInt(10) - Seq.fill(numElements)(new VariantVal(value, metadata)) + // Generate a valid metadata, otherwise the shredded reader will fail. + Seq.fill(numElements)(new VariantVal(value, Array[Byte](VERSION, 0, 0) ++ metadata)) } } @@ -299,7 +302,9 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval df.write.parquet(file) val schema = StructType(Seq(StructField("v", VariantType))) val result = spark.read.schema(schema).parquet(file).selectExpr("to_json(v)") - val e = intercept[org.apache.spark.SparkException](result.collect()) + val e = withSQLConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED.key -> "false") { + intercept[org.apache.spark.SparkException](result.collect()) + } checkError( exception = e.getCause.asInstanceOf[AnalysisException], condition = condition, From 2093bae154e062d1845083c14c9787f5b65f00b1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 10 Dec 2024 14:54:02 -0800 Subject: [PATCH 144/438] [SPARK-50536][CORE] Log downloaded archive file sizes in `SparkContext` and `Executor` ### What changes were proposed in this pull request? This PR aims to log downloaded archive file sizes in `SparkContext` and `Executor`. ### Why are the changes needed? To improve the UX by showing the file size intuitively in order to help check file health. ### Does this PR introduce _any_ user-facing change? No, this is a log message. ### How was this patch tested? Manual check with the Apache Spark test jar file. - https://github.com/apache/spark/raw/refs/heads/master/data/artifact-tests/smallJar.jar ``` $ ls -al data/artifact-tests/smallJar.jar -rwxr-xr-x 1 dongjoon staff 787 Dec 3 09:06 data/artifact-tests/smallJar.jar ``` Run the spark-shell and grep `(787 bytes)` from the log message like the following. ``` $ bin/spark-shell --archives https://github.com/apache/spark/raw/refs/heads/master/data/artifact-tests/smallJar.jar -c spark.log.level=DEBUG 2>&1 | grep '787 bytes' 24/12/10 10:05:09 INFO SparkContext: Unpacking an archive https://github.com/apache/spark/raw/refs/heads/master/data/artifact-tests/smallJar.jar (787 bytes) from /private/var/folders/c8/9r5k3d7n19g2m72c86qm08580000gn/T/spark-28ceb587-8376-403e-b364-e675529aeef0/smallJar.jar to /private/var/folders/c8/9r5k3d7n19g2m72c86qm08580000gn/T/spark-d97fc8fc-358e-4044-a904-dfc6823d742d/userFiles-89ff1008-a0d7-4c83-bd9f-9c4e2e830645/smallJar.jar 24/12/10 10:05:09 INFO Executor: Unpacking an archive https://github.com/apache/spark/raw/refs/heads/master/data/artifact-tests/smallJar.jar (787 bytes) from /private/var/folders/c8/9r5k3d7n19g2m72c86qm08580000gn/T/spark-b9020649-cb2b-476b-89f5-5d502f1e0ba8/smallJar.jar to /private/var/folders/c8/9r5k3d7n19g2m72c86qm08580000gn/T/spark-d97fc8fc-358e-4044-a904-dfc6823d742d/userFiles-89ff1008-a0d7-4c83-bd9f-9c4e2e830645/smallJar.jar ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49132 from dongjoon-hyun/SPARK-50536. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + core/src/main/scala/org/apache/spark/executor/Executor.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 042179d86c31a..20180e142f57f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1878,6 +1878,7 @@ class SparkContext(config: SparkConf) extends Logging { if (uri.getFragment != null) uri.getFragment else source.getName) logInfo( log"Unpacking an archive ${MDC(LogKeys.PATH, path)}" + + log" (${MDC(LogKeys.BYTE_SIZE, source.length)} bytes)" + log" from ${MDC(LogKeys.SOURCE_PATH, source.getAbsolutePath)}" + log" to ${MDC(LogKeys.DESTINATION_PATH, dest.getAbsolutePath)}") Utils.deleteRecursively(dest) 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 c299f38526aeb..f1087b695a7da 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -1210,6 +1210,7 @@ private[spark] class Executor( if (sourceURI.getFragment != null) sourceURI.getFragment else source.getName) logInfo( log"Unpacking an archive ${LogMDC(ARCHIVE_NAME, name)}" + + log" (${LogMDC(BYTE_SIZE, source.length)} bytes)" + log" from ${LogMDC(SOURCE_PATH, source.getAbsolutePath)}" + log" to ${LogMDC(DESTINATION_PATH, dest.getAbsolutePath)}") Utils.deleteRecursively(dest) From f903efb9e49aa80a948abc0e08dbbc889a9d99dc Mon Sep 17 00:00:00 2001 From: Alex Khakhlyuk Date: Wed, 11 Dec 2024 08:58:03 +0900 Subject: [PATCH 145/438] [SPARK-50537][CONNECT][PYTHON] Fix compression option being overwritten in `df.write.parquet` ### What changes were proposed in this pull request? There is a small bug in Spark Connect's DataFrameWriter. ``` df.write.option("compression", "gzip").parquet(path) ``` When this code is used, the specified compression option "gzip" gets overwritten by None. This happens because `parquet()` function has a default `compression=None` parameter which is used to set the compression option directly `with self.option("compression", compression)`. The Spark Connect server then receives a request without a specified compression option and instead uses the default "snappy" compression. The fix is to use the `self._set_opts(compression=compression)`. This method of setting options is used by most other DataFrameWriter APIs. ### Why are the changes needed? Bug described above and repro is provided. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New unit test. Ran `./python/run-tests --python-executables=python3.12 --modules=pyspark-connect -p 16` ### Was this patch authored or co-authored using generative AI tooling? No Closes #49133 from khakhlyuk/connect-parquet-compression-fix. Authored-by: Alex Khakhlyuk Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/readwriter.py | 2 +- .../sql/tests/connect/test_connect_readwriter.py | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/readwriter.py b/python/pyspark/sql/connect/readwriter.py index aeb0f98d71076..6cc38aca4fc4c 100644 --- a/python/pyspark/sql/connect/readwriter.py +++ b/python/pyspark/sql/connect/readwriter.py @@ -751,7 +751,7 @@ def parquet( self.mode(mode) if partitionBy is not None: self.partitionBy(partitionBy) - self.option("compression", compression) + self._set_opts(compression=compression) self.format("parquet").save(path) parquet.__doc__ = PySparkDataFrameWriter.parquet.__doc__ diff --git a/python/pyspark/sql/tests/connect/test_connect_readwriter.py b/python/pyspark/sql/tests/connect/test_connect_readwriter.py index db1e94cb6863e..06266b86de3ff 100644 --- a/python/pyspark/sql/tests/connect/test_connect_readwriter.py +++ b/python/pyspark/sql/tests/connect/test_connect_readwriter.py @@ -146,6 +146,16 @@ def test_parquet(self): self.connect.read.parquet(d).toPandas(), self.spark.read.parquet(d).toPandas() ) + def test_parquet_compression_option(self): + # SPARK-50537: Fix compression option being overwritten in df.write.parquet + with tempfile.TemporaryDirectory(prefix="test_parquet") as d: + self.connect.range(10).write.mode("overwrite").option("compression", "gzip").parquet(d) + self.assertTrue(any(file.endswith(".gz.parquet") for file in os.listdir(d))) + # Read the Parquet file as a DataFrame. + self.assert_eq( + self.connect.read.parquet(d).toPandas(), self.spark.read.parquet(d).toPandas() + ) + def test_text(self): # SPARK-41849: Implement DataFrameReader.text with tempfile.TemporaryDirectory(prefix="test_text") as d: From 6cdc96f7399ea7c47d0c0c0ec221558d54d66c53 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 11 Dec 2024 09:16:03 +0900 Subject: [PATCH 146/438] [SPARK-50517][PYTHON][TESTS][FOLLOW-UP] Add refactored package into pure Python test ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/49104 that adds refactored package into pure Python test ### Why are the changes needed? In order to fix the pure Python build https://github.com/apache/spark/actions/runs/12262937305/job/34213372670 ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Will monitor the build. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49137 from HyukjinKwon/SPARK-50517-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/packaging/connect/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index cf8ca7d3dbd84..89c9c205b5bfd 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -73,6 +73,7 @@ "pyspark.resource.tests", "pyspark.sql.tests", "pyspark.sql.tests.connect", + "pyspark.sql.tests.connect.arrow", "pyspark.sql.tests.connect.streaming", "pyspark.sql.tests.connect.client", "pyspark.sql.tests.connect.pandas", From 58c77bab93173969e37fd5dd4619f514816e1196 Mon Sep 17 00:00:00 2001 From: Zhihong Yu Date: Wed, 11 Dec 2024 09:19:40 +0900 Subject: [PATCH 147/438] [SPARK-49349][SQL][FOLLOWUP] Rename isContainsUnsupportedLCA function ### What changes were proposed in this pull request? As follow-up to https://github.com/apache/spark/pull/48915, this PR renames `isContainsUnsupportedLCA` function ### Why are the changes needed? The renamed function is easier to understand. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No change in functionality. Existing tests suffice. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49134 from tedyu/cont-unsup. Authored-by: Zhihong Yu Signed-off-by: Hyukjin Kwon --- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index aaae763991f10..8f4431cb1ac11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -192,7 +192,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB ) } - private def isContainsUnsupportedLCA(e: Expression, operator: LogicalPlan): Boolean = { + private def containsUnsupportedLCA(e: Expression, operator: LogicalPlan): Boolean = { e.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE) && operator.expressions.exists { case a: Alias if e.collect { case l: LateralColumnAliasReference => l.nameParts.head }.contains(a.name) => @@ -369,7 +369,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case GetMapValue(map, key: Attribute) if isMapWithStringKey(map) && !key.resolved => failUnresolvedAttribute(operator, key, "UNRESOLVED_MAP_KEY") - case e: Expression if isContainsUnsupportedLCA(e, operator) => + case e: Expression if containsUnsupportedLCA(e, operator) => val lcaRefNames = e.collect { case lcaRef: LateralColumnAliasReference => lcaRef.name }.distinct failAnalysis( From 0f27d73b6c8b4139220fd0da33d5e8f0973283be Mon Sep 17 00:00:00 2001 From: Jiashen Cao Date: Tue, 10 Dec 2024 20:03:04 -0800 Subject: [PATCH 148/438] [SPARK-49565][SQL] Add SQL pipe syntax for the FROM operator ### What changes were proposed in this pull request? This PR adds SQL pipe syntax support for the FROM operator. Note: this PR includes all content in https://github.com/apache/spark/pull/48724 and also fixes remaining tests. For example: ``` CREATE TABLE t(x INT, y STRING) USING CSV; INSERT INTO t VALUES (0, 'abc'), (1, 'def'); CREATE TABLE other(a INT, b INT) USING JSON; INSERT INTO other VALUES (1, 1), (1, 2), (2, 4); FROM t |> SELECT 1 AS X; ``` ### Why are the changes needed? This allows users to use FROM in the pipe syntax format. ### Does this PR introduce any user-facing change? Yes, as indicated in the example. ### How was this patch tested? * Unit tests in `SparkSqlParserSuite` and `PlanParserSuite`. * Golden file based end to end tests in `pipe-operators.sql`. Was this patch authored or co-authored using generative AI tooling? No Closes #49120 from dtenedor/pipe-syntax-from-clause. Lead-authored-by: Jiashen Cao Co-authored-by: Daniel Tenedorio Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 5 +- .../sql/catalyst/parser/PlanParserSuite.scala | 16 +- .../analyzer-results/pipe-operators.sql.out | 146 ++++++++++++++++-- .../sql-tests/inputs/pipe-operators.sql | 67 +++++++- .../sql-tests/results/pipe-operators.sql.out | 143 ++++++++++++++++- .../sql/execution/SparkSqlParserSuite.scala | 3 + 7 files changed, 339 insertions(+), 43 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index a0f447dba798e..8ef7ab90c6ff7 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -648,7 +648,7 @@ sortItem ; fromStatement - : fromClause fromStatementBody+ + : fromClause fromStatementBody* ; fromStatementBody diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index fad4fcefc1d1d..47139810528df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -548,7 +548,10 @@ class AstBuilder extends DataTypeAstBuilder optionalMap(body.queryOrganization)(withQueryResultClauses(_, _, forPipeOperators = false)) } // If there are multiple SELECT just UNION them together into one query. - if (selects.length == 1) { + if (selects.length == 0) { + // This is a "FROM " clause with no other syntax. + from + } else if (selects.length == 1) { selects.head } else { Union(selects.toSeq) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index c556a92373954..9e5555c4c6c0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -323,19 +323,9 @@ class PlanParserSuite extends AnalysisTest { assertEqual( "from db.a select b, c where d < 1", table("db", "a").where($"d" < 1).select($"b", $"c")) assertEqual("from a select distinct b, c", Distinct(table("a").select($"b", $"c"))) - - // Weird "FROM table" queries, should be invalid anyway - val sql1 = "from a" - checkError( - exception = parseException(sql1), - condition = "PARSE_SYNTAX_ERROR", - parameters = Map("error" -> "end of input", "hint" -> "")) - - val sql2 = "from (from a union all from b) c select *" - checkError( - exception = parseException(sql2), - condition = "PARSE_SYNTAX_ERROR", - parameters = Map("error" -> "'union'", "hint" -> "")) + assertEqual("from a", table("a")) + assertEqual("from (from a union all from b) c select *", + table("a").union(table("b")).subquery("c").select(star())) } test("multi select query") { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index 1e1ad90946f88..1121d8baf5dba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -265,6 +265,131 @@ CreateViewCommand `windowTestData`, select * from values +- LocalRelation [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x] +-- !query +from t +-- !query analysis +SubqueryAlias spark_catalog.default.t ++- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +-- !query analysis +SubqueryAlias spark_catalog.default.t ++- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +from t +|> select 1 as x +-- !query analysis +Project [pipeexpression(1, false, SELECT) AS x#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +from t as t_alias +|> select t_alias.x +-- !query analysis +Project [x#x] ++- SubqueryAlias t_alias + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +from t as t_alias +|> select t_alias.x as tx, t_alias.y as ty +|> where ty = 'def' +|> select tx +-- !query analysis +Project [tx#x] ++- Filter (ty#x = def) + +- SubqueryAlias __auto_generated_subquery_name + +- Project [pipeexpression(x#x, false, SELECT) AS tx#x, pipeexpression(y#x, false, SELECT) AS ty#x] + +- SubqueryAlias t_alias + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +from t, other +|> select t.x + other.a as z +-- !query analysis +Project [pipeexpression((x#x + a#x), false, SELECT) AS z#x] ++- Join Inner + :- SubqueryAlias spark_catalog.default.t + : +- Relation spark_catalog.default.t[x#x,y#x] csv + +- SubqueryAlias spark_catalog.default.other + +- Relation spark_catalog.default.other[a#x,b#x] json + + +-- !query +from t join other on (t.x = other.a) +|> select t.x + other.a as z +-- !query analysis +Project [pipeexpression((x#x + a#x), false, SELECT) AS z#x] ++- Join Inner, (x#x = a#x) + :- SubqueryAlias spark_catalog.default.t + : +- Relation spark_catalog.default.t[x#x,y#x] csv + +- SubqueryAlias spark_catalog.default.other + +- Relation spark_catalog.default.other[a#x,b#x] json + + +-- !query +from t lateral view explode(array(100, 101)) as ly +|> select t.x + ly as z +-- !query analysis +Project [pipeexpression((x#x + ly#x), false, SELECT) AS z#x] ++- Generate explode(array(100, 101)), false, as, [ly#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +from st +|> select col.i1 +-- !query analysis +Project [col#x.i1 AS i1#x] ++- SubqueryAlias spark_catalog.default.st + +- Relation spark_catalog.default.st[x#x,col#x] parquet + + +-- !query +from st as st_alias +|> select st_alias.col.i1 +-- !query analysis +Project [col#x.i1 AS i1#x] ++- SubqueryAlias st_alias + +- SubqueryAlias spark_catalog.default.st + +- Relation spark_catalog.default.st[x#x,col#x] parquet + + +-- !query +from values (0), (1) tab(col) +|> select col as x +-- !query analysis +Project [pipeexpression(col#x, false, SELECT) AS x#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +from t +|> from t +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'from'", + "hint" : "" + } +} + + -- !query table t |> select 1 as x @@ -3661,13 +3786,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query with customer_total_return as - (table store_returns + (from store_returns |> join date_dim |> where sr_returned_date_sk = d_date_sk and d_year = 2000 |> aggregate sum(sr_return_amt) as ctr_total_return group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) -table customer_total_return -|> as ctr1 +from customer_total_return ctr1 |> join store |> join customer |> where ctr1.ctr_total_return > @@ -3692,8 +3816,8 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 40, - "stopIndex" : 52, + "startIndex" : 39, + "stopIndex" : 51, "fragment" : "store_returns" } ] } @@ -4109,13 +4233,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query -table store_sales -|> as ss1 +from store_sales ss1 |> where ss_store_sk = 4 |> aggregate avg(ss_net_profit) rank_col group by ss_item_sk as item_sk |> where rank_col > 0.9 * ( - table store_sales + from store_sales |> where ss_store_sk = 4 and ss_addr_sk is null |> aggregate avg(ss_net_profit) rank_col @@ -4130,8 +4253,7 @@ table store_sales |> where rnk < 11 |> as asceding |> join ( - table store_sales - |> as ss1 + from store_sales ss1 |> where ss_store_sk = 4 |> aggregate avg(ss_net_profit) rank_col group by ss_item_sk as item_sk @@ -4170,8 +4292,8 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 7, - "stopIndex" : 17, + "startIndex" : 6, + "stopIndex" : 16, "fragment" : "store_sales" } ] } diff --git a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql index 924b42d9d3055..1299da3020d59 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql @@ -71,6 +71,60 @@ create temporary view windowTestData as select * from values (3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); +-- FROM operators: positive tests. +---------------------------------- + +-- FromClause alone. +from t; + +-- Table alone. +table t; + +-- Selecting from a constant. +from t +|> select 1 as x; + +-- Selecting using a table alias. +from t as t_alias +|> select t_alias.x; + +-- Selecting using a table alias. +from t as t_alias +|> select t_alias.x as tx, t_alias.y as ty +|> where ty = 'def' +|> select tx; + +-- Selecting from multiple relations. +from t, other +|> select t.x + other.a as z; + +-- Selecting from multiple relations with join. +from t join other on (t.x = other.a) +|> select t.x + other.a as z; + +-- Selecting from lateral view. +from t lateral view explode(array(100, 101)) as ly +|> select t.x + ly as z; + +-- Selecting struct fields. +from st +|> select col.i1; + +-- Selecting struct fields using a table alias. +from st as st_alias +|> select st_alias.col.i1; + +-- Selecting from a VALUES list. +from values (0), (1) tab(col) +|> select col as x; + +-- FROM operators: negative tests. +---------------------------------- + +-- It is not possible to use the FROM operator accepting an input relation. +from t +|> from t; + -- SELECT operators: positive tests. --------------------------------------- @@ -1158,13 +1212,12 @@ order by c_customer_id limit 100; with customer_total_return as - (table store_returns + (from store_returns |> join date_dim |> where sr_returned_date_sk = d_date_sk and d_year = 2000 |> aggregate sum(sr_return_amt) as ctr_total_return group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) -table customer_total_return -|> as ctr1 +from customer_total_return ctr1 |> join store |> join customer |> where ctr1.ctr_total_return > @@ -1466,13 +1519,12 @@ where asceding.rnk = descending.rnk order by asceding.rnk limit 100; -table store_sales -|> as ss1 +from store_sales ss1 |> where ss_store_sk = 4 |> aggregate avg(ss_net_profit) rank_col group by ss_item_sk as item_sk |> where rank_col > 0.9 * ( - table store_sales + from store_sales |> where ss_store_sk = 4 and ss_addr_sk is null |> aggregate avg(ss_net_profit) rank_col @@ -1487,8 +1539,7 @@ table store_sales |> where rnk < 11 |> as asceding |> join ( - table store_sales - |> as ss1 + from store_sales ss1 |> where ss_store_sk = 4 |> aggregate avg(ss_net_profit) rank_col group by ss_item_sk as item_sk diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index 570b61f388ead..cc603903712a6 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -232,6 +232,136 @@ struct<> +-- !query +from t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +from t +|> select 1 as x +-- !query schema +struct +-- !query output +1 +1 + + +-- !query +from t as t_alias +|> select t_alias.x +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +from t as t_alias +|> select t_alias.x as tx, t_alias.y as ty +|> where ty = 'def' +|> select tx +-- !query schema +struct +-- !query output +1 + + +-- !query +from t, other +|> select t.x + other.a as z +-- !query schema +struct +-- !query output +1 +1 +2 +2 +2 +3 + + +-- !query +from t join other on (t.x = other.a) +|> select t.x + other.a as z +-- !query schema +struct +-- !query output +2 +2 + + +-- !query +from t lateral view explode(array(100, 101)) as ly +|> select t.x + ly as z +-- !query schema +struct +-- !query output +100 +101 +101 +102 + + +-- !query +from st +|> select col.i1 +-- !query schema +struct +-- !query output +2 + + +-- !query +from st as st_alias +|> select st_alias.col.i1 +-- !query schema +struct +-- !query output +2 + + +-- !query +from values (0), (1) tab(col) +|> select col as x +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +from t +|> from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'from'", + "hint" : "" + } +} + + -- !query table t |> select 1 as x @@ -3344,13 +3474,12 @@ struct -- !query with customer_total_return as - (table store_returns + (from store_returns |> join date_dim |> where sr_returned_date_sk = d_date_sk and d_year = 2000 |> aggregate sum(sr_return_amt) as ctr_total_return group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) -table customer_total_return -|> as ctr1 +from customer_total_return ctr1 |> join store |> join customer |> where ctr1.ctr_total_return > @@ -3696,13 +3825,12 @@ struct -- !query -table store_sales -|> as ss1 +from store_sales ss1 |> where ss_store_sk = 4 |> aggregate avg(ss_net_profit) rank_col group by ss_item_sk as item_sk |> where rank_col > 0.9 * ( - table store_sales + from store_sales |> where ss_store_sk = 4 and ss_addr_sk is null |> aggregate avg(ss_net_profit) rank_col @@ -3717,8 +3845,7 @@ table store_sales |> where rnk < 11 |> as asceding |> join ( - table store_sales - |> as ss1 + from store_sales ss1 |> where ss_store_sk = 4 |> aggregate avg(ss_net_profit) rank_col group by ss_item_sk as item_sk diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 36a003883a771..e698c50e5631a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -901,6 +901,9 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y") checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y, X + 2 Z") checkPipeSelect("TABLE t |> EXTEND 1 AS z, 2 AS Z |> SET z = 1, Z = 2") + // FROM operators. + def checkPipeSelectFrom(query: String): Unit = check(query, Seq(PROJECT)) + checkPipeSelectFrom("FROM t |> SELECT 1 AS X") // Basic WHERE operators. def checkPipeWhere(query: String): Unit = check(query, Seq(FILTER)) checkPipeWhere("TABLE t |> WHERE X = 1") From 2f5728f3ea7511e568fc992924f92b7c5c56d1ab Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Wed, 11 Dec 2024 12:40:48 +0800 Subject: [PATCH 149/438] [SPARK-50543][BUILD] Fix log printing in `dev/check-protos.py` ### What changes were proposed in this pull request? This pr makes the following fixes to the log printing in `dev/check-protos.py`: - Removed extra `$` symbols from the log printing. - For the log message `"Generated files for pyspark-connect are out of sync! "`, change `connect` to `{module_name}`. ### Why are the changes needed? Fix log printing in `dev/check-protos.py` ### Does this PR introduce _any_ user-facing change? No, dev only ### How was this patch tested? Manually check the log printing ### Was this patch authored or co-authored using generative AI tooling? No Closes #49141 from LuciferYang/SPARK-50543. Authored-by: yangjie01 Signed-off-by: Ruifeng Zheng --- dev/check-protos.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/check-protos.py b/dev/check-protos.py index bfca8b27be21c..4ddd1f1058820 100755 --- a/dev/check-protos.py +++ b/dev/check-protos.py @@ -44,8 +44,8 @@ def run_cmd(cmd): def check_protos(module_name, cmp_path, proto_path): - print(f"Start checking the generated codes in pyspark-${module_name}.") - with tempfile.TemporaryDirectory(prefix=f"check_${module_name}__protos") as tmp: + print(f"Start checking the generated codes in pyspark-{module_name}.") + with tempfile.TemporaryDirectory(prefix=f"check_{module_name}__protos") as tmp: run_cmd(f"{SPARK_HOME}/dev/gen-protos.sh {module_name} {tmp}") result = filecmp.dircmp( f"{SPARK_HOME}/{cmp_path}", @@ -71,12 +71,12 @@ def check_protos(module_name, cmp_path, proto_path): success = False if success: - print(f"Finish checking the generated codes in pyspark-${module_name}: SUCCESS") + print(f"Finish checking the generated codes in pyspark-{module_name}: SUCCESS") else: fail( - "Generated files for pyspark-connect are out of sync! " - f"If you have touched files under ${proto_path}, " - f"please run ./dev/${module_name}-gen-protos.sh. " + f"Generated files for pyspark-{module_name} are out of sync! " + f"If you have touched files under {proto_path}, " + f"please run ./dev/{module_name}-gen-protos.sh. " "If you haven't touched any file above, please rebase your PR against main branch." ) From d268e0c4ac64b14c2ef9ec3c199faafa45f91c13 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 11 Dec 2024 13:37:58 +0800 Subject: [PATCH 150/438] [SPARK-46934][SQL][FOLLOWUP] Read/write roundtrip for struct type with special characters with HMS - a backward compatible approach ### What changes were proposed in this pull request? A backward-compatible approach for #45039 to make older versions of spark properly read struct-typed columns created by spark 4.x or later with special characters. Compared with #45039, only the datasource tables are supported now, as we have a special way to store hive incompatible schema to the table properties. This is a safe removal because we don't have any release to support that. ### Why are the changes needed? backward-compatibility improvement ### Does this PR introduce _any_ user-facing change? Users can store/read struct-typed columns with special characters. ### How was this patch tested? #### tests provided by SPARK-22431 ```scala DDLSuite.scala: test("SPARK-22431: table with nested type col with special char") DDLSuite.scala: test("SPARK-22431: view with nested type") HiveDDLSuite.scala: test("SPARK-22431: table with nested type") { HiveDDLSuite.scala: test("SPARK-22431: view with nested type") { HiveDDLSuite.scala: test("SPARK-22431: alter table tests with nested types") { ``` #### tests provided by the previous PR towards SPARK-46934 ```scala HiveMetastoreCatalogSuite.scala: test("SPARK-46934: HMS columns cannot handle quoted columns") HiveMetastoreCatalogSuite.scala: test("SPARK-46934: Handle special characters in struct types") { HiveMetastoreCatalogSuite.scala: test("SPARK-46934: Handle special characters in struct types with CTAS") { HiveMetastoreCatalogSuite.scala: test("SPARK-46934: Handle special characters in struct types with hive DDL") { HiveDDLSuite.scala: test("SPARK-46934: quote element name before parsing struct") { HiveDDLSuite.scala: test("SPARK-46934: alter table tests with nested types") { ``` #### manually backward compatibility test 1. create a tarball with the current revison 2. cd dist 3. using spark-sql to mock data ``` spark-sql (default)> CREATE TABLE t AS SELECT named_struct('a.b.b', array('a'), 'a b c', map(1, 'a')) AS `a.b`; ``` 4. copy metadata to 3.5.3 release ``` cp -r ~/spark/dist/metastore_db . ``` 5. Fix derby version restrictions ``` rm jars/derby-10.14.2.0.jar cp -r ~/spark/dist/jars/derby-10.16.1.1.jar ./jars ``` 6. read data ``` spark-sql (default)> select version(); 6.5.3 32232e9ed33bb16b93ad58cfde8b82e0f07c0970 Time taken: 0.103 seconds, Fetched 1 row(s) spark-sql (default)> select * from t; {"a.b.b":["a"],"a b c":{1:"a"}} Time taken: 0.09 seconds, Fetched 1 row(s) spark-sql (default)> desc formatted t; a.b struct,a b c:map> # Detailed Table Information Catalog spark_catalog Database default Table t Owner hzyaoqin Created Time Wed Nov 27 17:40:53 CST 2024 Last Access UNKNOWN Created By Spark 4.0.0-SNAPSHOT Type MANAGED Provider parquet Statistics 1245 bytes Location file:/Users/hzyaoqin/spark/dist/spark-warehouse/t Serde Library org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat org.apache.hadoop.mapred.SequenceFileInputFormat OutputFormat org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat Time taken: 0.054 seconds, Fetched 17 row(s) ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48986 from yaooqinn/SPARK-46934-F. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../org/apache/spark/sql/hive/HiveUtils.scala | 9 +++ .../sql/hive/client/HiveClientImpl.scala | 53 ++++++++++--- .../sql/hive/HiveMetastoreCatalogSuite.scala | 41 +++++++++- .../sql/hive/execution/HiveDDLSuite.scala | 77 +++++++++---------- 4 files changed, 128 insertions(+), 52 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 478f486eeb213..776f7045f5965 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -221,6 +221,15 @@ private[spark] object HiveUtils extends Logging { .booleanConf .createWithDefault(true) + val QUOTE_HIVE_STRUCT_FIELD_NAME = + buildConf("spark.sql.hive.quoteHiveStructFieldName") + .doc("When true, for a column defined in struct type, when it contains special characters " + + "in the field name, Spark will quote it for verification. E.g. struct" + + " is read as struct<`x`:int,`y.z`:int> for verification.") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + /** * The version of the hive client that will be used to communicate with the metastore. Note that * this does not necessarily need to be the same version of Hive that is used internally by diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index ba03b7fe3cee1..fd4d3220f367d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -56,12 +56,13 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} -import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, QuotingUtils, StringConcat} import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_SCHEMA +import org.apache.spark.sql.hive.HiveUtils.QUOTE_HIVE_STRUCT_FIELD_NAME import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.{CircularBuffer, Utils} @@ -580,7 +581,6 @@ private[hive] class HiveClientImpl( } override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState { - verifyColumnDataType(table.dataSchema) shim.createTable(client, toHiveTable(table, Some(userName)), ignoreIfExists) } @@ -600,7 +600,6 @@ private[hive] class HiveClientImpl( // these properties are still available to the others that share the same Hive metastore. // If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect // these user-specified values. - verifyColumnDataType(table.dataSchema) val hiveTable = toHiveTable( table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName)) // Do not use `table.qualifiedName` here because this may be a rename @@ -624,7 +623,6 @@ private[hive] class HiveClientImpl( newDataSchema: StructType, schemaProps: Map[String, String]): Unit = withHiveState { val oldTable = shim.getTable(client, dbName, tableName) - verifyColumnDataType(newDataSchema) val hiveCols = newDataSchema.map(toHiveColumn) oldTable.setFields(hiveCols.asJava) @@ -1092,13 +1090,45 @@ private[hive] object HiveClientImpl extends Logging { // When reading data in parquet, orc, or avro file format with string type for char, // the tailing spaces may lost if we are not going to pad it. val typeString = if (SQLConf.get.charVarcharAsString) { - c.dataType.catalogString + catalogString(c.dataType) } else { - CharVarcharUtils.getRawTypeString(c.metadata).getOrElse(c.dataType.catalogString) + CharVarcharUtils.getRawTypeString(c.metadata).getOrElse(catalogString(c.dataType)) } new FieldSchema(c.name, typeString, c.getComment().orNull) } + /** + * This a a variant of `DataType.catalogString` that does the same thing in general but + * it will not quote the field names in the struct type. HMS API uses unquoted field names + * to store the schema of a struct type. This is fine if we in the write path, we might encounter + * issues in the read path to parse the unquoted schema strings in the Spark SQL parser. You can + * see the tricks we play in the `getSparkSQLDataType` method to handle this. To avoid the + * flakiness of those tricks, we quote the field names, make them unrecognized by HMS API, and + * then store them in custom spark properties in a fallback way. + * + * And the reason we don't add quoting in `DataType.catalogString` directly is that we don't + * want to break the compatibility of the existing query output schema. + */ + def catalogString(dataType: DataType): String = dataType match { + case ArrayType(et, _) => s"array<${catalogString(et)}>" + case MapType(k, v, _) => s"map<${catalogString(k)},${catalogString(v)}>" + case StructType(fields) => + val stringConcat = new StringConcat() + val len = fields.length + stringConcat.append("struct<") + var i = 0 + while (i < len) { + val name = QuotingUtils.quoteIfNeeded(fields(i).name) + stringConcat.append(s"$name:${catalogString(fields(i).dataType)}") + i += 1 + if (i < len) stringConcat.append(",") + } + stringConcat.append(">") + stringConcat.toString + case udt: UserDefinedType[_] => catalogString(udt.sqlType) + case _ => dataType.catalogString + } + /** Get the Spark SQL native DataType from Hive's FieldSchema. */ private def getSparkSQLDataType(hc: FieldSchema): DataType = { // For struct types, Hive metastore API uses unquoted element names, so does the spark catalyst @@ -1111,7 +1141,12 @@ private[hive] object HiveClientImpl extends Logging { // struct -> struct<`x`:int,`y.z`:int> // array> -> array> // map> -> map> - val typeStr = hc.getType.replaceAll("(?<=struct<|,)([^,<:]+)(?=:)", "`$1`") + val typeStr = if (SQLConf.get.getConf(QUOTE_HIVE_STRUCT_FIELD_NAME) && + hc.getType.indexOf('`') < 0) { // This a defensive code for possible changes in HMS + hc.getType.replaceAll("(?<=struct<|,)([^,<:]+)(?=:)", "`$1`") + } else { + hc.getType + } try { CatalystSqlParser.parseDataType(typeStr) } catch { @@ -1130,10 +1165,6 @@ private[hive] object HiveClientImpl extends Logging { Option(hc.getComment).map(field.withComment).getOrElse(field) } - private def verifyColumnDataType(schema: StructType): Unit = { - schema.foreach(col => getSparkSQLDataType(toHiveColumn(col))) - } - private def toInputFormat(name: String) = Utils.classForName[org.apache.hadoop.mapred.InputFormat[_, _]](name) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 72c570d1f9097..bb274ce0578b1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.hive +import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.hive.HiveUtils.QUOTE_HIVE_STRUCT_FIELD_NAME import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} @@ -129,6 +131,26 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton with SQLTestUtils { assert(schema == expectedSchema) } } + + test("SPARK-46934: HMS columns cannot handle quoted columns") { + withTable("t") { + val schema = + "a struct<" + + "`a.a`:int," + + "`a.b`:struct<" + + " `a b b`:array," + + " `a b c`:map" + + " >" + + ">" + val e = intercept[AnalysisException](sql("CREATE TABLE t(" + schema + ") USING hive")) + checkError( + exception = e, + condition = "_LEGACY_ERROR_TEMP_3065", + parameters = Map( + "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", + "msg" -> e.getCause.getMessage)) + } + } } class DataSourceWithHiveMetastoreCatalogSuite @@ -441,7 +463,7 @@ class DataSourceWithHiveMetastoreCatalogSuite } test("SPARK-46934: Handle special characters in struct types with hive DDL") { - withTable("t") { + try { val schema = "a struct<" + "`a.a`:int," + @@ -451,7 +473,22 @@ class DataSourceWithHiveMetastoreCatalogSuite " >" + ">" sparkSession.metadataHive.runSqlHive(s"CREATE TABLE t($schema)") - assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema(schema)) + withSQLConf(QUOTE_HIVE_STRUCT_FIELD_NAME.key -> "true") { + assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema(schema)) + } + + withSQLConf(QUOTE_HIVE_STRUCT_FIELD_NAME.key -> "false") { + checkError(exception = + intercept[SparkException](spark.table("t")).getCause.asInstanceOf[SparkException], + condition = "CANNOT_RECOGNIZE_HIVE_TYPE", + parameters = Map( + "fieldType" -> + "\"STRUCT,A B C:MAP>>\"", + "fieldName" -> "`a`" + )) + } + } finally { + sparkSession.metadataHive.runSqlHive("DROP TABLE IF EXISTS t") } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 53a65e195e3f0..4efc159a3ed44 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER -import org.apache.spark.sql.errors.DataTypeErrors.toSQLType import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, ParquetFooterReader} @@ -167,17 +166,23 @@ class HiveDDLSuite } test("SPARK-46934: quote element name before parsing struct") { - withTable("t") { - sql("CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q") - assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema( - "q STRUCT<`$a`: STRING, b: INT>")) - } + val e = intercept[AnalysisException]( + sql("CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q")) + checkError( + exception = e, + condition = "_LEGACY_ERROR_TEMP_3065", + parameters = Map( + "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", + "msg" -> e.getCause.getMessage)) - withTable("t") { - sql("CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT) USING hive") - assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema( - "q STRUCT<`$a`:INT, col2:STRING>, i1 INT")) - } + val e1 = intercept[AnalysisException]( + sql("CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT) USING hive")) + checkError( + exception = e1, + condition = "_LEGACY_ERROR_TEMP_3065", + parameters = Map( + "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", + "msg" -> e1.getCause.getMessage)) withView("v") { spark.sql("CREATE VIEW v AS SELECT STRUCT('a' AS `a`, 1 AS b) q") @@ -233,15 +238,29 @@ class HiveDDLSuite } } - test("SPARK-46934: alter table tests with nested types") { + test("SPARK-46934: alter datasource table tests with nested types") { withTable("t1") { - sql("CREATE TABLE t1 (q STRUCT, i1 INT) USING hive") + sql("CREATE TABLE t1 (q STRUCT, i1 INT) USING parquet") sql("ALTER TABLE t1 ADD COLUMNS (newcol1 STRUCT<`$col1`:STRING, col2:Int>)") assert(spark.table("t1").schema == CatalystSqlParser.parseTableSchema( "q STRUCT, i1 INT,newcol1 STRUCT<`$col1`:STRING, col2:Int>")) } } + test("SPARK-46934: alter hive table tests with nested types") { + withTable("t1") { + sql("CREATE TABLE t1 (q STRUCT, i1 INT) USING hive") + val e = intercept[AnalysisException]( + sql("ALTER TABLE t1 ADD COLUMNS (newcol1 STRUCT<`$col1`:STRING, col2:Int>)")) + checkError( + exception = e, + condition = "_LEGACY_ERROR_TEMP_3065", + parameters = Map( + "clazz" -> "java.lang.IllegalArgumentException", + "msg" -> e.getCause.getMessage)) + } + } + test("SPARK-26630: table with old input format and without partitioned will use HadoopRDD") { withTable("table_old", "table_ctas_old") { sql( @@ -2849,38 +2868,18 @@ class HiveDDLSuite } test("SPARK-47101 checks if nested column names do not include invalid characters") { - // delimiter characters - Seq(",", ":").foreach { c => + Seq(",", ":", ";", "^", "\\", "/", "%").foreach { c => val typ = s"array>" - // The regex is from HiveClientImpl.getSparkSQLDataType, please keep them in sync. - val replaced = typ.replaceAll("`", "").replaceAll("(?<=struct<|,)([^,<:]+)(?=:)", "`$1`") withTable("t") { - checkError( - exception = intercept[SparkException] { + val e = intercept[AnalysisException] { sql(s"CREATE TABLE t (a $typ) USING hive") - }, - condition = "CANNOT_RECOGNIZE_HIVE_TYPE", - parameters = Map( - "fieldType" -> toSQLType(replaced), - "fieldName" -> "`a`") - ) - } - } - // other special characters - Seq(";", "^", "\\", "/", "%").foreach { c => - val typ = s"array>" - val replaced = typ.replaceAll("`", "") - val msg = s"java.lang.IllegalArgumentException: Error: : expected at the position " + - s"16 of '$replaced' but '$c' is found." - withTable("t") { + } checkError( - exception = intercept[AnalysisException] { - sql(s"CREATE TABLE t (a $typ) USING hive") - }, + exception = e, condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( - "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", - "msg" -> msg) + "clazz" -> e.getCause.getClass.getName, + "msg" -> e.getCause.getMessage) ) } } From 2fa72d251f6f13bf50941c32de28f349bf420cbf Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 11 Dec 2024 18:35:52 +0800 Subject: [PATCH 151/438] [SPARK-50542][INFRA] Add a separate docker file for python 3.13 daily build ### What changes were proposed in this pull request? Add a separate docker file for python 3.13 daily build ### Why are the changes needed? to isolate the testing environments ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? PR builder with env ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-313", "PYTHON_TO_TEST": "python3.13"}' ``` https://github.com/zhengruifeng/spark/runs/34188570406 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49140 from zhengruifeng/py_image_313. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../workflows/build_infra_images_cache.yml | 14 ++++ .github/workflows/build_python_3.13.yml | 1 + dev/spark-test-image/python-313/Dockerfile | 79 +++++++++++++++++++ 3 files changed, 94 insertions(+) create mode 100644 dev/spark-test-image/python-313/Dockerfile diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index b4e7a2cbd0b37..adccc64261f55 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -33,6 +33,7 @@ on: - 'dev/spark-test-image/python-309/Dockerfile' - 'dev/spark-test-image/python-310/Dockerfile' - 'dev/spark-test-image/python-312/Dockerfile' + - 'dev/spark-test-image/python-313/Dockerfile' - '.github/workflows/build_infra_images_cache.yml' # Create infra image when cutting down branches/tags create: @@ -144,3 +145,16 @@ jobs: - name: Image digest (PySpark with Python 3.12) if: hashFiles('dev/spark-test-image/python-312/Dockerfile') != '' run: echo ${{ steps.docker_build_pyspark_python_312.outputs.digest }} + - name: Build and push (PySpark with Python 3.13) + if: hashFiles('dev/spark-test-image/python-313/Dockerfile') != '' + id: docker_build_pyspark_python_313 + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/python-313/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-313-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-313-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-313-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark with Python 3.13) + if: hashFiles('dev/spark-test-image/python-313/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_python_313.outputs.digest }} diff --git a/.github/workflows/build_python_3.13.yml b/.github/workflows/build_python_3.13.yml index 6f67cf383584f..5c62a431e3f0d 100644 --- a/.github/workflows/build_python_3.13.yml +++ b/.github/workflows/build_python_3.13.yml @@ -36,6 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-313", "PYTHON_TO_TEST": "python3.13" } jobs: >- diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile new file mode 100644 index 0000000000000..86232da71097c --- /dev/null +++ b/dev/spark-test-image/python-313/Dockerfile @@ -0,0 +1,79 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark with Python 3.13" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE 20241210 + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pkg-config \ + qpdf \ + tzdata \ + software-properties-common \ + wget \ + zlib1g-dev + +# Install Python 3.13 +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt-get update && apt-get install -y \ + python3.13 \ + && apt-get autoremove --purge -y \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" + + +# Install Python 3.13 packages +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 +# TODO(SPARK-49862) Add BASIC_PIP_PKGS and CONNECT_PIP_PKGS to Python 3.13 image when it supports Python 3.13 +RUN python3.13 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.13 -m pip install numpy>=2.1 pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy coverage matplotlib openpyxl grpcio==1.67.0 grpcio-status==1.67.0 lxml jinja2 && \ + python3.13 -m pip cache purge From 9394b35ed7bb57a343ab6038dc1c1345db104604 Mon Sep 17 00:00:00 2001 From: cashmand Date: Wed, 11 Dec 2024 08:12:27 -0800 Subject: [PATCH 152/438] [SPARK-48898][SQL] Set nullability correctly in the Variant schema ### What changes were proposed in this pull request? The `variantShreddingSchema` method converts a human-readable schema for Variant to one that's a valid shredding schema. According to the shredding schema in https://github.com/apache/parquet-format/pull/461, each shredded field in an object should be a required group - i.e. a non-nullable struct. This PR fixes the `variantShreddingSchema` to mark that struct as non-nullable. ### Why are the changes needed? If we use `variantShreddingSchema` to construct a schema for Parquet, the schema would be technically non-conformant with the spec by setting the group as optional. I don't think this should really matter to readers, but it would waste a bit of space in the Parquet file by adding an extra definition level. ### Does this PR introduce _any_ user-facing change? No, this code is not used yet. ### How was this patch tested? Added a test to do some minimal validation of the `variantShreddingSchema` function. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49118 from cashmand/SPARK-48898-nullability. Authored-by: cashmand Signed-off-by: Wenchen Fan --- .../parquet/SparkShreddingUtils.scala | 5 +++- .../sql/VariantWriteShreddingSuite.scala | 30 +++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index 41244e20c369f..507633abfb285 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -81,8 +81,11 @@ case object SparkShreddingUtils { StructField(TypedValueFieldName, arrayShreddingSchema, nullable = true) ) case StructType(fields) => + // The field name level is always non-nullable: Variant null values are represented in the + // "value" columna as "00", and missing values are represented by setting both "value" and + // "typed_value" to null. val objectShreddingSchema = StructType(fields.map(f => - f.copy(dataType = variantShreddingSchema(f.dataType, false)))) + f.copy(dataType = variantShreddingSchema(f.dataType, false), nullable = false))) Seq( StructField(VariantValueFieldName, BinaryType, nullable = true), StructField(TypedValueFieldName, objectShreddingSchema, nullable = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala index a62c6e4462464..9022d8cfdca49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala @@ -67,6 +67,36 @@ class VariantWriteShreddingSuite extends SparkFunSuite with ExpressionEvalHelper private val emptyMetadata: Array[Byte] = parseJson("null").getMetadata + test("variantShreddingSchema") { + // Validate the schema produced by SparkShreddingUtils.variantShreddingSchema for a few simple + // cases. + // metadata is always non-nullable. + assert(SparkShreddingUtils.variantShreddingSchema(IntegerType) == + StructType(Seq( + StructField("metadata", BinaryType, nullable = false), + StructField("value", BinaryType, nullable = true), + StructField("typed_value", IntegerType, nullable = true)))) + + val fieldA = StructType(Seq( + StructField("value", BinaryType, nullable = true), + StructField("typed_value", TimestampNTZType, nullable = true))) + val arrayType = ArrayType(StructType(Seq( + StructField("value", BinaryType, nullable = true), + StructField("typed_value", StringType, nullable = true)))) + val fieldB = StructType(Seq( + StructField("value", BinaryType, nullable = true), + StructField("typed_value", arrayType, nullable = true))) + val objectType = StructType(Seq( + StructField("a", fieldA, nullable = false), + StructField("b", fieldB, nullable = false))) + val structSchema = DataType.fromDDL("a timestamp_ntz, b array") + assert(SparkShreddingUtils.variantShreddingSchema(structSchema) == + StructType(Seq( + StructField("metadata", BinaryType, nullable = false), + StructField("value", BinaryType, nullable = true), + StructField("typed_value", objectType, nullable = true)))) + } + test("shredding as fixed numeric types") { /* Cast integer to any wider numeric type. */ testWithSchema("1", IntegerType, Row(emptyMetadata, null, 1)) From 3bb9a72c8691786584eb2856ff6e3571f0118283 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 11 Dec 2024 08:21:07 -0800 Subject: [PATCH 153/438] [SPARK-50545][CORE][SQL] `AccessControlException` should be thrown even if `ignoreCorruptFiles` is enabled ### What changes were proposed in this pull request? `AccessControlException` extends `IOException` but we should not treat it as a data corruption issue. This is similar to SPARK-50483 which handles `BlockMissingException` in the same way. ``` 2024-12-11 06:29:05 WARN HadoopRDD: Skipped the rest content in the corrupted file: hdfs://hadoop-master1.orb.local:8020/warehouse/region/part-00000-2dc8a6f6-8cea-4652-8ba1-762c1b65e2b4-c000:192+192 org.apache.hadoop.security.AccessControlException: Permission denied: user=hive, access=READ, inode="/warehouse/region/part-00000-2dc8a6f6-8cea-4652-8ba1-762c1b65e2b4-c000":kyuubi.hadoop:hadoop:-rw------- at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:506) ``` image ### Why are the changes needed? Avoid data issue if `ignoreCorruptFiles` is enabled when `AccessControlException` occurred. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Manual test. Task fails with `org.apache.hadoop.security.AccessControlException` even with `spark.sql.files.ignoreCorruptFiles=true` and `spark.files.ignoreCorruptFiles=true` image ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49143 from pan3793/SPARK-50545. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 5 +++-- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 5 +++-- .../org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala | 3 ++- .../org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala | 3 ++- .../apache/spark/sql/execution/datasources/FileScanRDD.scala | 3 ++- .../sql/execution/datasources/v2/FilePartitionReader.scala | 5 +++-- .../spark/sql/execution/datasources/xml/XmlDataSource.scala | 3 ++- 7 files changed, 17 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 7f12d3833b638..01bc46fc0b623 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.mapred.lib.CombineFileSplit import org.apache.hadoop.mapreduce.TaskType import org.apache.hadoop.mapreduce.lib.input.FileInputFormat +import org.apache.hadoop.security.AccessControlException import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ @@ -320,7 +321,7 @@ class HadoopRDD[K, V]( null // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e - case e: BlockMissingException => throw e + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e case e: IOException if ignoreCorruptFiles => logWarning(log"Skipped the rest content in the corrupted file: " + log"${MDC(PATH, split.inputSplit)}", e) @@ -347,7 +348,7 @@ class HadoopRDD[K, V]( finished = true // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e - case e: BlockMissingException => throw e + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e case e: IOException if ignoreCorruptFiles => logWarning(log"Skipped the rest content in the corrupted file: " + log"${MDC(PATH, split.inputSplit)}", e) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index bedd0aa3a67aa..d619602305890 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -33,6 +33,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileInputFormat, FileSplit, InvalidInputException} import org.apache.hadoop.mapreduce.task.{JobContextImpl, TaskAttemptContextImpl} +import org.apache.hadoop.security.AccessControlException import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi @@ -256,7 +257,7 @@ class NewHadoopRDD[K, V]( null // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e - case e: BlockMissingException => throw e + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e case e: IOException if ignoreCorruptFiles => logWarning( log"Skipped the rest content in the corrupted file: " + @@ -286,7 +287,7 @@ class NewHadoopRDD[K, V]( finished = true // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e - case e: BlockMissingException => throw e + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e case e: IOException if ignoreCorruptFiles => logWarning( log"Skipped the rest content in the corrupted file: " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala index a615ae61bd674..4b892da9db255 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala @@ -33,6 +33,7 @@ import scala.xml.SAXException import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hdfs.BlockMissingException +import org.apache.hadoop.security.AccessControlException import org.apache.spark.{SparkIllegalArgumentException, SparkUpgradeException} import org.apache.spark.internal.Logging @@ -656,7 +657,7 @@ class XmlTokenizer( e) case NonFatal(e) => ExceptionUtils.getRootCause(e) match { - case _: BlockMissingException => + case _: AccessControlException | _: BlockMissingException => reader.close() reader = null throw e diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala index 00bba00584fdd..ecde7c1715bd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala @@ -31,6 +31,7 @@ import scala.util.control.NonFatal import scala.xml.SAXException import org.apache.hadoop.hdfs.BlockMissingException +import org.apache.hadoop.security.AccessControlException import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.internal.Logging @@ -159,7 +160,7 @@ class XmlInferSchema(options: XmlOptions, caseSensitive: Boolean) logWarning("Skipped missing file", e) Some(StructType(Nil)) case e: FileNotFoundException if !options.ignoreMissingFiles => throw e - case e: BlockMissingException => throw e + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e case e @ (_: IOException | _: RuntimeException) if options.ignoreCorruptFiles => logWarning("Skipped the rest of the content in the corrupted file", e) Some(StructType(Nil)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index b9b34238719c8..e468807f4ffd1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -22,6 +22,7 @@ import java.net.URI import org.apache.hadoop.fs.Path import org.apache.hadoop.hdfs.BlockMissingException +import org.apache.hadoop.security.AccessControlException import org.apache.spark.{Partition => RDDPartition, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil @@ -267,7 +268,7 @@ class FileScanRDD( null // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e - case e: BlockMissingException => throw e + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning(log"Skipped the rest of the content in the corrupted file: " + log"${MDC(PATH, currentFile)}", e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala index 2aaca708b04bb..be4f5dcb65aa1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import java.io.{FileNotFoundException, IOException} import org.apache.hadoop.hdfs.BlockMissingException +import org.apache.hadoop.security.AccessControlException import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.{CURRENT_FILE, PARTITIONED_FILE_READER} @@ -50,7 +51,7 @@ class FilePartitionReader[T]( case e: FileNotFoundException if ignoreMissingFiles => logWarning(s"Skipped missing file.", e) currentReader = null - case e: BlockMissingException => + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw FileDataSourceV2.attachFilePath(file.urlEncodedPath, e) case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning( @@ -68,7 +69,7 @@ class FilePartitionReader[T]( val hasNext = try { currentReader != null && currentReader.next() } catch { - case e: BlockMissingException => + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw FileDataSourceV2.attachFilePath(currentReader.file.urlEncodedPath, e) case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning(log"Skipped the rest of the content in the corrupted file: " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala index f0aefcb9b2a1d..8a179afb0f357 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hdfs.BlockMissingException import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.mapreduce.lib.input.FileInputFormat +import org.apache.hadoop.security.AccessControlException import org.apache.spark.TaskContext import org.apache.spark.input.{PortableDataStream, StreamInputFormat} @@ -191,7 +192,7 @@ object MultiLineXmlDataSource extends XmlDataSource { Iterator.empty[String] case NonFatal(e) => ExceptionUtils.getRootCause(e) match { - case e: BlockMissingException => throw e + case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e case _: RuntimeException | _: IOException if parsedOptions.ignoreCorruptFiles => logWarning("Skipped the rest of the content in the corrupted file", e) Iterator.empty[String] From b2c8b3069ef4f5288a5964af0da6f6b23a769e6b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 11 Dec 2024 10:35:36 -0800 Subject: [PATCH 154/438] Revert "[SPARK-48898][SQL] Set nullability correctly in the Variant schema" This reverts commit 9394b35ed7bb57a343ab6038dc1c1345db104604. --- .../parquet/SparkShreddingUtils.scala | 5 +--- .../sql/VariantWriteShreddingSuite.scala | 30 ------------------- 2 files changed, 1 insertion(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index 507633abfb285..41244e20c369f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -81,11 +81,8 @@ case object SparkShreddingUtils { StructField(TypedValueFieldName, arrayShreddingSchema, nullable = true) ) case StructType(fields) => - // The field name level is always non-nullable: Variant null values are represented in the - // "value" columna as "00", and missing values are represented by setting both "value" and - // "typed_value" to null. val objectShreddingSchema = StructType(fields.map(f => - f.copy(dataType = variantShreddingSchema(f.dataType, false), nullable = false))) + f.copy(dataType = variantShreddingSchema(f.dataType, false)))) Seq( StructField(VariantValueFieldName, BinaryType, nullable = true), StructField(TypedValueFieldName, objectShreddingSchema, nullable = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala index 9022d8cfdca49..a62c6e4462464 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala @@ -67,36 +67,6 @@ class VariantWriteShreddingSuite extends SparkFunSuite with ExpressionEvalHelper private val emptyMetadata: Array[Byte] = parseJson("null").getMetadata - test("variantShreddingSchema") { - // Validate the schema produced by SparkShreddingUtils.variantShreddingSchema for a few simple - // cases. - // metadata is always non-nullable. - assert(SparkShreddingUtils.variantShreddingSchema(IntegerType) == - StructType(Seq( - StructField("metadata", BinaryType, nullable = false), - StructField("value", BinaryType, nullable = true), - StructField("typed_value", IntegerType, nullable = true)))) - - val fieldA = StructType(Seq( - StructField("value", BinaryType, nullable = true), - StructField("typed_value", TimestampNTZType, nullable = true))) - val arrayType = ArrayType(StructType(Seq( - StructField("value", BinaryType, nullable = true), - StructField("typed_value", StringType, nullable = true)))) - val fieldB = StructType(Seq( - StructField("value", BinaryType, nullable = true), - StructField("typed_value", arrayType, nullable = true))) - val objectType = StructType(Seq( - StructField("a", fieldA, nullable = false), - StructField("b", fieldB, nullable = false))) - val structSchema = DataType.fromDDL("a timestamp_ntz, b array") - assert(SparkShreddingUtils.variantShreddingSchema(structSchema) == - StructType(Seq( - StructField("metadata", BinaryType, nullable = false), - StructField("value", BinaryType, nullable = true), - StructField("typed_value", objectType, nullable = true)))) - } - test("shredding as fixed numeric types") { /* Cast integer to any wider numeric type. */ testWithSchema("1", IntegerType, Row(emptyMetadata, null, 1)) From 48efe3f57f9f7b9d9fc5b68498ccbe14784a54ab Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Wed, 11 Dec 2024 15:38:34 -0800 Subject: [PATCH 155/438] [SPARK-50134][SPARK-50132][SQL][CONNECT][PYTHON] Support DataFrame API for Lateral Join in Spark Connect ### What changes were proposed in this pull request? Supports DataFrame API for Lateral Join in Spark Connect. ### Why are the changes needed? DataFrame API for Lateral Join is missing in Spark Connect. ### Does this PR introduce _any_ user-facing change? Yes, Lateral Join will be available in Spark Connect. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49083 from ueshin/issues/SPARK-50132/lateral_join_connect. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../scala/org/apache/spark/sql/Dataset.scala | 38 ++- .../org/apache/spark/sql/SQLImplicits.scala | 5 +- .../org/apache/spark/sql/SparkSession.scala | 4 +- .../sql/internal/columnNodeSupport.scala | 3 + .../spark/sql/DataFrameSubquerySuite.scala | 222 +++++++++++++ .../DataFrameTableValuedFunctionsSuite.scala | 254 ++++++++++++++ .../org/apache/spark/sql/test/SQLHelper.scala | 25 +- python/pyspark/errors/error-conditions.json | 2 +- python/pyspark/sql/connect/column.py | 8 +- python/pyspark/sql/connect/dataframe.py | 13 +- python/pyspark/sql/connect/expressions.py | 15 + python/pyspark/sql/connect/plan.py | 98 +++++- .../sql/connect/proto/expressions_pb2.py | 150 ++++----- .../sql/connect/proto/expressions_pb2.pyi | 28 ++ .../sql/connect/proto/relations_pb2.py | 310 +++++++++--------- .../sql/connect/proto/relations_pb2.pyi | 62 ++++ .../sql/tests/connect/test_parity_subquery.py | 33 +- .../sql/tests/connect/test_parity_tvf.py | 40 +-- .../sql/tests/connect/test_parity_udtf.py | 8 - python/pyspark/sql/tests/test_subquery.py | 17 + .../spark/sql/catalyst/plans/joinTypes.scala | 22 ++ .../protobuf/spark/connect/expressions.proto | 6 + .../protobuf/spark/connect/relations.proto | 18 + .../connect/planner/SparkConnectPlanner.scala | 22 +- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- 25 files changed, 1103 insertions(+), 304 deletions(-) create mode 100644 connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index eb166a1e8003b..ffaa8a70cc7c4 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -383,20 +383,44 @@ class Dataset[T] private[sql] ( } } - // TODO(SPARK-50134): Support Lateral Join API in Spark Connect - // scalastyle:off not.implemented.error.usage + private def lateralJoin( + right: DS[_], + joinExprs: Option[Column], + joinType: String): DataFrame = { + val joinTypeValue = toJoinType(joinType) + joinTypeValue match { + case proto.Join.JoinType.JOIN_TYPE_INNER | proto.Join.JoinType.JOIN_TYPE_LEFT_OUTER | + proto.Join.JoinType.JOIN_TYPE_CROSS => + case _ => + throw new IllegalArgumentException(s"Unsupported lateral join type $joinType") + } + sparkSession.newDataFrame { builder => + val lateralJoinBuilder = builder.getLateralJoinBuilder + lateralJoinBuilder.setLeft(plan.getRoot).setRight(right.plan.getRoot) + joinExprs.foreach(c => lateralJoinBuilder.setJoinCondition(c.expr)) + lateralJoinBuilder.setJoinType(joinTypeValue) + } + } + /** @inheritdoc */ - def lateralJoin(right: DS[_]): DataFrame = ??? + def lateralJoin(right: DS[_]): DataFrame = { + lateralJoin(right, None, "inner") + } /** @inheritdoc */ - def lateralJoin(right: DS[_], joinExprs: Column): DataFrame = ??? + def lateralJoin(right: DS[_], joinExprs: Column): DataFrame = { + lateralJoin(right, Some(joinExprs), "inner") + } /** @inheritdoc */ - def lateralJoin(right: DS[_], joinType: String): DataFrame = ??? + def lateralJoin(right: DS[_], joinType: String): DataFrame = { + lateralJoin(right, None, joinType) + } /** @inheritdoc */ - def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = ??? - // scalastyle:on not.implemented.error.usage + def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = { + lateralJoin(right, Some(joinExprs), joinType) + } override protected def sortInternal(global: Boolean, sortCols: Seq[Column]): Dataset[T] = { val sortExprs = sortCols.map { c => diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 4690253da808b..993b09ace9139 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql /** @inheritdoc */ -abstract class SQLImplicits private[sql] (override val session: SparkSession) - extends api.SQLImplicits { +abstract class SQLImplicits extends api.SQLImplicits { type DS[U] = Dataset[U] + + protected def session: SparkSession } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 3183a155c16a3..939a1341a8911 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -301,7 +301,9 @@ class SparkSession private[sql] ( // scalastyle:off /** @inheritdoc */ - object implicits extends SQLImplicits(this) + object implicits extends SQLImplicits { + override protected def session: SparkSession = SparkSession.this + } // scalastyle:on /** @inheritdoc */ diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala index 34a8a91a0ddf8..0e8889e19de22 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala @@ -164,6 +164,9 @@ object ColumnNodeToProtoConverter extends (ColumnNode => proto.Expression) { b.addArguments(apply(value, e)) } + case LazyExpression(child, _) => + builder.getLazyExpressionBuilder.setChild(apply(child, e)) + case ProtoColumnNode(e, _) => return e diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala new file mode 100644 index 0000000000000..91f60b1fefb91 --- /dev/null +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -0,0 +1,222 @@ +/* + * 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 + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} + +class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { + import testImplicits._ + + private def table1() = { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + spark.table("t1") + } + + private def table2() = { + sql("CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3)") + spark.table("t2") + } + + private def table3() = { + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + spark.table("t3") + } + + test("lateral join with single column select") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c1".outer())).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c1)").toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.select($"c1")).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c1 FROM t2)").toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.select($"t1.c1".outer())).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.c1 FROM t2)").toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.select($"t1.c1".outer() + $"t2.c1")).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2)").toDF("c1", "c2", "c3")) + } + } + + test("lateral join with different join types") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), + $"c2" === $"c3"), + sql("SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3")) + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), + $"c2" === $"c3", + "left"), + sql("SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3")) + checkAnswer( + t1.lateralJoin(spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), "cross"), + sql("SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3)")) + } + } + + test("lateral join with correlated equality / non-equality predicates") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2")) + .toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1)") + .toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.where($"t1.c1".outer() < $"t2.c1").select($"c2")) + .toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 < t2.c1)") + .toDF("c1", "c2", "c3")) + } + } + + test("lateral join with aggregation and correlated non-equality predicates") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.where($"t1.c2".outer() < $"t2.c2").select(max($"c2").as("m"))), + sql("SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2)")) + } + } + + test("lateral join can reference preceding FROM clause items") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.join(t2) + .lateralJoin(spark.range(1).select($"t1.c2".outer() + $"t2.c2".outer())) + .toDF("c1", "c2", "c3", "c4", "c5"), + sql("SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2)") + .toDF("c1", "c2", "c3", "c4", "c5")) + } + } + + test("multiple lateral joins") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin(spark.range(1).select(($"c1".outer() + $"c2".outer()).as("a"))) + .lateralJoin(spark.range(1).select(($"c1".outer() - $"c2".outer()).as("b"))) + .lateralJoin(spark.range(1).select(($"a".outer() * $"b".outer()).as("c"))), + sql(""" + |SELECT * FROM t1, + |LATERAL (SELECT c1 + c2 AS a), + |LATERAL (SELECT c1 - c2 AS b), + |LATERAL (SELECT a * b AS c) + |""".stripMargin)) + } + } + + test("lateral join in between regular joins") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2"), "left") + .join(t1.as("t3"), $"t2.c2" === $"t3.c2", "left") + .toDF("c1", "c2", "c3", "c4", "c5"), + sql(""" + |SELECT * FROM t1 + |LEFT OUTER JOIN LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) s + |LEFT OUTER JOIN t1 t3 ON s.c2 = t3.c2 + |""".stripMargin) + .toDF("c1", "c2", "c3", "c4", "c5")) + } + } + + test("nested lateral joins") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.lateralJoin(spark.range(1).select($"c1".outer()))) + .toDF("c1", "c2", "c3", "c4", "c5"), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT c1))") + .toDF("c1", "c2", "c3", "c4", "c5")) + checkAnswer( + t1.lateralJoin( + spark + .range(1) + .select(($"c1".outer() + lit(1)).as("c1")) + .lateralJoin(spark.range(1).select($"c1".outer()))) + .toDF("c1", "c2", "c3", "c4"), + sql( + "SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1 + 1 AS c1), LATERAL (SELECT c1))") + .toDF("c1", "c2", "c3", "c4")) + } + } + + test("lateral join with table-valued functions") { + withView("t1", "t3") { + val t1 = table1() + val t3 = table3() + + checkAnswer(t1.lateralJoin(spark.tvf.range(3)), sql("SELECT * FROM t1, LATERAL RANGE(3)")) + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)")) + checkAnswer( + spark.tvf + .explode(array(lit(1), lit(2))) + .toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + 1)), + sql("SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)")) + } + } + + test("lateral join with table-valued functions and join conditions") { + withView("t1", "t3") { + val t1 = table1() + val t3 = table3() + + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer())), $"c1" === $"col"), + sql("SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode($"c2".outer()), $"c1" === $"col"), + sql("SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode($"c2".outer()), $"c1" === $"col", "left"), + sql("SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3")) + } + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala index f4a0d1bcf5d6f..aeef2e8f0fcf2 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSession { + import testImplicits._ test("explode") { val actual1 = spark.tvf.explode(array(lit(1), lit(2))) @@ -50,6 +51,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("explode - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)")) + checkAnswer( + spark.tvf + .explode(array(lit(1), lit(2))) + .toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)")) + } + } + test("explode_outer") { val actual1 = spark.tvf.explode_outer(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM explode_outer(array(1, 2))") @@ -78,6 +103,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("explode_outer - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)")) + checkAnswer( + spark.tvf + .explode_outer(array(lit(1), lit(2))) + .toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM EXPLODE_OUTER(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)")) + } + } + test("inline") { val actual1 = spark.tvf.inline(array(struct(lit(1), lit("a")), struct(lit(2), lit("b")))) val expected1 = spark.sql("SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b')))") @@ -98,6 +147,25 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual3, expected3) } + test("inline - lateral join") { + withView("array_struct") { + sql(""" + |CREATE VIEW array_struct(id, arr) AS VALUES + | (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + | (2, ARRAY()), + | (3, ARRAY(STRUCT(3, 'c'))) + |""".stripMargin) + val arrayStruct = spark.table("array_struct") + + checkAnswer( + arrayStruct.lateralJoin(spark.tvf.inline($"arr".outer())), + sql("SELECT * FROM array_struct JOIN LATERAL INLINE(arr)")) + checkAnswer( + arrayStruct.lateralJoin(spark.tvf.inline($"arr".outer()), $"id" === $"col1", "left"), + sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k")) + } + } + test("inline_outer") { val actual1 = spark.tvf.inline_outer(array(struct(lit(1), lit("a")), struct(lit(2), lit("b")))) @@ -119,6 +187,28 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual3, expected3) } + test("inline_outer - lateral join") { + withView("array_struct") { + sql(""" + |CREATE VIEW array_struct(id, arr) AS VALUES + | (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + | (2, ARRAY()), + | (3, ARRAY(STRUCT(3, 'c'))) + |""".stripMargin) + val arrayStruct = spark.table("array_struct") + + checkAnswer( + arrayStruct.lateralJoin(spark.tvf.inline_outer($"arr".outer())), + sql("SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr)")) + checkAnswer( + arrayStruct.lateralJoin( + spark.tvf.inline_outer($"arr".outer()), + $"id" === $"col1", + "left"), + sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k")) + } + } + test("json_tuple") { val actual = spark.tvf.json_tuple(lit("""{"a":1,"b":2}"""), lit("a"), lit("b")) val expected = spark.sql("""SELECT * FROM json_tuple('{"a":1,"b":2}', 'a', 'b')""") @@ -131,6 +221,54 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi assert(ex.messageParameters("functionName") == "`json_tuple`") } + test("json_tuple - lateral join") { + withView("json_table") { + sql(""" + |CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES + | ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + | ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + | ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + | ('4', cast(null as string)), + | ('5', '{"f1": null, "f5": ""}'), + | ('6', '[invalid JSON string]') + |""".stripMargin) + val jsonTable = spark.table("json_table") + + checkAnswer( + jsonTable + .as("t1") + .lateralJoin( + spark.tvf.json_tuple( + $"t1.jstring".outer(), + lit("f1"), + lit("f2"), + lit("f3"), + lit("f4"), + lit("f5"))) + .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + sql( + "SELECT t1.key, t2.* FROM json_table t1, " + + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2")) + checkAnswer( + jsonTable + .as("t1") + .lateralJoin( + spark.tvf.json_tuple( + $"jstring".outer(), + lit("f1"), + lit("f2"), + lit("f3"), + lit("f4"), + lit("f5"))) + .where($"c0".isNotNull) + .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + sql( + "SELECT t1.key, t2.* FROM json_table t1, " + + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 " + + "WHERE t2.c0 IS NOT NULL")) + } + } + test("posexplode") { val actual1 = spark.tvf.posexplode(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM posexplode(array(1, 2))") @@ -159,6 +297,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("posexplode - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.posexplode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL POSEXPLODE(ARRAY(c1, c2))")) + checkAnswer( + t3.lateralJoin(spark.tvf.posexplode($"c2".outer())), + sql("SELECT * FROM t3, LATERAL POSEXPLODE(c2)")) + checkAnswer( + spark.tvf + .posexplode(array(lit(1), lit(2))) + .toDF("p", "v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM POSEXPLODE(ARRAY(1, 2)) t(p, v), LATERAL (SELECT v + 1)")) + } + } + test("posexplode_outer") { val actual1 = spark.tvf.posexplode_outer(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM posexplode_outer(array(1, 2))") @@ -187,12 +349,62 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("posexplode_outer - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.posexplode_outer(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL POSEXPLODE_OUTER(ARRAY(c1, c2))")) + checkAnswer( + t3.lateralJoin(spark.tvf.posexplode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL POSEXPLODE_OUTER(c2)")) + checkAnswer( + spark.tvf + .posexplode_outer(array(lit(1), lit(2))) + .toDF("p", "v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM POSEXPLODE_OUTER(ARRAY(1, 2)) t(p, v), LATERAL (SELECT v + 1)")) + } + } + test("stack") { val actual = spark.tvf.stack(lit(2), lit(1), lit(2), lit(3)) val expected = spark.sql("SELECT * FROM stack(2, 1, 2, 3)") checkAnswer(actual, expected) } + test("stack - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin( + spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer())) + .select($"col0", $"col1"), + sql("SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t")) + checkAnswer( + t1.lateralJoin(spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer())) + .select($"col0".as("x"), $"col1".as("y")), + sql("SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)")) + checkAnswer( + t1.join(t3, $"t1.c1" === $"t3.c1") + .lateralJoin(spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer())) + .select($"col0", $"col1"), + sql("SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t")) + } + } + test("collations") { val actual = spark.tvf.collations() val expected = spark.sql("SELECT * FROM collations()") @@ -236,6 +448,27 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("variant_explode - lateral join") { + withView("variant_table") { + sql(""" + |CREATE VIEW variant_table(id, v) AS + |SELECT id, parse_json(v) AS v FROM VALUES + |(0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + |(2, '[]'), (3, '{}'), + |(4, NULL), (5, '1') + |AS t(id, v) + |""".stripMargin) + val variantTable = spark.table("variant_table") + + checkAnswer( + variantTable + .as("t1") + .lateralJoin(spark.tvf.variant_explode($"v".outer())) + .select($"id", $"pos", $"key", $"value"), + sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode(v) AS t")) + } + } + test("variant_explode_outer") { val actual1 = spark.tvf.variant_explode_outer(parse_json(lit("""["hello", "world"]"""))) val expected1 = @@ -266,4 +499,25 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi val expected6 = spark.sql("SELECT * FROM variant_explode_outer(parse_json('1'))") checkAnswer(actual6, expected6) } + + test("variant_explode_outer - lateral join") { + withView("variant_table") { + sql(""" + |CREATE VIEW variant_table(id, v) AS + |SELECT id, parse_json(v) AS v FROM VALUES + |(0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + |(2, '[]'), (3, '{}'), + |(4, NULL), (5, '1') + |AS t(id, v) + |""".stripMargin) + val variantTable = spark.table("variant_table") + + checkAnswer( + variantTable + .as("t1") + .lateralJoin(spark.tvf.variant_explode_outer($"v".outer())) + .select($"id", $"pos", $"key", $"value"), + sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode_outer(v) AS t")) + } + } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala index 4a574a15f7ab8..007d4f0648e44 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala @@ -21,13 +21,27 @@ import java.util.UUID import org.scalatest.Assertions.fail -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession, SQLImplicits} import org.apache.spark.util.{SparkErrorUtils, SparkFileUtils} trait SQLHelper { def spark: SparkSession + // Shorthand for running a query using our SparkSession + protected lazy val sql: String => DataFrame = spark.sql _ + + /** + * A helper object for importing SQL implicits. + * + * Note that the alternative of importing `spark.implicits._` is not possible here. This is + * because we create the `SparkSession` immediately before the first test is run, but the + * implicits import is needed in the constructor. + */ + protected object testImplicits extends SQLImplicits { + override protected def session: SparkSession = spark + } + /** * Sets all SQL configurations specified in `pairs`, calls `f`, and then restores all SQL * configurations. @@ -106,4 +120,13 @@ trait SQLHelper { } } } + + /** + * Drops view `viewName` after calling `f`. + */ + protected def withView(viewNames: String*)(f: => Unit): Unit = { + SparkErrorUtils.tryWithSafeFinally(f)(viewNames.foreach { name => + spark.sql(s"DROP VIEW IF EXISTS $name") + }) + } } diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index b2a68a83bfa70..dbb4bb0087e06 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -1075,7 +1075,7 @@ }, "UNSUPPORTED_JOIN_TYPE": { "message": [ - "Unsupported join type: . Supported join types include: 'inner', 'outer', 'full', 'fullouter', 'full_outer', 'leftouter', 'left', 'left_outer', 'rightouter', 'right', 'right_outer', 'leftsemi', 'left_semi', 'semi', 'leftanti', 'left_anti', 'anti', 'cross'." + "Unsupported join type: ''. Supported join types include: ." ] }, "UNSUPPORTED_LITERAL": { diff --git a/python/pyspark/sql/connect/column.py b/python/pyspark/sql/connect/column.py index 1440c4c2792bf..1e49afc0b158a 100644 --- a/python/pyspark/sql/connect/column.py +++ b/python/pyspark/sql/connect/column.py @@ -34,7 +34,6 @@ PySparkTypeError, PySparkAttributeError, PySparkValueError, - PySparkNotImplementedError, ) from pyspark.sql.types import DataType from pyspark.sql.utils import enum_to_value @@ -44,6 +43,7 @@ Expression, UnresolvedFunction, UnresolvedExtractValue, + LazyExpression, LiteralExpression, CaseWhen, SortOrder, @@ -466,11 +466,7 @@ def over(self, window: "WindowSpec") -> ParentColumn: # type: ignore[override] return Column(WindowExpression(windowFunction=self._expr, windowSpec=window)) def outer(self) -> ParentColumn: - # TODO(SPARK-50134): Implement this method - raise PySparkNotImplementedError( - errorClass="NOT_IMPLEMENTED", - messageParameters={"feature": "outer()"}, - ) + return Column(LazyExpression(self._expr)) def isin(self, *cols: Any) -> ParentColumn: if len(cols) == 1 and isinstance(cols[0], (list, set)): diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 124ce5e0d39a5..ee1886b8ef290 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -692,10 +692,14 @@ def lateralJoin( on: Optional[Column] = None, how: Optional[str] = None, ) -> ParentDataFrame: - # TODO(SPARK-50134): Implement this method - raise PySparkNotImplementedError( - errorClass="NOT_IMPLEMENTED", - messageParameters={"feature": "lateralJoin()"}, + self._check_same_session(other) + if how is not None and isinstance(how, str): + how = how.lower().replace("_", "") + return DataFrame( + plan.LateralJoin( + left=self._plan, right=cast(plan.LogicalPlan, other._plan), on=on, how=how + ), + session=self._session, ) def _joinAsOf( @@ -2277,7 +2281,6 @@ def _test() -> None: # TODO(SPARK-50134): Support subquery in connect del pyspark.sql.dataframe.DataFrame.scalar.__doc__ del pyspark.sql.dataframe.DataFrame.exists.__doc__ - del pyspark.sql.dataframe.DataFrame.lateralJoin.__doc__ globs["spark"] = ( PySparkSession.builder.appName("sql.connect.dataframe tests") diff --git a/python/pyspark/sql/connect/expressions.py b/python/pyspark/sql/connect/expressions.py index 5a5320366f666..5d7b348f6d383 100644 --- a/python/pyspark/sql/connect/expressions.py +++ b/python/pyspark/sql/connect/expressions.py @@ -1153,3 +1153,18 @@ def to_plan(self, session: "SparkConnectClient") -> "proto.Expression": def __repr__(self) -> str: return f"{self._key} => {self._value}" + + +class LazyExpression(Expression): + def __init__(self, expr: Expression): + assert isinstance(expr, Expression) + super().__init__() + self._expr = expr + + def to_plan(self, session: "SparkConnectClient") -> proto.Expression: + expr = self._create_proto_expression() + expr.lazy_expression.child.CopyFrom(self._expr.to_plan(session)) + return expr + + def __repr__(self) -> str: + return f"lazy({self._expr})" diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 34f11768bcbc0..c411baf17ce94 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -893,7 +893,35 @@ def __init__( else: raise AnalysisException( errorClass="UNSUPPORTED_JOIN_TYPE", - messageParameters={"join_type": how}, + messageParameters={ + "typ": how, + "supported": ( + "'" + + "', '".join( + [ + "inner", + "outer", + "full", + "fullouter", + "full_outer", + "leftouter", + "left", + "left_outer", + "rightouter", + "right", + "right_outer", + "leftsemi", + "left_semi", + "semi", + "leftanti", + "left_anti", + "anti", + "cross", + ] + ) + + "'" + ), + }, ) self.how = join_type @@ -1028,6 +1056,74 @@ def _repr_html_(self) -> str: """ +class LateralJoin(LogicalPlan): + def __init__( + self, + left: Optional[LogicalPlan], + right: LogicalPlan, + on: Optional[Column], + how: Optional[str], + ) -> None: + super().__init__(left) + self.left = cast(LogicalPlan, left) + self.right = right + self.on = on + if how is None: + join_type = proto.Join.JoinType.JOIN_TYPE_INNER + elif how == "inner": + join_type = proto.Join.JoinType.JOIN_TYPE_INNER + elif how in ["leftouter", "left"]: + join_type = proto.Join.JoinType.JOIN_TYPE_LEFT_OUTER + elif how == "cross": + join_type = proto.Join.JoinType.JOIN_TYPE_CROSS + else: + raise AnalysisException( + errorClass="UNSUPPORTED_JOIN_TYPE", + messageParameters={ + "typ": how, + "supported": ( + "'" + + "', '".join(["inner", "leftouter", "left", "left_outer", "cross"]) + + "'" + ), + }, + ) + self.how = join_type + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.lateral_join.left.CopyFrom(self.left.plan(session)) + plan.lateral_join.right.CopyFrom(self.right.plan(session)) + if self.on is not None: + plan.lateral_join.join_condition.CopyFrom(self.on.to_plan(session)) + plan.lateral_join.join_type = self.how + return plan + + @property + def observations(self) -> Dict[str, "Observation"]: + return dict(**super().observations, **self.right.observations) + + def print(self, indent: int = 0) -> str: + i = " " * indent + o = " " * (indent + LogicalPlan.INDENT) + n = indent + LogicalPlan.INDENT * 2 + return ( + f"{i}\n{o}" + f"left=\n{self.left.print(n)}\n{o}right=\n{self.right.print(n)}" + ) + + def _repr_html_(self) -> str: + return f""" +
    +
  • + LateralJoin
    + Left: {self.left._repr_html_()} + Right: {self.right._repr_html_()} +
  • +
+ """ + + class SetOperation(LogicalPlan): def __init__( self, diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index 0d4730ac736e3..87070fd5ad3c5 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -40,7 +40,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xc1\x30\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_conditionB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\x8b\x31\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hildB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -54,77 +54,79 @@ "DESCRIPTOR" ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_EXPRESSION"]._serialized_start = 133 - _globals["_EXPRESSION"]._serialized_end = 6342 - _globals["_EXPRESSION_WINDOW"]._serialized_start = 1900 - _globals["_EXPRESSION_WINDOW"]._serialized_end = 2683 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2190 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_end = 2683 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_start = 2457 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_end = 2602 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_start = 2604 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_end = 2683 - _globals["_EXPRESSION_SORTORDER"]._serialized_start = 2686 - _globals["_EXPRESSION_SORTORDER"]._serialized_end = 3111 - _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_start = 2916 - _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_end = 3024 - _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_start = 3026 - _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_end = 3111 - _globals["_EXPRESSION_CAST"]._serialized_start = 3114 - _globals["_EXPRESSION_CAST"]._serialized_end = 3429 - _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3315 - _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3413 - _globals["_EXPRESSION_LITERAL"]._serialized_start = 3432 - _globals["_EXPRESSION_LITERAL"]._serialized_end = 4995 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4267 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4384 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4386 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4484 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4487 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4617 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4620 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 4847 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 4850 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 4979 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 4998 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5184 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5187 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5391 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5393 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5443 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5445 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5569 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5571 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5657 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5660 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 5792 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 5795 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 5982 - _globals["_EXPRESSION_ALIAS"]._serialized_start = 5984 - _globals["_EXPRESSION_ALIAS"]._serialized_end = 6104 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6107 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6265 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6267 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6329 - _globals["_EXPRESSIONCOMMON"]._serialized_start = 6344 - _globals["_EXPRESSIONCOMMON"]._serialized_end = 6409 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6412 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6776 - _globals["_PYTHONUDF"]._serialized_start = 6779 - _globals["_PYTHONUDF"]._serialized_end = 6983 - _globals["_SCALARSCALAUDF"]._serialized_start = 6986 - _globals["_SCALARSCALAUDF"]._serialized_end = 7200 - _globals["_JAVAUDF"]._serialized_start = 7203 - _globals["_JAVAUDF"]._serialized_end = 7352 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7354 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7453 - _globals["_CALLFUNCTION"]._serialized_start = 7455 - _globals["_CALLFUNCTION"]._serialized_end = 7563 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7565 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7657 - _globals["_MERGEACTION"]._serialized_start = 7660 - _globals["_MERGEACTION"]._serialized_end = 8172 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 7882 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 7988 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 7991 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8158 + _globals["_EXPRESSION"]._serialized_end = 6416 + _globals["_EXPRESSION_WINDOW"]._serialized_start = 1974 + _globals["_EXPRESSION_WINDOW"]._serialized_end = 2757 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2264 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_end = 2757 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_start = 2531 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_end = 2676 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_start = 2678 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_end = 2757 + _globals["_EXPRESSION_SORTORDER"]._serialized_start = 2760 + _globals["_EXPRESSION_SORTORDER"]._serialized_end = 3185 + _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_start = 2990 + _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_end = 3098 + _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_start = 3100 + _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_end = 3185 + _globals["_EXPRESSION_CAST"]._serialized_start = 3188 + _globals["_EXPRESSION_CAST"]._serialized_end = 3503 + _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3389 + _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3487 + _globals["_EXPRESSION_LITERAL"]._serialized_start = 3506 + _globals["_EXPRESSION_LITERAL"]._serialized_end = 5069 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4341 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4458 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4460 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4558 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4561 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4691 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4694 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 4921 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 4924 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5053 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5072 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5258 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5261 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5465 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5467 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5517 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5519 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5643 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5645 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5731 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5734 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 5866 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 5869 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6056 + _globals["_EXPRESSION_ALIAS"]._serialized_start = 6058 + _globals["_EXPRESSION_ALIAS"]._serialized_end = 6178 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6181 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6339 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6341 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6403 + _globals["_EXPRESSIONCOMMON"]._serialized_start = 6418 + _globals["_EXPRESSIONCOMMON"]._serialized_end = 6483 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6486 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6850 + _globals["_PYTHONUDF"]._serialized_start = 6853 + _globals["_PYTHONUDF"]._serialized_end = 7057 + _globals["_SCALARSCALAUDF"]._serialized_start = 7060 + _globals["_SCALARSCALAUDF"]._serialized_end = 7274 + _globals["_JAVAUDF"]._serialized_start = 7277 + _globals["_JAVAUDF"]._serialized_end = 7426 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7428 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7527 + _globals["_CALLFUNCTION"]._serialized_start = 7529 + _globals["_CALLFUNCTION"]._serialized_end = 7637 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7639 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7731 + _globals["_MERGEACTION"]._serialized_start = 7734 + _globals["_MERGEACTION"]._serialized_end = 8246 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 7956 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8062 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8065 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8232 + _globals["_LAZYEXPRESSION"]._serialized_start = 8248 + _globals["_LAZYEXPRESSION"]._serialized_end = 8313 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index 1566eb1b1e9e2..df4106cfc5f79 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -1184,6 +1184,7 @@ class Expression(google.protobuf.message.Message): NAMED_ARGUMENT_EXPRESSION_FIELD_NUMBER: builtins.int MERGE_ACTION_FIELD_NUMBER: builtins.int TYPED_AGGREGATE_EXPRESSION_FIELD_NUMBER: builtins.int + LAZY_EXPRESSION_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def common(self) -> global___ExpressionCommon: ... @@ -1228,6 +1229,8 @@ class Expression(google.protobuf.message.Message): @property def typed_aggregate_expression(self) -> global___TypedAggregateExpression: ... @property + def lazy_expression(self) -> global___LazyExpression: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary relations they can add them here. During the planning the correct resolution is done. @@ -1256,6 +1259,7 @@ class Expression(google.protobuf.message.Message): named_argument_expression: global___NamedArgumentExpression | None = ..., merge_action: global___MergeAction | None = ..., typed_aggregate_expression: global___TypedAggregateExpression | None = ..., + lazy_expression: global___LazyExpression | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -1279,6 +1283,8 @@ class Expression(google.protobuf.message.Message): b"extension", "lambda_function", b"lambda_function", + "lazy_expression", + b"lazy_expression", "literal", b"literal", "merge_action", @@ -1328,6 +1334,8 @@ class Expression(google.protobuf.message.Message): b"extension", "lambda_function", b"lambda_function", + "lazy_expression", + b"lazy_expression", "literal", b"literal", "merge_action", @@ -1379,6 +1387,7 @@ class Expression(google.protobuf.message.Message): "named_argument_expression", "merge_action", "typed_aggregate_expression", + "lazy_expression", "extension", ] | None @@ -1801,3 +1810,22 @@ class MergeAction(google.protobuf.message.Message): ) -> typing_extensions.Literal["condition"] | None: ... global___MergeAction = MergeAction + +class LazyExpression(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + CHILD_FIELD_NUMBER: builtins.int + @property + def child(self) -> global___Expression: + """(Required) The expression to be marked as lazy.""" + def __init__( + self, + *, + child: global___Expression | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["child", b"child"] + ) -> builtins.bool: ... + def ClearField(self, field_name: typing_extensions.Literal["child", b"child"]) -> None: ... + +global___LazyExpression = LazyExpression diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 479abcfb597a1..506b266f60145 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -42,7 +42,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto"\x9c\x1c\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xfb\x04\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_conf"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirectionB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto"\xdd\x1c\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xfb\x04\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_conf"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -78,157 +78,159 @@ _globals["_PARSE_OPTIONSENTRY"]._loaded_options = None _globals["_PARSE_OPTIONSENTRY"]._serialized_options = b"8\001" _globals["_RELATION"]._serialized_start = 193 - _globals["_RELATION"]._serialized_end = 3805 - _globals["_UNKNOWN"]._serialized_start = 3807 - _globals["_UNKNOWN"]._serialized_end = 3816 - _globals["_RELATIONCOMMON"]._serialized_start = 3819 - _globals["_RELATIONCOMMON"]._serialized_end = 3961 - _globals["_SQL"]._serialized_start = 3964 - _globals["_SQL"]._serialized_end = 4442 - _globals["_SQL_ARGSENTRY"]._serialized_start = 4258 - _globals["_SQL_ARGSENTRY"]._serialized_end = 4348 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 4350 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 4442 - _globals["_WITHRELATIONS"]._serialized_start = 4444 - _globals["_WITHRELATIONS"]._serialized_end = 4561 - _globals["_READ"]._serialized_start = 4564 - _globals["_READ"]._serialized_end = 5227 - _globals["_READ_NAMEDTABLE"]._serialized_start = 4742 - _globals["_READ_NAMEDTABLE"]._serialized_end = 4934 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 4876 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 4934 - _globals["_READ_DATASOURCE"]._serialized_start = 4937 - _globals["_READ_DATASOURCE"]._serialized_end = 5214 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 4876 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 4934 - _globals["_PROJECT"]._serialized_start = 5229 - _globals["_PROJECT"]._serialized_end = 5346 - _globals["_FILTER"]._serialized_start = 5348 - _globals["_FILTER"]._serialized_end = 5460 - _globals["_JOIN"]._serialized_start = 5463 - _globals["_JOIN"]._serialized_end = 6124 - _globals["_JOIN_JOINDATATYPE"]._serialized_start = 5802 - _globals["_JOIN_JOINDATATYPE"]._serialized_end = 5894 - _globals["_JOIN_JOINTYPE"]._serialized_start = 5897 - _globals["_JOIN_JOINTYPE"]._serialized_end = 6105 - _globals["_SETOPERATION"]._serialized_start = 6127 - _globals["_SETOPERATION"]._serialized_end = 6606 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 6443 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 6557 - _globals["_LIMIT"]._serialized_start = 6608 - _globals["_LIMIT"]._serialized_end = 6684 - _globals["_OFFSET"]._serialized_start = 6686 - _globals["_OFFSET"]._serialized_end = 6765 - _globals["_TAIL"]._serialized_start = 6767 - _globals["_TAIL"]._serialized_end = 6842 - _globals["_AGGREGATE"]._serialized_start = 6845 - _globals["_AGGREGATE"]._serialized_end = 7611 - _globals["_AGGREGATE_PIVOT"]._serialized_start = 7260 - _globals["_AGGREGATE_PIVOT"]._serialized_end = 7371 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 7373 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 7449 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 7452 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 7611 - _globals["_SORT"]._serialized_start = 7614 - _globals["_SORT"]._serialized_end = 7774 - _globals["_DROP"]._serialized_start = 7777 - _globals["_DROP"]._serialized_end = 7918 - _globals["_DEDUPLICATE"]._serialized_start = 7921 - _globals["_DEDUPLICATE"]._serialized_end = 8161 - _globals["_LOCALRELATION"]._serialized_start = 8163 - _globals["_LOCALRELATION"]._serialized_end = 8252 - _globals["_CACHEDLOCALRELATION"]._serialized_start = 8254 - _globals["_CACHEDLOCALRELATION"]._serialized_end = 8326 - _globals["_CACHEDREMOTERELATION"]._serialized_start = 8328 - _globals["_CACHEDREMOTERELATION"]._serialized_end = 8383 - _globals["_SAMPLE"]._serialized_start = 8386 - _globals["_SAMPLE"]._serialized_end = 8659 - _globals["_RANGE"]._serialized_start = 8662 - _globals["_RANGE"]._serialized_end = 8807 - _globals["_SUBQUERYALIAS"]._serialized_start = 8809 - _globals["_SUBQUERYALIAS"]._serialized_end = 8923 - _globals["_REPARTITION"]._serialized_start = 8926 - _globals["_REPARTITION"]._serialized_end = 9068 - _globals["_SHOWSTRING"]._serialized_start = 9071 - _globals["_SHOWSTRING"]._serialized_end = 9213 - _globals["_HTMLSTRING"]._serialized_start = 9215 - _globals["_HTMLSTRING"]._serialized_end = 9329 - _globals["_STATSUMMARY"]._serialized_start = 9331 - _globals["_STATSUMMARY"]._serialized_end = 9423 - _globals["_STATDESCRIBE"]._serialized_start = 9425 - _globals["_STATDESCRIBE"]._serialized_end = 9506 - _globals["_STATCROSSTAB"]._serialized_start = 9508 - _globals["_STATCROSSTAB"]._serialized_end = 9609 - _globals["_STATCOV"]._serialized_start = 9611 - _globals["_STATCOV"]._serialized_end = 9707 - _globals["_STATCORR"]._serialized_start = 9710 - _globals["_STATCORR"]._serialized_end = 9847 - _globals["_STATAPPROXQUANTILE"]._serialized_start = 9850 - _globals["_STATAPPROXQUANTILE"]._serialized_end = 10014 - _globals["_STATFREQITEMS"]._serialized_start = 10016 - _globals["_STATFREQITEMS"]._serialized_end = 10141 - _globals["_STATSAMPLEBY"]._serialized_start = 10144 - _globals["_STATSAMPLEBY"]._serialized_end = 10453 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 10345 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 10444 - _globals["_NAFILL"]._serialized_start = 10456 - _globals["_NAFILL"]._serialized_end = 10590 - _globals["_NADROP"]._serialized_start = 10593 - _globals["_NADROP"]._serialized_end = 10727 - _globals["_NAREPLACE"]._serialized_start = 10730 - _globals["_NAREPLACE"]._serialized_end = 11026 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 10885 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11026 - _globals["_TODF"]._serialized_start = 11028 - _globals["_TODF"]._serialized_end = 11116 - _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11119 - _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 11501 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 11363 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 11430 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 11432 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 11501 - _globals["_WITHCOLUMNS"]._serialized_start = 11503 - _globals["_WITHCOLUMNS"]._serialized_end = 11622 - _globals["_WITHWATERMARK"]._serialized_start = 11625 - _globals["_WITHWATERMARK"]._serialized_end = 11759 - _globals["_HINT"]._serialized_start = 11762 - _globals["_HINT"]._serialized_end = 11894 - _globals["_UNPIVOT"]._serialized_start = 11897 - _globals["_UNPIVOT"]._serialized_end = 12224 - _globals["_UNPIVOT_VALUES"]._serialized_start = 12154 - _globals["_UNPIVOT_VALUES"]._serialized_end = 12213 - _globals["_TRANSPOSE"]._serialized_start = 12226 - _globals["_TRANSPOSE"]._serialized_end = 12348 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 12350 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 12475 - _globals["_TOSCHEMA"]._serialized_start = 12477 - _globals["_TOSCHEMA"]._serialized_end = 12583 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 12586 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 12789 - _globals["_MAPPARTITIONS"]._serialized_start = 12792 - _globals["_MAPPARTITIONS"]._serialized_end = 13024 - _globals["_GROUPMAP"]._serialized_start = 13027 - _globals["_GROUPMAP"]._serialized_end = 13662 - _globals["_COGROUPMAP"]._serialized_start = 13665 - _globals["_COGROUPMAP"]._serialized_end = 14191 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 14194 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 14551 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 14554 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 14798 - _globals["_PYTHONUDTF"]._serialized_start = 14801 - _globals["_PYTHONUDTF"]._serialized_end = 14978 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 14981 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 15132 - _globals["_PYTHONDATASOURCE"]._serialized_start = 15134 - _globals["_PYTHONDATASOURCE"]._serialized_end = 15209 - _globals["_COLLECTMETRICS"]._serialized_start = 15212 - _globals["_COLLECTMETRICS"]._serialized_end = 15348 - _globals["_PARSE"]._serialized_start = 15351 - _globals["_PARSE"]._serialized_end = 15739 - _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 4876 - _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 4934 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 15640 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 15728 - _globals["_ASOFJOIN"]._serialized_start = 15742 - _globals["_ASOFJOIN"]._serialized_end = 16217 + _globals["_RELATION"]._serialized_end = 3870 + _globals["_UNKNOWN"]._serialized_start = 3872 + _globals["_UNKNOWN"]._serialized_end = 3881 + _globals["_RELATIONCOMMON"]._serialized_start = 3884 + _globals["_RELATIONCOMMON"]._serialized_end = 4026 + _globals["_SQL"]._serialized_start = 4029 + _globals["_SQL"]._serialized_end = 4507 + _globals["_SQL_ARGSENTRY"]._serialized_start = 4323 + _globals["_SQL_ARGSENTRY"]._serialized_end = 4413 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 4415 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 4507 + _globals["_WITHRELATIONS"]._serialized_start = 4509 + _globals["_WITHRELATIONS"]._serialized_end = 4626 + _globals["_READ"]._serialized_start = 4629 + _globals["_READ"]._serialized_end = 5292 + _globals["_READ_NAMEDTABLE"]._serialized_start = 4807 + _globals["_READ_NAMEDTABLE"]._serialized_end = 4999 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 4941 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 4999 + _globals["_READ_DATASOURCE"]._serialized_start = 5002 + _globals["_READ_DATASOURCE"]._serialized_end = 5279 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 4941 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 4999 + _globals["_PROJECT"]._serialized_start = 5294 + _globals["_PROJECT"]._serialized_end = 5411 + _globals["_FILTER"]._serialized_start = 5413 + _globals["_FILTER"]._serialized_end = 5525 + _globals["_JOIN"]._serialized_start = 5528 + _globals["_JOIN"]._serialized_end = 6189 + _globals["_JOIN_JOINDATATYPE"]._serialized_start = 5867 + _globals["_JOIN_JOINDATATYPE"]._serialized_end = 5959 + _globals["_JOIN_JOINTYPE"]._serialized_start = 5962 + _globals["_JOIN_JOINTYPE"]._serialized_end = 6170 + _globals["_SETOPERATION"]._serialized_start = 6192 + _globals["_SETOPERATION"]._serialized_end = 6671 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 6508 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 6622 + _globals["_LIMIT"]._serialized_start = 6673 + _globals["_LIMIT"]._serialized_end = 6749 + _globals["_OFFSET"]._serialized_start = 6751 + _globals["_OFFSET"]._serialized_end = 6830 + _globals["_TAIL"]._serialized_start = 6832 + _globals["_TAIL"]._serialized_end = 6907 + _globals["_AGGREGATE"]._serialized_start = 6910 + _globals["_AGGREGATE"]._serialized_end = 7676 + _globals["_AGGREGATE_PIVOT"]._serialized_start = 7325 + _globals["_AGGREGATE_PIVOT"]._serialized_end = 7436 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 7438 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 7514 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 7517 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 7676 + _globals["_SORT"]._serialized_start = 7679 + _globals["_SORT"]._serialized_end = 7839 + _globals["_DROP"]._serialized_start = 7842 + _globals["_DROP"]._serialized_end = 7983 + _globals["_DEDUPLICATE"]._serialized_start = 7986 + _globals["_DEDUPLICATE"]._serialized_end = 8226 + _globals["_LOCALRELATION"]._serialized_start = 8228 + _globals["_LOCALRELATION"]._serialized_end = 8317 + _globals["_CACHEDLOCALRELATION"]._serialized_start = 8319 + _globals["_CACHEDLOCALRELATION"]._serialized_end = 8391 + _globals["_CACHEDREMOTERELATION"]._serialized_start = 8393 + _globals["_CACHEDREMOTERELATION"]._serialized_end = 8448 + _globals["_SAMPLE"]._serialized_start = 8451 + _globals["_SAMPLE"]._serialized_end = 8724 + _globals["_RANGE"]._serialized_start = 8727 + _globals["_RANGE"]._serialized_end = 8872 + _globals["_SUBQUERYALIAS"]._serialized_start = 8874 + _globals["_SUBQUERYALIAS"]._serialized_end = 8988 + _globals["_REPARTITION"]._serialized_start = 8991 + _globals["_REPARTITION"]._serialized_end = 9133 + _globals["_SHOWSTRING"]._serialized_start = 9136 + _globals["_SHOWSTRING"]._serialized_end = 9278 + _globals["_HTMLSTRING"]._serialized_start = 9280 + _globals["_HTMLSTRING"]._serialized_end = 9394 + _globals["_STATSUMMARY"]._serialized_start = 9396 + _globals["_STATSUMMARY"]._serialized_end = 9488 + _globals["_STATDESCRIBE"]._serialized_start = 9490 + _globals["_STATDESCRIBE"]._serialized_end = 9571 + _globals["_STATCROSSTAB"]._serialized_start = 9573 + _globals["_STATCROSSTAB"]._serialized_end = 9674 + _globals["_STATCOV"]._serialized_start = 9676 + _globals["_STATCOV"]._serialized_end = 9772 + _globals["_STATCORR"]._serialized_start = 9775 + _globals["_STATCORR"]._serialized_end = 9912 + _globals["_STATAPPROXQUANTILE"]._serialized_start = 9915 + _globals["_STATAPPROXQUANTILE"]._serialized_end = 10079 + _globals["_STATFREQITEMS"]._serialized_start = 10081 + _globals["_STATFREQITEMS"]._serialized_end = 10206 + _globals["_STATSAMPLEBY"]._serialized_start = 10209 + _globals["_STATSAMPLEBY"]._serialized_end = 10518 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 10410 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 10509 + _globals["_NAFILL"]._serialized_start = 10521 + _globals["_NAFILL"]._serialized_end = 10655 + _globals["_NADROP"]._serialized_start = 10658 + _globals["_NADROP"]._serialized_end = 10792 + _globals["_NAREPLACE"]._serialized_start = 10795 + _globals["_NAREPLACE"]._serialized_end = 11091 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 10950 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11091 + _globals["_TODF"]._serialized_start = 11093 + _globals["_TODF"]._serialized_end = 11181 + _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11184 + _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 11566 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 11428 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 11495 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 11497 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 11566 + _globals["_WITHCOLUMNS"]._serialized_start = 11568 + _globals["_WITHCOLUMNS"]._serialized_end = 11687 + _globals["_WITHWATERMARK"]._serialized_start = 11690 + _globals["_WITHWATERMARK"]._serialized_end = 11824 + _globals["_HINT"]._serialized_start = 11827 + _globals["_HINT"]._serialized_end = 11959 + _globals["_UNPIVOT"]._serialized_start = 11962 + _globals["_UNPIVOT"]._serialized_end = 12289 + _globals["_UNPIVOT_VALUES"]._serialized_start = 12219 + _globals["_UNPIVOT_VALUES"]._serialized_end = 12278 + _globals["_TRANSPOSE"]._serialized_start = 12291 + _globals["_TRANSPOSE"]._serialized_end = 12413 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 12415 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 12540 + _globals["_TOSCHEMA"]._serialized_start = 12542 + _globals["_TOSCHEMA"]._serialized_end = 12648 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 12651 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 12854 + _globals["_MAPPARTITIONS"]._serialized_start = 12857 + _globals["_MAPPARTITIONS"]._serialized_end = 13089 + _globals["_GROUPMAP"]._serialized_start = 13092 + _globals["_GROUPMAP"]._serialized_end = 13727 + _globals["_COGROUPMAP"]._serialized_start = 13730 + _globals["_COGROUPMAP"]._serialized_end = 14256 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 14259 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 14616 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 14619 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 14863 + _globals["_PYTHONUDTF"]._serialized_start = 14866 + _globals["_PYTHONUDTF"]._serialized_end = 15043 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15046 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 15197 + _globals["_PYTHONDATASOURCE"]._serialized_start = 15199 + _globals["_PYTHONDATASOURCE"]._serialized_end = 15274 + _globals["_COLLECTMETRICS"]._serialized_start = 15277 + _globals["_COLLECTMETRICS"]._serialized_end = 15413 + _globals["_PARSE"]._serialized_start = 15416 + _globals["_PARSE"]._serialized_end = 15804 + _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 4941 + _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 4999 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 15705 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 15793 + _globals["_ASOFJOIN"]._serialized_start = 15807 + _globals["_ASOFJOIN"]._serialized_end = 16282 + _globals["_LATERALJOIN"]._serialized_start = 16285 + _globals["_LATERALJOIN"]._serialized_end = 16515 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index 03753056c6bf1..e5a6bff9e430a 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -106,6 +106,7 @@ class Relation(google.protobuf.message.Message): WITH_RELATIONS_FIELD_NUMBER: builtins.int TRANSPOSE_FIELD_NUMBER: builtins.int UNRESOLVED_TABLE_VALUED_FUNCTION_FIELD_NUMBER: builtins.int + LATERAL_JOIN_FIELD_NUMBER: builtins.int FILL_NA_FIELD_NUMBER: builtins.int DROP_NA_FIELD_NUMBER: builtins.int REPLACE_FIELD_NUMBER: builtins.int @@ -211,6 +212,8 @@ class Relation(google.protobuf.message.Message): @property def unresolved_table_valued_function(self) -> global___UnresolvedTableValuedFunction: ... @property + def lateral_join(self) -> global___LateralJoin: ... + @property def fill_na(self) -> global___NAFill: """NA functions""" @property @@ -292,6 +295,7 @@ class Relation(google.protobuf.message.Message): with_relations: global___WithRelations | None = ..., transpose: global___Transpose | None = ..., unresolved_table_valued_function: global___UnresolvedTableValuedFunction | None = ..., + lateral_join: global___LateralJoin | None = ..., fill_na: global___NAFill | None = ..., drop_na: global___NADrop | None = ..., replace: global___NAReplace | None = ..., @@ -364,6 +368,8 @@ class Relation(google.protobuf.message.Message): b"html_string", "join", b"join", + "lateral_join", + b"lateral_join", "limit", b"limit", "local_relation", @@ -485,6 +491,8 @@ class Relation(google.protobuf.message.Message): b"html_string", "join", b"join", + "lateral_join", + b"lateral_join", "limit", b"limit", "local_relation", @@ -595,6 +603,7 @@ class Relation(google.protobuf.message.Message): "with_relations", "transpose", "unresolved_table_valued_function", + "lateral_join", "fill_na", "drop_na", "replace", @@ -4109,3 +4118,56 @@ class AsOfJoin(google.protobuf.message.Message): ) -> None: ... global___AsOfJoin = AsOfJoin + +class LateralJoin(google.protobuf.message.Message): + """Relation of type [[LateralJoin]]. + + `left` and `right` must be present. + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + LEFT_FIELD_NUMBER: builtins.int + RIGHT_FIELD_NUMBER: builtins.int + JOIN_CONDITION_FIELD_NUMBER: builtins.int + JOIN_TYPE_FIELD_NUMBER: builtins.int + @property + def left(self) -> global___Relation: + """(Required) Left input relation for a Join.""" + @property + def right(self) -> global___Relation: + """(Required) Right input relation for a Join.""" + @property + def join_condition(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression: + """(Optional) The join condition.""" + join_type: global___Join.JoinType.ValueType + """(Required) The join type.""" + def __init__( + self, + *, + left: global___Relation | None = ..., + right: global___Relation | None = ..., + join_condition: pyspark.sql.connect.proto.expressions_pb2.Expression | None = ..., + join_type: global___Join.JoinType.ValueType = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "join_condition", b"join_condition", "left", b"left", "right", b"right" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "join_condition", + b"join_condition", + "join_type", + b"join_type", + "left", + b"left", + "right", + b"right", + ], + ) -> None: ... + +global___LateralJoin = LateralJoin diff --git a/python/pyspark/sql/tests/connect/test_parity_subquery.py b/python/pyspark/sql/tests/connect/test_parity_subquery.py index 1cba3a7d49956..cffb6fc39059d 100644 --- a/python/pyspark/sql/tests/connect/test_parity_subquery.py +++ b/python/pyspark/sql/tests/connect/test_parity_subquery.py @@ -21,9 +21,38 @@ from pyspark.testing.connectutils import ReusedConnectTestCase -@unittest.skip("TODO(SPARK-50134): Support subquery in connect") class SubqueryParityTests(SubqueryTestsMixin, ReusedConnectTestCase): - pass + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_simple_uncorrelated_scalar_subquery(self): + super().test_simple_uncorrelated_scalar_subquery() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_uncorrelated_scalar_subquery_with_view(self): + super().test_uncorrelated_scalar_subquery_with_view() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_scalar_subquery_against_local_relations(self): + super().test_scalar_subquery_against_local_relations() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_correlated_scalar_subquery(self): + super().test_correlated_scalar_subquery() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_exists_subquery(self): + super().test_exists_subquery() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_scalar_subquery_with_outer_reference_errors(self): + super().test_scalar_subquery_with_outer_reference_errors() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_scalar_subquery_inside_lateral_join(self): + super().test_scalar_subquery_inside_lateral_join() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_lateral_join_inside_subquery(self): + super().test_lateral_join_inside_subquery() if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/connect/test_parity_tvf.py b/python/pyspark/sql/tests/connect/test_parity_tvf.py index c5edff02810ff..61e3decf562c3 100644 --- a/python/pyspark/sql/tests/connect/test_parity_tvf.py +++ b/python/pyspark/sql/tests/connect/test_parity_tvf.py @@ -21,45 +21,7 @@ class TVFParityTestsMixin(TVFTestsMixin, ReusedConnectTestCase): - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_explode_with_lateral_join(self): - super().test_explode_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_explode_outer_with_lateral_join(self): - super().test_explode_outer_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_inline_with_lateral_join(self): - super().test_inline_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_inline_outer_with_lateral_join(self): - super().test_inline_outer_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_json_tuple_with_lateral_join(self): - super().test_json_tuple_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_posexplode_with_lateral_join(self): - super().test_posexplode_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_posexplode_outer_with_lateral_join(self): - super().test_posexplode_outer_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_stack_with_lateral_join(self): - super().test_stack_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_variant_explode_with_lateral_join(self): - super().test_variant_explode_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_variant_explode_outer_with_lateral_join(self): - super().test_variant_explode_outer_with_lateral_join() + pass if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py b/python/pyspark/sql/tests/connect/test_parity_udtf.py index 29d1718fe3781..6955e7377b4c4 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udtf.py +++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py @@ -85,14 +85,6 @@ def _add_archive(self, path): def _add_file(self, path): self.spark.addArtifacts(path, file=True) - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_udtf_with_lateral_join_dataframe(self): - super().test_udtf_with_lateral_join_dataframe() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_udtf_with_conditional_return_dataframe(self): - super().test_udtf_with_conditional_return_dataframe() - class ArrowUDTFParityTests(UDTFArrowTestsMixin, UDTFParityTests): @classmethod diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 1b657e075c59e..91789f74d9da6 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -556,6 +556,23 @@ def test_lateral_join_with_different_join_types(self): self.spark.sql("""SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3)"""), ) + with self.assertRaises(AnalysisException) as pe: + t1.lateralJoin( + self.spark.range(1).select( + (sf.col("c1").outer() + sf.col("c2").outer()).alias("c3") + ), + how="right", + ).collect() + + self.check_error( + pe.exception, + errorClass="UNSUPPORTED_JOIN_TYPE", + messageParameters={ + "typ": "right", + "supported": "'inner', 'leftouter', 'left', 'left_outer', 'cross'", + }, + ) + def test_lateral_join_with_correlated_predicates(self): with self.tempView("t1", "t2"): t1 = self.table1() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 41bba99673a2b..9f8c62fe58408 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -159,3 +159,25 @@ sealed abstract class AsOfJoinDirection case object Forward extends AsOfJoinDirection case object Backward extends AsOfJoinDirection case object Nearest extends AsOfJoinDirection + +object LateralJoinType { + + val supported = Seq( + "inner", + "leftouter", "left", "left_outer", + "cross" + ) + + def apply(typ: String): JoinType = typ.toLowerCase(Locale.ROOT).replace("_", "") match { + case "inner" => Inner + case "leftouter" | "left" => LeftOuter + case "cross" => Cross + case _ => + throw new AnalysisException( + errorClass = "UNSUPPORTED_JOIN_TYPE", + messageParameters = Map( + "typ" -> typ, + "supported" -> supported.mkString("'", "', '", "'")) + ) + } +} diff --git a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto index 3a91371fd3b25..811dd032aa419 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -52,6 +52,7 @@ message Expression { NamedArgumentExpression named_argument_expression = 17; MergeAction merge_action = 19; TypedAggregateExpression typed_aggregate_expression = 20; + LazyExpression lazy_expression = 21; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // relations they can add them here. During the planning the correct resolution is done. @@ -451,3 +452,8 @@ message MergeAction { Expression value = 2; } } + +message LazyExpression { + // (Required) The expression to be marked as lazy. + Expression child = 1; +} diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index a7b9137c3400a..7a86db2799149 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -78,6 +78,7 @@ message Relation { WithRelations with_relations = 41; Transpose transpose = 42; UnresolvedTableValuedFunction unresolved_table_valued_function = 43; + LateralJoin lateral_join = 44; // NA functions NAFill fill_na = 90; @@ -1140,3 +1141,20 @@ message AsOfJoin { // (Required) Whether to search for prior, subsequent, or closest matches. string direction = 10; } + +// Relation of type [[LateralJoin]]. +// +// `left` and `right` must be present. +message LateralJoin { + // (Required) Left input relation for a Join. + Relation left = 1; + + // (Required) Right input relation for a Join. + Relation right = 2; + + // (Optional) The join condition. + Expression join_condition = 3; + + // (Required) The join type. + Join.JoinType join_type = 4; +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 4aa567f19e16c..ec67c57a37f57 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -45,7 +45,7 @@ import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTableValuedFunction, UnresolvedTranspose} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ @@ -154,6 +154,7 @@ class SparkConnectPlanner( case proto.Relation.RelTypeCase.TAIL => transformTail(rel.getTail) case proto.Relation.RelTypeCase.JOIN => transformJoinOrJoinWith(rel.getJoin) case proto.Relation.RelTypeCase.AS_OF_JOIN => transformAsOfJoin(rel.getAsOfJoin) + case proto.Relation.RelTypeCase.LATERAL_JOIN => transformLateralJoin(rel.getLateralJoin) case proto.Relation.RelTypeCase.DEDUPLICATE => transformDeduplicate(rel.getDeduplicate) case proto.Relation.RelTypeCase.SET_OP => transformSetOperation(rel.getSetOp) case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort) @@ -1556,6 +1557,8 @@ class SparkConnectPlanner( transformMergeAction(exp.getMergeAction) case proto.Expression.ExprTypeCase.TYPED_AGGREGATE_EXPRESSION => transformTypedAggregateExpression(exp.getTypedAggregateExpression, baseRelationOpt) + case proto.Expression.ExprTypeCase.LAZY_EXPRESSION => + transformLazyExpression(exp.getLazyExpression) case _ => throw InvalidPlanInput( s"Expression with ID: ${exp.getExprTypeCase.getNumber} is not supported") @@ -2140,6 +2143,19 @@ class SparkConnectPlanner( joined.logicalPlan } + private def transformLateralJoin(rel: proto.LateralJoin): LogicalPlan = { + assert(rel.hasLeft && rel.hasRight, "Both join sides must be present") + val joinCondition = + if (rel.hasJoinCondition) Some(transformExpression(rel.getJoinCondition)) else None + val joinType = transformJoinType( + if (rel.getJoinType != null) rel.getJoinType else proto.Join.JoinType.JOIN_TYPE_INNER) + logical.LateralJoin( + left = transformRelation(rel.getLeft), + right = LateralSubquery(transformRelation(rel.getRight)), + joinType = joinType, + condition = joinCondition) + } + private def transformSort(sort: proto.Sort): LogicalPlan = { assert(sort.getOrderCount > 0, "'order' must be present and contain elements.") logical.Sort( @@ -3704,4 +3720,8 @@ class SparkConnectPlanner( session.catalog.listCatalogs().logicalPlan } } + + private def transformLazyExpression(getLazyExpression: proto.LazyExpression): Expression = { + LazyExpression(transformExpression(getLazyExpression.getChild)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 8726ee268a477..c5c9dde901c9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -733,12 +733,12 @@ class Dataset[T] private[sql]( /** @inheritdoc */ def lateralJoin(right: DS[_], joinType: String): DataFrame = { - lateralJoin(right, None, JoinType(joinType)) + lateralJoin(right, None, LateralJoinType(joinType)) } /** @inheritdoc */ def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = { - lateralJoin(right, Some(joinExprs), JoinType(joinType)) + lateralJoin(right, Some(joinExprs), LateralJoinType(joinType)) } // TODO(SPARK-22947): Fix the DataFrame API. From d464e85b0a3c6494250ba09ecfa8d1fdd5ee9daf Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 12 Dec 2024 09:11:13 +0900 Subject: [PATCH 156/438] [SPARK-50549][DOCS] Use `rouge` `4.5.x` by removing the upper bound `3.x` ### What changes were proposed in this pull request? This PR aims to use the latest `rouge` `4.5.x` by removing the upper-bound `3.x` (2022-07-29) ### Why are the changes needed? We are using `Ruby 3.3` in Apache Spark 4 documentation. So, we don't need to be blocked in the old `rouge` versions. https://github.com/apache/spark/blob/3bb9a72c8691786584eb2856ff6e3571f0118283/.github/workflows/pages.yml#L71 https://github.com/apache/spark/blob/b2c8b3069ef4f5288a5964af0da6f6b23a769e6b/docs/README.md?plain=1#L33 `Rouge` release notes: - https://github.com/rouge-ruby/rouge/releases/tag/v4.5.1 (Latest) - https://github.com/rouge-ruby/rouge/releases/tag/v4.5.0 (2024-11-10) ... - https://github.com/rouge-ruby/rouge/releases/tag/v4.0.0 (2022-09-06) - https://github.com/rouge-ruby/rouge/releases/tag/v3.30.0 (2022-07-29) ### Does this PR introduce _any_ user-facing change? No behavior change. This only affects documentation. ### How was this patch tested? Manual review. I checked the following. ``` $ cd docs $ SKIP_API=1 bundle exec jekyll build ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49149 from dongjoon-hyun/SPARK-50549. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- docs/Gemfile | 3 --- docs/Gemfile.lock | 3 +-- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/Gemfile b/docs/Gemfile index 8177425cfb681..db1ba08204f67 100644 --- a/docs/Gemfile +++ b/docs/Gemfile @@ -24,9 +24,6 @@ source "https://rubygems.org" gem "jekyll", "~> 4.3" gem "jekyll-redirect-from", "~> 0.16" -# Rouge 4.0 drops support for Ruby < 2.7, which is EOL. -# See: https://github.com/rouge-ruby/rouge/blob/61bdda18f204a661413daa93d9624bc65ad219a5/CHANGELOG.md#version-400-2022-09-04 -gem "rouge", "~> 3.26" # This resolves a build issue on Apple Silicon. # See: https://issues.apache.org/jira/browse/SPARK-38488 gem "ffi", "~> 1.15" diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index e137f0f039b97..286c8fe97948d 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -54,7 +54,7 @@ GEM rb-inotify (0.10.1) ffi (~> 1.0) rexml (3.2.6) - rouge (3.30.0) + rouge (4.5.1) safe_yaml (1.0.5) sass-embedded (1.63.6) google-protobuf (~> 3.23) @@ -71,7 +71,6 @@ DEPENDENCIES ffi (~> 1.15) jekyll (~> 4.3) jekyll-redirect-from (~> 0.16) - rouge (~> 3.26) BUNDLED WITH 2.4.22 From bfc5c22cb9346412a36db15e4f9c91c31ee16a62 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 12 Dec 2024 10:22:27 +0900 Subject: [PATCH 157/438] [SPARK-50517][PYTHON][TESTS][FOLLOW-UP] Add refactored package into pure Python test ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/49104 that adds refactored package into pure Python test ### Why are the changes needed? In order to fix the pure Python build https://github.com/apache/spark/actions/runs/12283159330/job/34275992419 ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Will monitor the build. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49158 from HyukjinKwon/SPARK-50517-followup2. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/packaging/connect/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 89c9c205b5bfd..5f67e5306b3ff 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -72,6 +72,7 @@ "pyspark.testing", "pyspark.resource.tests", "pyspark.sql.tests", + "pyspark.sql.tests.arrow", "pyspark.sql.tests.connect", "pyspark.sql.tests.connect.arrow", "pyspark.sql.tests.connect.streaming", From 1f6cb60e3c78dc8592c0b76435f22d0f36194ed7 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 12 Dec 2024 10:24:09 +0800 Subject: [PATCH 158/438] [SPARK-50544][PYTHON][CONNECT] Implement `StructType.toDDL` ### What changes were proposed in this pull request? Implement `StructType.toDDL` ### Why are the changes needed? we has `fromDDL` in python side, but the `toDDL` is still missing ### Does this PR introduce _any_ user-facing change? new feature ``` In [1]: spark.range(100).schema.toDDL() Out[1]: 'id BIGINT NOT NULL' ``` ### How was this patch tested? added tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49142 from zhengruifeng/struct_type_to_ddl. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/connect/client/core.py | 8 + python/pyspark/sql/connect/proto/base_pb2.py | 344 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 47 +++ python/pyspark/sql/tests/test_types.py | 16 + python/pyspark/sql/types.py | 19 + .../main/protobuf/spark/connect/base.proto | 11 + .../service/SparkConnectAnalyzeHandler.scala | 12 + .../spark/sql/api/python/PythonSQLUtils.scala | 4 + 8 files changed, 291 insertions(+), 170 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 78d4e0fc1c4f4..af24af90a3dcd 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -494,6 +494,7 @@ def __init__( is_same_semantics: Optional[bool], semantic_hash: Optional[int], storage_level: Optional[StorageLevel], + ddl_string: Optional[str], ): self.schema = schema self.explain_string = explain_string @@ -506,6 +507,7 @@ def __init__( self.is_same_semantics = is_same_semantics self.semantic_hash = semantic_hash self.storage_level = storage_level + self.ddl_string = ddl_string @classmethod def fromProto(cls, pb: Any) -> "AnalyzeResult": @@ -520,6 +522,7 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": is_same_semantics: Optional[bool] = None semantic_hash: Optional[int] = None storage_level: Optional[StorageLevel] = None + ddl_string: Optional[str] = None if pb.HasField("schema"): schema = types.proto_schema_to_pyspark_data_type(pb.schema.schema) @@ -547,6 +550,8 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": pass elif pb.HasField("get_storage_level"): storage_level = proto_to_storage_level(pb.get_storage_level.storage_level) + elif pb.HasField("json_to_ddl"): + ddl_string = pb.json_to_ddl.ddl_string else: raise SparkConnectException("No analyze result found!") @@ -562,6 +567,7 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": is_same_semantics, semantic_hash, storage_level, + ddl_string, ) @@ -1284,6 +1290,8 @@ def _analyze(self, method: str, **kwargs: Any) -> AnalyzeResult: req.unpersist.blocking = cast(bool, kwargs.get("blocking")) elif method == "get_storage_level": req.get_storage_level.relation.CopyFrom(cast(pb2.Relation, kwargs.get("relation"))) + elif method == "json_to_ddl": + req.json_to_ddl.json_string = cast(str, kwargs.get("json_string")) else: raise PySparkValueError( errorClass="UNSUPPORTED_OPERATION", diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 620f413f62c00..97694c33abeb2 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -43,7 +43,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf8\x13\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relationB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xce\r\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevelB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\x87\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xab\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\x87\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xab\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -73,181 +73,185 @@ _globals["_USERCONTEXT"]._serialized_start = 337 _globals["_USERCONTEXT"]._serialized_end = 459 _globals["_ANALYZEPLANREQUEST"]._serialized_start = 462 - _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3014 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1745 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1794 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1797 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2112 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 1940 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2112 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2114 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2204 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2206 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2256 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2258 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2312 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2314 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2367 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2369 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2383 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2385 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2426 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2428 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2549 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2551 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2606 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2609 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2760 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2762 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 2872 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 2874 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 2944 - _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3017 - _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 4759 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4178 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4235 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4237 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4285 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4287 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4332 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4334 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4370 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4372 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4420 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4422 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4456 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4458 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4498 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4500 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4559 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4561 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4600 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4602 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4640 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2609 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2618 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2762 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2773 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4666 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 4749 - _globals["_EXECUTEPLANREQUEST"]._serialized_start = 4762 - _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5437 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5196 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5361 - _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5440 - _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 8358 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7132 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 7203 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 7205 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 7323 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 7326 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 7843 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 7421 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 7753 + _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3139 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1824 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1873 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1876 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2191 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2019 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2191 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2193 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2283 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2285 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2335 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2337 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2391 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2393 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2446 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2448 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2462 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2464 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2505 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2507 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2628 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2630 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2685 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2688 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2839 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2841 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 2951 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 2953 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3023 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3025 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3069 + _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3142 + _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5008 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4383 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4440 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4442 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4490 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4492 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4537 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4539 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4575 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4577 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4625 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4627 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4661 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4663 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4703 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4705 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4764 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4766 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4805 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4807 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4845 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2688 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2697 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2841 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2852 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4871 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 4954 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 4956 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 4998 + _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5011 + _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5686 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5445 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5610 + _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5689 + _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 8607 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7381 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 7452 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 7454 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 7572 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 7575 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8092 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 7670 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8002 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 7630 + ]._serialized_start = 7879 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_end = 7753 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 7755 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 7843 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 7846 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 7987 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 7989 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8005 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8008 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 8341 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 8164 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 8341 - _globals["_KEYVALUE"]._serialized_start = 8360 - _globals["_KEYVALUE"]._serialized_end = 8425 - _globals["_CONFIGREQUEST"]._serialized_start = 8428 - _globals["_CONFIGREQUEST"]._serialized_end = 9587 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 8736 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 9234 - _globals["_CONFIGREQUEST_SET"]._serialized_start = 9236 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 9288 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 9290 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 9315 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9317 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9380 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9382 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9413 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9415 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9463 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9465 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9492 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9494 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9528 - _globals["_CONFIGRESPONSE"]._serialized_start = 9590 - _globals["_CONFIGRESPONSE"]._serialized_end = 9765 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 9768 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 10770 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10243 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10296 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10298 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10409 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10411 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10504 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10507 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 10700 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 10773 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11045 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 10964 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11045 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11048 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11374 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11377 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 11729 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11572 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 11687 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 11689 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 11729 - _globals["_INTERRUPTREQUEST"]._serialized_start = 11732 - _globals["_INTERRUPTREQUEST"]._serialized_end = 12335 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12135 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12263 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 12338 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 12482 - _globals["_REATTACHOPTIONS"]._serialized_start = 12484 - _globals["_REATTACHOPTIONS"]._serialized_end = 12537 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12540 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 12946 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 12949 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13534 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13403 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13415 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13417 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13464 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13537 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 13702 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 13705 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 13876 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 13878 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 13986 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 13989 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14321 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14324 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 15879 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14553 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 14727 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 14730 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15098 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15061 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15098 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15101 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15510 + ]._serialized_end = 8002 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8004 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8092 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8095 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8236 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8238 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8254 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8257 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 8590 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 8413 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 8590 + _globals["_KEYVALUE"]._serialized_start = 8609 + _globals["_KEYVALUE"]._serialized_end = 8674 + _globals["_CONFIGREQUEST"]._serialized_start = 8677 + _globals["_CONFIGREQUEST"]._serialized_end = 9836 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 8985 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 9483 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 9485 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 9537 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 9539 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 9564 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9566 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9629 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9631 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9662 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9664 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9712 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9714 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9741 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9743 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9777 + _globals["_CONFIGRESPONSE"]._serialized_start = 9839 + _globals["_CONFIGRESPONSE"]._serialized_end = 10014 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10017 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11019 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10492 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10545 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10547 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10658 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10660 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10753 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10756 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 10949 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11022 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11294 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11213 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11294 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11297 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11623 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11626 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 11978 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11821 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 11936 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 11938 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 11978 + _globals["_INTERRUPTREQUEST"]._serialized_start = 11981 + _globals["_INTERRUPTREQUEST"]._serialized_end = 12584 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12384 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12512 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 12587 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 12731 + _globals["_REATTACHOPTIONS"]._serialized_start = 12733 + _globals["_REATTACHOPTIONS"]._serialized_end = 12786 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12789 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 13195 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 13198 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13783 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13652 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13664 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13666 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13713 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13786 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 13951 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 13954 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14125 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14127 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14235 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14238 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14570 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14573 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16128 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14802 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 14976 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 14979 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15347 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15310 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15347 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15350 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15759 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 15412 + ]._serialized_start = 15661 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 15480 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15513 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 15860 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 15881 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 15971 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 15974 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 16920 + ]._serialized_end = 15729 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15762 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16109 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16130 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16220 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16223 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17169 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 5db25569828b7..253f8a58166a6 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -477,6 +477,21 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["relation", b"relation"] ) -> None: ... + class JsonToDDL(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + JSON_STRING_FIELD_NUMBER: builtins.int + json_string: builtins.str + """(Required) The JSON formatted string to be converted to DDL.""" + def __init__( + self, + *, + json_string: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["json_string", b"json_string"] + ) -> None: ... + SESSION_ID_FIELD_NUMBER: builtins.int CLIENT_OBSERVED_SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int USER_CONTEXT_FIELD_NUMBER: builtins.int @@ -494,6 +509,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): PERSIST_FIELD_NUMBER: builtins.int UNPERSIST_FIELD_NUMBER: builtins.int GET_STORAGE_LEVEL_FIELD_NUMBER: builtins.int + JSON_TO_DDL_FIELD_NUMBER: builtins.int session_id: builtins.str """(Required) @@ -542,6 +558,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): def unpersist(self) -> global___AnalyzePlanRequest.Unpersist: ... @property def get_storage_level(self) -> global___AnalyzePlanRequest.GetStorageLevel: ... + @property + def json_to_ddl(self) -> global___AnalyzePlanRequest.JsonToDDL: ... def __init__( self, *, @@ -562,6 +580,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): persist: global___AnalyzePlanRequest.Persist | None = ..., unpersist: global___AnalyzePlanRequest.Unpersist | None = ..., get_storage_level: global___AnalyzePlanRequest.GetStorageLevel | None = ..., + json_to_ddl: global___AnalyzePlanRequest.JsonToDDL | None = ..., ) -> None: ... def HasField( self, @@ -588,6 +607,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): b"is_local", "is_streaming", b"is_streaming", + "json_to_ddl", + b"json_to_ddl", "persist", b"persist", "same_semantics", @@ -631,6 +652,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): b"is_local", "is_streaming", b"is_streaming", + "json_to_ddl", + b"json_to_ddl", "persist", b"persist", "same_semantics", @@ -680,6 +703,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): "persist", "unpersist", "get_storage_level", + "json_to_ddl", ] | None ): ... @@ -877,6 +901,20 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["storage_level", b"storage_level"] ) -> None: ... + class JsonToDDL(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + DDL_STRING_FIELD_NUMBER: builtins.int + ddl_string: builtins.str + def __init__( + self, + *, + ddl_string: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] + ) -> None: ... + SESSION_ID_FIELD_NUMBER: builtins.int SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int SCHEMA_FIELD_NUMBER: builtins.int @@ -892,6 +930,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): PERSIST_FIELD_NUMBER: builtins.int UNPERSIST_FIELD_NUMBER: builtins.int GET_STORAGE_LEVEL_FIELD_NUMBER: builtins.int + JSON_TO_DDL_FIELD_NUMBER: builtins.int session_id: builtins.str server_side_session_id: builtins.str """Server-side generated idempotency key that the client can use to assert that the server side @@ -923,6 +962,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): def unpersist(self) -> global___AnalyzePlanResponse.Unpersist: ... @property def get_storage_level(self) -> global___AnalyzePlanResponse.GetStorageLevel: ... + @property + def json_to_ddl(self) -> global___AnalyzePlanResponse.JsonToDDL: ... def __init__( self, *, @@ -941,6 +982,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): persist: global___AnalyzePlanResponse.Persist | None = ..., unpersist: global___AnalyzePlanResponse.Unpersist | None = ..., get_storage_level: global___AnalyzePlanResponse.GetStorageLevel | None = ..., + json_to_ddl: global___AnalyzePlanResponse.JsonToDDL | None = ..., ) -> None: ... def HasField( self, @@ -957,6 +999,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): b"is_local", "is_streaming", b"is_streaming", + "json_to_ddl", + b"json_to_ddl", "persist", b"persist", "result", @@ -990,6 +1034,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): b"is_local", "is_streaming", b"is_streaming", + "json_to_ddl", + b"json_to_ddl", "persist", b"persist", "result", @@ -1029,6 +1075,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): "persist", "unpersist", "get_storage_level", + "json_to_ddl", ] | None ): ... diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index dcc383b7add5a..432ddd083c802 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -2240,6 +2240,22 @@ def test_variant_type(self): PySparkValueError, lambda: str(VariantVal(bytes([32, 10, 1, 0, 0, 0]), metadata)) ) + def test_to_ddl(self): + schema = StructType().add("a", NullType()).add("b", BooleanType()).add("c", BinaryType()) + self.assertEqual(schema.toDDL(), "a VOID,b BOOLEAN,c BINARY") + + schema = StructType().add("a", IntegerType()).add("b", StringType()) + self.assertEqual(schema.toDDL(), "a INT,b STRING") + + schema = StructType().add("a", FloatType()).add("b", LongType(), False) + self.assertEqual(schema.toDDL(), "a FLOAT,b BIGINT NOT NULL") + + schema = StructType().add("a", ArrayType(DoubleType()), False).add("b", DateType()) + self.assertEqual(schema.toDDL(), "a ARRAY NOT NULL,b DATE") + + schema = StructType().add("a", TimestampType()).add("b", TimestampNTZType()) + self.assertEqual(schema.toDDL(), "a TIMESTAMP,b TIMESTAMP_NTZ") + def test_from_ddl(self): self.assertEqual(DataType.fromDDL("long"), LongType()) self.assertEqual( diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 03227c8c8760f..303c0c3c8bd07 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1555,6 +1555,25 @@ def treeString(self, maxDepth: int = JVM_INT_MAX) -> str: field._build_formatted_string(prefix, stringConcat, depth) return stringConcat.toString() + def toDDL(self) -> str: + from pyspark.sql.utils import is_remote + + if is_remote(): + from pyspark.sql.connect.session import SparkSession + + session = SparkSession.getActiveSession() + assert session is not None + return session._client._analyze( # type: ignore[return-value] + method="json_to_ddl", json_string=self.json() + ).ddl_string + + else: + from py4j.java_gateway import JVMView + + sc = get_active_spark_context() + assert sc._jvm is not None + return cast(JVMView, sc._jvm).PythonSQLUtils.jsonToDDL(self.json()) + class VariantType(AtomicType): """ diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index e3c84ddd5e887..e27049d2114d7 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -94,6 +94,7 @@ message AnalyzePlanRequest { Persist persist = 14; Unpersist unpersist = 15; GetStorageLevel get_storage_level = 16; + JsonToDDL json_to_ddl = 18; } message Schema { @@ -199,6 +200,11 @@ message AnalyzePlanRequest { // (Required) The logical plan to get the storage level. Relation relation = 1; } + + message JsonToDDL { + // (Required) The JSON formatted string to be converted to DDL. + string json_string = 1; + } } // Response to performing analysis of the query. Contains relevant metadata to be able to @@ -224,6 +230,7 @@ message AnalyzePlanResponse { Persist persist = 12; Unpersist unpersist = 13; GetStorageLevel get_storage_level = 14; + JsonToDDL json_to_ddl = 16; } message Schema { @@ -275,6 +282,10 @@ message AnalyzePlanResponse { // (Required) The StorageLevel as a result of get_storage_level request. StorageLevel storage_level = 1; } + + message JsonToDDL { + string ddl_string = 1; + } } // A request to be executed by the service. diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala index 6c5d95ac67d3d..8ca021c5be39e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.Dataset import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, InvalidPlanInput, StorageLevelProtoConverter} import org.apache.spark.sql.connect.planner.SparkConnectPlanner import org.apache.spark.sql.execution.{CodegenMode, CostMode, ExtendedMode, FormattedMode, SimpleMode} +import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.ArrayImplicits._ private[connect] class SparkConnectAnalyzeHandler( @@ -206,6 +207,17 @@ private[connect] class SparkConnectAnalyzeHandler( .setStorageLevel(StorageLevelProtoConverter.toConnectProtoType(storageLevel)) .build()) + case proto.AnalyzePlanRequest.AnalyzeCase.JSON_TO_DDL => + val ddl = DataType + .fromJson(request.getJsonToDdl.getJsonString) + .asInstanceOf[StructType] + .toDDL + builder.setJsonToDdl( + proto.AnalyzePlanResponse.JsonToDDL + .newBuilder() + .setDdlString(ddl) + .build()) + case other => throw InvalidPlanInput(s"Unknown Analyze Method $other!") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index da03293ce743b..e33fe38b160af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -144,6 +144,10 @@ private[sql] object PythonSQLUtils extends Logging { } } + def jsonToDDL(json: String): String = { + DataType.fromJson(json).asInstanceOf[StructType].toDDL + } + def unresolvedNamedLambdaVariable(name: String): Column = Column(internal.UnresolvedNamedLambdaVariable.apply(name)) From d84b2d4565c5e29c912de4e86d6960fff49ffbd2 Mon Sep 17 00:00:00 2001 From: bogao007 Date: Thu, 12 Dec 2024 12:04:35 +0900 Subject: [PATCH 159/438] [SPARK-50428][SS][PYTHON] Support TransformWithStateInPandas in batch queries ### What changes were proposed in this pull request? Support TransformWithStateInPandas in batch queries. ### Why are the changes needed? Bring parity as Scala. Scala batch support for TransformWithState is done in https://issues.apache.org/jira/browse/SPARK-46865. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Added new unit test cases. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49113 from bogao007/tws-batch. Authored-by: bogao007 Signed-off-by: Jungtaek Lim --- .../test_pandas_transform_with_state.py | 48 +++++ .../spark/sql/execution/SparkStrategies.scala | 21 +- .../TransformWithStateInPandasExec.scala | 181 ++++++++++++++---- 3 files changed, 207 insertions(+), 43 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 60f2c9348db3f..15089f2cb0d6d 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -859,6 +859,54 @@ def test_transform_with_state_with_timers_single_partition(self): self.test_transform_with_state_in_pandas_event_time() self.test_transform_with_state_in_pandas_proc_timer() + def test_transform_with_state_in_pandas_batch_query(self): + data = [("0", 123), ("0", 46), ("1", 146), ("1", 346)] + df = self.spark.createDataFrame(data, "id string, temperature int") + + output_schema = StructType( + [ + StructField("id", StringType(), True), + StructField("countAsString", StringType(), True), + ] + ) + batch_result = df.groupBy("id").transformWithStateInPandas( + statefulProcessor=MapStateProcessor(), + outputStructType=output_schema, + outputMode="Update", + timeMode="None", + ) + assert set(batch_result.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } + + def test_transform_with_state_in_pandas_batch_query_initial_state(self): + data = [("0", 123), ("0", 46), ("1", 146), ("1", 346)] + df = self.spark.createDataFrame(data, "id string, temperature int") + + init_data = [("0", 789), ("3", 987)] + initial_state = self.spark.createDataFrame(init_data, "id string, initVal int").groupBy( + "id" + ) + + output_schema = StructType( + [ + StructField("id", StringType(), True), + StructField("value", StringType(), True), + ] + ) + batch_result = df.groupBy("id").transformWithStateInPandas( + statefulProcessor=SimpleStatefulProcessorWithInitialState(), + outputStructType=output_schema, + outputMode="Update", + timeMode="None", + initialState=initial_state, + ) + assert set(batch_result.sort("id").collect()) == { + Row(id="0", value=str(789 + 123 + 46)), + Row(id="1", value=str(146 + 346)), + } + class SimpleStatefulProcessorWithInitialState(StatefulProcessor): # this dict is the same as input initial state dataframe diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index e77c050fe8872..36e25773f8342 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.{SparkException, SparkUnsupportedOperationException} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{execution, AnalysisException, Strategy} -import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, InternalRow} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper, NormalizeFloatingNumbers} @@ -794,8 +794,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object TransformWithStateInPandasStrategy extends Strategy { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case t @ TransformWithStateInPandas( - func, _, outputAttrs, outputMode, timeMode, child, - hasInitialState, initialState, _, initialStateSchema) => + func, _, outputAttrs, outputMode, timeMode, child, + hasInitialState, initialState, _, initialStateSchema) => val execPlan = TransformWithStateInPandasExec( func, t.leftAttributes, outputAttrs, outputMode, timeMode, stateInfo = None, @@ -803,6 +803,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { eventTimeWatermarkForLateEvents = None, eventTimeWatermarkForEviction = None, planLater(child), + isStreaming = true, hasInitialState, planLater(initialState), t.rightAttributes, @@ -967,18 +968,16 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { keyEncoder, outputObjAttr, planLater(child), hasInitialState, initialStateGroupingAttrs, initialStateDataAttrs, initialStateDeserializer, planLater(initialState)) :: Nil + case t @ TransformWithStateInPandas( + func, _, outputAttrs, outputMode, timeMode, child, + hasInitialState, initialState, _, initialStateSchema) => + TransformWithStateInPandasExec.generateSparkPlanForBatchQueries(func, + t.leftAttributes, outputAttrs, outputMode, timeMode, planLater(child), hasInitialState, + planLater(initialState), t.rightAttributes, initialStateSchema) :: Nil case _: FlatMapGroupsInPandasWithState => // TODO(SPARK-40443): support applyInPandasWithState in batch query throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3176") - case t: TransformWithStateInPandas => - // TODO(SPARK-50428): support TransformWithStateInPandas in batch query - throw new ExtendedAnalysisException( - new AnalysisException( - "_LEGACY_ERROR_TEMP_3102", - Map( - "msg" -> "TransformWithStateInPandas is not supported with batch DataFrames/Datasets") - ), plan = t) case logical.CoGroup( f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, lOrder, rOrder, oAttr, left, right) => execution.CoGroupExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala index 617c20c3a782a..f8e9f11f4d738 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala @@ -16,12 +16,15 @@ */ package org.apache.spark.sql.execution.python +import java.util.UUID + import scala.concurrent.duration.NANOSECONDS import org.apache.hadoop.conf.Configuration import org.apache.spark.JobArtifactSet import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -34,10 +37,11 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.PandasGroupUtils.{executePython, groupAndProject, resolveArgOffsets} import org.apache.spark.sql.execution.streaming.{StatefulOperatorCustomMetric, StatefulOperatorCustomSumMetric, StatefulOperatorPartitioning, StatefulOperatorStateInfo, StatefulProcessorHandleImpl, StateStoreWriter, WatermarkSupport} import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper -import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateSchemaValidationResult, StateStore, StateStoreConf, StateStoreId, StateStoreOps, StateStoreProviderId} +import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, RocksDBStateStoreProvider, StateSchemaValidationResult, StateStore, StateStoreConf, StateStoreId, StateStoreOps, StateStoreProvider, StateStoreProviderId} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, TimeMode} import org.apache.spark.sql.types.{BinaryType, StructField, StructType} -import org.apache.spark.util.{CompletionIterator, SerializableConfiguration} +import org.apache.spark.util.{CompletionIterator, SerializableConfiguration, Utils} /** * Physical operator for executing @@ -53,8 +57,11 @@ import org.apache.spark.util.{CompletionIterator, SerializableConfiguration} * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events * @param eventTimeWatermarkForEviction event time watermark for state eviction * @param child the physical plan for the underlying data + * @param isStreaming defines whether the query is streaming or batch + * @param hasInitialState defines whether the query has initial state * @param initialState the physical plan for the input initial state * @param initialStateGroupingAttrs grouping attributes for initial state + * @param initialStateSchema schema for initial state */ case class TransformWithStateInPandasExec( functionExpr: Expression, @@ -67,6 +74,7 @@ case class TransformWithStateInPandasExec( eventTimeWatermarkForLateEvents: Option[Long], eventTimeWatermarkForEviction: Option[Long], child: SparkPlan, + isStreaming: Boolean = true, hasInitialState: Boolean, initialState: SparkPlan, initialStateGroupingAttrs: Seq[Attribute], @@ -190,18 +198,32 @@ case class TransformWithStateInPandasExec( metrics if (!hasInitialState) { - child.execute().mapPartitionsWithStateStore[InternalRow]( - getStateInfo, - schemaForKeyRow, - schemaForValueRow, - NoPrefixKeyStateEncoderSpec(schemaForKeyRow), - session.sqlContext.sessionState, - Some(session.sqlContext.streams.stateStoreCoordinator), - useColumnFamilies = true, - useMultipleValuesPerKey = true - ) { - case (store: StateStore, dataIterator: Iterator[InternalRow]) => - processDataWithPartition(store, dataIterator) + if (isStreaming) { + child.execute().mapPartitionsWithStateStore[InternalRow]( + getStateInfo, + schemaForKeyRow, + schemaForValueRow, + NoPrefixKeyStateEncoderSpec(schemaForKeyRow), + session.sqlContext.sessionState, + Some(session.sqlContext.streams.stateStoreCoordinator), + useColumnFamilies = true, + useMultipleValuesPerKey = true + ) { + case (store: StateStore, dataIterator: Iterator[InternalRow]) => + processDataWithPartition(store, dataIterator) + } + } else { + // If the query is running in batch mode, we need to create a new StateStore and instantiate + // a temp directory on the executors in mapPartitionsWithIndex. + val hadoopConfBroadcast = sparkContext.broadcast( + new SerializableConfiguration(session.sessionState.newHadoopConf())) + child.execute().mapPartitionsWithIndex[InternalRow]( + (partitionId: Int, dataIterator: Iterator[InternalRow]) => { + initNewStateStoreAndProcessData(partitionId, hadoopConfBroadcast) { store => + processDataWithPartition(store, dataIterator) + } + } + ) } } else { val storeConf = new StateStoreConf(session.sqlContext.sessionState.conf) @@ -216,25 +238,71 @@ case class TransformWithStateInPandasExec( // The state store aware zip partitions will provide us with two iterators, // child data iterator and the initial state iterator per partition. case (partitionId, childDataIterator, initStateIterator) => - val stateStoreId = StateStoreId(stateInfo.get.checkpointLocation, - stateInfo.get.operatorId, partitionId) - val storeProviderId = StateStoreProviderId(stateStoreId, stateInfo.get.queryRunId) - val store = StateStore.get( - storeProviderId = storeProviderId, - keySchema = schemaForKeyRow, - valueSchema = schemaForValueRow, - NoPrefixKeyStateEncoderSpec(schemaForKeyRow), - version = stateInfo.get.storeVersion, - stateStoreCkptId = stateInfo.get.getStateStoreCkptId(partitionId).map(_.head), - useColumnFamilies = true, - storeConf = storeConf, - hadoopConf = hadoopConfBroadcast.value.value - ) - processDataWithPartition(store, childDataIterator, initStateIterator) + if (isStreaming) { + val stateStoreId = StateStoreId(stateInfo.get.checkpointLocation, + stateInfo.get.operatorId, partitionId) + val storeProviderId = StateStoreProviderId(stateStoreId, stateInfo.get.queryRunId) + val store = StateStore.get( + storeProviderId = storeProviderId, + keySchema = schemaForKeyRow, + valueSchema = schemaForValueRow, + NoPrefixKeyStateEncoderSpec(schemaForKeyRow), + version = stateInfo.get.storeVersion, + stateStoreCkptId = stateInfo.get.getStateStoreCkptId(partitionId).map(_.head), + useColumnFamilies = true, + storeConf = storeConf, + hadoopConf = hadoopConfBroadcast.value.value + ) + processDataWithPartition(store, childDataIterator, initStateIterator) + } else { + initNewStateStoreAndProcessData(partitionId, hadoopConfBroadcast) { store => + processDataWithPartition(store, childDataIterator, initStateIterator) + } + } } } } + /** + * Create a new StateStore for given partitionId and instantiate a temp directory + * on the executors. Process data and close the stateStore provider afterwards. + */ + private def initNewStateStoreAndProcessData( + partitionId: Int, + hadoopConfBroadcast: Broadcast[SerializableConfiguration]) + (f: StateStore => Iterator[InternalRow]): Iterator[InternalRow] = { + + val providerId = { + val tempDirPath = Utils.createTempDir().getAbsolutePath + new StateStoreProviderId( + StateStoreId(tempDirPath, 0, partitionId), getStateInfo.queryRunId) + } + + val sqlConf = new SQLConf() + sqlConf.setConfString(SQLConf.STATE_STORE_PROVIDER_CLASS.key, + classOf[RocksDBStateStoreProvider].getName) + val storeConf = new StateStoreConf(sqlConf) + + // Create StateStoreProvider for this partition + val stateStoreProvider = StateStoreProvider.createAndInit( + providerId, + schemaForKeyRow, + schemaForValueRow, + NoPrefixKeyStateEncoderSpec(schemaForKeyRow), + useColumnFamilies = true, + storeConf = storeConf, + hadoopConf = hadoopConfBroadcast.value.value, + useMultipleValuesPerKey = true) + + val store = stateStoreProvider.getStore(0, None) + val outputIterator = f(store) + CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator.iterator, { + stateStoreProvider.close() + }).map { row => + row + } + } + private def processDataWithPartition( store: StateStore, dataIterator: Iterator[InternalRow], @@ -259,7 +327,7 @@ case class TransformWithStateInPandasExec( val data = groupAndProject(filteredIter, groupingAttributes, child.output, dedupAttributes) val processorHandle = new StatefulProcessorHandleImpl(store, getStateInfo.queryRunId, - groupingKeyExprEncoder, timeMode, isStreaming = true, batchTimestampMs, metrics) + groupingKeyExprEncoder, timeMode, isStreaming, batchTimestampMs, metrics) val outputIterator = if (!hasInitialState) { val runner = new TransformWithStateInPandasPythonRunner( @@ -311,8 +379,12 @@ case class TransformWithStateInPandasExec( // by the upstream (consumer) operators in addition to the processing in this operator. allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs) commitTimeMs += timeTakenMs { - processorHandle.doTtlCleanup() - store.commit() + if (isStreaming) { + processorHandle.doTtlCleanup() + store.commit() + } else { + store.abort() + } } setStoreMetrics(store) setOperatorMetrics() @@ -334,3 +406,48 @@ case class TransformWithStateInPandasExec( override def right: SparkPlan = initialState } + +// scalastyle:off argcount +object TransformWithStateInPandasExec { + + // Plan logical transformWithStateInPandas for batch queries + def generateSparkPlanForBatchQueries( + functionExpr: Expression, + groupingAttributes: Seq[Attribute], + output: Seq[Attribute], + outputMode: OutputMode, + timeMode: TimeMode, + child: SparkPlan, + hasInitialState: Boolean = false, + initialState: SparkPlan, + initialStateGroupingAttrs: Seq[Attribute], + initialStateSchema: StructType): SparkPlan = { + val shufflePartitions = child.session.sessionState.conf.numShufflePartitions + val statefulOperatorStateInfo = StatefulOperatorStateInfo( + checkpointLocation = "", // empty checkpointLocation will be populated in doExecute + queryRunId = UUID.randomUUID(), + operatorId = 0, + storeVersion = 0, + numPartitions = shufflePartitions, + stateStoreCkptIds = None + ) + + new TransformWithStateInPandasExec( + functionExpr, + groupingAttributes, + output, + outputMode, + timeMode, + Some(statefulOperatorStateInfo), + Some(System.currentTimeMillis), + None, + None, + child, + isStreaming = false, + hasInitialState, + initialState, + initialStateGroupingAttrs, + initialStateSchema) + } +} +// scalastyle:on argcount From 032623f62c071d3b0f2879f1951cbf4a6b3e55d3 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 12 Dec 2024 16:29:47 +0900 Subject: [PATCH 160/438] [SPARK-50553][CONNECT] Throw `InvalidPlanInput` for invalid plan message ### What changes were proposed in this pull request? Throw `InvalidPlanInput` for invalid plan message ### Why are the changes needed? Should throw `InvalidPlanInput` for invalid plan message, instead of `AssertionError` ### Does this PR introduce _any_ user-facing change? error message improvement: `AssertionError -> InvalidPlanInput` ### How was this patch tested? updated tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49161 from zhengruifeng/assert_plan. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .../connect/planner/SparkConnectPlanner.scala | 42 ++++++++++--------- .../planner/SparkConnectPlannerSuite.scala | 2 +- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index ec67c57a37f57..82dfcf7a3694e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -558,7 +558,7 @@ class SparkConnectPlanner( private def transformToSchema(rel: proto.ToSchema): LogicalPlan = { val schema = transformDataType(rel.getSchema) - assert(schema.isInstanceOf[StructType]) + assertPlan(schema.isInstanceOf[StructType]) Dataset .ofRows(session, transformRelation(rel.getInput)) @@ -876,7 +876,7 @@ class SparkConnectPlanner( logicalPlan: LogicalPlan, groupingExprs: java.util.List[proto.Expression], sortOrder: Seq[SortOrder]): UntypedKeyValueGroupedDataset = { - assert(groupingExprs.size() >= 1) + assertPlan(groupingExprs.size() >= 1) val dummyFunc = TypedScalaUdf(groupingExprs.get(0), None) val groupExprs = groupingExprs.asScala.toSeq.drop(1).map(expr => transformExpression(expr)) @@ -896,7 +896,7 @@ class SparkConnectPlanner( logicalPlan: LogicalPlan, groupingExprs: java.util.List[proto.Expression], sortOrder: Seq[SortOrder]): UntypedKeyValueGroupedDataset = { - assert(groupingExprs.size() == 1) + assertPlan(groupingExprs.size() == 1) val groupFunc = TypedScalaUdf(groupingExprs.get(0), Some(logicalPlan.output)) val vEnc = groupFunc.inEnc val kEnc = groupFunc.outEnc @@ -952,7 +952,7 @@ class SparkConnectPlanner( // Most typed API takes one UDF input. // For the few that takes more than one inputs, e.g. grouping function mapping UDFs, // the first input which is the key of the grouping function. - assert(udf.inputEncoders.nonEmpty) + assertPlan(udf.inputEncoders.nonEmpty) val inEnc = udf.inputEncoders.head // single input encoder or key encoder TypedScalaUdf(udf.function, udf.outputEncoder, inEnc, inputAttrs) } @@ -1431,7 +1431,7 @@ class SparkConnectPlanner( } private def transformFilter(rel: proto.Filter): LogicalPlan = { - assert(rel.hasInput) + assertPlan(rel.hasInput) val baseRel = transformRelation(rel.getInput) val cond = rel.getCondition if (isTypedScalaUdfExpr(cond)) { @@ -1754,7 +1754,7 @@ class SparkConnectPlanner( val udf = fun.getScalarScalaUdf val udfPacket = unpackUdf(fun) if (udf.getAggregate) { - assert(udfPacket.inputEncoders.size == 1, "UDAF should have exactly one input encoder") + assertPlan(udfPacket.inputEncoders.size == 1, "UDAF should have exactly one input encoder") UserDefinedAggregator( aggregator = udfPacket.function.asInstanceOf[Aggregator[Any, Any, Any]], inputEncoder = ExpressionEncoder(udfPacket.inputEncoders.head), @@ -2072,7 +2072,7 @@ class SparkConnectPlanner( } private def transformJoin(rel: proto.Join): LogicalPlan = { - assert(rel.hasLeft && rel.hasRight, "Both join sides must be present") + assertPlan(rel.hasLeft && rel.hasRight, "Both join sides must be present") if (rel.hasJoinCondition && rel.getUsingColumnsCount > 0) { throw InvalidPlanInput( s"Using columns or join conditions cannot be set at the same time in Join") @@ -2144,7 +2144,7 @@ class SparkConnectPlanner( } private def transformLateralJoin(rel: proto.LateralJoin): LogicalPlan = { - assert(rel.hasLeft && rel.hasRight, "Both join sides must be present") + assertPlan(rel.hasLeft && rel.hasRight, "Both join sides must be present") val joinCondition = if (rel.hasJoinCondition) Some(transformExpression(rel.getJoinCondition)) else None val joinType = transformJoinType( @@ -2157,7 +2157,7 @@ class SparkConnectPlanner( } private def transformSort(sort: proto.Sort): LogicalPlan = { - assert(sort.getOrderCount > 0, "'order' must be present and contain elements.") + assertPlan(sort.getOrderCount > 0, "'order' must be present and contain elements.") logical.Sort( child = transformRelation(sort.getInput), global = sort.getIsGlobal, @@ -2287,10 +2287,8 @@ class SparkConnectPlanner( private def transformTypedReduceExpression( fun: proto.Expression.UnresolvedFunction, dataAttributes: Seq[Attribute]): Expression = { - assert(fun.getFunctionName == "reduce") - if (fun.getArgumentsCount != 1) { - throw InvalidPlanInput("reduce requires single child expression") - } + assertPlan(fun.getFunctionName == "reduce") + assertPlan(fun.getArgumentsCount == 1, "reduce requires single child expression") val udf = fun.getArgumentsList.asScala match { case collection.Seq(e) if e.hasCommonInlineUserDefinedFunction && @@ -2320,10 +2318,10 @@ class SparkConnectPlanner( expr: proto.TypedAggregateExpression, baseRelationOpt: Option[LogicalPlan]): AggregateExpression = { val udf = expr.getScalarScalaUdf - assert(udf.getAggregate) + assertPlan(udf.getAggregate) val udfPacket = unpackScalaUDF[UdfPacket](udf) - assert(udfPacket.inputEncoders.size == 1, "UDAF should have exactly one input encoder") + assertPlan(udfPacket.inputEncoders.size == 1, "UDAF should have exactly one input encoder") val aggregator = udfPacket.function.asInstanceOf[Aggregator[Any, Any, Any]] val tae = @@ -2356,17 +2354,17 @@ class SparkConnectPlanner( }.toSeq action.getActionType match { case proto.MergeAction.ActionType.ACTION_TYPE_DELETE => - assert(assignments.isEmpty, "Delete action should not have assignment.") + assertPlan(assignments.isEmpty, "Delete action should not have assignment.") DeleteAction(condition) case proto.MergeAction.ActionType.ACTION_TYPE_INSERT => InsertAction(condition, assignments) case proto.MergeAction.ActionType.ACTION_TYPE_INSERT_STAR => - assert(assignments.isEmpty, "InsertStar action should not have assignment.") + assertPlan(assignments.isEmpty, "InsertStar action should not have assignment.") InsertStarAction(condition) case proto.MergeAction.ActionType.ACTION_TYPE_UPDATE => UpdateAction(condition, assignments) case proto.MergeAction.ActionType.ACTION_TYPE_UPDATE_STAR => - assert(assignments.isEmpty, "UpdateStar action should not have assignment.") + assertPlan(assignments.isEmpty, "UpdateStar action should not have assignment.") UpdateStarAction(condition) case _ => throw InvalidPlanInput(s"Unsupported merge action type ${action.getActionType}.") @@ -3578,7 +3576,7 @@ class SparkConnectPlanner( getCreateExternalTable: proto.CreateExternalTable): LogicalPlan = { val schema = if (getCreateExternalTable.hasSchema) { val struct = transformDataType(getCreateExternalTable.getSchema) - assert(struct.isInstanceOf[StructType]) + assertPlan(struct.isInstanceOf[StructType]) struct.asInstanceOf[StructType] } else { new StructType @@ -3608,7 +3606,7 @@ class SparkConnectPlanner( private def transformCreateTable(getCreateTable: proto.CreateTable): LogicalPlan = { val schema = if (getCreateTable.hasSchema) { val struct = transformDataType(getCreateTable.getSchema) - assert(struct.isInstanceOf[StructType]) + assertPlan(struct.isInstanceOf[StructType]) struct.asInstanceOf[StructType] } else { new StructType @@ -3724,4 +3722,8 @@ class SparkConnectPlanner( private def transformLazyExpression(getLazyExpression: proto.LazyExpression): Expression = { LazyExpression(transformExpression(getLazyExpression.getChild)) } + + private def assertPlan(assertion: Boolean, message: String = ""): Unit = { + if (!assertion) throw InvalidPlanInput(message) + } } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala index e44d3eacc66df..84d5fd68d4c73 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala @@ -274,7 +274,7 @@ class SparkConnectPlannerSuite extends SparkFunSuite with SparkConnectPlanTest { test("Simple Join") { val incompleteJoin = proto.Relation.newBuilder.setJoin(proto.Join.newBuilder.setLeft(readRel)).build() - intercept[AssertionError](transform(incompleteJoin)) + intercept[InvalidPlanInput](transform(incompleteJoin)) // Join type JOIN_TYPE_UNSPECIFIED is not supported. intercept[InvalidPlanInput] { From e4be5e67fefc73b600954352d10c1e1bce5e35d9 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 12 Dec 2024 16:09:18 +0800 Subject: [PATCH 161/438] [SPARK-50552][INFRA] Add a separate docker file for PyPy 3.10 daily build ### What changes were proposed in this pull request? Add a separate docker file for PyPy 3.10 daily build ### Why are the changes needed? to isolate the environments ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with `env`: ``` default: '{"PYSPARK_IMAGE_TO_TEST": "pypy-310", "PYTHON_TO_TEST": "pypy3"}' ``` https://github.com/zhengruifeng/spark/runs/34293616750 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49160 from zhengruifeng/py_image_pypy_310. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../workflows/build_infra_images_cache.yml | 14 ++++ .github/workflows/build_python_pypy3.10.yml | 1 + dev/spark-test-image/pypy-310/Dockerfile | 71 +++++++++++++++++++ 3 files changed, 86 insertions(+) create mode 100644 dev/spark-test-image/pypy-310/Dockerfile diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index adccc64261f55..df927da57c240 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -30,6 +30,7 @@ on: - 'dev/spark-test-image/docs/Dockerfile' - 'dev/spark-test-image/lint/Dockerfile' - 'dev/spark-test-image/sparkr/Dockerfile' + - 'dev/spark-test-image/pypy-310/Dockerfile' - 'dev/spark-test-image/python-309/Dockerfile' - 'dev/spark-test-image/python-310/Dockerfile' - 'dev/spark-test-image/python-312/Dockerfile' @@ -106,6 +107,19 @@ jobs: - name: Image digest (SparkR) if: hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' run: echo ${{ steps.docker_build_sparkr.outputs.digest }} + - name: Build and push (PySpark with PyPy 3.10) + if: hashFiles('dev/spark-test-image/pypy-310/Dockerfile') != '' + id: docker_build_pyspark_pypy_310 + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/pypy-310/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-pypy-310-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-pypy-310-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-pypy-310-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark with PyPy 3.10) + if: hashFiles('dev/spark-test-image/pypy-310/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_pypy_310.outputs.digest }} - name: Build and push (PySpark with Python 3.9) if: hashFiles('dev/spark-test-image/python-309/Dockerfile') != '' id: docker_build_pyspark_python_309 diff --git a/.github/workflows/build_python_pypy3.10.yml b/.github/workflows/build_python_pypy3.10.yml index 163af2f4aec8b..c590427f6f49e 100644 --- a/.github/workflows/build_python_pypy3.10.yml +++ b/.github/workflows/build_python_pypy3.10.yml @@ -36,6 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "pypy-310", "PYTHON_TO_TEST": "pypy3" } jobs: >- diff --git a/dev/spark-test-image/pypy-310/Dockerfile b/dev/spark-test-image/pypy-310/Dockerfile new file mode 100644 index 0000000000000..7d6ff26566de3 --- /dev/null +++ b/dev/spark-test-image/pypy-310/Dockerfile @@ -0,0 +1,71 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark with PyPy 3.10" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE 20241212 + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pkg-config \ + qpdf \ + tzdata \ + software-properties-common \ + wget \ + zlib1g-dev \ + && apt-get autoremove --purge -y \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +RUN add-apt-repository ppa:pypy/ppa +RUN mkdir -p /usr/local/pypy/pypy3.10 && \ + curl -sqL https://downloads.python.org/pypy/pypy3.10-v7.3.17-linux64.tar.bz2 | tar xjf - -C /usr/local/pypy/pypy3.10 --strip-components=1 && \ + ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3.10 && \ + ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3 +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 +RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.3' scipy coverage matplotlib lxml From b5d195a5cd1b027d6cf1d7b992a6ef6c2f18520f Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Thu, 12 Dec 2024 16:41:30 +0300 Subject: [PATCH 162/438] [SPARK-48344][SQL] Add Frames and Scopes to support Exception Handlers and Local Variables ### What changes were proposed in this pull request? This PR is **third** in series of refactoring and introducing SQL Scripting Execution Framework: - Introducing `SqlScriptingExecutionContext`, object to keep current state of script execution. - Introducing `Frames` and `Scopes` to support Local Variables and Error Handlers resolution. - Decoupled `SqlScriptingIterator` from `SqlScriptExecution`. - Enabling execution of SQL Scripting using `sql()` API. - Updated `SqlScriptingExecutionNodeSuite` so tests remain independent of concept of Frames and Scopes. First [PR](https://github.com/apache/spark/pull/48879) Second [PR](https://github.com/apache/spark/pull/48950) ### Why are the changes needed? This changes are needed to enable introduction of Error Handling mechanism and Local Variables. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests that were updated to support new concepts introduced in this PR. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49006 from miland-db/milan-dankovic_data/refactor-execution-3. Lead-authored-by: Milan Dankovic Co-authored-by: David Milicevic Signed-off-by: Max Gekk --- .../sql/catalyst/parser/AstBuilder.scala | 80 +++- .../logical/SqlScriptingLogicalPlans.scala | 5 +- .../sql/scripting/SqlScriptingExecution.scala | 44 ++- .../SqlScriptingExecutionContext.scala | 91 +++++ .../scripting/SqlScriptingExecutionNode.scala | 86 ++++- .../scripting/SqlScriptingInterpreter.scala | 62 +-- .../SqlScriptingExecutionNodeSuite.scala | 355 ++++++++++-------- .../SqlScriptingInterpreterSuite.scala | 10 +- 8 files changed, 508 insertions(+), 225 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 47139810528df..64491264f3e9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -144,16 +144,27 @@ class AstBuilder extends DataTypeAstBuilder override def visitSingleCompoundStatement(ctx: SingleCompoundStatementContext): CompoundBody = { val labelCtx = new SqlScriptingLabelContext() - Option(ctx.compoundBody()) - .map(visitCompoundBodyImpl(_, None, allowVarDeclare = true, labelCtx)) - .getOrElse(CompoundBody(Seq.empty, None)) + val labelText = labelCtx.enterLabeledScope(None, None) + + val script = Option(ctx.compoundBody()) + .map(visitCompoundBodyImpl( + _, + Some(labelText), + allowVarDeclare = true, + labelCtx, + isScope = true + )).getOrElse(CompoundBody(Seq.empty, Some(labelText), isScope = true)) + + labelCtx.exitLabeledScope(None) + script } private def visitCompoundBodyImpl( ctx: CompoundBodyContext, label: Option[String], allowVarDeclare: Boolean, - labelCtx: SqlScriptingLabelContext): CompoundBody = { + labelCtx: SqlScriptingLabelContext, + isScope: Boolean): CompoundBody = { val buff = ListBuffer[CompoundPlanStatement]() ctx.compoundStatements.forEach( compoundStatement => buff += visitCompoundStatementImpl(compoundStatement, labelCtx)) @@ -185,7 +196,7 @@ class AstBuilder extends DataTypeAstBuilder case _ => } - CompoundBody(buff.toSeq, label) + CompoundBody(buff.toSeq, label, isScope) } private def visitBeginEndCompoundBlockImpl( @@ -194,8 +205,13 @@ class AstBuilder extends DataTypeAstBuilder val labelText = labelCtx.enterLabeledScope(Option(ctx.beginLabel()), Option(ctx.endLabel())) val body = Option(ctx.compoundBody()) - .map(visitCompoundBodyImpl(_, Some(labelText), allowVarDeclare = true, labelCtx)) - .getOrElse(CompoundBody(Seq.empty, Some(labelText))) + .map(visitCompoundBodyImpl( + _, + Some(labelText), + allowVarDeclare = true, + labelCtx, + isScope = true + )).getOrElse(CompoundBody(Seq.empty, Some(labelText), isScope = true)) labelCtx.exitLabeledScope(Option(ctx.beginLabel())) body } @@ -246,10 +262,12 @@ class AstBuilder extends DataTypeAstBuilder OneRowRelation())) }), conditionalBodies = ctx.conditionalBodies.asScala.toList.map( - body => visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx) + body => + visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx, isScope = false) ), elseBody = Option(ctx.elseBody).map( - body => visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx) + body => + visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx, isScope = false) ) ) } @@ -266,7 +284,13 @@ class AstBuilder extends DataTypeAstBuilder Project( Seq(Alias(expression(boolExpr), "condition")()), OneRowRelation()))} - val body = visitCompoundBodyImpl(ctx.compoundBody(), None, allowVarDeclare = false, labelCtx) + val body = visitCompoundBodyImpl( + ctx.compoundBody(), + None, + allowVarDeclare = false, + labelCtx, + isScope = false + ) labelCtx.exitLabeledScope(Option(ctx.beginLabel())) WhileStatement(condition, body, Some(labelText)) @@ -283,7 +307,8 @@ class AstBuilder extends DataTypeAstBuilder }) val conditionalBodies = ctx.conditionalBodies.asScala.toList.map( - body => visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx) + body => + visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx, isScope = false) ) if (conditions.length != conditionalBodies.length) { @@ -296,7 +321,8 @@ class AstBuilder extends DataTypeAstBuilder conditions = conditions, conditionalBodies = conditionalBodies, elseBody = Option(ctx.elseBody).map( - body => visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx) + body => + visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx, isScope = false) )) } @@ -313,7 +339,8 @@ class AstBuilder extends DataTypeAstBuilder }) val conditionalBodies = ctx.conditionalBodies.asScala.toList.map( - body => visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx) + body => + visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx, isScope = false) ) if (conditions.length != conditionalBodies.length) { @@ -326,7 +353,8 @@ class AstBuilder extends DataTypeAstBuilder conditions = conditions, conditionalBodies = conditionalBodies, elseBody = Option(ctx.elseBody).map( - body => visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx) + body => + visitCompoundBodyImpl(body, None, allowVarDeclare = false, labelCtx, isScope = false) )) } @@ -342,7 +370,13 @@ class AstBuilder extends DataTypeAstBuilder Project( Seq(Alias(expression(boolExpr), "condition")()), OneRowRelation()))} - val body = visitCompoundBodyImpl(ctx.compoundBody(), None, allowVarDeclare = false, labelCtx) + val body = visitCompoundBodyImpl( + ctx.compoundBody(), + None, + allowVarDeclare = false, + labelCtx, + isScope = false + ) labelCtx.exitLabeledScope(Option(ctx.beginLabel())) RepeatStatement(condition, body, Some(labelText)) @@ -358,7 +392,13 @@ class AstBuilder extends DataTypeAstBuilder SingleStatement(visitQuery(queryCtx)) } val varName = Option(ctx.multipartIdentifier()).map(_.getText) - val body = visitCompoundBodyImpl(ctx.compoundBody(), None, allowVarDeclare = false, labelCtx) + val body = visitCompoundBodyImpl( + ctx.compoundBody(), + None, + allowVarDeclare = false, + labelCtx, + isScope = false + ) labelCtx.exitLabeledScope(Option(ctx.beginLabel())) ForStatement(query, varName, body, Some(labelText)) @@ -431,7 +471,13 @@ class AstBuilder extends DataTypeAstBuilder labelCtx: SqlScriptingLabelContext): LoopStatement = { val labelText = labelCtx.enterLabeledScope(Option(ctx.beginLabel()), Option(ctx.endLabel())) - val body = visitCompoundBodyImpl(ctx.compoundBody(), None, allowVarDeclare = false, labelCtx) + val body = visitCompoundBodyImpl( + ctx.compoundBody(), + None, + allowVarDeclare = false, + labelCtx, + isScope = false + ) labelCtx.exitLabeledScope(Option(ctx.beginLabel())) LoopStatement(body, Some(labelText)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala index 4faf1f5d26672..207c586996fd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala @@ -65,13 +65,14 @@ case class SingleStatement(parsedPlan: LogicalPlan) */ case class CompoundBody( collection: Seq[CompoundPlanStatement], - label: Option[String]) extends Command with CompoundPlanStatement { + label: Option[String], + isScope: Boolean) extends Command with CompoundPlanStatement { override def children: Seq[LogicalPlan] = collection override protected def withNewChildrenInternal( newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { - CompoundBody(newChildren.map(_.asInstanceOf[CompoundPlanStatement]), label) + CompoundBody(newChildren.map(_.asInstanceOf[CompoundPlanStatement]), label, isScope) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala index 59252f6229180..71b44cbbd0704 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala @@ -35,27 +35,47 @@ class SqlScriptingExecution( session: SparkSession, args: Map[String, Expression]) extends Iterator[DataFrame] { - // Build the execution plan for the script. - private val executionPlan: Iterator[CompoundStatementExec] = - SqlScriptingInterpreter(session).buildExecutionPlan(sqlScript, args) + private val interpreter = SqlScriptingInterpreter(session) - private var current = getNextResult + // Frames to keep what is being executed. + private val context: SqlScriptingExecutionContext = { + val ctx = new SqlScriptingExecutionContext() + val executionPlan = interpreter.buildExecutionPlan(sqlScript, args, ctx) + // Add frame which represents SQL Script to the context. + ctx.frames.addOne(new SqlScriptingExecutionFrame(executionPlan.getTreeIterator)) + // Enter the scope of the top level compound. + // We don't need to exit this scope explicitly as it will be done automatically + // when the frame is removed during iteration. + executionPlan.enterScope() + ctx + } + + private var current: Option[DataFrame] = getNextResult override def hasNext: Boolean = current.isDefined override def next(): DataFrame = { - if (!hasNext) throw SparkException.internalError("No more elements to iterate through.") - val nextDataFrame = current.get - current = getNextResult - nextDataFrame + current match { + case None => throw SparkException.internalError("No more elements to iterate through.") + case Some(result) => + current = getNextResult + result + } + } + + /** Helper method to iterate get next statements from the first available frame. */ + private def getNextStatement: Option[CompoundStatementExec] = { + while (context.frames.nonEmpty && !context.frames.last.hasNext) { + context.frames.remove(context.frames.size - 1) + } + if (context.frames.nonEmpty) { + return Some(context.frames.last.next()) + } + None } /** Helper method to iterate through statements until next result statement is encountered. */ private def getNextResult: Option[DataFrame] = { - - def getNextStatement: Option[CompoundStatementExec] = - if (executionPlan.hasNext) Some(executionPlan.next()) else None - var currentStatement = getNextStatement // While we don't have a result statement, execute the statements. while (currentStatement.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala new file mode 100644 index 0000000000000..5a2ef62e3bb7d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala @@ -0,0 +1,91 @@ +/* + * 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.scripting + +import scala.collection.mutable.ListBuffer + +import org.apache.spark.SparkException + +/** + * SQL scripting execution context - keeps track of the current execution state. + */ +class SqlScriptingExecutionContext { + // List of frames that are currently active. + val frames: ListBuffer[SqlScriptingExecutionFrame] = ListBuffer.empty + + def enterScope(label: String): Unit = { + if (frames.isEmpty) { + throw SparkException.internalError("Cannot enter scope: no frames.") + } + frames.last.enterScope(label) + } + + def exitScope(label: String): Unit = { + if (frames.isEmpty) { + throw SparkException.internalError("Cannot exit scope: no frames.") + } + frames.last.exitScope(label) + } +} + +/** + * SQL scripting executor - executes script and returns result statements. + * This supports returning multiple result statements from a single script. + * + * @param executionPlan CompoundBody which need to be executed. + */ +class SqlScriptingExecutionFrame( + executionPlan: Iterator[CompoundStatementExec]) extends Iterator[CompoundStatementExec] { + + // List of scopes that are currently active. + private val scopes: ListBuffer[SqlScriptingExecutionScope] = ListBuffer.empty + + override def hasNext: Boolean = executionPlan.hasNext + + override def next(): CompoundStatementExec = { + if (!hasNext) throw SparkException.internalError("No more elements to iterate through.") + executionPlan.next() + } + + def enterScope(label: String): Unit = { + scopes.addOne(new SqlScriptingExecutionScope(label)) + } + + def exitScope(label: String): Unit = { + if (scopes.isEmpty) { + throw SparkException.internalError("Cannot exit scope: no scopes to exit.") + } + + // Remove all scopes until the one with the given label. + while (scopes.nonEmpty && scopes.last.label != label) { + scopes.remove(scopes.length - 1) + } + + if (scopes.nonEmpty) { + scopes.remove(scopes.length - 1) + } + } +} + +/** + * SQL scripting execution scope - keeps track of the current execution scope. + * + * @param label + * Label of the scope. + */ +class SqlScriptingExecutionScope(val label: String) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index 99719ce19e3de..2d50d37e2cb83 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -115,12 +115,15 @@ trait NonLeafStatementExec extends CompoundStatementExec { * Whether the statement originates from the SQL script or it is created during the * interpretation. Example: DropVariable statements are automatically created at the end of each * compound. + * @param context + * SqlScriptingExecutionContext keeps the execution state of current script. */ class SingleStatementExec( var parsedPlan: LogicalPlan, override val origin: Origin, val args: Map[String, Expression], - override val isInternal: Boolean) + override val isInternal: Boolean, + context: SqlScriptingExecutionContext) extends LeafStatementExec with WithOrigin { /** @@ -177,12 +180,54 @@ class NoOpStatementExec extends LeafStatementExec { * Executable nodes for nested statements within the CompoundBody. * @param label * Label set by user to CompoundBody or None otherwise. + * @param isScope + * Flag that indicates whether Compound Body is scope or not. + * @param context + * SqlScriptingExecutionContext keeps the execution state of current script. */ -class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[String] = None) +class CompoundBodyExec( + statements: Seq[CompoundStatementExec], + label: Option[String] = None, + isScope: Boolean, + context: SqlScriptingExecutionContext) extends NonLeafStatementExec { + private object ScopeStatus extends Enumeration { + type ScopeStatus = Value + val NOT_ENTERED, INSIDE, EXITED = Value + } + private var localIterator = statements.iterator private var curr = if (localIterator.hasNext) Some(localIterator.next()) else None + private var scopeStatus = ScopeStatus.NOT_ENTERED + + /** + * Enter scope represented by this compound statement. + * + * This operation needs to be idempotent because it is called multiple times during + * iteration, but it should be executed only once when compound body that represent + * scope is encountered for the first time. + */ + def enterScope(): Unit = { + // This check makes this operation idempotent. + if (isScope && scopeStatus == ScopeStatus.NOT_ENTERED) { + scopeStatus = ScopeStatus.INSIDE + context.enterScope(label.get) + } + } + + /** + * Exit scope represented by this compound statement. + * + * Even though this operation is called exactly once, we are making it idempotent. + */ + protected def exitScope(): Unit = { + // This check makes this operation idempotent. + if (isScope && scopeStatus == ScopeStatus.INSIDE) { + scopeStatus = ScopeStatus.EXITED + context.exitScope(label.get) + } + } /** Used to stop the iteration in cases when LEAVE statement is encountered. */ private var stopIteration = false @@ -218,6 +263,11 @@ class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[Str statement case Some(body: NonLeafStatementExec) => if (body.getTreeIterator.hasNext) { + body match { + // Scope will be entered only once per compound because enter scope is idempotent. + case compoundBodyExec: CompoundBodyExec => compoundBodyExec.enterScope() + case _ => // pass + } body.getTreeIterator.next() match { case leaveStatement: LeaveStatementExec => handleLeaveStatement(leaveStatement) @@ -228,6 +278,11 @@ class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[Str case other => other } } else { + body match { + // Exit scope when there are no more statements to iterate through. + case compoundBodyExec: CompoundBodyExec => compoundBodyExec.exitScope() + case _ => // pass + } curr = if (localIterator.hasNext) Some(localIterator.next()) else None next() } @@ -244,6 +299,7 @@ class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[Str localIterator = statements.iterator curr = if (localIterator.hasNext) Some(localIterator.next()) else None stopIteration = false + scopeStatus = ScopeStatus.NOT_ENTERED } /** Actions to do when LEAVE statement is encountered, to stop the execution of this compound. */ @@ -252,6 +308,9 @@ class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[Str // Stop the iteration. stopIteration = true + // Exit scope if leave statement is encountered. + exitScope() + // TODO: Variable cleanup (once we add SQL script execution logic). // TODO: Add interpreter tests as well. @@ -268,6 +327,9 @@ class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[Str // Stop the iteration. stopIteration = true + // Exit scope if iterate statement is encountered. + exitScope() + // TODO: Variable cleanup (once we add SQL script execution logic). // TODO: Add interpreter tests as well. @@ -680,13 +742,15 @@ class LoopStatementExec( * @param body Executable node for the body. * @param label Label set to ForStatement by user or None otherwise. * @param session Spark session that SQL script is executed within. + * @param context SqlScriptingExecutionContext keeps the execution state of current script. */ class ForStatementExec( query: SingleStatementExec, variableName: Option[String], body: CompoundBodyExec, val label: Option[String], - session: SparkSession) extends NonLeafStatementExec { + session: SparkSession, + context: SqlScriptingExecutionContext) extends NonLeafStatementExec { private object ForState extends Enumeration { val VariableAssignment, Body, VariableCleanup = Value @@ -848,7 +912,10 @@ class ForStatementExec( else { // create compound body for dropping nodes after execution is complete dropVariablesExec = new CompoundBodyExec( - variablesMap.keys.toSeq.map(colName => createDropVarExec(colName)) + variablesMap.keys.toSeq.map(colName => createDropVarExec(colName)), + None, + isScope = false, + context ) ForState.VariableCleanup } @@ -861,7 +928,7 @@ class ForStatementExec( defaultExpression, replace = true ) - new SingleStatementExec(declareVariable, Origin(), Map.empty, isInternal = true) + new SingleStatementExec(declareVariable, Origin(), Map.empty, isInternal = true, context) } private def createSetVarExec(varName: String, variable: Expression): SingleStatementExec = { @@ -871,12 +938,17 @@ class ForStatementExec( ) val setIdentifierToCurrentRow = SetVariable(Seq(UnresolvedAttribute(varName)), projectNamedStruct) - new SingleStatementExec(setIdentifierToCurrentRow, Origin(), Map.empty, isInternal = true) + new SingleStatementExec( + setIdentifierToCurrentRow, + Origin(), + Map.empty, + isInternal = true, + context) } private def createDropVarExec(varName: String): SingleStatementExec = { val dropVar = DropVariable(UnresolvedIdentifier(Seq(varName)), ifExists = true) - new SingleStatementExec(dropVar, Origin(), Map.empty, isInternal = true) + new SingleStatementExec(dropVar, Origin(), Map.empty, isInternal = true, context) } override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala index 5d3edeefc532b..7d00bbb3538df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala @@ -40,13 +40,14 @@ case class SqlScriptingInterpreter(session: SparkSession) { * @param args * A map of parameter names to SQL literal expressions. * @return - * Iterator through collection of statements to be executed. + * Top level CompoundBodyExec representing SQL Script to be executed. */ def buildExecutionPlan( compound: CompoundBody, - args: Map[String, Expression]): Iterator[CompoundStatementExec] = { - transformTreeIntoExecutable(compound, args) - .asInstanceOf[CompoundBodyExec].getTreeIterator + args: Map[String, Expression], + context: SqlScriptingExecutionContext): CompoundBodyExec = { + transformTreeIntoExecutable(compound, args, context) + .asInstanceOf[CompoundBodyExec] } /** @@ -74,9 +75,10 @@ case class SqlScriptingInterpreter(session: SparkSession) { */ private def transformTreeIntoExecutable( node: CompoundPlanStatement, - args: Map[String, Expression]): CompoundStatementExec = + args: Map[String, Expression], + context: SqlScriptingExecutionContext): CompoundStatementExec = node match { - case CompoundBody(collection, label) => + case CompoundBody(collection, label, isScope) => // TODO [SPARK-48530]: Current logic doesn't support scoped variables and shadowing. val variables = collection.flatMap { case st: SingleStatement => getDeclareVarNameFromPlan(st.parsedPlan) @@ -84,16 +86,20 @@ case class SqlScriptingInterpreter(session: SparkSession) { } val dropVariables = variables .map(varName => DropVariable(varName, ifExists = true)) - .map(new SingleStatementExec(_, Origin(), args, isInternal = true)) + .map(new SingleStatementExec(_, Origin(), args, isInternal = true, context)) .reverse - val statements = - collection.map(st => transformTreeIntoExecutable(st, args)) ++ dropVariables match { + val statements = collection + .map(st => transformTreeIntoExecutable(st, args, context)) ++ dropVariables match { case Nil => Seq(new NoOpStatementExec) case s => s } - new CompoundBodyExec(statements, label) + new CompoundBodyExec( + statements, + label, + isScope, + context) case IfElseStatement(conditions, conditionalBodies, elseBody) => val conditionsExec = conditions.map(condition => @@ -101,11 +107,12 @@ case class SqlScriptingInterpreter(session: SparkSession) { condition.parsedPlan, condition.origin, args, - isInternal = false)) + isInternal = false, + context)) val conditionalBodiesExec = conditionalBodies.map(body => - transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args, context).asInstanceOf[CompoundBodyExec]) val unconditionalBodiesExec = elseBody.map(body => - transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args, context).asInstanceOf[CompoundBodyExec]) new IfElseStatementExec( conditionsExec, conditionalBodiesExec, unconditionalBodiesExec, session) @@ -115,11 +122,12 @@ case class SqlScriptingInterpreter(session: SparkSession) { condition.parsedPlan, condition.origin, args, - isInternal = false)) + isInternal = false, + context)) val conditionalBodiesExec = conditionalBodies.map(body => - transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args, context).asInstanceOf[CompoundBodyExec]) val unconditionalBodiesExec = elseBody.map(body => - transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec]) + transformTreeIntoExecutable(body, args, context).asInstanceOf[CompoundBodyExec]) new CaseStatementExec( conditionsExec, conditionalBodiesExec, unconditionalBodiesExec, session) @@ -129,9 +137,10 @@ case class SqlScriptingInterpreter(session: SparkSession) { condition.parsedPlan, condition.origin, args, - isInternal = false) + isInternal = false, + context) val bodyExec = - transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec] + transformTreeIntoExecutable(body, args, context).asInstanceOf[CompoundBodyExec] new WhileStatementExec(conditionExec, bodyExec, label, session) case RepeatStatement(condition, body, label) => @@ -140,13 +149,14 @@ case class SqlScriptingInterpreter(session: SparkSession) { condition.parsedPlan, condition.origin, args, - isInternal = false) + isInternal = false, + context) val bodyExec = - transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec] + transformTreeIntoExecutable(body, args, context).asInstanceOf[CompoundBodyExec] new RepeatStatementExec(conditionExec, bodyExec, label, session) case LoopStatement(body, label) => - val bodyExec = transformTreeIntoExecutable(body, args) + val bodyExec = transformTreeIntoExecutable(body, args, context) .asInstanceOf[CompoundBodyExec] new LoopStatementExec(bodyExec, label) @@ -156,10 +166,11 @@ case class SqlScriptingInterpreter(session: SparkSession) { query.parsedPlan, query.origin, args, - isInternal = false) + isInternal = false, + context) val bodyExec = - transformTreeIntoExecutable(body, args).asInstanceOf[CompoundBodyExec] - new ForStatementExec(queryExec, variableNameOpt, bodyExec, label, session) + transformTreeIntoExecutable(body, args, context).asInstanceOf[CompoundBodyExec] + new ForStatementExec(queryExec, variableNameOpt, bodyExec, label, session, context) case leaveStatement: LeaveStatement => new LeaveStatementExec(leaveStatement.label) @@ -172,6 +183,7 @@ case class SqlScriptingInterpreter(session: SparkSession) { sparkStatement.parsedPlan, sparkStatement.origin, args, - isInternal = false) + isInternal = false, + context) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala index a997b5beadd34..325c8ce380c63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala @@ -32,6 +32,35 @@ import org.apache.spark.sql.types.{IntegerType, StructField, StructType} */ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSession { // Helpers + case class TestCompoundBody( + statements: Seq[CompoundStatementExec], + label: Option[String] = None, + isScope: Boolean = false, + context: SqlScriptingExecutionContext = null) + extends CompoundBodyExec(statements, label, isScope, context) { + + // No-op to remove unnecessary logic for these tests. + override def enterScope(): Unit = () + + // No-op to remove unnecessary logic for these tests. + override def exitScope(): Unit = () + } + + case class TestForStatement( + query: SingleStatementExec, + variableName: Option[String], + body: CompoundBodyExec, + override val label: Option[String], + session: SparkSession, + context: SqlScriptingExecutionContext = null) + extends ForStatementExec( + query, + variableName, + body, + label, + session, + context) + case class TestLeafStatement(testVal: String) extends LeafStatementExec { override def reset(): Unit = () } @@ -41,7 +70,9 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi parsedPlan = Project(Seq(Alias(Literal(condVal), description)()), OneRowRelation()), Origin(startIndex = Some(0), stopIndex = Some(description.length)), Map.empty, - isInternal = false) + isInternal = false, + null + ) case class DummyLogicalPlan() extends LeafNode { override def output: Seq[Attribute] = Seq.empty @@ -53,7 +84,9 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi parsedPlan = DummyLogicalPlan(), Origin(startIndex = Some(0), stopIndex = Some(description.length)), Map.empty, - isInternal = false) + isInternal = false, + null + ) class LoopBooleanConditionEvaluator(condition: TestLoopCondition) { private var callCount: Int = 0 @@ -71,7 +104,7 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi case class TestWhile( condition: TestLoopCondition, - body: CompoundBodyExec, + body: TestCompoundBody, label: Option[String] = None) extends WhileStatementExec(condition, body, label, spark) { @@ -84,7 +117,7 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi case class TestRepeat( condition: TestLoopCondition, - body: CompoundBodyExec, + body: TestCompoundBody, label: Option[String] = None) extends RepeatStatementExec(condition, body, label, spark) { @@ -100,7 +133,8 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi DummyLogicalPlan(), Origin(startIndex = Some(0), stopIndex = Some(description.length)), Map.empty, - isInternal = false) { + isInternal = false, + null) { override def buildDataFrame(session: SparkSession): DataFrame = { val data = Seq.range(0, numberOfRows).map(Row(_)) val schema = List(StructField(columnName, IntegerType)) @@ -120,7 +154,7 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi case loopStmt: LoopStatementExec => loopStmt.label.get case leaveStmt: LeaveStatementExec => leaveStmt.label case iterateStmt: IterateStatementExec => iterateStmt.label - case forStmt: ForStatementExec => forStmt.label.get + case forStmt: TestForStatement => forStmt.label.get case dropStmt: SingleStatementExec if dropStmt.parsedPlan.isInstanceOf[DropVariable] => "DropVariable" case _ => fail("Unexpected statement type") @@ -128,13 +162,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi // Tests test("test body - single statement") { - val iter = new CompoundBodyExec(Seq(TestLeafStatement("one"))).getTreeIterator + val iter = TestCompoundBody(Seq(TestLeafStatement("one"))).getTreeIterator val statements = iter.map(extractStatementValue).toSeq assert(statements === Seq("one")) } test("test body - no nesting") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( Seq( TestLeafStatement("one"), TestLeafStatement("two"), @@ -145,26 +179,26 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("test body - nesting") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( Seq( - new CompoundBodyExec(Seq(TestLeafStatement("one"), TestLeafStatement("two"))), + TestCompoundBody(Seq(TestLeafStatement("one"), TestLeafStatement("two"))), TestLeafStatement("three"), - new CompoundBodyExec(Seq(TestLeafStatement("four"), TestLeafStatement("five"))))) + TestCompoundBody(Seq(TestLeafStatement("four"), TestLeafStatement("five"))))) .getTreeIterator val statements = iter.map(extractStatementValue).toSeq assert(statements === Seq("one", "two", "three", "four", "five")) } test("if else - enter body of the IF clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = true, description = "con1") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body2")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body2")))), session = spark ) )).getTreeIterator @@ -173,15 +207,15 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("if else - enter body of the ELSE clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body2")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body2")))), session = spark ) )).getTreeIterator @@ -190,17 +224,17 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("if else if - enter body of the IF clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = true, description = "con1"), TestIfElseCondition(condVal = false, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body3")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body3")))), session = spark ) )).getTreeIterator @@ -209,17 +243,17 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("if else if - enter body of the ELSE IF clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), TestIfElseCondition(condVal = true, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body3")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body3")))), session = spark ) )).getTreeIterator @@ -228,7 +262,7 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("if else if - enter body of the second ELSE IF clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), @@ -236,11 +270,11 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi TestIfElseCondition(condVal = true, description = "con3") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))), - new CompoundBodyExec(Seq(TestLeafStatement("body3"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))), + TestCompoundBody(Seq(TestLeafStatement("body3"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body4")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body4")))), session = spark ) )).getTreeIterator @@ -249,17 +283,17 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("if else if - enter body of the ELSE clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), TestIfElseCondition(condVal = false, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body3")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body3")))), session = spark ) )).getTreeIterator @@ -268,15 +302,15 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("if else if - without else (successful check)") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), TestIfElseCondition(condVal = true, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), elseBody = None, session = spark @@ -287,15 +321,15 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("if else if - without else (unsuccessful checks)") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new IfElseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), TestIfElseCondition(condVal = false, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), elseBody = None, session = spark @@ -306,10 +340,10 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("while - doesn't enter body") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 0, description = "con1"), - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -317,10 +351,10 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("while - enters body once") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 1, description = "con1"), - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -328,10 +362,10 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("while - enters body with multiple statements multiple times") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("statement1"), TestLeafStatement("statement2"))) ) @@ -342,13 +376,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("nested while - 2 times outer 2 times inner") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con2"), - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) )) ) ) @@ -361,10 +395,10 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("repeat - true condition") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 0, description = "con1"), - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -372,10 +406,10 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("repeat - condition false once") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 1, description = "con1"), - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -383,10 +417,10 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("repeat - enters body with multiple statements multiple times") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("statement1"), TestLeafStatement("statement2"))) ) @@ -397,13 +431,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("nested repeat") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con2"), - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) )) ) ) @@ -419,7 +453,7 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("leave compound block") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestLeafStatement("one"), new LeaveStatementExec("lbl") @@ -431,11 +465,11 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("leave while loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new LeaveStatementExec("lbl")) ), @@ -448,11 +482,11 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("leave repeat loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new LeaveStatementExec("lbl")) ), @@ -465,11 +499,11 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("iterate while loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new IterateStatementExec("lbl"), TestLeafStatement("body2")) @@ -483,11 +517,11 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("iterate repeat loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new IterateStatementExec("lbl"), TestLeafStatement("body2")) @@ -502,14 +536,14 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("leave outer loop from nested while loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con2"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new LeaveStatementExec("lbl")) ), @@ -525,14 +559,14 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("leave outer loop from nested repeat loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con2"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new LeaveStatementExec("lbl")) ), @@ -548,14 +582,14 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("iterate outer loop from nested while loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestWhile( condition = TestLoopCondition(condVal = true, reps = 2, description = "con2"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new IterateStatementExec("lbl"), TestLeafStatement("body2")) @@ -575,14 +609,14 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("iterate outer loop from nested repeat loop") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con1"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestRepeat( condition = TestLoopCondition(condVal = false, reps = 2, description = "con2"), - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new IterateStatementExec("lbl"), TestLeafStatement("body2")) @@ -602,17 +636,17 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("searched case - enter first WHEN clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new CaseStatementExec( conditions = Seq( TestIfElseCondition(condVal = true, description = "con1"), TestIfElseCondition(condVal = false, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body3")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body3")))), session = spark ) )).getTreeIterator @@ -621,15 +655,15 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("searched case - enter body of the ELSE clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new CaseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body2")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body2")))), session = spark ) )).getTreeIterator @@ -638,17 +672,17 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("searched case - enter second WHEN clause") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new CaseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), TestIfElseCondition(condVal = true, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), - elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body3")))), + elseBody = Some(TestCompoundBody(Seq(TestLeafStatement("body3")))), session = spark ) )).getTreeIterator @@ -657,15 +691,15 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("searched case - without else (successful check)") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new CaseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), TestIfElseCondition(condVal = true, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), elseBody = None, session = spark @@ -676,15 +710,15 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("searched case - without else (unsuccessful checks)") { - val iter = new CompoundBodyExec(Seq( + val iter = TestCompoundBody(Seq( new CaseStatementExec( conditions = Seq( TestIfElseCondition(condVal = false, description = "con1"), TestIfElseCondition(condVal = false, description = "con2") ), conditionalBodies = Seq( - new CompoundBodyExec(Seq(TestLeafStatement("body1"))), - new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + TestCompoundBody(Seq(TestLeafStatement("body1"))), + TestCompoundBody(Seq(TestLeafStatement("body2"))) ), elseBody = None, session = spark @@ -695,10 +729,10 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("loop statement with leave") { - val iter = new CompoundBodyExec( + val iter = TestCompoundBody( statements = Seq( new LoopStatementExec( - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new LeaveStatementExec("lbl")) ), @@ -711,13 +745,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - enters body once") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(1, "intCol", "query1"), variableName = Some("x"), label = Some("for1"), session = spark, - body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + body = TestCompoundBody(Seq(TestLeafStatement("body"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -729,13 +763,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - enters body with multiple statements multiple times") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = Some("x"), label = Some("for1"), session = spark, - body = new CompoundBodyExec( + body = TestCompoundBody( Seq(TestLeafStatement("statement1"), TestLeafStatement("statement2")) ) ) @@ -752,13 +786,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - empty result") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(0, "intCol", "query1"), variableName = Some("x"), label = Some("for1"), session = spark, - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -766,23 +800,24 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - nested") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = Some("x"), label = Some("for1"), session = spark, - body = new CompoundBodyExec(Seq( - new ForStatementExec( + body = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol1", "query2"), variableName = Some("y"), label = Some("for2"), session = spark, - body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + body = TestCompoundBody(Seq(TestLeafStatement("body"))) ) )) - ) - )).getTreeIterator + )), + label = Some("lbl") + ).getTreeIterator val statements = iter.map(extractStatementValue).toSeq assert(statements === Seq( "body", @@ -799,13 +834,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - enters body once") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(1, "intCol", "query1"), variableName = None, label = Some("for1"), session = spark, - body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + body = TestCompoundBody(Seq(TestLeafStatement("body"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -816,13 +851,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - enters body with multiple statements multiple times") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = None, label = Some("for1"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("statement1"), TestLeafStatement("statement2"))) ) @@ -835,13 +870,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - empty result") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(0, "intCol", "query1"), variableName = None, label = Some("for1"), session = spark, - body = new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + body = TestCompoundBody(Seq(TestLeafStatement("body1"))) ) )).getTreeIterator val statements = iter.map(extractStatementValue).toSeq @@ -849,19 +884,19 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - nested") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = None, label = Some("for1"), session = spark, - body = new CompoundBodyExec(Seq( - new ForStatementExec( + body = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol1", "query2"), variableName = None, label = Some("for2"), session = spark, - body = new CompoundBodyExec(Seq(TestLeafStatement("body"))) + body = TestCompoundBody(Seq(TestLeafStatement("body"))) ) )) ) @@ -877,13 +912,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - iterate") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = Some("x"), label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("statement1"), new IterateStatementExec("lbl1"), TestLeafStatement("statement2"))) @@ -901,13 +936,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - leave") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = Some("x"), label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("statement1"), new LeaveStatementExec("lbl1"), TestLeafStatement("statement2"))) @@ -918,20 +953,20 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - nested - iterate outer loop") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = Some("x"), label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("outer_body"), - new ForStatementExec( + TestForStatement( query = MockQuery(2, "intCol1", "query2"), variableName = Some("y"), label = Some("lbl2"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new IterateStatementExec("lbl1"), TestLeafStatement("body2"))) @@ -953,19 +988,19 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement - nested - leave outer loop") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = Some("x"), label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( - new ForStatementExec( + body = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query2"), variableName = Some("y"), label = Some("lbl2"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new LeaveStatementExec("lbl1"), TestLeafStatement("body2"))) @@ -978,13 +1013,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - iterate") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = None, label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("statement1"), new IterateStatementExec("lbl1"), TestLeafStatement("statement2"))) @@ -998,13 +1033,13 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - leave") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = None, label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("statement1"), new LeaveStatementExec("lbl1"), TestLeafStatement("statement2"))) @@ -1015,20 +1050,20 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - nested - iterate outer loop") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = None, label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("outer_body"), - new ForStatementExec( + TestForStatement( query = MockQuery(2, "intCol1", "query2"), variableName = None, label = Some("lbl2"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new IterateStatementExec("lbl1"), TestLeafStatement("body2"))) @@ -1044,19 +1079,19 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi } test("for statement no variable - nested - leave outer loop") { - val iter = new CompoundBodyExec(Seq( - new ForStatementExec( + val iter = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol", "query1"), variableName = None, label = Some("lbl1"), session = spark, - body = new CompoundBodyExec(Seq( - new ForStatementExec( + body = TestCompoundBody(Seq( + TestForStatement( query = MockQuery(2, "intCol1", "query2"), variableName = None, label = Some("lbl2"), session = spark, - body = new CompoundBodyExec(Seq( + body = TestCompoundBody(Seq( TestLeafStatement("body1"), new LeaveStatementExec("lbl1"), TestLeafStatement("body2"))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 2ec42c4554e09..20997504b15eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -45,8 +45,14 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { args: Map[String, Expression] = Map.empty): Array[DataFrame] = { val interpreter = SqlScriptingInterpreter(spark) val compoundBody = spark.sessionState.sqlParser.parsePlan(sqlText).asInstanceOf[CompoundBody] - val executionPlan = interpreter.buildExecutionPlan(compoundBody, args) - executionPlan.flatMap { + + // Initialize context so scopes can be entered correctly. + val context = new SqlScriptingExecutionContext() + val executionPlan = interpreter.buildExecutionPlan(compoundBody, args, context) + context.frames.addOne(new SqlScriptingExecutionFrame(executionPlan.getTreeIterator)) + executionPlan.enterScope() + + executionPlan.getTreeIterator.flatMap { case statement: SingleStatementExec => if (statement.isExecuted) { None From f979bc8d37d241d65986dfcf2105db481fef6eef Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 12 Dec 2024 10:07:48 -0800 Subject: [PATCH 163/438] [SPARK-50554][INFRA] Add a separate docker file for Python 3.11 daily coverage build ### What changes were proposed in this pull request? 1, Add a separate docker file for Python 3.11; 2, use it in daily coverage build; (will use it for other jobs later) ### Why are the changes needed? to isolate the environments ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with: ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11", "PYSPARK_CODECOV": "true"}' ``` https://github.com/zhengruifeng/spark/runs/34308729714 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49165 from zhengruifeng/py_image_311. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- .github/workflows/build_coverage.yml | 1 + .../workflows/build_infra_images_cache.yml | 14 ++++ dev/spark-test-image/python-311/Dockerfile | 80 +++++++++++++++++++ 3 files changed, 95 insertions(+) create mode 100644 dev/spark-test-image/python-311/Dockerfile diff --git a/.github/workflows/build_coverage.yml b/.github/workflows/build_coverage.yml index 64f65bd777a02..8da65927f37f7 100644 --- a/.github/workflows/build_coverage.yml +++ b/.github/workflows/build_coverage.yml @@ -36,6 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11", "PYSPARK_CODECOV": "true" } diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index df927da57c240..cd57e070e21b3 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -33,6 +33,7 @@ on: - 'dev/spark-test-image/pypy-310/Dockerfile' - 'dev/spark-test-image/python-309/Dockerfile' - 'dev/spark-test-image/python-310/Dockerfile' + - 'dev/spark-test-image/python-311/Dockerfile' - 'dev/spark-test-image/python-312/Dockerfile' - 'dev/spark-test-image/python-313/Dockerfile' - '.github/workflows/build_infra_images_cache.yml' @@ -146,6 +147,19 @@ jobs: - name: Image digest (PySpark with Python 3.10) if: hashFiles('dev/spark-test-image/python-310/Dockerfile') != '' run: echo ${{ steps.docker_build_pyspark_python_310.outputs.digest }} + - name: Build and push (PySpark with Python 3.11) + if: hashFiles('dev/spark-test-image/python-311/Dockerfile') != '' + id: docker_build_pyspark_python_311 + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/python-311/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-311-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-311-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-311-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark with Python 3.11) + if: hashFiles('dev/spark-test-image/python-311/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_python_311.outputs.digest }} - name: Build and push (PySpark with Python 3.12) if: hashFiles('dev/spark-test-image/python-312/Dockerfile') != '' id: docker_build_pyspark_python_312 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile new file mode 100644 index 0000000000000..6287caca648f2 --- /dev/null +++ b/dev/spark-test-image/python-311/Dockerfile @@ -0,0 +1,80 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark with Python 3.11" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE 20241212 + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pkg-config \ + qpdf \ + tzdata \ + software-properties-common \ + wget \ + zlib1g-dev + +# Install Python 3.11 +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt-get update && apt-get install -y \ + python3.11 \ + && apt-get autoremove --purge -y \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +# Python deps for Spark Connect +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" + +# Install Python 3.11 packages +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 +RUN python3.11 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this +RUN python3.11 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS && \ + python3.11 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ + python3.11 -m pip install deepspeed torcheval && \ + python3.11 -m pip cache purge From df08177de2fd2b177caf79ca533eb0cd2c6a4ba6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 12 Dec 2024 15:36:09 -0800 Subject: [PATCH 164/438] [SPARK-48416][SQL] Support nested correlated With expression ### What changes were proposed in this pull request? The inner `With` may reference common expressions of an outer `With`. This PR supports this case by making the rule `RewriteWithExpression` only rewrite top-level `With` expressions, and run the rule repeatedly so that the inner `With` expression becomes top-level `With` after one iteration, and gets rewritten in the next iteration. ### Why are the changes needed? To support optimized filter pushdown with `With` expression. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? updated the unit test ### Was this patch authored or co-authored using generative AI tooling? no Closes #49093 from cloud-fan/with. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../optimizer/RewriteWithExpression.scala | 25 ++++---- .../RewriteWithExpressionSuite.scala | 61 ++++++++++++------- 3 files changed, 50 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 7ec467badce5c..31c1f89177632 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -160,7 +160,7 @@ abstract class Optimizer(catalogManager: CatalogManager) Batch("Finish Analysis", FixedPoint(1), FinishAnalysis) :: // We must run this batch after `ReplaceExpressions`, as `RuntimeReplaceable` expression // may produce `With` expressions that need to be rewritten. - Batch("Rewrite With expression", Once, RewriteWithExpression) :: + Batch("Rewrite With expression", fixedPoint, RewriteWithExpression) :: ////////////////////////////////////////////////////////////////////////////////////////// // Optimizer rules start here ////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala index 393a66f7c1e4f..d0c5d8158644b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala @@ -85,21 +85,19 @@ object RewriteWithExpression extends Rule[LogicalPlan] { private def rewriteWithExprAndInputPlans( e: Expression, - inputPlans: Array[LogicalPlan]): Expression = { + inputPlans: Array[LogicalPlan], + isNestedWith: Boolean = false): Expression = { if (!e.containsPattern(WITH_EXPRESSION)) return e e match { - case w: With => + // Do not handle nested With in one pass. Leave it to the next rule executor batch. + case w: With if !isNestedWith => // Rewrite nested With expressions first - val child = rewriteWithExprAndInputPlans(w.child, inputPlans) - val defs = w.defs.map(rewriteWithExprAndInputPlans(_, inputPlans)) + val child = rewriteWithExprAndInputPlans(w.child, inputPlans, isNestedWith = true) + val defs = w.defs.map(rewriteWithExprAndInputPlans(_, inputPlans, isNestedWith = true)) val refToExpr = mutable.HashMap.empty[CommonExpressionId, Expression] val childProjections = Array.fill(inputPlans.length)(mutable.ArrayBuffer.empty[Alias]) defs.zipWithIndex.foreach { case (CommonExpressionDef(child, id), index) => - if (child.containsPattern(COMMON_EXPR_REF)) { - throw SparkException.internalError( - "Common expression definition cannot reference other Common expression definitions") - } if (id.canonicalized) { throw SparkException.internalError( "Cannot rewrite canonicalized Common expression definitions") @@ -148,10 +146,9 @@ object RewriteWithExpression extends Rule[LogicalPlan] { } child.transformWithPruning(_.containsPattern(COMMON_EXPR_REF)) { - case ref: CommonExpressionRef => - if (!refToExpr.contains(ref.id)) { - throw SparkException.internalError("Undefined common expression id " + ref.id) - } + // `child` may contain nested With and we only replace `CommonExpressionRef` that + // references common expressions in the current `With`. + case ref: CommonExpressionRef if refToExpr.contains(ref.id) => if (ref.id.canonicalized) { throw SparkException.internalError( "Cannot rewrite canonicalized Common expression references") @@ -161,7 +158,7 @@ object RewriteWithExpression extends Rule[LogicalPlan] { case c: ConditionalExpression => val newAlwaysEvaluatedInputs = c.alwaysEvaluatedInputs.map( - rewriteWithExprAndInputPlans(_, inputPlans)) + rewriteWithExprAndInputPlans(_, inputPlans, isNestedWith)) val newExpr = c.withNewAlwaysEvaluatedInputs(newAlwaysEvaluatedInputs) // Use transformUp to handle nested With. newExpr.transformUpWithPruning(_.containsPattern(WITH_EXPRESSION)) { @@ -174,7 +171,7 @@ object RewriteWithExpression extends Rule[LogicalPlan] { } } - case other => other.mapChildren(rewriteWithExprAndInputPlans(_, inputPlans)) + case other => other.mapChildren(rewriteWithExprAndInputPlans(_, inputPlans, isNestedWith)) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala index 0aeca961aa513..0be6ae6494643 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.analysis.TempResolvedColumn import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -29,7 +28,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor class RewriteWithExpressionSuite extends PlanTest { object Optimizer extends RuleExecutor[LogicalPlan] { - val batches = Batch("Rewrite With expression", Once, + val batches = Batch("Rewrite With expression", FixedPoint(5), PullOutGroupingExpressions, RewriteWithExpression) :: Nil } @@ -84,13 +83,11 @@ class RewriteWithExpressionSuite extends PlanTest { ref * ref } - val plan = testRelation.select(outerExpr.as("col")) comparePlans( - Optimizer.execute(plan), + Optimizer.execute(testRelation.select(outerExpr.as("col"))), testRelation - .select((testRelation.output :+ (a + a).as("_common_expr_0")): _*) - .select((testRelation.output ++ Seq($"_common_expr_0", - ($"_common_expr_0" + $"_common_expr_0" + b).as("_common_expr_1"))): _*) + .select(star(), (a + a).as("_common_expr_0")) + .select(a, b, ($"_common_expr_0" + $"_common_expr_0" + b).as("_common_expr_1")) .select(($"_common_expr_1" * $"_common_expr_1").as("col")) .analyze ) @@ -104,42 +101,60 @@ class RewriteWithExpressionSuite extends PlanTest { val outerExpr = With(b + b) { case Seq(ref) => ref * ref + innerExpr } - - val plan = testRelation.select(outerExpr.as("col")) - val rewrittenInnerExpr = (a + a).as("_common_expr_0") - val rewrittenOuterExpr = (b + b).as("_common_expr_1") - val finalExpr = rewrittenOuterExpr.toAttribute * rewrittenOuterExpr.toAttribute + - (rewrittenInnerExpr.toAttribute + rewrittenInnerExpr.toAttribute) + val finalExpr = $"_common_expr_1" * $"_common_expr_1" + ($"_common_expr_0" + $"_common_expr_0") comparePlans( - Optimizer.execute(plan), + Optimizer.execute(testRelation.select(outerExpr.as("col"))), testRelation - .select((testRelation.output :+ rewrittenInnerExpr): _*) - .select((testRelation.output :+ rewrittenInnerExpr.toAttribute :+ rewrittenOuterExpr): _*) + .select(star(), (b + b).as("_common_expr_1")) + .select(star(), (a + a).as("_common_expr_0")) .select(finalExpr.as("col")) .analyze ) } - test("correlated nested WITH expression is not supported") { + test("correlated nested WITH expression is supported") { val Seq(a, b) = testRelation.output val outerCommonExprDef = CommonExpressionDef(b + b, CommonExpressionId(0)) val outerRef = new CommonExpressionRef(outerCommonExprDef) + val rewrittenOuterExpr = (b + b).as("_common_expr_0") // The inner expression definition references the outer expression val commonExprDef1 = CommonExpressionDef(a + a + outerRef, CommonExpressionId(1)) val ref1 = new CommonExpressionRef(commonExprDef1) val innerExpr1 = With(ref1 + ref1, Seq(commonExprDef1)) - val outerExpr1 = With(outerRef + innerExpr1, Seq(outerCommonExprDef)) - intercept[SparkException](Optimizer.execute(testRelation.select(outerExpr1.as("col")))) + comparePlans( + Optimizer.execute(testRelation.select(outerExpr1.as("col"))), + testRelation + // The first Project contains the common expression of the outer With + .select(star(), rewrittenOuterExpr) + // The second Project contains the common expression of the inner With, which references + // the common expression of the outer With. + .select(star(), (a + a + $"_common_expr_0").as("_common_expr_1")) + // The final Project contains the final result expression, which references both common + // expressions. + .select(($"_common_expr_0" + ($"_common_expr_1" + $"_common_expr_1")).as("col")) + .analyze + ) - val commonExprDef2 = CommonExpressionDef(a + a) + val commonExprDef2 = CommonExpressionDef(a + a, CommonExpressionId(2)) val ref2 = new CommonExpressionRef(commonExprDef2) // The inner main expression references the outer expression - val innerExpr2 = With(ref2 + outerRef, Seq(commonExprDef1)) - + val innerExpr2 = With(ref2 + outerRef, Seq(commonExprDef2)) val outerExpr2 = With(outerRef + innerExpr2, Seq(outerCommonExprDef)) - intercept[SparkException](Optimizer.execute(testRelation.select(outerExpr2.as("col")))) + comparePlans( + Optimizer.execute(testRelation.select(outerExpr2.as("col"))), + testRelation + // The first Project contains the common expression of the outer With + .select(star(), rewrittenOuterExpr) + // The second Project contains the common expression of the inner With, which does not + // reference the common expression of the outer With. + .select(star(), (a + a).as("_common_expr_2")) + // The final Project contains the final result expression, which references both common + // expressions. + .select(($"_common_expr_0" + ($"_common_expr_2" + $"_common_expr_0")).as("col")) + .analyze + ) } test("WITH expression in filter") { From 429402b7c3d8b657545bf60c04048c5043a24aa7 Mon Sep 17 00:00:00 2001 From: WweiL Date: Fri, 13 Dec 2024 11:08:28 +0900 Subject: [PATCH 165/438] [SPARK-49883][SS] State Store Checkpoint Structure V2 Integration with RocksDB and RocksDBFileManager ### What changes were proposed in this pull request? This PR enables RocksDB to read file watermarked with unique ids (e.g. `version_uniqueId.zip`, `version_uniqueId.changelog`). Below is a explanation on the changes and rationale. Now for each changelog file, we put a "version: uniqueId" to its first line, from it's current version to the previous snapshot version. For each snapshot (zip) file, there is no change other than their name (`version_uniqueId.zip`), because snapshot files are single source of truth. In addition to `LastCommitBasedCheckpointId, lastCommittedCheckpointId, loadedCheckpointId` added in https://github.com/apache/spark/pull/47895#pullrequestreview-2349097435, also add a in-memory map `versionToUniqueIdLineage` that maps version to unique Id. This is useful when you reuse the same rocksDB instance in the executor, so you don't need to load the lineage from the changelog file again. ## RocksDB: #### Load - When `loadedVersion != version`, try to load a changelog file with `version_checkpointUniqueId.changelog`. There could be multiple cases: 1. Version corresponds to a zip file: 1. `version_uniqueId.zip`, this means (either changelog was not enabled, or it was enabled but version happens to be a checkpoint file), and previously query run ckpt v2 2. `version.zip`, this means (either changelog was not enabled, or it was enabled but version happens to be a checkpoint file), and previously query run ckpt v1 2. Version corresponds to a changelog file: 1. `version_uniqueId.changelog`, this means changelog was enabled, and previously query run ckpt v2 2. `version.changelog`, this means changelog was enabled, and previously query run ckpt v1 - For case i.a, we construct a new empty lineage `(version, sessionCheckpointId)`. `version_uniqueId.changelog`. - For case ii.a, we read the lineage file stored in - For case i.b and ii.b, there is no need to load the lineage as they were not presented before, we just load the corresponding file without `uniqueId`, but newer files will be constructed with uniqueId. checkpoint version v1 to checkpoint version v2. Next the code finds the latest snapshot version through file listing. When there are multiple snapshot files with the same version but different unique Id (main problem this project was trying to solve), the correct one will be loaded based on the checkpoint id. Then changelog is replayed with the awareness of lineage. The lineage is stored in memory for next load(). Last, load the changelog writer for version + 1, and write the lineage (version + 1, sessionCheckpointId) to the first line of the file. While it seems that the lineage is written early, it is safe because the change log writer is not committed yet. - When `loadedVersion == version`, the same rocks db instance is reused and the lineage is stored in memory to `versionToUniqueIdLineage`. #### Commit - Also save `sessionCheckpointId` to `latestSnapshot` - Add `(newVersion, sessionCheckpointId)` to `versionToUniqueIdLineage` #### Abort Also clear up `versionToUniqueIdLineage` ## RocksDBFileManager: - A bunch of add-ups to make until code uniqueId aware. Now all places that return version returns a pair, in v1 format, the option is None. - ### deleteOldVersions: If there are multiple `version_uniqueId1.zip`(changelog) and `versioion.uniqueId2.zip`, all are deleted. ## Changelog Reader / Writer We purpose to save the lineage to the first line of the changelog files. For changelog reader, there is an abstract function `readLineage` created. In `RocksDBCheckpointManager.getChangelogReader` function, the `readLineage` will be called right after the initialization of the changelog reader to update the file pointer to after the lineage. Subsequent `getNext` function won't be affecter because of this. For changelog writer, there is an abstract function `writeLineage` that writes the lineage. This function will be called before any actual changelog data is written in `RocksDB.load()`. ### Why are the changes needed? Improve fault tolerance to RocksDB State Store. ### Does this PR introduce _any_ user-facing change? Not yet, after the V2 format project is finished, customer can use the new config to enable it with better rocksDB state store fault tolerance ### How was this patch tested? Modified existing unit tests. For unit tests and backward compatibility tests please refer to: https://github.com/apache/spark/pull/48356 ### Was this patch authored or co-authored using generative AI tooling? No Closes #48355 from WweiL/integration-for-review. Lead-authored-by: WweiL Co-authored-by: Wei Liu Signed-off-by: Jungtaek Lim --- .../resources/error/error-conditions.json | 5 + .../org/apache/spark/internal/LogKey.scala | 4 + .../sql/errors/QueryExecutionErrors.scala | 8 + .../streaming/MicroBatchExecution.scala | 4 +- .../execution/streaming/state/RocksDB.scala | 423 ++++-- .../streaming/state/RocksDBFileManager.scala | 178 ++- ...ateDataSourceTransformWithStateSuite.scala | 6 +- ...sDBStateStoreCheckpointFormatV2Suite.scala | 7 +- .../RocksDBStateStoreIntegrationSuite.scala | 2 +- .../state/RocksDBStateStoreSuite.scala | 2 +- .../streaming/state/RocksDBSuite.scala | 1227 ++++++++++------- .../TransformWithListStateSuite.scala | 5 +- .../TransformWithMapStateSuite.scala | 5 +- .../TransformWithStateInitialStateSuite.scala | 4 +- .../streaming/TransformWithStateSuite.scala | 2 +- .../streaming/TransformWithStateTTLTest.scala | 4 +- 16 files changed, 1235 insertions(+), 651 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index ae75992da2f6d..fca6a0d451de1 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -233,6 +233,11 @@ "An error occurred during loading state." ], "subClass" : { + "CANNOT_FIND_BASE_SNAPSHOT_CHECKPOINT" : { + "message" : [ + "Cannot find a base snapshot checkpoint with lineage: ." + ] + }, "CANNOT_READ_CHECKPOINT" : { "message" : [ "Cannot read RocksDB checkpoint metadata. Expected , but found ." 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 4fe6d96f9aaba..740a56cc84ef8 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 @@ -353,6 +353,8 @@ private[spark] object LogKeys { case object LABEL_COLUMN extends LogKey case object LARGEST_CLUSTER_INDEX extends LogKey case object LAST_ACCESS_TIME extends LogKey + case object LAST_COMMITTED_CHECKPOINT_ID extends LogKey + case object LAST_COMMIT_BASED_CHECKPOINT_ID extends LogKey case object LAST_VALID_TIME extends LogKey case object LATEST_BATCH_ID extends LogKey case object LATEST_COMMITTED_BATCH_ID extends LogKey @@ -361,8 +363,10 @@ private[spark] object LogKeys { case object LEFT_EXPR extends LogKey case object LEFT_LOGICAL_PLAN_STATS_SIZE_IN_BYTES extends LogKey case object LINE extends LogKey + case object LINEAGE extends LogKey case object LINE_NUM extends LogKey case object LISTENER extends LogKey + case object LOADED_CHECKPOINT_ID extends LogKey case object LOADED_VERSION extends LogKey case object LOAD_FACTOR extends LogKey case object LOAD_TIME extends LogKey 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 0852e773c87b4..95e2f30bbf2db 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 @@ -2602,6 +2602,14 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE cause = null) } + def cannotFindBaseSnapshotCheckpoint(lineage: String): Throwable = { + new SparkException ( + errorClass = + "CANNOT_LOAD_STATE_STORE.CANNOT_FIND_BASE_SNAPSHOT_CHECKPOINT", + messageParameters = Map("lineage" -> lineage), + cause = null) + } + def unexpectedFileSize( dfsFile: Path, localFile: File, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 40d58e5a402a1..23e72fc4e3e2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -513,6 +513,7 @@ class MicroBatchExecution( execCtx.startOffsets ++= execCtx.endOffsets watermarkTracker.setWatermark( math.max(watermarkTracker.currentWatermark, commitMetadata.nextBatchWatermarkMs)) + currentStateStoreCkptId ++= commitMetadata.stateUniqueIds } else if (latestCommittedBatchId == latestBatchId - 1) { execCtx.endOffsets.foreach { case (source: Source, end: Offset) => @@ -965,7 +966,8 @@ class MicroBatchExecution( updateStateStoreCkptId(execCtx, latestExecPlan) } execCtx.reportTimeTaken("commitOffsets") { - if (!commitLog.add(execCtx.batchId, CommitMetadata(watermarkTracker.currentWatermark))) { + if (!commitLog.add(execCtx.batchId, + CommitMetadata(watermarkTracker.currentWatermark, currentStateStoreCkptId.toMap))) { throw QueryExecutionErrors.concurrentStreamLogUpdate(execCtx.batchId) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 709197cd56527..c13884a948195 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -84,7 +84,8 @@ class RocksDB( columnFamilyMapping: Map[String, Short], maxColumnFamilyId: Short, dfsFileSuffix: String, - fileMapping: Map[String, RocksDBSnapshotFile]) { + fileMapping: Map[String, RocksDBSnapshotFile], + uniqueId: Option[String] = None) { def close(): Unit = { silentDeleteRecursively(checkpointDir, s"Free up local checkpoint of snapshot $version") } @@ -158,7 +159,7 @@ class RocksDB( @volatile private var db: NativeRocksDB = _ @volatile private var changelogWriter: Option[StateStoreChangelogWriter] = None private val enableChangelogCheckpointing: Boolean = conf.enableChangelogCheckpointing - @volatile private var loadedVersion = -1L // -1 = nothing valid is loaded + @volatile protected var loadedVersion: Long = -1L // -1 = nothing valid is loaded // variables to manage checkpoint ID. Once a checkpointing finishes, it needs to return // `lastCommittedStateStoreCkptId` as the committed checkpointID, as well as @@ -170,10 +171,11 @@ class RocksDB( // we have to use a new one. We have to update `sessionStateStoreCkptId` if we reload a previous // batch version, as we would have to use a new checkpointID for re-committing a version. // The reusing is to help debugging but is not required for the algorithm to work. - private var lastCommitBasedStateStoreCkptId: Option[String] = None - private var lastCommittedStateStoreCkptId: Option[String] = None - private var loadedStateStoreCkptId: Option[String] = None - private var sessionStateStoreCkptId: Option[String] = None + protected var lastCommitBasedStateStoreCkptId: Option[String] = None + protected var lastCommittedStateStoreCkptId: Option[String] = None + protected var loadedStateStoreCkptId: Option[String] = None + protected var sessionStateStoreCkptId: Option[String] = None + protected[sql] val lineageManager: RocksDBLineageManager = new RocksDBLineageManager @volatile private var numKeysOnLoadedVersion = 0L @volatile private var numKeysOnWritingVersion = 0L @@ -278,27 +280,167 @@ class RocksDB( // We send snapshots that needs to be uploaded by the maintenance thread to this queue private val snapshotsToUploadQueue = new ConcurrentLinkedQueue[RocksDBSnapshot]() + /** + * Read the lineage from the changelog files. It first get the changelog reader + * of the correct changelog version and then read the lineage information from the file. + * The changelog file is named as version_stateStoreCkptId.changelog + * @param version version of the changelog file, used to load changelog file. + * @param stateStoreCkptId uniqueId of the changelog file, used to load changelog file. + * @return the lineage stored in the changelog file + */ + private def getLineageFromChangelogFile( + version: Long, + stateStoreCkptId: Option[String]): Array[LineageItem] = { + var changelogReader: StateStoreChangelogReader = null + var currLineage: Array[LineageItem] = Array.empty + try { + changelogReader = fileManager.getChangelogReader(version, stateStoreCkptId) + currLineage = changelogReader.lineage + logInfo(log"Loading lineage: " + + log"${MDC(LogKeys.LINEAGE, lineageManager)} from " + + log"changelog version: ${MDC(LogKeys.VERSION_NUM, version)} " + + log"uniqueId: ${MDC(LogKeys.UUID, stateStoreCkptId.getOrElse(""))}.") + } finally { + if (changelogReader != null) { + changelogReader.closeIfNeeded() + } + } + currLineage + } + + /** * Load the given version of data in a native RocksDB instance. * Note that this will copy all the necessary file from DFS to local disk as needed, * and possibly restart the native RocksDB instance. */ - def load( + private def loadWithCheckpointId( + version: Long, + stateStoreCkptId: Option[String], + readOnly: Boolean = false): RocksDB = { + // An array contains lineage information from [snapShotVersion, version] (inclusive in both ends) + var currVersionLineage: Array[LineageItem] = lineageManager.getLineageForCurrVersion() + try { + if (loadedVersion != version || (loadedStateStoreCkptId.isEmpty || + stateStoreCkptId.get != loadedStateStoreCkptId.get)) { + closeDB(ignoreException = false) + + val (latestSnapshotVersion, latestSnapshotUniqueId) = { + // Special handling when version is 0. + // When loading the very first version (0), stateStoreCkptId does not need to be defined + // because there won't be 0.changelog / 0.zip file created in RocksDB under v2. + if (version == 0) { + assert(stateStoreCkptId.isEmpty, "stateStoreCkptId should be empty when version is zero") + (0L, None) + // When there is a snapshot file, it is the ground truth, we can skip + // reconstructing the lineage from changelog file. + } else if (fileManager.existsSnapshotFile(version, stateStoreCkptId)) { + currVersionLineage = Array(LineageItem(version, stateStoreCkptId.get)) + (version, stateStoreCkptId) + } else { + currVersionLineage = getLineageFromChangelogFile(version, stateStoreCkptId) :+ + LineageItem(version, stateStoreCkptId.get) + currVersionLineage = currVersionLineage.sortBy(_.version) + + val latestSnapshotVersionsAndUniqueId = + fileManager.getLatestSnapshotVersionAndUniqueIdFromLineage(currVersionLineage) + latestSnapshotVersionsAndUniqueId match { + case Some(pair) => (pair._1, Option(pair._2)) + case None if currVersionLineage.head.version == 1L => + logDebug(log"Cannot find latest snapshot based on lineage but first version " + + log"is 1, use 0 as default. Lineage: ${MDC(LogKeys.LINEAGE, lineageManager)}") + (0L, None) + case _ => + throw QueryExecutionErrors.cannotFindBaseSnapshotCheckpoint( + printLineageItems(currVersionLineage)) + } + } + } + + logInfo(log"Loaded latestSnapshotVersion: ${ + MDC(LogKeys.SNAPSHOT_VERSION, latestSnapshotVersion)}, latestSnapshotUniqueId: ${ + MDC(LogKeys.UUID, latestSnapshotUniqueId)}") + + val metadata = fileManager.loadCheckpointFromDfs(latestSnapshotVersion, + workingDir, rocksDBFileMapping, latestSnapshotUniqueId) + + loadedVersion = latestSnapshotVersion + + // reset the last snapshot version to the latest available snapshot version + lastSnapshotVersion = latestSnapshotVersion + lineageManager.resetLineage(currVersionLineage) + + // Initialize maxVersion upon successful load from DFS + fileManager.setMaxSeenVersion(version) + + openLocalRocksDB(metadata) + + if (loadedVersion != version) { + val versionsAndUniqueIds = currVersionLineage.collect { + case i if i.version > loadedVersion && i.version <= version => + (i.version, Option(i.checkpointUniqueId)) + } + replayChangelog(versionsAndUniqueIds) + loadedVersion = version + lineageManager.resetLineage(currVersionLineage) + } + // After changelog replay the numKeysOnWritingVersion will be updated to + // the correct number of keys in the loaded version. + numKeysOnLoadedVersion = numKeysOnWritingVersion + fileManagerMetrics = fileManager.latestLoadCheckpointMetrics + } + + lastCommitBasedStateStoreCkptId = None + loadedStateStoreCkptId = stateStoreCkptId + sessionStateStoreCkptId = Some(java.util.UUID.randomUUID.toString) + lastCommittedStateStoreCkptId = None + + if (conf.resetStatsOnLoad) { + nativeStats.reset + } + + logInfo(log"Loaded ${MDC(LogKeys.VERSION_NUM, version)} " + + log"with uniqueId ${MDC(LogKeys.UUID, stateStoreCkptId)}") + } catch { + case t: Throwable => + loadedVersion = -1 // invalidate loaded data + lastCommitBasedStateStoreCkptId = None + lastCommittedStateStoreCkptId = None + loadedStateStoreCkptId = None + sessionStateStoreCkptId = None + lineageManager.clear() + throw t + } + if (enableChangelogCheckpointing && !readOnly) { + // Make sure we don't leak resource. + changelogWriter.foreach(_.abort()) + // Initialize the changelog writer with lineage info + // The lineage stored in changelog files should normally start with + // the version of a snapshot, except for the first few versions. + // Because they are solely loaded from changelog file. + // (e.g. with default minDeltasForSnapshot, there is only 1_uuid1.changelog, no 1_uuid1.zip) + // It should end with exactly one version before the change log's version. + changelogWriter = Some(fileManager.getChangeLogWriter( + version + 1, + useColumnFamilies, + sessionStateStoreCkptId, + Some(currVersionLineage))) + } + this +} + + private def loadWithoutCheckpointId( version: Long, - stateStoreCkptId: Option[String] = None, readOnly: Boolean = false): RocksDB = { - assert(version >= 0) - acquire(LoadStore) - recordedMetrics = None - logInfo(log"Loading ${MDC(LogKeys.VERSION_NUM, version)}") try { - if (loadedVersion != version || - (enableStateStoreCheckpointIds && stateStoreCkptId.isDefined && - (loadedStateStoreCkptId.isEmpty || stateStoreCkptId.get != loadedStateStoreCkptId.get))) { + if (loadedVersion != version) { closeDB(ignoreException = false) val latestSnapshotVersion = fileManager.getLatestSnapshotVersion(version) - val metadata = fileManager.loadCheckpointFromDfs(latestSnapshotVersion, - workingDir, rocksDBFileMapping) + val metadata = fileManager.loadCheckpointFromDfs( + latestSnapshotVersion, + workingDir, + rocksDBFileMapping) + loadedVersion = latestSnapshotVersion // reset the last snapshot version to the latest available snapshot version @@ -307,37 +449,19 @@ class RocksDB( // Initialize maxVersion upon successful load from DFS fileManager.setMaxSeenVersion(version) - setInitialCFInfo() - metadata.columnFamilyMapping.foreach { mapping => - colFamilyNameToIdMap.putAll(mapping.asJava) - } + openLocalRocksDB(metadata) - metadata.maxColumnFamilyId.foreach { maxId => - maxColumnFamilyId.set(maxId) + if (loadedVersion != version) { + val versionsAndUniqueIds: Array[(Long, Option[String])] = + (loadedVersion + 1 to version).map((_, None)).toArray + replayChangelog(versionsAndUniqueIds) + loadedVersion = version } - openDB() - numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) { - // we don't track the total number of rows - discard the number being track - -1L - } else if (metadata.numKeys < 0) { - // we track the total number of rows, but the snapshot doesn't have tracking number - // need to count keys now - countKeys() - } else { - metadata.numKeys - } - if (loadedVersion != version) replayChangelog(version) // After changelog replay the numKeysOnWritingVersion will be updated to // the correct number of keys in the loaded version. numKeysOnLoadedVersion = numKeysOnWritingVersion fileManagerMetrics = fileManager.latestLoadCheckpointMetrics } - if (enableStateStoreCheckpointIds) { - lastCommitBasedStateStoreCkptId = None - loadedStateStoreCkptId = stateStoreCkptId - sessionStateStoreCkptId = Some(java.util.UUID.randomUUID.toString) - } - lastCommittedStateStoreCkptId = None if (conf.resetStatsOnLoad) { nativeStats.reset } @@ -345,10 +469,6 @@ class RocksDB( } catch { case t: Throwable => loadedVersion = -1 // invalidate loaded data - lastCommitBasedStateStoreCkptId = None - lastCommittedStateStoreCkptId = None - loadedStateStoreCkptId = None - sessionStateStoreCkptId = None throw t } if (enableChangelogCheckpointing && !readOnly) { @@ -359,6 +479,49 @@ class RocksDB( this } + /** + * Initialize key metrics based on the metadata loaded from DFS and open local RocksDB. + */ + private def openLocalRocksDB(metadata: RocksDBCheckpointMetadata): Unit = { + + setInitialCFInfo() + metadata.columnFamilyMapping.foreach { mapping => + colFamilyNameToIdMap.putAll(mapping.asJava) + } + + metadata.maxColumnFamilyId.foreach { maxId => + maxColumnFamilyId.set(maxId) + } + openDB() + numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) { + // we don't track the total number of rows - discard the number being track + -1L + } else if (metadata.numKeys < 0) { + // we track the total number of rows, but the snapshot doesn't have tracking number + // need to count keys now + countKeys() + } else { + metadata.numKeys + } + } + + def load( + version: Long, + stateStoreCkptId: Option[String] = None, + readOnly: Boolean = false): RocksDB = { + assert(version >= 0) + acquire(LoadStore) + recordedMetrics = None + logInfo(log"Loading ${MDC(LogKeys.VERSION_NUM, version)} with stateStoreCkptId: ${ + MDC(LogKeys.UUID, stateStoreCkptId.getOrElse(""))}") + if (stateStoreCkptId.isDefined || enableStateStoreCheckpointIds && version == 0) { + loadWithCheckpointId(version, stateStoreCkptId, readOnly) + } else { + loadWithoutCheckpointId(version, readOnly) + } + this + } + /** * Load from the start snapshot version and apply all the changelog records to reach the * end version. Note that this will copy all the necessary files from DFS to local disk as needed, @@ -417,7 +580,12 @@ class RocksDB( } else { metadata.numKeys } - if (loadedVersion != endVersion) replayChangelog(endVersion) + if (loadedVersion != endVersion) { + val versionsAndUniqueIds: Array[(Long, Option[String])] = + (loadedVersion + 1 to endVersion).map((_, None)).toArray + replayChangelog(versionsAndUniqueIds) + loadedVersion = endVersion + } // After changelog replay the numKeysOnWritingVersion will be updated to // the correct number of keys in the loaded version. numKeysOnLoadedVersion = numKeysOnWritingVersion @@ -431,16 +599,23 @@ class RocksDB( /** * Replay change log from the loaded version to the target version. */ - private def replayChangelog(endVersion: Long): Unit = { + private def replayChangelog(versionsAndUniqueIds: Array[(Long, Option[String])]): Unit = { + assert(!versionsAndUniqueIds.isEmpty && versionsAndUniqueIds.head._1 == loadedVersion + 1, + s"Replay changelog should start from one version after loadedVersion: $loadedVersion," + + s" but it is not." + ) + logInfo(log"Replaying changelog from version " + log"${MDC(LogKeys.LOADED_VERSION, loadedVersion)} -> " + - log"${MDC(LogKeys.END_VERSION, endVersion)}") - for (v <- loadedVersion + 1 to endVersion) { - logInfo(log"Replaying changelog on version " + - log"${MDC(LogKeys.VERSION_NUM, v)}") + log"${MDC(LogKeys.END_VERSION, versionsAndUniqueIds.lastOption.map(_._1))}") + + versionsAndUniqueIds.foreach { case (v, uniqueId) => + logInfo(log"replaying changelog from version ${MDC(LogKeys.VERSION_NUM, v)} with " + + log"unique Id: ${MDC(LogKeys.UUID, uniqueId)}") + var changelogReader: StateStoreChangelogReader = null try { - changelogReader = fileManager.getChangelogReader(v) + changelogReader = fileManager.getChangelogReader(v, uniqueId) changelogReader.foreach { case (recordType, key, value) => recordType match { case RecordType.PUT_RECORD => @@ -457,7 +632,6 @@ class RocksDB( if (changelogReader != null) changelogReader.closeIfNeeded() } } - loadedVersion = endVersion } /** @@ -648,8 +822,12 @@ class RocksDB( // inside the uploadSnapshot() called below. // If changelog checkpointing is enabled, snapshot will be uploaded asynchronously // during state store maintenance. - snapshot = Some(createSnapshot(checkpointDir, newVersion, - colFamilyNameToIdMap.asScala.toMap, maxColumnFamilyId.get().toShort)) + snapshot = Some(createSnapshot( + checkpointDir, + newVersion, + colFamilyNameToIdMap.asScala.toMap, + maxColumnFamilyId.get().toShort, + sessionStateStoreCkptId)) lastSnapshotVersion = newVersion } } @@ -695,6 +873,18 @@ class RocksDB( } } + if (enableStateStoreCheckpointIds) { + lastCommitBasedStateStoreCkptId = loadedStateStoreCkptId + lastCommittedStateStoreCkptId = sessionStateStoreCkptId + loadedStateStoreCkptId = sessionStateStoreCkptId + lineageManager.appendLineageItem(LineageItem(newVersion, sessionStateStoreCkptId.get)) + logInfo(log"Update checkpoint IDs and lineage: ${MDC( + LogKeys.LOADED_CHECKPOINT_ID, loadedStateStoreCkptId)}," + + log" ${MDC(LogKeys.LAST_COMMITTED_CHECKPOINT_ID, lastCommittedStateStoreCkptId)}," + + log" ${MDC(LogKeys.LAST_COMMIT_BASED_CHECKPOINT_ID, lastCommitBasedStateStoreCkptId)}," + + log" ${MDC(LogKeys.LINEAGE, lineageManager)}") + } + // Set maxVersion when checkpoint files are synced to DFS successfully // We need to handle this explicitly in RocksDB as we could use different // changeLogWriter instances in fileManager instance when committing @@ -702,11 +892,6 @@ class RocksDB( numKeysOnLoadedVersion = numKeysOnWritingVersion loadedVersion = newVersion - if (enableStateStoreCheckpointIds) { - lastCommitBasedStateStoreCkptId = loadedStateStoreCkptId - lastCommittedStateStoreCkptId = sessionStateStoreCkptId - loadedStateStoreCkptId = sessionStateStoreCkptId - } commitLatencyMs ++= Map( "flush" -> flushTimeMs, "compact" -> compactTimeMs, @@ -748,6 +933,7 @@ class RocksDB( lastCommittedStateStoreCkptId = None loadedStateStoreCkptId = None sessionStateStoreCkptId = None + lineageManager.clear() changelogWriter.foreach(_.abort()) // Make sure changelogWriter gets recreated next time. changelogWriter = None @@ -917,15 +1103,16 @@ class RocksDB( } private def createSnapshot( - checkpointDir: File, - version: Long, - columnFamilyMapping: Map[String, Short], - maxColumnFamilyId: Short): RocksDBSnapshot = { + checkpointDir: File, + version: Long, + columnFamilyMapping: Map[String, Short], + maxColumnFamilyId: Short, + uniqueId: Option[String] = None): RocksDBSnapshot = { val (dfsFileSuffix, immutableFileMapping) = rocksDBFileMapping.createSnapshotFileMapping( fileManager, checkpointDir, version) RocksDBSnapshot(checkpointDir, version, numKeysOnWritingVersion, - columnFamilyMapping, maxColumnFamilyId, dfsFileSuffix, immutableFileMapping) + columnFamilyMapping, maxColumnFamilyId, dfsFileSuffix, immutableFileMapping, uniqueId) } /** @@ -1036,6 +1223,43 @@ class RocksDB( Option(acquiredThreadInfo).map(_.copy()) } + /** Upload the snapshot to DFS and remove it from snapshots pending */ + private def uploadSnapshot( + snapshot: RocksDB#RocksDBSnapshot, + fileManager: RocksDBFileManager, + snapshotsPendingUpload: Set[RocksDBVersionSnapshotInfo], + loggingId: String): RocksDBFileManagerMetrics = { + var fileManagerMetrics: RocksDBFileManagerMetrics = null + try { + val uploadTime = timeTakenMs { + fileManager.saveCheckpointToDfs(snapshot.checkpointDir, + snapshot.version, snapshot.numKeys, snapshot.fileMapping, + Some(snapshot.columnFamilyMapping), Some(snapshot.maxColumnFamilyId), snapshot.uniqueId) + fileManagerMetrics = fileManager.latestSaveCheckpointMetrics + + val snapshotInfo = RocksDBVersionSnapshotInfo(snapshot.version, snapshot.dfsFileSuffix) + // We are only removing the uploaded snapshot info from the pending set, + // to let the file mapping (i.e. query threads) know that the snapshot (i.e. and its files) + // have been uploaded to DFS. We don't touch the file mapping here to avoid corrupting it. + snapshotsPendingUpload.remove(snapshotInfo) + } + // This is relative aggressive because that even if the uploading succeeds, + // it is not necessarily the one written to the commit log. But we can always load lineage + // from commit log so it is fine. + lineageManager.resetLineage(lineageManager.getLineageForCurrVersion() + .filter(i => i.version >= snapshot.version)) + logInfo(log"${MDC(LogKeys.LOG_ID, loggingId)}: " + + log"Upload snapshot of version ${MDC(LogKeys.VERSION_NUM, snapshot.version)}, " + + log"with uniqueId: ${MDC(LogKeys.UUID, snapshot.uniqueId)} " + + log"time taken: ${MDC(LogKeys.TIME_UNITS, uploadTime)} ms. " + + log"Current lineage: ${MDC(LogKeys.LINEAGE, lineageManager)}") + } finally { + snapshot.close() + } + + fileManagerMetrics + } + /** Create a native RocksDB logger that forwards native logs to log4j with correct log levels. */ private def createLogger(): Logger = { val dbLogger = new Logger(rocksDbOptions.infoLogLevel()) { @@ -1087,36 +1311,10 @@ class RocksDB( } object RocksDB extends Logging { + private def printLineageItems(lineage: Array[LineageItem]): String = lineage.map { + case LineageItem(l, optStr) => s"$l:$optStr" + }.mkString(" ") - /** Upload the snapshot to DFS and remove it from snapshots pending */ - private def uploadSnapshot( - snapshot: RocksDB#RocksDBSnapshot, - fileManager: RocksDBFileManager, - snapshotsPendingUpload: Set[RocksDBVersionSnapshotInfo], - loggingId: String): RocksDBFileManagerMetrics = { - var fileManagerMetrics: RocksDBFileManagerMetrics = null - try { - val uploadTime = timeTakenMs { - fileManager.saveCheckpointToDfs(snapshot.checkpointDir, - snapshot.version, snapshot.numKeys, snapshot.fileMapping, - Some(snapshot.columnFamilyMapping), Some(snapshot.maxColumnFamilyId)) - fileManagerMetrics = fileManager.latestSaveCheckpointMetrics - - val snapshotInfo = RocksDBVersionSnapshotInfo(snapshot.version, snapshot.dfsFileSuffix) - // We are only removing the uploaded snapshot info from the pending set, - // to let the file mapping (i.e. query threads) know that the snapshot (i.e. and its files) - // have been uploaded to DFS. We don't touch the file mapping here to avoid corrupting it. - snapshotsPendingUpload.remove(snapshotInfo) - } - logInfo(log"${MDC(LogKeys.LOG_ID, loggingId)}: Upload snapshot of version " + - log"${MDC(LogKeys.VERSION_NUM, snapshot.version)}," + - log" time taken: ${MDC(LogKeys.TIME_UNITS, uploadTime)} ms") - } finally { - snapshot.close() - } - - fileManagerMetrics - } /** Records the duration of running `body` for the next query progress update. */ private def timeTakenMs(body: => Unit): Long = Utils.timeTakenMs(body)._2 @@ -1533,3 +1731,40 @@ case class AcquiredThreadInfo( } } +/** + * A helper class to manage the lineage information when checkpoint unique id is enabled. + * "lineage" is an array of LineageItem (version, uniqueId) pair. + * + * The first item of "lineage" should normally be the version of a snapshot, except + * for the first few versions. Because they are solely loaded from changelog file. + * (i.e. with default minDeltasForSnapshot, there is only 1_uuid1.changelog, no 1_uuid1.zip) + * + * The last item of "lineage" corresponds to one version before the to-be-committed version. + */ +private[sql] class RocksDBLineageManager { + @volatile private var lineage: Array[LineageItem] = Array.empty + + override def toString: String = lineage.map { + case LineageItem(version, uuid) => s"$version: $uuid" + }.mkString(" ") + + def appendLineageItem(item: LineageItem): Unit = { + lineage = lineage :+ item + } + + def resetLineage(newLineage: Array[LineageItem]): Unit = { + lineage = newLineage + } + + def getLineageForCurrVersion(): Array[LineageItem] = { + lineage.clone() + } + + def contains(item: LineageItem): Boolean = { + lineage.contains(item) + } + + def clear(): Unit = { + lineage = Array.empty + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 483e4a32cd85b..e42a46dfbe15a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -41,6 +41,7 @@ import org.apache.spark.internal.{Logging, LogKeys, MDC, MessageWithContext} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -151,7 +152,10 @@ class RocksDBFileManager( private var minSeenVersion = 1L @volatile private var rootDirChecked: Boolean = false - private val versionToRocksDBFiles = new ConcurrentHashMap[Long, Seq[RocksDBImmutableFile]] + + // (version, checkpointUniqueId) -> immutable files + private val versionToRocksDBFiles = + new ConcurrentHashMap[(Long, Option[String]), Seq[RocksDBImmutableFile]]() /** * Get the changelog version based on rocksDB features. @@ -184,6 +188,7 @@ class RocksDBFileManager( val enableStateStoreCheckpointIds = checkpointUniqueId.isDefined val changelogVersion = getChangelogWriterVersion( useColumnFamilies, enableStateStoreCheckpointIds) + val changelogWriter = changelogVersion match { case 1 => new StateStoreChangelogWriterV1(fm, changelogFile, codec) @@ -245,13 +250,15 @@ class RocksDBFileManager( numKeys: Long, fileMapping: Map[String, RocksDBSnapshotFile], columnFamilyMapping: Option[Map[String, Short]] = None, - maxColumnFamilyId: Option[Short] = None): Unit = { + maxColumnFamilyId: Option[Short] = None, + checkpointUniqueId: Option[String] = None): Unit = { logFilesInDir(checkpointDir, log"Saving checkpoint files " + log"for version ${MDC(LogKeys.VERSION_NUM, version)}") val (localImmutableFiles, localOtherFiles) = listRocksDBFiles(checkpointDir) - val rocksDBFiles = saveImmutableFilesToDfs(version, localImmutableFiles, fileMapping) - val metadata = RocksDBCheckpointMetadata( - rocksDBFiles, numKeys, columnFamilyMapping, maxColumnFamilyId) + val rocksDBFiles = saveImmutableFilesToDfs( + version, localImmutableFiles, fileMapping, checkpointUniqueId) + val metadata = RocksDBCheckpointMetadata(rocksDBFiles, numKeys, columnFamilyMapping, + maxColumnFamilyId) val metadataFile = localMetadataFile(checkpointDir) metadata.writeToFile(metadataFile) logInfo(log"Written metadata for version ${MDC(LogKeys.VERSION_NUM, version)}:\n" + @@ -270,8 +277,9 @@ class RocksDBFileManager( rootDirChecked = true } } - zipToDfsFile(localOtherFiles :+ metadataFile, dfsBatchZipFile(version)) - logInfo(log"Saved checkpoint file for version ${MDC(LogKeys.VERSION_NUM, version)}") + zipToDfsFile(localOtherFiles :+ metadataFile, dfsBatchZipFile(version, checkpointUniqueId)) + logInfo(log"Saved checkpoint file for version ${MDC(LogKeys.VERSION_NUM, version)} " + + log"checkpointUniqueId: ${MDC(LogKeys.UUID, checkpointUniqueId.getOrElse(""))}") } /** @@ -283,12 +291,14 @@ class RocksDBFileManager( def loadCheckpointFromDfs( version: Long, localDir: File, - rocksDBFileMapping: RocksDBFileMapping): RocksDBCheckpointMetadata = { - logInfo(log"Loading checkpoint files for version ${MDC(LogKeys.VERSION_NUM, version)}") + rocksDBFileMapping: RocksDBFileMapping, + checkpointUniqueId: Option[String] = None): RocksDBCheckpointMetadata = { + logInfo(log"Loading checkpoint files for version ${MDC(LogKeys.VERSION_NUM, version)} " + + log"checkpointUniqueId: ${MDC(LogKeys.UUID, checkpointUniqueId.getOrElse(""))}") // The unique ids of SST files are checked when opening a rocksdb instance. The SST files // in larger versions can't be reused even if they have the same size and name because // they belong to another rocksdb instance. - versionToRocksDBFiles.keySet().removeIf(_ >= version) + versionToRocksDBFiles.keySet().removeIf(_._1 >= version) val metadata = if (version == 0) { if (localDir.exists) Utils.deleteRecursively(localDir) localDir.mkdirs() @@ -296,7 +306,7 @@ class RocksDBFileManager( } else { // Delete all non-immutable files in local dir, and unzip new ones from DFS commit file listRocksDBFiles(localDir)._2.foreach(_.delete()) - Utils.unzipFilesFromFile(fs, dfsBatchZipFile(version), localDir) + Utils.unzipFilesFromFile(fs, dfsBatchZipFile(version, checkpointUniqueId), localDir) // Copy the necessary immutable files val metadataFile = localMetadataFile(localDir) @@ -304,7 +314,7 @@ class RocksDBFileManager( logInfo(log"Read metadata for version ${MDC(LogKeys.VERSION_NUM, version)}:\n" + log"${MDC(LogKeys.METADATA_JSON, metadata.prettyJson)}") loadImmutableFilesFromDfs(metadata.immutableFiles, localDir, rocksDBFileMapping, version) - versionToRocksDBFiles.put(version, metadata.immutableFiles) + versionToRocksDBFiles.put((version, checkpointUniqueId), metadata.immutableFiles) metadataFile.delete() metadata } @@ -313,6 +323,17 @@ class RocksDBFileManager( metadata } + // Return if there is a snapshot file at the corresponding version + // and optionally with checkpointunique id, e.g. version.zip or version_uniqueId.zip + def existsSnapshotFile(version: Long, checkpointUniqueId: Option[String] = None): Boolean = { + if (!rootDirChecked) { + val path = new Path(dfsRootDir) + if (!fm.exists(path)) fm.mkdirs(path) + rootDirChecked = true + } + fm.exists(dfsBatchZipFile(version, checkpointUniqueId)) + } + // Get latest snapshot version <= version def getLatestSnapshotVersion(version: Long): Long = { val path = new Path(dfsRootDir) @@ -331,20 +352,52 @@ class RocksDBFileManager( } } + /** + * Based on the ground truth lineage loaded from changelog file (lineage), this function + * does file listing to find all snapshot (version, uniqueId) pairs, and finds + * the ground truth latest snapshot (version, uniqueId) the db instance needs to load. + * + * @param lineage The ground truth lineage loaded from changelog file, sorted by id + * @return The ground truth latest snapshot (version, uniqueId) the db instance needs to load, + * when the return value is None it means ther is no such snapshot found. + */ + def getLatestSnapshotVersionAndUniqueIdFromLineage( + lineage: Array[LineageItem]): Option[(Long, String)] = { + val path = new Path(dfsRootDir) + if (fm.exists(path)) { + fm.list(path, onlyZipFiles) + .map(_.getPath.getName.stripSuffix(".zip").split("_")) + .collect { + case Array(ver, id) if lineage.contains(LineageItem(ver.toLong, id)) => + (ver.toLong, id) + } + .sortBy(_._1) + .reverse + .headOption + } else { + None + } + } /** Get the latest version available in the DFS directory. If no data present, it returns 0. */ def getLatestVersion(): Long = { val path = new Path(dfsRootDir) if (fm.exists(path)) { val files = fm.list(path).map(_.getPath) - val changelogFileVersions = files - .filter(onlyChangelogFiles.accept) - .map(_.getName.stripSuffix(".changelog")) - .map(_.toLong) - val snapshotFileVersions = files - .filter(onlyZipFiles.accept) - .map(_.getName.stripSuffix(".zip")) - .map(_.toLong) + val changelogFileVersions = files.filter(onlyChangelogFiles.accept) + .map { fileName => + fileName.getName.stripSuffix(".changelog").split("_") match { + case Array(version, _) => version.toLong + case Array(version) => version.toLong + } + } + val snapshotFileVersions = files.filter(onlyZipFiles.accept) + .map { fileName => + fileName.getName.stripSuffix(".zip").split("_") match { + case Array(version, _) => version.toLong + case Array(version) => version.toLong + } + } val versions = changelogFileVersions ++ snapshotFileVersions versions.foldLeft(0L)(math.max) } else { @@ -385,15 +438,18 @@ class RocksDBFileManager( } } - private def deleteChangelogFiles(versionsToDelete: Array[Long]): Unit = { - versionsToDelete.foreach { version => + private def deleteChangelogFiles( + versionsAndUniqueIdsToDelete: Array[(Long, Option[String])]): Unit = { + versionsAndUniqueIdsToDelete.foreach { case (version, uniqueId) => try { - fm.delete(dfsChangelogFile(version)) - logInfo(log"Deleted changelog file ${MDC(LogKeys.VERSION_NUM, version)}") + fm.delete(dfsChangelogFile(version, uniqueId)) + logInfo(log"Deleted changelog file ${MDC(LogKeys.VERSION_NUM, version)} uniqueId: " + + log"${MDC(LogKeys.UUID, uniqueId.getOrElse(""))}") } catch { case e: Exception => logWarning( - log"Error deleting changelog file for version ${MDC(LogKeys.FILE_VERSION, version)}", e) + log"Error deleting changelog file for version ${MDC(LogKeys.FILE_VERSION, version)} " + + log"uniqueId: ${MDC(LogKeys.UUID, uniqueId.getOrElse(""))}", e) } } } @@ -483,38 +539,43 @@ class RocksDBFileManager( val snapshotFiles = allFiles.filter(file => onlyZipFiles.accept(file)) val changelogFiles = allFiles.filter(file => onlyChangelogFiles.accept(file)) // All versions present in DFS, sorted - val sortedSnapshotVersions = snapshotFiles - .map(_.getName.stripSuffix(".zip")) - .map(_.toLong) - .sorted + val sortedSnapshotVersionsAndUniqueIds = snapshotFiles + .map(_.getName.stripSuffix(".zip").split("_")) + .map { + case Array(version, uniqueId) => (version.toLong, Some(uniqueId)) + case Array(version) => (version.toLong, None) + } + .sortBy(_._1) // Return if no versions generated yet - if (sortedSnapshotVersions.isEmpty) return + if (sortedSnapshotVersionsAndUniqueIds.isEmpty) return // Find the versions to delete - val maxSnapshotVersionPresent = sortedSnapshotVersions.last + val maxSnapshotVersionPresent = sortedSnapshotVersionsAndUniqueIds.last._1 // In order to reconstruct numVersionsToRetain version, retain the latest snapshot // that satisfies (version <= maxSnapshotVersionPresent - numVersionsToRetain + 1). // If none of the snapshots satisfy the condition, minVersionToRetain will be 0 and // no version gets deleted. - val minVersionToRetain = sortedSnapshotVersions + val minVersionToRetain = sortedSnapshotVersionsAndUniqueIds + .map(_._1) .filter(_ <= maxSnapshotVersionPresent - numVersionsToRetain + 1) .foldLeft(0L)(math.max) // When snapshotVersionToDelete is non-empty, there are at least 2 snapshot versions. // We only delete orphan files when there are at least 2 versions, // which avoid deleting files for running tasks. - val snapshotVersionsToDelete = sortedSnapshotVersions.filter(_ < minVersionToRetain) - if (snapshotVersionsToDelete.isEmpty) return - + val snapshotVersionsAndUniqueIdsToDelete = sortedSnapshotVersionsAndUniqueIds + .filter(_._1 < minVersionToRetain) + val snapshotVersionsToDelete = snapshotVersionsAndUniqueIdsToDelete.map(_._1) + if (snapshotVersionsAndUniqueIdsToDelete.isEmpty) return // Resolve RocksDB files for all the versions and find the max version each file is used val fileToMaxUsedVersion = new mutable.HashMap[String, Long] - sortedSnapshotVersions.foreach { version => - val files = Option(versionToRocksDBFiles.get(version)).getOrElse { - val newResolvedFiles = getImmutableFilesFromVersionZip(version) - versionToRocksDBFiles.put(version, newResolvedFiles) + sortedSnapshotVersionsAndUniqueIds.foreach { case (version, uniqueId) => + val files = Option(versionToRocksDBFiles.get((version, uniqueId))).getOrElse { + val newResolvedFiles = getImmutableFilesFromVersionZip(version, uniqueId) + versionToRocksDBFiles.put((version, uniqueId), newResolvedFiles) newResolvedFiles } files.foreach(f => fileToMaxUsedVersion(f.dfsFileName) = @@ -557,11 +618,11 @@ class RocksDBFileManager( } // Delete the version files and forget about them - snapshotVersionsToDelete.foreach { version => - val versionFile = dfsBatchZipFile(version) + snapshotVersionsAndUniqueIdsToDelete.foreach { case (version, uniqueId) => + val versionFile = dfsBatchZipFile(version, uniqueId) try { fm.delete(versionFile) - versionToRocksDBFiles.remove(version) + versionToRocksDBFiles.remove((version, uniqueId)) logDebug(s"Deleted version $version") } catch { case e: Exception => @@ -573,10 +634,16 @@ class RocksDBFileManager( log"(failed to delete" + log"${MDC(LogKeys.NUM_FILES_FAILED_TO_DELETE, failedToDelete)} files) " + log"not used in versions >= ${MDC(LogKeys.MIN_VERSION_NUM, minVersionToRetain)}") - val changelogVersionsToDelete = changelogFiles - .map(_.getName.stripSuffix(".changelog")).map(_.toLong) - .filter(_ < minVersionToRetain) - deleteChangelogFiles(changelogVersionsToDelete) + + val changelogVersionsAndUniqueIdsToDelete: Array[(Long, Option[String])] = changelogFiles + .map(_.getName.stripSuffix(".changelog").split("_")) + .map { + case Array(version, uniqueId) => (version.toLong, Option(uniqueId)) + case Array(version) => (version.toLong, None) + } + .filter(_._1 < minVersionToRetain) + + deleteChangelogFiles(changelogVersionsAndUniqueIdsToDelete) // Always set minSeenVersion for regular deletion frequency even if deletion fails. // This is safe because subsequent calls retry deleting old version files @@ -587,10 +654,12 @@ class RocksDBFileManager( private def saveImmutableFilesToDfs( version: Long, localFiles: Seq[File], - fileMappings: Map[String, RocksDBSnapshotFile]): Seq[RocksDBImmutableFile] = { + fileMappings: Map[String, RocksDBSnapshotFile], + checkpointUniqueId: Option[String] = None): Seq[RocksDBImmutableFile] = { // Get the immutable files used in previous versions, as some of those uploaded files can be // reused for this version - logInfo(log"Saving RocksDB files to DFS for ${MDC(LogKeys.VERSION_NUM, version)}") + logInfo(log"Saving RocksDB files to DFS for version ${MDC(LogKeys.VERSION_NUM, version)} " + + log"uniqueId: ${MDC(LogKeys.UUID, checkpointUniqueId.getOrElse(""))}") var bytesCopied = 0L var filesCopied = 0L @@ -626,7 +695,7 @@ class RocksDBFileManager( log"(${MDC(LogKeys.NUM_BYTES, bytesCopied)} bytes) from local to" + log" DFS for version ${MDC(LogKeys.VERSION_NUM, version)}. " + log"${MDC(LogKeys.NUM_FILES_REUSED, filesReused)} files reused without copying.") - versionToRocksDBFiles.put(version, immutableFiles) + versionToRocksDBFiles.put((version, checkpointUniqueId), immutableFiles) saveCheckpointMetrics = RocksDBFileManagerMetrics( bytesCopied = bytesCopied, filesCopied = filesCopied, @@ -714,10 +783,11 @@ class RocksDBFileManager( } /** Get the SST files required for a version from the version zip file in DFS */ - private def getImmutableFilesFromVersionZip(version: Long): Seq[RocksDBImmutableFile] = { + private def getImmutableFilesFromVersionZip( + version: Long, checkpointUniqueId: Option[String] = None): Seq[RocksDBImmutableFile] = { Utils.deleteRecursively(localTempDir) localTempDir.mkdirs() - Utils.unzipFilesFromFile(fs, dfsBatchZipFile(version), localTempDir) + Utils.unzipFilesFromFile(fs, dfsBatchZipFile(version, checkpointUniqueId), localTempDir) val metadataFile = localMetadataFile(localTempDir) val metadata = RocksDBCheckpointMetadata.readFromFile(metadataFile) metadata.immutableFiles @@ -789,7 +859,9 @@ class RocksDBFileManager( immutableFile.dfsFileName.substring(suffixStart + 1, suffixEnd) } - private def dfsBatchZipFile(version: Long): Path = new Path(s"$dfsRootDir/$version.zip") + private def dfsBatchZipFile(version: Long, checkpointUniqueId: Option[String] = None): Path = + checkpointUniqueId.map(id => new Path(s"$dfsRootDir/${version}_$id.zip")) + .getOrElse(new Path(s"$dfsRootDir/$version.zip")) // We use changelog suffix intentionally so that we can tell the difference from changelog file of // HDFSBackedStateStore which is named version.delta. private def dfsChangelogFile(version: Long, checkpointUniqueId: Option[String] = None): Path = @@ -890,7 +962,7 @@ case class RocksDBCheckpointMetadata( /** Helper class for [[RocksDBCheckpointMetadata]] */ object RocksDBCheckpointMetadata { - val VERSION = 1 + val VERSION = SQLConf.get.stateStoreCheckpointFormatVersion implicit val format: Formats = Serialization.formats(NoTypeHints) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala index 1b63180171be8..b402344a73177 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.{Encoders, Row} import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBFileManager, RocksDBStateStoreProvider, TestClass} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBFileManager, RocksDBStateStoreProvider, TestClass} import org.apache.spark.sql.functions.{col, explode, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{InputMapRow, ListState, MapInputEvent, MapOutputEvent, MapStateTTLProcessor, MaxEventTimeStatefulProcessor, OutputMode, RunningCountStatefulProcessor, RunningCountStatefulProcessorWithProcTimeTimerUpdates, StatefulProcessor, StateStoreMetricsTest, TestMapStateProcessor, TimeMode, TimerValues, TransformWithStateSuiteUtils, Trigger, TTLConfig, ValueState} @@ -126,7 +126,7 @@ class SessionGroupsStatefulProcessorWithTTL extends * Test suite to verify integration of state data source reader with the transformWithState operator */ class StateDataSourceTransformWithStateSuite extends StateStoreMetricsTest - with AlsoTestWithChangelogCheckpointingEnabled with AlsoTestWithEncodingTypes { + with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { import testImplicits._ @@ -1085,7 +1085,7 @@ class StateDataSourceTransformWithStateSuite extends StateStoreMetricsTest Utils.deleteRecursively(new File(changelogFilePath)) // Write the retained entry back to the changelog - val changelogWriter = fileManager.getChangeLogWriter(3, true) + val changelogWriter = fileManager.getChangeLogWriter(3) changelogWriter.put(retainEntry._2, retainEntry._3) changelogWriter.commit() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala index 9ac74eb5b9e8f..24c99bffd8bab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala @@ -182,7 +182,7 @@ class CkptIdCollectingStateStoreProviderWrapper extends StateStoreProvider { // return their own state store checkpointID. This can happen because of task retry or // speculative execution. class RocksDBStateStoreCheckpointFormatV2Suite extends StreamTest - with AlsoTestWithChangelogCheckpointingEnabled { + with AlsoTestWithRocksDBFeatures { import testImplicits._ val providerClassName = classOf[CkptIdCollectingStateStoreProviderWrapper].getCanonicalName @@ -445,11 +445,12 @@ class RocksDBStateStoreCheckpointFormatV2Suite extends StreamTest val numBatches = checkpointInfoList.size / 8 // We don't pass batch versions that would need base checkpoint IDs because we don't know - // batchIDs for that. We only know that there are 3 batches without it. + // batchIDs for that. We only know that there are 1 batches without it. + // The two checkpoint IDs in between are stored in the commit log. validateCheckpointInfo(numBatches, 4, Set()) assert(CkptIdCollectingStateStoreWrapper .getStateStoreCheckpointInfos - .count(_.baseStateStoreCkptId.isDefined) == (numBatches - 3) * 8) + .count(_.baseStateStoreCkptId.isDefined) == (numBatches - 1) * 8) } testWithCheckpointInfoTracked(s"checkpointFormatVersion2 validate DropDuplicates") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala index d20cfb04f8e81..f170de66ee9df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.streaming.OutputMode.Update import org.apache.spark.util.Utils class RocksDBStateStoreIntegrationSuite extends StreamTest - with AlsoTestWithChangelogCheckpointingEnabled { + with AlsoTestWithRocksDBFeatures { import testImplicits._ testWithColumnFamilies("RocksDBStateStore", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala index 0abdcadefbd55..a8e9d6db01e24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.Utils @ExtendedSQLTest class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvider] - with AlsoTestWithChangelogCheckpointingEnabled + with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes with SharedSparkSession with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 2b48bc2e501bd..3ec81a7ef13e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming.state import java.io._ import java.nio.charset.Charset +import java.util.UUID import java.util.concurrent.Executors import scala.collection.mutable @@ -48,7 +49,6 @@ import org.apache.spark.tags.SlowSQLTest import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.util.ArrayImplicits._ - class NoOverwriteFileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration) extends FileSystemBasedCheckpointFileManager(path, hadoopConf) { @@ -86,16 +86,22 @@ trait RocksDBStateStoreChangelogCheckpointingTestUtil { def snapshotVersionsPresent(dir: File): Seq[Long] = { dir.listFiles.filter(_.getName.endsWith(".zip")) - .map(_.getName.stripSuffix(".zip")) - .map(_.toLong) + .map(_.getName.stripSuffix(".zip").split("_")) + .map { + case Array(version, _) => version.toLong + case Array(version) => version.toLong + } .sorted .toImmutableArraySeq } def changelogVersionsPresent(dir: File): Seq[Long] = { dir.listFiles.filter(_.getName.endsWith(".changelog")) - .map(_.getName.stripSuffix(".changelog")) - .map(_.toLong) + .map(_.getName.stripSuffix(".changelog").split("_")) + .map { + case Array(version, _) => version.toLong + case Array(version) => version.toLong + } .sorted .toImmutableArraySeq } @@ -114,7 +120,7 @@ trait AlsoTestWithEncodingTypes extends SQLTestUtils { } } -trait AlsoTestWithChangelogCheckpointingEnabled +trait AlsoTestWithRocksDBFeatures extends SQLTestUtils with RocksDBStateStoreChangelogCheckpointingTestUtil { sealed trait TestMode @@ -156,6 +162,20 @@ trait AlsoTestWithChangelogCheckpointingEnabled } } + def testWithRocksDBStateStore( + testName: String, testTags: Tag*)(testBody: => Any): Unit = { + super.test(testName, testTags: _*) { + // in case tests have any code that needs to execute before every test + super.beforeEach() + withSQLConf( + SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName) { + testBody + } + // in case tests have any code that needs to execute after every test + super.afterEach() + } + } + def testWithColumnFamilies( testName: String, testMode: TestMode, @@ -190,6 +210,46 @@ trait AlsoTestWithChangelogCheckpointingEnabled } } } + + def testWithStateStoreCheckpointIdsAndColumnFamilies( + testName: String, + testMode: TestMode, + testTags: Tag*) + (testBody: (Boolean, Boolean) => Any): Unit = { + Seq(true, false).foreach { enableStateStoreCheckpointIds => + val newTestName = s"$testName - with enableStateStoreCheckpointIds = " + + s"$enableStateStoreCheckpointIds" + testWithColumnFamilies(newTestName, testMode, testTags: _*) { colFamiliesEnabled => + testBody(enableStateStoreCheckpointIds, colFamiliesEnabled) + } + } + } + + def testWithStateStoreCheckpointIds( + testName: String, + testTags: Tag*) + (testBody: Boolean => Any): Unit = { + Seq(true, false).foreach { enableStateStoreCheckpointIds => + val newTestName = s"$testName - with enableStateStoreCheckpointIds = " + + s"$enableStateStoreCheckpointIds" + test(newTestName, testTags: _*) { enableStateStoreCheckpointIds => + testBody(enableStateStoreCheckpointIds) + } + } + } + + def testWithStateStoreCheckpointIdsAndChangelogEnabled( + testName: String, + testTags: Tag*) + (testBody: Boolean => Any): Unit = { + Seq(true, false).foreach { enableStateStoreCheckpointIds => + val newTestName = s"$testName - with enableStateStoreCheckpointIds = " + + s"$enableStateStoreCheckpointIds" + testWithChangelogCheckpointingDisabled(newTestName, testTags: _*) { + enableStateStoreCheckpointIds => testBody(enableStateStoreCheckpointIds) + } + } + } } class OpenNumCountedTestInputStream(in: InputStream) extends FSDataInputStream(in) { @@ -241,7 +301,7 @@ object OpenNumCountedTestInputStream extends Logging { } @SlowSQLTest -class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with SharedSparkSession +class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession with PrivateMethodTester { override protected def sparkConf: SparkConf = { @@ -268,6 +328,26 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } + testWithStateStoreCheckpointIdsAndColumnFamilies("RocksDB: check changelog and snapshot version", + TestWithChangelogCheckpointingEnabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = 1) + new File(remoteDir).delete() // to make sure that the directory gets created + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, + useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + for (version <- 0 to 49) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + if ((version + 1) % 5 == 0) db.doMaintenance() + } + } + } + testWithColumnFamilies( "RocksDB: check changelog and snapshot version", TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => @@ -283,231 +363,253 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - if (isChangelogCheckpointingEnabled) { - assert(changelogVersionsPresent(remoteDir) === (1 to 50)) - assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) - } else { - assert(changelogVersionsPresent(remoteDir) === Seq.empty) - assert(snapshotVersionsPresent(remoteDir) === (1 to 50)) - } + if (isChangelogCheckpointingEnabled) { + assert(changelogVersionsPresent(remoteDir) === (1 to 50)) + assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) + } else { + assert(changelogVersionsPresent(remoteDir) === Seq.empty) + assert(snapshotVersionsPresent(remoteDir) === (1 to 50)) + } } - testWithColumnFamilies(s"RocksDB: load version that doesn't exist", + testWithStateStoreCheckpointIdsAndColumnFamilies(s"RocksDB: load version that doesn't exist", TestWithBothChangelogCheckpointingEnabledAndDisabled) { - colFamiliesEnabled => - val provider = new RocksDBStateStoreProvider() - var ex = intercept[SparkException] { - provider.getStore(-1) - } - checkError( - ex, - condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", - parameters = Map("version" -> "-1") - ) - ex = intercept[SparkException] { - provider.getReadStore(-1) - } - checkError( - ex, - condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", - parameters = Map("version" -> "-1") - ) - - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, useColumnFamilies = colFamiliesEnabled) { db => + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val provider = new RocksDBStateStoreProvider() + var ex = intercept[SparkException] { + provider.getStore(-1) + } + checkError( + ex, + condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", + parameters = Map("version" -> "-1") + ) ex = intercept[SparkException] { - db.load(1) + provider.getReadStore(-1) } checkError( ex, - condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE", - parameters = Map( - "fileToRead" -> s"$remoteDir/1.changelog" - ) + condition = "CANNOT_LOAD_STATE_STORE.UNEXPECTED_VERSION", + parameters = Map("version" -> "-1") ) - } - } - testWithColumnFamilies( - "RocksDB: purge changelog and snapshots with minVersionsToDelete = 0", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - val conf = dbConf.copy(enableChangelogCheckpointing = true, - minVersionsToRetain = 3, minDeltasForSnapshot = 1, minVersionsToDelete = 0) - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - db.load(0) - db.commit() - for (version <- 1 to 2) { - db.load(version) - db.commit() - db.doMaintenance() + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + ex = intercept[SparkException] { + db.load(1) + } + checkError( + ex, + condition = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE", + parameters = Map( + "fileToRead" -> s"$remoteDir/1.changelog" + ) + ) } - assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) - assert(changelogVersionsPresent(remoteDir) == Seq(1, 2, 3)) + } - for (version <- 3 to 4) { - db.load(version) + testWithStateStoreCheckpointIdsAndColumnFamilies( + "RocksDB: purge changelog and snapshots with minVersionsToDelete = 0", + TestWithChangelogCheckpointingEnabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created + val conf = dbConf.copy(enableChangelogCheckpointing = true, + minVersionsToRetain = 3, minDeltasForSnapshot = 1, minVersionsToDelete = 0) + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => + db.load(0) db.commit() - } - assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) - assert(changelogVersionsPresent(remoteDir) == (1 to 5)) - db.doMaintenance() - // 3 is the latest snapshot <= maxSnapshotVersionPresent - minVersionsToRetain + 1 - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 5)) - assert(changelogVersionsPresent(remoteDir) == (3 to 5)) + for (version <- 1 to 2) { + db.load(version) + db.commit() + db.doMaintenance() + } + assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) + assert(changelogVersionsPresent(remoteDir) == Seq(1, 2, 3)) - for (version <- 5 to 7) { - db.load(version) - db.commit() + for (version <- 3 to 4) { + db.load(version) + db.commit() + } + assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) + assert(changelogVersionsPresent(remoteDir) == (1 to 5)) + db.doMaintenance() + // 3 is the latest snapshot <= maxSnapshotVersionPresent - minVersionsToRetain + 1 + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 5)) + assert(changelogVersionsPresent(remoteDir) == (3 to 5)) + + for (version <- 5 to 7) { + db.load(version) + db.commit() + } + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 5)) + assert(changelogVersionsPresent(remoteDir) == (3 to 8)) + db.doMaintenance() + // 5 is the latest snapshot <= maxSnapshotVersionPresent - minVersionsToRetain + 1 + assert(snapshotVersionsPresent(remoteDir) === Seq(5, 8)) + assert(changelogVersionsPresent(remoteDir) == (5 to 8)) } - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 5)) - assert(changelogVersionsPresent(remoteDir) == (3 to 8)) - db.doMaintenance() - // 5 is the latest snapshot <= maxSnapshotVersionPresent - minVersionsToRetain + 1 - assert(snapshotVersionsPresent(remoteDir) === Seq(5, 8)) - assert(changelogVersionsPresent(remoteDir) == (5 to 8)) - } } - testWithColumnFamilies( + testWithStateStoreCheckpointIdsAndColumnFamilies( "RocksDB: purge version files with minVersionsToDelete > 0", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - val conf = dbConf.copy( - minVersionsToRetain = 3, minDeltasForSnapshot = 1, minVersionsToDelete = 3) - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - // Commit 5 versions - // stale versions: (1, 2) - // keep versions: (3, 4, 5) - for (version <- 0 to 4) { - // Should upload latest snapshot but not delete any files - // since number of stale versions < minVersionsToDelete - db.load(version) - db.commit() - db.doMaintenance() - } + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created + val conf = dbConf.copy( + minVersionsToRetain = 3, minDeltasForSnapshot = 1, minVersionsToDelete = 3) + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => + // Commit 5 versions + // stale versions: (1, 2) + // keep versions: (3, 4, 5) + for (version <- 0 to 4) { + // Should upload latest snapshot but not delete any files + // since number of stale versions < minVersionsToDelete + db.load(version) + db.commit() + db.doMaintenance() + } - // Commit 1 more version - // stale versions: (1, 2, 3) - // keep versions: (4, 5, 6) - db.load(5) - db.commit() + // Commit 1 more version + // stale versions: (1, 2, 3) + // keep versions: (4, 5, 6) + db.load(5) + db.commit() - // Checkpoint directory before maintenance - if (isChangelogCheckpointingEnabled) { - assert(snapshotVersionsPresent(remoteDir) == (1 to 5)) - assert(changelogVersionsPresent(remoteDir) == (1 to 6)) - } else { - assert(snapshotVersionsPresent(remoteDir) == (1 to 6)) - } + // Checkpoint directory before maintenance + if (isChangelogCheckpointingEnabled) { + assert(snapshotVersionsPresent(remoteDir) == (1 to 5)) + assert(changelogVersionsPresent(remoteDir) == (1 to 6)) + } else { + assert(snapshotVersionsPresent(remoteDir) == (1 to 6)) + } - // Should delete stale versions for zip files and change log files - // since number of stale versions >= minVersionsToDelete - db.doMaintenance() + // Should delete stale versions for zip files and change log files + // since number of stale versions >= minVersionsToDelete + db.doMaintenance() - // Checkpoint directory after maintenance - assert(snapshotVersionsPresent(remoteDir) == Seq(4, 5, 6)) - if (isChangelogCheckpointingEnabled) { - assert(changelogVersionsPresent(remoteDir) == Seq(4, 5, 6)) + // Checkpoint directory after maintenance + assert(snapshotVersionsPresent(remoteDir) == Seq(4, 5, 6)) + if (isChangelogCheckpointingEnabled) { + assert(changelogVersionsPresent(remoteDir) == Seq(4, 5, 6)) + } } - } } - testWithColumnFamilies( + testWithStateStoreCheckpointIdsAndColumnFamilies( "RocksDB: minDeltasForSnapshot", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - val conf = dbConf.copy(enableChangelogCheckpointing = true, minDeltasForSnapshot = 3) - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - for (version <- 0 to 1) { - db.load(version) + TestWithChangelogCheckpointingEnabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created + val conf = dbConf.copy(enableChangelogCheckpointing = true, minDeltasForSnapshot = 3) + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + for (version <- 0 to 1) { + db.load(version) + db.commit() + db.doMaintenance() + } + // Snapshot should not be created because minDeltasForSnapshot = 3 + assert(snapshotVersionsPresent(remoteDir) === Seq.empty) + assert(changelogVersionsPresent(remoteDir) == Seq(1, 2)) + db.load(2) db.commit() db.doMaintenance() - } - // Snapshot should not be created because minDeltasForSnapshot = 3 - assert(snapshotVersionsPresent(remoteDir) === Seq.empty) - assert(changelogVersionsPresent(remoteDir) == Seq(1, 2)) - db.load(2) - db.commit() - db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3)) - db.load(3) + assert(snapshotVersionsPresent(remoteDir) === Seq(3)) + db.load(3) - for (version <- 3 to 7) { - db.load(version) - db.commit() + for (version <- 3 to 7) { + db.load(version) + db.commit() + db.doMaintenance() + } + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6)) + for (version <- 8 to 17) { + db.load(version) + db.commit() + } db.doMaintenance() + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) } - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6)) - for (version <- 8 to 17) { - db.load(version) - db.commit() - } - db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) - } - // pick up from the last snapshot and the next upload will be for version 21 - withDB(remoteDir, conf = conf) { db => - db.load(18) - db.commit() - db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) - - for (version <- 19 to 20) { - db.load(version) + // pick up from the last snapshot and the next upload will be for version 21 + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + db.load(18) db.commit() + db.doMaintenance() + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) + + for (version <- 19 to 20) { + db.load(version) + db.commit() + } + db.doMaintenance() + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18, 21)) } - db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18, 21)) - } } - testWithColumnFamilies("SPARK-45419: Do not reuse SST files" + + testWithStateStoreCheckpointIdsAndColumnFamilies("SPARK-45419: Do not reuse SST files" + " in different RocksDB instances", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - val conf = dbConf.copy(minDeltasForSnapshot = 0, compactOnCommit = false) - new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - for (version <- 0 to 2) { - db.load(version) - db.put(version.toString, version.toString) - db.commit() - } - // upload snapshot 3.zip - db.doMaintenance() - // Roll back to version 1 and start to process data. - for (version <- 1 to 3) { - db.load(version) - db.put(version.toString, version.toString) - db.commit() + TestWithChangelogCheckpointingEnabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = 0, compactOnCommit = false) + new File(remoteDir).delete() // to make sure that the directory gets created + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + for (version <- 0 to 2) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 3.zip + db.doMaintenance() + // Roll back to version 1 and start to process data. + for (version <- 1 to 3) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // Upload snapshot 4.zip, should not reuse the SST files in 3.zip + db.doMaintenance() } - // Upload snapshot 4.zip, should not reuse the SST files in 3.zip - db.doMaintenance() - } - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - // Open the db to verify that the state in 4.zip is no corrupted. - db.load(4) - } + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + // Open the db to verify that the state in 4.zip is no corrupted. + db.load(4) + } } // A rocksdb instance with changelog checkpointing enabled should be able to load // an existing checkpoint without changelog. - testWithColumnFamilies( + testWithStateStoreCheckpointIdsAndColumnFamilies( "RocksDB: changelog checkpointing backward compatibility", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled) { (enableStateStoreCheckpointIds, colFamiliesEnabled) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val disableChangelogCheckpointingConf = dbConf.copy(enableChangelogCheckpointing = false, minVersionsToRetain = 30) + val versionToUniqueId = new mutable.HashMap[Long, String]() withDB(remoteDir, conf = disableChangelogCheckpointingConf, - useColumnFamilies = colFamiliesEnabled) { db => + useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version - 1) db.put(version.toString, version.toString) @@ -523,7 +625,9 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared dbConf.copy(enableChangelogCheckpointing = true, minVersionsToRetain = 30, minDeltasForSnapshot = 1) withDB(remoteDir, conf = enableChangelogCheckpointingConf, - useColumnFamilies = colFamiliesEnabled) { db => + useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) @@ -551,7 +655,13 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // Check that snapshots and changelogs get purged correctly. db.doMaintenance() assert(snapshotVersionsPresent(remoteDir) === Seq(30, 60)) - assert(changelogVersionsPresent(remoteDir) === (30 to 60)) + if (enableStateStoreCheckpointIds) { + // recommit version 60 creates another changelog file with different unique id + assert(changelogVersionsPresent(remoteDir) === (30 to 60) :+ 60) + } else { + assert(changelogVersionsPresent(remoteDir) === (30 to 60)) + } + // Verify the content of retained versions. for (version <- 30 to 60) { db.load(version, readOnly = true) @@ -562,16 +672,19 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // A rocksdb instance with changelog checkpointing disabled should be able to load // an existing checkpoint with changelog. - testWithColumnFamilies( + testWithStateStoreCheckpointIdsAndColumnFamilies( "RocksDB: changelog checkpointing forward compatibility", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled) { (enableStateStoreCheckpointIds, colFamiliesEnabled) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val enableChangelogCheckpointingConf = dbConf.copy(enableChangelogCheckpointing = true, minVersionsToRetain = 20, minDeltasForSnapshot = 3) + val versionToUniqueId = new mutable.HashMap[Long, String]() withDB(remoteDir, conf = enableChangelogCheckpointingConf, - useColumnFamilies = colFamiliesEnabled) { db => + useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version - 1) db.put(version.toString, version.toString) @@ -583,10 +696,12 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // Now disable changelog checkpointing in a checkpoint created by a state store // that enable changelog checkpointing. val disableChangelogCheckpointingConf = - dbConf.copy(enableChangelogCheckpointing = false, minVersionsToRetain = 20, - minDeltasForSnapshot = 1) + dbConf.copy(enableChangelogCheckpointing = false, minVersionsToRetain = 20, + minDeltasForSnapshot = 1) withDB(remoteDir, conf = disableChangelogCheckpointingConf, - useColumnFamilies = colFamiliesEnabled) { db => + useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) @@ -631,145 +746,165 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithColumnFamilies(s"RocksDB: get, put, iterator, commit, load", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - def testOps(compactOnCommit: Boolean): Unit = { - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created + testWithStateStoreCheckpointIdsAndColumnFamilies(s"RocksDB: get, put, iterator, commit, load", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + def testOps(compactOnCommit: Boolean): Unit = { + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created - val conf = RocksDBConf().copy(compactOnCommit = compactOnCommit) - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - assert(db.get("a") === null) - assert(iterator(db).isEmpty) + val conf = RocksDBConf().copy(compactOnCommit = compactOnCommit) + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + assert(db.get("a") === null) + assert(iterator(db).isEmpty) - db.put("a", "1") - assert(toStr(db.get("a")) === "1") - db.commit() - } + db.put("a", "1") + assert(toStr(db.get("a")) === "1") + db.commit() + } - withDB(remoteDir, conf = conf, version = 0, useColumnFamilies = colFamiliesEnabled) { db => - // version 0 can be loaded again - assert(toStr(db.get("a")) === null) - assert(iterator(db).isEmpty) - } + withDB(remoteDir, conf = conf, version = 0, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + // version 0 can be loaded again + assert(toStr(db.get("a")) === null) + assert(iterator(db).isEmpty) + } - withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = colFamiliesEnabled) { db => - // version 1 data recovered correctly - assert(toStr(db.get("a")) === "1") - assert(db.iterator().map(toStr).toSet === Set(("a", "1"))) + withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + // version 1 data recovered correctly + assert(toStr(db.get("a")) === "1") + assert(db.iterator().map(toStr).toSet === Set(("a", "1"))) - // make changes but do not commit version 2 - db.put("b", "2") - assert(toStr(db.get("b")) === "2") - assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - } + // make changes but do not commit version 2 + db.put("b", "2") + assert(toStr(db.get("b")) === "2") + assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2"))) + } - withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = colFamiliesEnabled) { db => - // version 1 data not changed - assert(toStr(db.get("a")) === "1") - assert(db.get("b") === null) - assert(db.iterator().map(toStr).toSet === Set(("a", "1"))) + withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + // version 1 data not changed + assert(toStr(db.get("a")) === "1") + assert(db.get("b") === null) + assert(db.iterator().map(toStr).toSet === Set(("a", "1"))) - // commit version 2 - db.put("b", "2") - assert(toStr(db.get("b")) === "2") - db.commit() - assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - } + // commit version 2 + db.put("b", "2") + assert(toStr(db.get("b")) === "2") + db.commit() + assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2"))) + } - withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = colFamiliesEnabled) { db => - // version 1 data not changed - assert(toStr(db.get("a")) === "1") - assert(db.get("b") === null) - } + withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + // version 1 data not changed + assert(toStr(db.get("a")) === "1") + assert(db.get("b") === null) + } - withDB(remoteDir, conf = conf, version = 2, useColumnFamilies = colFamiliesEnabled) { db => - // version 2 can be loaded again - assert(toStr(db.get("b")) === "2") - assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2"))) + withDB(remoteDir, conf = conf, version = 2, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => + // version 2 can be loaded again + assert(toStr(db.get("b")) === "2") + assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - db.load(1) - assert(toStr(db.get("b")) === null) - assert(db.iterator().map(toStr).toSet === Set(("a", "1"))) + db.load(1) + assert(toStr(db.get("b")) === null) + assert(db.iterator().map(toStr).toSet === Set(("a", "1"))) + } } - } - for (compactOnCommit <- Seq(false, true)) { - withClue(s"compactOnCommit = $compactOnCommit") { - testOps(compactOnCommit) + for (compactOnCommit <- Seq(false, true)) { + withClue(s"compactOnCommit = $compactOnCommit") { + testOps(compactOnCommit) + } } - } } - testWithColumnFamilies(s"RocksDB: handle commit failures and aborts", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val hadoopConf = new Configuration() - hadoopConf.set( - SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, - classOf[CreateAtomicTestManager].getName) - val remoteDir = Utils.createTempDir().getAbsolutePath - withDB(remoteDir, hadoopConf = hadoopConf, useColumnFamilies = colFamiliesEnabled) { db => - // Disable failure of output stream and generate versions - CreateAtomicTestManager.shouldFailInCreateAtomic = false - for (version <- 1 to 10) { - db.load(version - 1) - db.put(version.toString, version.toString) // update "1" -> "1", "2" -> "2", ... - db.commit() - } - val version10Data = (1L to 10).map(_.toString).map(x => x -> x).toSet - - // Fail commit for next version and verify that reloading resets the files - CreateAtomicTestManager.shouldFailInCreateAtomic = true - db.load(10) - db.put("11", "11") - intercept[IOException] { - quietly { + testWithStateStoreCheckpointIdsAndColumnFamilies(s"RocksDB: handle commit failures and aborts", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val hadoopConf = new Configuration() + hadoopConf.set( + SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, + classOf[CreateAtomicTestManager].getName) + val remoteDir = Utils.createTempDir().getAbsolutePath + withDB(remoteDir, hadoopConf = hadoopConf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => + // Disable failure of output stream and generate versions + CreateAtomicTestManager.shouldFailInCreateAtomic = false + for (version <- 1 to 10) { + db.load(version - 1) + db.put(version.toString, version.toString) // update "1" -> "1", "2" -> "2", ... db.commit() } - } - assert(db.load(10, readOnly = true).iterator().map(toStr).toSet === version10Data) - CreateAtomicTestManager.shouldFailInCreateAtomic = false + val version10Data = (1L to 10).map(_.toString).map(x => x -> x).toSet + + // Fail commit for next version and verify that reloading resets the files + CreateAtomicTestManager.shouldFailInCreateAtomic = true + db.load(10) + db.put("11", "11") + intercept[IOException] { + quietly { + db.commit() + } + } + assert(db.load(10, readOnly = true).iterator().map(toStr).toSet === version10Data) + CreateAtomicTestManager.shouldFailInCreateAtomic = false - // Abort commit for next version and verify that reloading resets the files - db.load(10) - db.put("11", "11") - db.rollback() - assert(db.load(10, readOnly = true).iterator().map(toStr).toSet === version10Data) - } + // Abort commit for next version and verify that reloading resets the files + db.load(10) + db.put("11", "11") + db.rollback() + assert(db.load(10, readOnly = true).iterator().map(toStr).toSet === version10Data) + } } - testWithColumnFamilies("RocksDB close tests - close before doMaintenance", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - val conf = dbConf.copy(minDeltasForSnapshot = 1, compactOnCommit = false) - new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf, - useColumnFamilies = colFamiliesEnabled) { db => - db.load(0) - db.put("foo", "bar") - db.commit() - // call close first and maintenance can be still be invoked in the context of the - // maintenance task's thread pool - db.close() - db.doMaintenance() - } + testWithStateStoreCheckpointIdsAndColumnFamilies("RocksDB close tests - " + + "close before doMaintenance", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = 1, compactOnCommit = false) + new File(remoteDir).delete() // to make sure that the directory gets created + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => + db.load(0) + db.put("foo", "bar") + db.commit() + // call close first and maintenance can be still be invoked in the context of the + // maintenance task's thread pool + db.close() + db.doMaintenance() + } } - testWithColumnFamilies("RocksDB close tests - close after doMaintenance", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - val conf = dbConf.copy(minDeltasForSnapshot = 1, compactOnCommit = false) - new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf, - useColumnFamilies = colFamiliesEnabled) { db => - db.load(0) - db.put("foo", "bar") - db.commit() - // maintenance can be invoked in the context of the maintenance task's thread pool - // and close is invoked after that - db.doMaintenance() - db.close() - } + testWithStateStoreCheckpointIdsAndColumnFamilies("RocksDB close tests - " + + "close after doMaintenance", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = 1, compactOnCommit = false) + new File(remoteDir).delete() // to make sure that the directory gets created + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => + db.load(0) + db.put("foo", "bar") + db.commit() + // maintenance can be invoked in the context of the maintenance task's thread pool + // and close is invoked after that + db.doMaintenance() + db.close() + } } testWithChangelogCheckpointingEnabled("RocksDB: Unsupported Operations" + @@ -1179,8 +1314,9 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithColumnFamilies("RocksDBFileManager: delete orphan files", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + testWithStateStoreCheckpointIdsAndColumnFamilies("RocksDBFileManager: delete orphan files", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath // Use 2 file managers here to emulate concurrent execution @@ -1204,9 +1340,12 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "archive/00001.log" -> 1000, "archive/00002.log" -> 2000 ) - + val uuid = enableStateStoreCheckpointIds match { + case false => None + case true => Some(UUID.randomUUID().toString) + } saveCheckpointFiles(fileManager, cpFiles1, version = 1, - numKeys = 101, rocksDBFileMapping) + numKeys = 101, rocksDBFileMapping, uuid) assert(fileManager.getLatestVersion() === 1) assert(numRemoteSSTFiles == 2) // 2 sst files copied assert(numRemoteLogFiles == 2) @@ -1221,7 +1360,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "archive/00003.log" -> 2000 ) saveCheckpointFiles(fileManager_, cpFiles1_, version = 1, - numKeys = 101, new RocksDBFileMapping()) + numKeys = 101, new RocksDBFileMapping(), uuid) assert(fileManager_.getLatestVersion() === 1) assert(numRemoteSSTFiles == 4) assert(numRemoteLogFiles == 4) @@ -1241,7 +1380,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "archive/00005.log" -> 2000 ) saveCheckpointFiles(fileManager_, cpFiles2, - version = 2, numKeys = 121, new RocksDBFileMapping()) + version = 2, numKeys = 121, new RocksDBFileMapping(), uuid) fileManager_.deleteOldVersions(1) assert(numRemoteSSTFiles <= 4) // delete files recorded in 1.zip assert(numRemoteLogFiles <= 5) // delete files recorded in 1.zip and orphan 00001.log @@ -1256,7 +1395,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "archive/00007.log" -> 2000 ) saveCheckpointFiles(fileManager_, cpFiles3, - version = 3, numKeys = 131, new RocksDBFileMapping()) + version = 3, numKeys = 131, new RocksDBFileMapping(), uuid) assert(fileManager_.getLatestVersion() === 3) fileManager_.deleteOldVersions(1) assert(numRemoteSSTFiles == 1) @@ -1264,9 +1403,10 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithColumnFamilies("RocksDBFileManager: don't delete orphan files " + - s"when there is only 1 version", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + testWithStateStoreCheckpointIdsAndColumnFamilies("RocksDBFileManager: don't delete " + + s"orphan files when there is only 1 version", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath val fileManager = new RocksDBFileManager( @@ -1294,8 +1434,14 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "archive/00002.log" -> 2000 ) val rocksDBFileMapping = new RocksDBFileMapping() - saveCheckpointFiles(fileManager, cpFiles1, - version = 1, numKeys = 101, rocksDBFileMapping) + val uuid = if (enableStateStoreCheckpointIds) { + Some(UUID.randomUUID().toString) + } else { + None + } + + saveCheckpointFiles( + fileManager, cpFiles1, version = 1, numKeys = 101, rocksDBFileMapping, uuid) fileManager.deleteOldVersions(1) // Should not delete orphan files even when they are older than all existing files // when there is only 1 version. @@ -1312,8 +1458,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "archive/00003.log" -> 1000, "archive/00004.log" -> 2000 ) - saveCheckpointFiles(fileManager, cpFiles2, - version = 2, numKeys = 101, rocksDBFileMapping) + saveCheckpointFiles( + fileManager, cpFiles2, version = 2, numKeys = 101, rocksDBFileMapping, uuid) assert(numRemoteSSTFiles == 5) assert(numRemoteLogFiles == 5) fileManager.deleteOldVersions(1) @@ -1323,122 +1469,131 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithColumnFamilies("RocksDBFileManager: upload only new immutable files", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - withTempDir { dir => - val dfsRootDir = dir.getAbsolutePath - val verificationDir = Utils.createTempDir().getAbsolutePath // local dir to load checkpoints - val fileManager = new RocksDBFileManager( - dfsRootDir, Utils.createTempDir(), hadoopConf) - val sstDir = s"$dfsRootDir/SSTs" - def numRemoteSSTFiles: Int = listFiles(sstDir).length - val logDir = s"$dfsRootDir/logs" - def numRemoteLogFiles: Int = listFiles(logDir).length - val fileMapping = new RocksDBFileMapping - - // Verify behavior before any saved checkpoints - assert(fileManager.getLatestVersion() === 0) + testWithStateStoreCheckpointIdsAndColumnFamilies("RocksDBFileManager: upload only " + + "new immutable files", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => + withTempDir { dir => + val dfsRootDir = dir.getAbsolutePath + val verificationDir = Utils.createTempDir().getAbsolutePath // local dir to load checkpoints + val fileManager = new RocksDBFileManager( + dfsRootDir, Utils.createTempDir(), hadoopConf) + val sstDir = s"$dfsRootDir/SSTs" + def numRemoteSSTFiles: Int = listFiles(sstDir).length + val logDir = s"$dfsRootDir/logs" + def numRemoteLogFiles: Int = listFiles(logDir).length + val fileMapping = new RocksDBFileMapping + + // Verify behavior before any saved checkpoints + assert(fileManager.getLatestVersion() === 0) + + // Try to load incorrect versions + intercept[FileNotFoundException] { + fileManager.loadCheckpointFromDfs(1, Utils.createTempDir(), fileMapping) + } - // Try to load incorrect versions - intercept[FileNotFoundException] { - fileManager.loadCheckpointFromDfs(1, Utils.createTempDir(), fileMapping) - } + // Save a version of checkpoint files + val cpFiles1 = Seq( + "sst-file1.sst" -> 10, + "sst-file2.sst" -> 20, + "other-file1" -> 100, + "other-file2" -> 200, + "archive/00001.log" -> 1000, + "archive/00002.log" -> 2000 + ) - // Save a version of checkpoint files - val cpFiles1 = Seq( - "sst-file1.sst" -> 10, - "sst-file2.sst" -> 20, - "other-file1" -> 100, - "other-file2" -> 200, - "archive/00001.log" -> 1000, - "archive/00002.log" -> 2000 - ) - saveCheckpointFiles(fileManager, cpFiles1, - version = 1, numKeys = 101, fileMapping) - assert(fileManager.getLatestVersion() === 1) - assert(numRemoteSSTFiles == 2) // 2 sst files copied - assert(numRemoteLogFiles == 2) // 2 log files copied - - // Load back the checkpoint files into another local dir with existing files and verify - generateFiles(verificationDir, Seq( - "sst-file1.sst" -> 11, // files with same name but different sizes, should get overwritten - "other-file1" -> 101, - "archive/00001.log" -> 1001, - "random-sst-file.sst" -> 100, // unnecessary files, should get deleted - "random-other-file" -> 9, - "00005.log" -> 101, - "archive/00007.log" -> 101 - )) - - // as we are loading version 1 again, the previously committed 1,zip and - // SST files would not be reused. - loadAndVerifyCheckpointFiles(fileManager, verificationDir, - version = 1, cpFiles1, 101, fileMapping) - - // Save SAME version again with different checkpoint files and load back again to verify - // whether files were overwritten. - val cpFiles1_ = Seq( - "sst-file1.sst" -> 10, // same SST file as before, but will be uploaded again - "sst-file2.sst" -> 25, // new SST file with same name as before, but different length - "sst-file3.sst" -> 30, // new SST file - "other-file1" -> 100, // same non-SST file as before, should not get copied - "other-file2" -> 210, // new non-SST file with same name as before, but different length - "other-file3" -> 300, // new non-SST file - "archive/00001.log" -> 1000, // same log file as before, this should get reused - "archive/00002.log" -> 2500, // new log file with same name as before, but different length - "archive/00003.log" -> 3000 // new log file - ) + val uuid = if (enableStateStoreCheckpointIds) { + Some(UUID.randomUUID().toString) + } else { + None + } - // upload version 1 again, new checkpoint will be created and SST files from - // previously committed version 1 will not be reused. - saveCheckpointFiles(fileManager, cpFiles1_, - version = 1, numKeys = 1001, fileMapping) - assert(numRemoteSSTFiles === 5, "shouldn't reuse old version 1 SST files" + - " while uploading version 1 again") // 2 old + 3 new SST files - assert(numRemoteLogFiles === 5, "shouldn't reuse old version 1 log files" + - " while uploading version 1 again") // 2 old + 3 new log files + saveCheckpointFiles( + fileManager, cpFiles1, version = 1, numKeys = 101, fileMapping, uuid) + assert(fileManager.getLatestVersion() === 1) + assert(numRemoteSSTFiles == 2) // 2 sst files copied + assert(numRemoteLogFiles == 2) // 2 log files copied + + // Load back the checkpoint files into another local dir with existing files and verify + generateFiles(verificationDir, Seq( + "sst-file1.sst" -> 11, // files with same name but different sizes, should get overwritten + "other-file1" -> 101, + "archive/00001.log" -> 1001, + "random-sst-file.sst" -> 100, // unnecessary files, should get deleted + "random-other-file" -> 9, + "00005.log" -> 101, + "archive/00007.log" -> 101 + )) + + // as we are loading version 1 again, the previously committed 1.zip and + // SST files would not be reused. + loadAndVerifyCheckpointFiles( + fileManager, verificationDir, version = 1, cpFiles1, 101, fileMapping, uuid) + + // Save SAME version again with different checkpoint files and load back again to verify + // whether files were overwritten. + val cpFiles1_ = Seq( + "sst-file1.sst" -> 10, // same SST file as before, but will be uploaded again + "sst-file2.sst" -> 25, // new SST file with same name as before, but different length + "sst-file3.sst" -> 30, // new SST file + "other-file1" -> 100, // same non-SST file as before, should not get copied + "other-file2" -> 210, // new non-SST file with same name as before, but different length + "other-file3" -> 300, // new non-SST file + "archive/00001.log" -> 1000, // same log file as before, this should get reused + "archive/00002.log" -> 2500, // new log file with same name but different length + "archive/00003.log" -> 3000 // new log file + ) - // verify checkpoint state is correct - loadAndVerifyCheckpointFiles(fileManager, verificationDir, - version = 1, cpFiles1_, 1001, fileMapping) + // upload version 1 again, new checkpoint will be created and SST files from + // previously committed version 1 will not be reused. + saveCheckpointFiles(fileManager, cpFiles1_, + version = 1, numKeys = 1001, fileMapping, uuid) + assert(numRemoteSSTFiles === 5, "shouldn't reuse old version 1 SST files" + + " while uploading version 1 again") // 2 old + 3 new SST files + assert(numRemoteLogFiles === 5, "shouldn't reuse old version 1 log files" + + " while uploading version 1 again") // 2 old + 3 new log files - // Save another version and verify - val cpFiles2 = Seq( - "sst-file1.sst" -> 10, // same SST file as version 1, should be reused - "sst-file2.sst" -> 25, // same SST file as version 1, should be reused - "sst-file3.sst" -> 30, // same SST file as version 1, should be reused - "sst-file4.sst" -> 40, // new sst file, should be uploaded - "other-file4" -> 400, - "archive/00004.log" -> 4000 - ) + // verify checkpoint state is correct + loadAndVerifyCheckpointFiles(fileManager, verificationDir, + version = 1, cpFiles1_, 1001, fileMapping, uuid) + + // Save another version and verify + val cpFiles2 = Seq( + "sst-file1.sst" -> 10, // same SST file as version 1, should be reused + "sst-file2.sst" -> 25, // same SST file as version 1, should be reused + "sst-file3.sst" -> 30, // same SST file as version 1, should be reused + "sst-file4.sst" -> 40, // new sst file, should be uploaded + "other-file4" -> 400, + "archive/00004.log" -> 4000 + ) + saveCheckpointFiles(fileManager, cpFiles2, + version = 2, numKeys = 1501, fileMapping, uuid) + assert(numRemoteSSTFiles === 6) // 1 new file over earlier 5 files + assert(numRemoteLogFiles === 6) // 1 new file over earlier 6 files + loadAndVerifyCheckpointFiles(fileManager, verificationDir, + version = 2, cpFiles2, 1501, fileMapping, uuid) - saveCheckpointFiles(fileManager, cpFiles2, - version = 2, numKeys = 1501, fileMapping) - assert(numRemoteSSTFiles === 6) // 1 new file over earlier 5 files - assert(numRemoteLogFiles === 6) // 1 new file over earlier 6 files - loadAndVerifyCheckpointFiles(fileManager, verificationDir, - version = 2, cpFiles2, 1501, fileMapping) + // Loading an older version should work + loadAndVerifyCheckpointFiles( + fileManager, verificationDir, version = 1, cpFiles1_, 1001, fileMapping, uuid) - // Loading an older version should work - loadAndVerifyCheckpointFiles(fileManager, verificationDir, - version = 1, cpFiles1_, 1001, fileMapping) + // Loading incorrect version should fail + intercept[FileNotFoundException] { + loadAndVerifyCheckpointFiles( + fileManager, verificationDir, version = 3, Nil, 1001, fileMapping, uuid) + } - // Loading incorrect version should fail - intercept[FileNotFoundException] { - loadAndVerifyCheckpointFiles(fileManager, verificationDir, - version = 3, Nil, 1001, fileMapping) + // Loading 0 should delete all files + require(verificationDir.list().length > 0) + loadAndVerifyCheckpointFiles( + fileManager, verificationDir, version = 0, Nil, 0, fileMapping, uuid) } - - // Loading 0 should delete all files - require(verificationDir.list().length > 0) - loadAndVerifyCheckpointFiles(fileManager, verificationDir, - version = 0, Nil, 0, fileMapping) - } } - testWithColumnFamilies("RocksDBFileManager: error writing [version].zip " + - s"cancels the output stream", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + testWithStateStoreCheckpointIdsAndColumnFamilies("RocksDBFileManager: error writing " + + s"[version].zip cancels the output stream", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => quietly { val hadoopConf = new Configuration() hadoopConf.set( @@ -1448,21 +1603,31 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared val fileManager = new RocksDBFileManager(dfsRootDir, Utils.createTempDir(), hadoopConf) val cpFiles = Seq("sst-file1.sst" -> 10, "sst-file2.sst" -> 20, "other-file1" -> 100) CreateAtomicTestManager.shouldFailInCreateAtomic = true + val uuid = if (enableStateStoreCheckpointIds) { + Some(UUID.randomUUID().toString) + } else { + None + } intercept[IOException] { - saveCheckpointFiles(fileManager, cpFiles, - version = 1, numKeys = 101, new RocksDBFileMapping()) + saveCheckpointFiles( + fileManager, cpFiles, version = 1, numKeys = 101, new RocksDBFileMapping(), uuid) } assert(CreateAtomicTestManager.cancelCalledInCreateAtomic) } } - testWithColumnFamilies("disallow concurrent updates to the same RocksDB instance", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + testWithStateStoreCheckpointIdsAndColumnFamilies("disallow concurrent updates to the same " + + "RocksDB instance", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + case (enableStateStoreCheckpointIds, colFamiliesEnabled) => quietly { + val versionToUniqueId = new mutable.HashMap[Long, String]() withDB( Utils.createTempDir().toString, conf = dbConf.copy(lockAcquireTimeoutMs = 20), - useColumnFamilies = colFamiliesEnabled) { db => + useColumnFamilies = colFamiliesEnabled, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => // DB has been loaded so current thread has already // acquired the lock on the RocksDB instance @@ -1964,11 +2129,14 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - test("time travel - validate successful RocksDB load") { + testWithStateStoreCheckpointIds("time travel - " + + "validate successful RocksDB load") { enableStateStoreCheckpointIds => val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 1, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf) { db => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { db.load(version) db.put(version.toString, version.toString) @@ -1984,7 +2152,10 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // upload snapshot 3.zip db.doMaintenance() // simulate db in another executor that override the zip file - withDB(remoteDir, conf = conf) { db1 => + // In checkpoint V2, reusing the same versionToUniqueId to simulate when two executors + // are scheduled with the same uniqueId in the same microbatch + withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db1 => for (version <- 0 to 1) { db1.load(version) db1.put(version.toString, version.toString) @@ -2005,13 +2176,16 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - test("time travel 2 - validate successful RocksDB load") { + testWithStateStoreCheckpointIds("time travel 2 - " + + "validate successful RocksDB load") { enableStateStoreCheckpointIds => Seq(1, 2).map(minDeltasForSnapshot => { val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = minDeltasForSnapshot, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf) { db => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { db.load(version) db.put(version.toString, version.toString) @@ -2026,7 +2200,11 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } db.load(0) // simulate db in another executor that override the zip file - withDB(remoteDir, conf = conf) { db1 => + // In checkpoint V2, reusing the same versionToUniqueId to simulate when two executors + // are scheduled with the same uniqueId in the same microbatch + withDB(remoteDir, conf = conf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db1 => for (version <- 0 to 1) { db1.load(version) db1.put(version.toString, version.toString) @@ -2047,11 +2225,14 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared }) } - test("time travel 3 - validate successful RocksDB load") { + testWithStateStoreCheckpointIds("time travel 3 - validate" + + " successful RocksDB load") { enableStateStoreCheckpointIds => val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 0, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf) { db => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 2) { db.load(version) db.put(version.toString, version.toString) @@ -2072,12 +2253,14 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithChangelogCheckpointingEnabled("time travel 4 -" + - " validate successful RocksDB load when metadata file is overwritten") { + testWithStateStoreCheckpointIdsAndChangelogEnabled("time travel 4 - validate successful" + + " RocksDB load when metadata file is overwritten") { enableStateStoreCheckpointIds => val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 2, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf) { db => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { db.load(version) db.put(version.toString, version.toString) @@ -2100,8 +2283,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithChangelogCheckpointingEnabled("time travel 5 -" + - "validate successful RocksDB load when metadata file is not overwritten") { + testWithStateStoreCheckpointIdsAndChangelogEnabled("time travel 5 - validate successful " + + "RocksDB load when metadata file is not overwritten") { enableStateStoreCheckpointIds => val fmClass = "org.apache.spark.sql.execution.streaming.state." + "NoOverwriteFileSystemBasedCheckpointFileManager" Seq(Some(fmClass), None).foreach { fm => @@ -2111,7 +2294,10 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared fm.foreach(value => hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, value)) val remoteDir = dir.getCanonicalPath - withDB(remoteDir, conf = conf, hadoopConf = hadoopConf) { db => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => db.load(0) db.put("a", "1") db.commit() @@ -2142,14 +2328,17 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithChangelogCheckpointingEnabled("reloading the same version") { + testWithStateStoreCheckpointIdsAndChangelogEnabled("reloading the " + + "same version") { enableStateStoreCheckpointIds => // Keep executing the same batch for two or more times. Some queries with ForEachBatch // will cause this behavior. // The test was accidentally fixed by SPARK-48586 (https://github.com/apache/spark/pull/47130) val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 2, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf) { db => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => // load the same version of pending snapshot uploading // This is possible because after committing version x, we can continue to x+1, and replay // x+1. The replay will load a checkpoint by version x. At this moment, the snapshot @@ -2224,18 +2413,23 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared for (randomSeed <- 1 to 8) { for (ifTestSkipBatch <- 0 to 1) { - testWithChangelogCheckpointingEnabled( - s"randomized snapshotting $randomSeed ifTestSkipBatch $ifTestSkipBatch") { - // The unit test simulates the case where batches can be reloaded and maintenance tasks + testWithStateStoreCheckpointIdsAndChangelogEnabled("randomized snapshotting " + + s"$randomSeed ifTestSkipBatch $ifTestSkipBatch") { enableStateStoreCheckpointIds => + // The unit test simulates the case where batches can be reloaded and maintenance tasks // can be delayed. After each batch, we randomly decide whether we would move onto the - // next batch, and whetehr maintenance task is executed. + // next batch, and whether maintenance task is executed. val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 3, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created - withDB(remoteDir, conf = conf) { db => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = dbConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db => // A second DB is opened to simulate another executor that runs some batches that // skipped in the current DB. - withDB(remoteDir, conf = conf) { db2 => + withDB(remoteDir, conf = dbConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db2 => val random = new Random(randomSeed) var curVer: Int = 0 for (i <- 1 to 100) { @@ -2274,8 +2468,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - test("validate Rocks DB SST files do not have a VersionIdMismatch" + - " when metadata file is not overwritten - scenario 1") { + testWithStateStoreCheckpointIds("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is not overwritten - scenario 1") { enableStateStoreCheckpointIds => val fmClass = "org.apache.spark.sql.execution.streaming.state." + "NoOverwriteFileSystemBasedCheckpointFileManager" withTempDir { dir => @@ -2284,8 +2478,13 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, fmClass) val remoteDir = dir.getCanonicalPath - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db1 db1.load(0) db1.put("a", "1") @@ -2319,14 +2518,19 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - test("validate Rocks DB SST files do not have a VersionIdMismatch" + - " when metadata file is overwritten - scenario 1") { + testWithStateStoreCheckpointIds("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is overwritten - scenario 1") { enableStateStoreCheckpointIds => withTempDir { dir => val dbConf = RocksDBConf(StateStoreConf(new SQLConf())) val hadoopConf = new Configuration() val remoteDir = dir.getCanonicalPath - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db1 db1.load(0) db1.put("a", "1") @@ -2360,8 +2564,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - test("validate Rocks DB SST files do not have a VersionIdMismatch" + - " when metadata file is not overwritten - scenario 2") { + testWithStateStoreCheckpointIds("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is not overwritten - scenario 2") { enableStateStoreCheckpointIds => val fmClass = "org.apache.spark.sql.execution.streaming.state." + "NoOverwriteFileSystemBasedCheckpointFileManager" withTempDir { dir => @@ -2370,8 +2574,13 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, fmClass) val remoteDir = dir.getCanonicalPath - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db2 db2.load(0) db2.put("a", "1") @@ -2405,14 +2614,19 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - test("validate Rocks DB SST files do not have a VersionIdMismatch" + - " when metadata file is overwritten - scenario 2") { + testWithStateStoreCheckpointIds("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is overwritten - scenario 2") { enableStateStoreCheckpointIds => withTempDir { dir => val dbConf = RocksDBConf(StateStoreConf(new SQLConf())) val hadoopConf = new Configuration() val remoteDir = dir.getCanonicalPath - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => - withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + val versionToUniqueId = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db2 db2.load(0) db2.put("a", "1") @@ -2718,24 +2932,65 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared private def dbConf = RocksDBConf(StateStoreConf(SQLConf.get.clone())) + class RocksDBCheckpointFormatV2( + dfsRootDir: String, + conf: RocksDBConf, + localRootDir: File = Utils.createTempDir(), + hadoopConf: Configuration = new Configuration, + loggingId: String = "", + useColumnFamilies: Boolean = false, + val versionToUniqueId : mutable.Map[Long, String] = mutable.Map[Long, String]()) + extends RocksDB(dfsRootDir, conf, localRootDir, hadoopConf, loggingId, + useColumnFamilies, enableStateStoreCheckpointIds = true) { + + override def load( + version: Long, + ckptId: Option[String] = None, + readOnly: Boolean = false): RocksDB = { + super.load(version, versionToUniqueId.get(version), readOnly) + } + + override def commit(): Long = { + val ret = super.commit() + // update versionToUniqueId from lineageManager + lineageManager.getLineageForCurrVersion().foreach { + case LineageItem(version, id) => versionToUniqueId.getOrElseUpdate(version, id) + } + ret + } + } + + // withDB override with checkpoint format v2 def withDB[T]( remoteDir: String, version: Int = 0, conf: RocksDBConf = dbConf, hadoopConf: Configuration = hadoopConf, useColumnFamilies: Boolean = false, + enableStateStoreCheckpointIds: Boolean = false, + versionToUniqueId : mutable.Map[Long, String] = mutable.Map[Long, String](), localDir: File = Utils.createTempDir())( func: RocksDB => T): T = { var db: RocksDB = null try { - db = new RocksDB( - remoteDir, - conf = conf, - localRootDir = localDir, - hadoopConf = hadoopConf, - loggingId = s"[Thread-${Thread.currentThread.getId}]", - useColumnFamilies = useColumnFamilies - ) + db = if (enableStateStoreCheckpointIds) { + new RocksDBCheckpointFormatV2( + remoteDir, + conf = conf, + localRootDir = localDir, + hadoopConf = hadoopConf, + loggingId = s"[Thread-${Thread.currentThread.getId}]", + useColumnFamilies = useColumnFamilies, + versionToUniqueId = versionToUniqueId) + } else { + new RocksDB( + remoteDir, + conf = conf, + localRootDir = localDir, + hadoopConf = hadoopConf, + loggingId = s"[Thread-${Thread.currentThread.getId}]", + useColumnFamilies = useColumnFamilies) + } db.load(version) func(db) } finally { @@ -2757,7 +3012,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared fileToLengths: Seq[(String, Int)], version: Int, numKeys: Int, - fileMapping: RocksDBFileMapping): Unit = { + fileMapping: RocksDBFileMapping, + checkpointUniqueId: Option[String] = None): Unit = { val checkpointDir = Utils.createTempDir().getAbsolutePath // local dir to create checkpoints generateFiles(checkpointDir, fileToLengths) val (dfsFileSuffix, immutableFileMapping) = fileMapping.createSnapshotFileMapping( @@ -2766,7 +3022,9 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared checkpointDir, version, numKeys, - immutableFileMapping) + immutableFileMapping, + checkpointUniqueId = checkpointUniqueId) + val snapshotInfo = RocksDBVersionSnapshotInfo(version, dfsFileSuffix) fileMapping.snapshotsPendingUpload.remove(snapshotInfo) } @@ -2777,9 +3035,10 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared version: Int, expectedFiles: Seq[(String, Int)], expectedNumKeys: Int, - fileMapping: RocksDBFileMapping): Unit = { - val metadata = fileManager.loadCheckpointFromDfs(version, - verificationDir, fileMapping) + fileMapping: RocksDBFileMapping, + checkpointUniqueId: Option[String] = None): Unit = { + val metadata = fileManager.loadCheckpointFromDfs( + version, verificationDir, fileMapping, checkpointUniqueId) val filesAndLengths = listFiles(verificationDir).map(f => f.getName -> f.length).toSet ++ listFiles(verificationDir + "/archive").map(f => s"archive/${f.getName}" -> f.length()).toSet diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala index 5d88db0d01ba4..f7606cd45949f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Encoders import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBStateStoreProvider} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf case class InputRow(key: String, action: String, value: String) @@ -127,8 +127,7 @@ class ToggleSaveAndEmitProcessor } class TransformWithListStateSuite extends StreamTest - with AlsoTestWithChangelogCheckpointingEnabled - with AlsoTestWithEncodingTypes { + with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { import testImplicits._ test("test appending null value in list state throw exception") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala index ec6ff4fcceb67..6884ef577f8ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Encoders import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBStateStoreProvider} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf case class InputMapRow(key: String, action: String, value: (String, String)) @@ -81,8 +81,7 @@ class TestMapStateProcessor * operators such as transformWithState. */ class TransformWithMapStateSuite extends StreamTest - with AlsoTestWithChangelogCheckpointingEnabled - with AlsoTestWithEncodingTypes { + with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { import testImplicits._ private def testMapStateWithNullUserKey(inputMapRow: InputMapRow): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala index c7ad8536ebd46..cf304301565ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.sql.{DataFrame, Dataset, Encoders, KeyValueGroupedDataset} import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.functions.{col, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock @@ -361,7 +361,7 @@ class StatefulProcessorWithInitialStateEventTimerClass * streaming operator with user-defined initial state */ class TransformWithStateInitialStateSuite extends StateStoreMetricsTest - with AlsoTestWithChangelogCheckpointingEnabled { + with AlsoTestWithRocksDBFeatures { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index d4c5a735ce6fa..601699c8320fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -442,7 +442,7 @@ class SleepingTimerProcessor extends StatefulProcessor[String, String, String] { * Class that adds tests for transformWithState stateful streaming operator */ class TransformWithStateSuite extends StateStoreMetricsTest - with AlsoTestWithChangelogCheckpointingEnabled with AlsoTestWithEncodingTypes { + with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala index 55a46f51f9f6f..e1df2d640f1fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala @@ -21,7 +21,7 @@ import java.sql.Timestamp import java.time.Duration import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, AlsoTestWithEncodingTypes, RocksDBStateStoreProvider} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock @@ -41,7 +41,7 @@ case class OutputEvent( * Test suite base for TransformWithState with TTL support. */ abstract class TransformWithStateTTLTest - extends StreamTest with AlsoTestWithChangelogCheckpointingEnabled + extends StreamTest with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { import testImplicits._ From 98cef08cae07be3d58b65585f57264495186cd59 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 13 Dec 2024 16:49:08 +0800 Subject: [PATCH 166/438] [SPARK-50562][INFRA] Apply Python 3.11 image in Java 21 daily build ### What changes were proposed in this pull request? Apply Python 3.11 image in Java 21 daily build ### Why are the changes needed? to switch to the new testing images ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with: ``` java: required: false type: string default: 21 ... envs: description: Additional environment variables to set when running the tests. Should be in JSON format. required: false type: string default: '{"PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11", "SKIP_MIMA": "true", "SKIP_UNIDOC": "true", "DEDICATED_JVM_SBT_TESTS": "org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormatV1Suite,org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormatV2Suite,org.apache.spark.sql.execution.datasources.orc.OrcSourceV1Suite,org.apache.spark.sql.execution.datasources.orc.OrcSourceV2Suite"}' ``` https://github.com/zhengruifeng/spark/actions/runs/12308077804/job/34352937432 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49168 from zhengruifeng/py_311_java21. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_java21.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/build_java21.yml b/.github/workflows/build_java21.yml index 871e1a9c07ef0..826e6f46e358a 100644 --- a/.github/workflows/build_java21.yml +++ b/.github/workflows/build_java21.yml @@ -36,6 +36,8 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-311", + "PYTHON_TO_TEST": "python3.11", "SKIP_MIMA": "true", "SKIP_UNIDOC": "true", "DEDICATED_JVM_SBT_TESTS": "org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormatV1Suite,org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormatV2Suite,org.apache.spark.sql.execution.datasources.orc.OrcSourceV1Suite,org.apache.spark.sql.execution.datasources.orc.OrcSourceV2Suite" From 819bac9903141e3ab8ce5ad163001a077899079c Mon Sep 17 00:00:00 2001 From: beliefer Date: Fri, 13 Dec 2024 19:07:13 +0800 Subject: [PATCH 167/438] [SPARK-50157][SQL] Using SQLConf provided by SparkSession first ### What changes were proposed in this pull request? This PR proposes to use `SQLConf` provided by `SparkSession` first. ### Why are the changes needed? `SQLConf` provided by `SparkSession` have better perf than `SQLConf.get`. ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? GA tests. The benchmark test. ``` object SQLConfBenchmark extends SqlBasedBenchmark { override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Get SQLConf") { val iters = 1000 val benchmark = new Benchmark("Benchmark SQLConf", iters, output = output) benchmark.addCase("SQLConf.get") { _ => for (_ <- 1 to iters) { val conf = SQLConf.get } } benchmark.addCase("sessionState.conf") { _ => for (_ <- 1 to iters) { val conf = spark.sessionState.conf } } benchmark.run() } } } ``` The benchmark output. ``` Benchmark SQLConf: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ SQLConf.get 0 0 1 22.9 43.6 1.0X sessionState.conf 0 0 0 1377.4 0.7 60.1X ``` ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #48693 from beliefer/SPARK-50157. Authored-by: beliefer Signed-off-by: beliefer --- .../org/apache/spark/sql/execution/SparkPlanner.scala | 3 +++ .../execution/adaptive/CoalesceShufflePartitions.scala | 2 ++ .../sql/execution/adaptive/InsertAdaptiveSparkPlan.scala | 2 ++ .../adaptive/PlanAdaptiveDynamicPruningFilters.scala | 4 ++++ .../org/apache/spark/sql/execution/command/ddl.scala | 5 +++-- .../org/apache/spark/sql/execution/command/tables.scala | 9 +++++---- .../org/apache/spark/sql/execution/command/views.scala | 3 ++- .../apache/spark/sql/execution/datasources/rules.scala | 4 ++++ .../dynamicpruning/PlanDynamicPruningFilters.scala | 3 +++ .../scala/org/apache/spark/sql/hive/HiveStrategies.scala | 5 ++++- .../sql/hive/execution/PruneHiveTablePartitions.scala | 3 +++ 11 files changed, 35 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index da3159319f98e..5dfe85548349c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -25,10 +25,13 @@ import org.apache.spark.sql.execution.adaptive.LogicalQueryStageStrategy import org.apache.spark.sql.execution.command.v2.V2CommandStrategy import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileSourceStrategy} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy +import org.apache.spark.sql.internal.SQLConf class SparkPlanner(val session: SparkSession, val experimentalMethods: ExperimentalMethods) extends SparkStrategies with SQLConfHelper { + override def conf: SQLConf = session.sessionState.conf + def numPartitions: Int = conf.numShufflePartitions override def strategies: Seq[Strategy] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala index 1bbc26f3e52ed..3fdcb17bdeae6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala @@ -33,6 +33,8 @@ import org.apache.spark.util.Utils */ case class CoalesceShufflePartitions(session: SparkSession) extends AQEShuffleReadRule { + override def conf: SQLConf = session.sessionState.conf + override val supportedShuffleOrigins: Seq[ShuffleOrigin] = Seq(ENSURE_REQUIREMENTS, REPARTITION_BY_COL, REBALANCE_PARTITIONS_BY_NONE, REBALANCE_PARTITIONS_BY_COL) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 8517911d70262..73fc9b1fe4e2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -44,6 +44,8 @@ import org.apache.spark.sql.internal.SQLConf case class InsertAdaptiveSparkPlan( adaptiveExecutionContext: AdaptiveExecutionContext) extends Rule[SparkPlan] { + override def conf: SQLConf = adaptiveExecutionContext.session.sessionState.conf + override def apply(plan: SparkPlan): SparkPlan = applyInternal(plan, false) private def applyInternal(plan: SparkPlan, isSubquery: Boolean): SparkPlan = plan match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveDynamicPruningFilters.scala index 3d35abff3c538..77c180b18aee0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveDynamicPruningFilters.scala @@ -25,12 +25,16 @@ import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashedRelationBroadcastMode, HashJoin} +import org.apache.spark.sql.internal.SQLConf /** * A rule to insert dynamic pruning predicates in order to reuse the results of broadcast. */ case class PlanAdaptiveDynamicPruningFilters( rootPlan: AdaptiveSparkPlanExec) extends Rule[SparkPlan] with AdaptiveSparkPlanHelper { + + override def conf: SQLConf = rootPlan.context.session.sessionState.conf + def apply(plan: SparkPlan): SparkPlan = { if (!conf.dynamicPartitionPruningEnabled) { return plan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index a8a91af1bdbc4..9dfe5c3e4c301 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -196,7 +196,8 @@ case class DescribeDatabaseCommand( if (properties.isEmpty) { "" } else { - conf.redactOptions(properties).toSeq.sortBy(_._1).mkString("(", ", ", ")") + sparkSession.sessionState.conf.redactOptions(properties).toSeq + .sortBy(_._1).mkString("(", ", ", ")") } result :+ Row("Properties", propertiesStr) } else { @@ -548,7 +549,7 @@ case class AlterTableAddPartitionCommand( // Hive metastore may not have enough memory to handle millions of partitions in single RPC. // Also the request to metastore times out when adding lot of partitions in one shot. // we should split them into smaller batches - val batchSize = conf.getConf(SQLConf.ADD_PARTITION_BATCH_SIZE) + val batchSize = sparkSession.sessionState.conf.getConf(SQLConf.ADD_PARTITION_BATCH_SIZE) parts.iterator.grouped(batchSize).foreach { batch => catalog.createPartitions(table.identifier, batch, ignoreIfExists = ifNotExists) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 84b73a74f3ab2..a58e8fac6e36d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -240,7 +240,7 @@ case class AlterTableAddColumnsCommand( SchemaUtils.checkColumnNameDuplication( (colsWithProcessedDefaults ++ catalogTable.schema).map(_.name), - conf.caseSensitiveAnalysis) + sparkSession.sessionState.conf.caseSensitiveAnalysis) if (!conf.allowCollationsInMapKeys) { colsToAdd.foreach(col => SchemaUtils.checkNoCollationsInMapKeys(col.dataType)) } @@ -501,7 +501,7 @@ case class TruncateTableCommand( partLocations } val hadoopConf = spark.sessionState.newHadoopConf() - val ignorePermissionAcl = conf.truncateTableIgnorePermissionAcl + val ignorePermissionAcl = spark.sessionState.conf.truncateTableIgnorePermissionAcl locations.foreach { location => if (location.isDefined) { val path = new Path(location.get) @@ -819,7 +819,8 @@ case class DescribeColumnCommand( val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) val colStatsMap = catalogTable.stats.map(_.colStats).getOrElse(Map.empty) - val colStats = if (conf.caseSensitiveAnalysis) colStatsMap else CaseInsensitiveMap(colStatsMap) + val colStats = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) colStatsMap + else CaseInsensitiveMap(colStatsMap) val cs = colStats.get(field.name) val comment = if (field.metadata.contains("comment")) { @@ -975,7 +976,7 @@ case class ShowTablePropertiesCommand( Seq.empty[Row] } else { val catalogTable = catalog.getTableMetadata(table) - val properties = conf.redactOptions(catalogTable.properties) + val properties = sparkSession.sessionState.conf.redactOptions(catalogTable.properties) propertyKey match { case Some(p) => val propValue = properties 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 071e3826b20a0..a98d9886a2730 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 @@ -729,7 +729,8 @@ object ViewHelper extends SQLConfHelper with Logging { val uncache = getRawTempView(name.table).map { r => needsToUncache(r, aliasedPlan) }.getOrElse(false) - val storeAnalyzedPlanForView = conf.storeAnalyzedPlanForView || originalText.isEmpty + val storeAnalyzedPlanForView = session.sessionState.conf.storeAnalyzedPlanForView || + originalText.isEmpty if (replace && uncache) { logDebug(s"Try to uncache ${name.quotedString} before replacing.") if (!storeAnalyzedPlanForView) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 23596861a647c..f1033c5cded18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.command.ViewHelper.generateViewProperties import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.util.PartitioningUtils.normalizePartitionSpec @@ -47,6 +48,9 @@ import org.apache.spark.util.ArrayImplicits._ * Replaces [[UnresolvedRelation]]s if the plan is for direct query on files. */ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { + + override def conf: SQLConf = sparkSession.sessionState.conf + object UnresolvedRelationResolution { def unapply(plan: LogicalPlan): Option[LogicalPlan] = { plan match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala index 3a08b13be0134..6907061d67703 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.DYNAMIC_PRUNING_SUBQUERY import org.apache.spark.sql.execution.{InSubqueryExec, QueryExecution, SparkPlan, SubqueryBroadcastExec} import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.internal.SQLConf /** * This planner rule aims at rewriting dynamic pruning predicates in order to reuse the @@ -36,6 +37,8 @@ import org.apache.spark.sql.execution.joins._ */ case class PlanDynamicPruningFilters(sparkSession: SparkSession) extends Rule[SparkPlan] { + override def conf: SQLConf = sparkSession.sessionState.conf + /** * Identify the shape in which keys of a given plan are broadcasted. */ 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 87ce809914e10..73d0327e2bcad 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 @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceStrate 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 +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} /** @@ -117,6 +117,9 @@ class ResolveHiveSerdeTable(session: SparkSession) extends Rule[LogicalPlan] { } class DetermineTableStats(session: SparkSession) extends Rule[LogicalPlan] { + + override def conf: SQLConf = session.sessionState.conf + private def hiveTableWithStats(relation: HiveTableRelation): HiveTableRelation = { val table = relation.tableMeta val partitionCols = relation.partitionCols diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala index 779562bed5b0f..6486904fe65af 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.FilterEstimation import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.internal.SQLConf /** * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned @@ -43,6 +44,8 @@ import org.apache.spark.sql.execution.datasources.DataSourceStrategy private[sql] class PruneHiveTablePartitions(session: SparkSession) extends Rule[LogicalPlan] with CastSupport with PredicateHelper { + override def conf: SQLConf = session.sessionState.conf + /** * Extract the partition filters from the filters on the table. */ From 885915f545370c5b38b8d022e17c129cd085814f Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Fri, 13 Dec 2024 21:56:01 +0900 Subject: [PATCH 168/438] [SPARK-50566][MINOR][SS] Fix code style violation for RocksDB file ### What changes were proposed in this pull request? Fix code style violation for RocksDB file, especially indentations. ### Why are the changes needed? Better code style ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No Need ### Was this patch authored or co-authored using generative AI tooling? No Closes #49174 from WweiL/SPARK-50566-style. Authored-by: Wei Liu Signed-off-by: Jungtaek Lim --- .../execution/streaming/state/RocksDB.scala | 213 +++++++++--------- 1 file changed, 106 insertions(+), 107 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index c13884a948195..795d29c16bcfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -315,119 +315,121 @@ class RocksDB( * and possibly restart the native RocksDB instance. */ private def loadWithCheckpointId( - version: Long, - stateStoreCkptId: Option[String], - readOnly: Boolean = false): RocksDB = { - // An array contains lineage information from [snapShotVersion, version] (inclusive in both ends) - var currVersionLineage: Array[LineageItem] = lineageManager.getLineageForCurrVersion() - try { - if (loadedVersion != version || (loadedStateStoreCkptId.isEmpty || - stateStoreCkptId.get != loadedStateStoreCkptId.get)) { - closeDB(ignoreException = false) - - val (latestSnapshotVersion, latestSnapshotUniqueId) = { - // Special handling when version is 0. - // When loading the very first version (0), stateStoreCkptId does not need to be defined - // because there won't be 0.changelog / 0.zip file created in RocksDB under v2. - if (version == 0) { - assert(stateStoreCkptId.isEmpty, "stateStoreCkptId should be empty when version is zero") - (0L, None) - // When there is a snapshot file, it is the ground truth, we can skip - // reconstructing the lineage from changelog file. - } else if (fileManager.existsSnapshotFile(version, stateStoreCkptId)) { - currVersionLineage = Array(LineageItem(version, stateStoreCkptId.get)) - (version, stateStoreCkptId) - } else { - currVersionLineage = getLineageFromChangelogFile(version, stateStoreCkptId) :+ - LineageItem(version, stateStoreCkptId.get) - currVersionLineage = currVersionLineage.sortBy(_.version) - - val latestSnapshotVersionsAndUniqueId = - fileManager.getLatestSnapshotVersionAndUniqueIdFromLineage(currVersionLineage) - latestSnapshotVersionsAndUniqueId match { - case Some(pair) => (pair._1, Option(pair._2)) - case None if currVersionLineage.head.version == 1L => - logDebug(log"Cannot find latest snapshot based on lineage but first version " + - log"is 1, use 0 as default. Lineage: ${MDC(LogKeys.LINEAGE, lineageManager)}") - (0L, None) - case _ => - throw QueryExecutionErrors.cannotFindBaseSnapshotCheckpoint( - printLineageItems(currVersionLineage)) + version: Long, + stateStoreCkptId: Option[String], + readOnly: Boolean = false): RocksDB = { + // An array contains lineage information from [snapShotVersion, version] + // (inclusive in both ends) + var currVersionLineage: Array[LineageItem] = lineageManager.getLineageForCurrVersion() + try { + if (loadedVersion != version || (loadedStateStoreCkptId.isEmpty || + stateStoreCkptId.get != loadedStateStoreCkptId.get)) { + closeDB(ignoreException = false) + + val (latestSnapshotVersion, latestSnapshotUniqueId) = { + // Special handling when version is 0. + // When loading the very first version (0), stateStoreCkptId does not need to be defined + // because there won't be 0.changelog / 0.zip file created in RocksDB under v2. + if (version == 0) { + assert(stateStoreCkptId.isEmpty, + "stateStoreCkptId should be empty when version is zero") + (0L, None) + // When there is a snapshot file, it is the ground truth, we can skip + // reconstructing the lineage from changelog file. + } else if (fileManager.existsSnapshotFile(version, stateStoreCkptId)) { + currVersionLineage = Array(LineageItem(version, stateStoreCkptId.get)) + (version, stateStoreCkptId) + } else { + currVersionLineage = getLineageFromChangelogFile(version, stateStoreCkptId) :+ + LineageItem(version, stateStoreCkptId.get) + currVersionLineage = currVersionLineage.sortBy(_.version) + + val latestSnapshotVersionsAndUniqueId = + fileManager.getLatestSnapshotVersionAndUniqueIdFromLineage(currVersionLineage) + latestSnapshotVersionsAndUniqueId match { + case Some(pair) => (pair._1, Option(pair._2)) + case None if currVersionLineage.head.version == 1L => + logDebug(log"Cannot find latest snapshot based on lineage but first version " + + log"is 1, use 0 as default. Lineage: ${MDC(LogKeys.LINEAGE, lineageManager)}") + (0L, None) + case _ => + throw QueryExecutionErrors.cannotFindBaseSnapshotCheckpoint( + printLineageItems(currVersionLineage)) + } } } - } - logInfo(log"Loaded latestSnapshotVersion: ${ - MDC(LogKeys.SNAPSHOT_VERSION, latestSnapshotVersion)}, latestSnapshotUniqueId: ${ - MDC(LogKeys.UUID, latestSnapshotUniqueId)}") + logInfo(log"Loaded latestSnapshotVersion: ${ + MDC(LogKeys.SNAPSHOT_VERSION, latestSnapshotVersion)}, latestSnapshotUniqueId: ${ + MDC(LogKeys.UUID, latestSnapshotUniqueId)}") - val metadata = fileManager.loadCheckpointFromDfs(latestSnapshotVersion, - workingDir, rocksDBFileMapping, latestSnapshotUniqueId) + val metadata = fileManager.loadCheckpointFromDfs(latestSnapshotVersion, + workingDir, rocksDBFileMapping, latestSnapshotUniqueId) - loadedVersion = latestSnapshotVersion + loadedVersion = latestSnapshotVersion - // reset the last snapshot version to the latest available snapshot version - lastSnapshotVersion = latestSnapshotVersion - lineageManager.resetLineage(currVersionLineage) + // reset the last snapshot version to the latest available snapshot version + lastSnapshotVersion = latestSnapshotVersion + lineageManager.resetLineage(currVersionLineage) - // Initialize maxVersion upon successful load from DFS - fileManager.setMaxSeenVersion(version) + // Initialize maxVersion upon successful load from DFS + fileManager.setMaxSeenVersion(version) - openLocalRocksDB(metadata) + openLocalRocksDB(metadata) - if (loadedVersion != version) { - val versionsAndUniqueIds = currVersionLineage.collect { - case i if i.version > loadedVersion && i.version <= version => - (i.version, Option(i.checkpointUniqueId)) - } - replayChangelog(versionsAndUniqueIds) - loadedVersion = version - lineageManager.resetLineage(currVersionLineage) + if (loadedVersion != version) { + val versionsAndUniqueIds = currVersionLineage.collect { + case i if i.version > loadedVersion && i.version <= version => + (i.version, Option(i.checkpointUniqueId)) + } + replayChangelog(versionsAndUniqueIds) + loadedVersion = version + lineageManager.resetLineage(currVersionLineage) + } + // After changelog replay the numKeysOnWritingVersion will be updated to + // the correct number of keys in the loaded version. + numKeysOnLoadedVersion = numKeysOnWritingVersion + fileManagerMetrics = fileManager.latestLoadCheckpointMetrics } - // After changelog replay the numKeysOnWritingVersion will be updated to - // the correct number of keys in the loaded version. - numKeysOnLoadedVersion = numKeysOnWritingVersion - fileManagerMetrics = fileManager.latestLoadCheckpointMetrics - } - - lastCommitBasedStateStoreCkptId = None - loadedStateStoreCkptId = stateStoreCkptId - sessionStateStoreCkptId = Some(java.util.UUID.randomUUID.toString) - lastCommittedStateStoreCkptId = None - if (conf.resetStatsOnLoad) { - nativeStats.reset - } - - logInfo(log"Loaded ${MDC(LogKeys.VERSION_NUM, version)} " + - log"with uniqueId ${MDC(LogKeys.UUID, stateStoreCkptId)}") - } catch { - case t: Throwable => - loadedVersion = -1 // invalidate loaded data lastCommitBasedStateStoreCkptId = None + loadedStateStoreCkptId = stateStoreCkptId + sessionStateStoreCkptId = Some(java.util.UUID.randomUUID.toString) lastCommittedStateStoreCkptId = None - loadedStateStoreCkptId = None - sessionStateStoreCkptId = None - lineageManager.clear() - throw t - } - if (enableChangelogCheckpointing && !readOnly) { - // Make sure we don't leak resource. - changelogWriter.foreach(_.abort()) - // Initialize the changelog writer with lineage info - // The lineage stored in changelog files should normally start with - // the version of a snapshot, except for the first few versions. - // Because they are solely loaded from changelog file. - // (e.g. with default minDeltasForSnapshot, there is only 1_uuid1.changelog, no 1_uuid1.zip) - // It should end with exactly one version before the change log's version. - changelogWriter = Some(fileManager.getChangeLogWriter( - version + 1, - useColumnFamilies, - sessionStateStoreCkptId, - Some(currVersionLineage))) + + if (conf.resetStatsOnLoad) { + nativeStats.reset + } + + logInfo(log"Loaded ${MDC(LogKeys.VERSION_NUM, version)} " + + log"with uniqueId ${MDC(LogKeys.UUID, stateStoreCkptId)}") + } catch { + case t: Throwable => + loadedVersion = -1 // invalidate loaded data + lastCommitBasedStateStoreCkptId = None + lastCommittedStateStoreCkptId = None + loadedStateStoreCkptId = None + sessionStateStoreCkptId = None + lineageManager.clear() + throw t + } + if (enableChangelogCheckpointing && !readOnly) { + // Make sure we don't leak resource. + changelogWriter.foreach(_.abort()) + // Initialize the changelog writer with lineage info + // The lineage stored in changelog files should normally start with + // the version of a snapshot, except for the first few versions. + // Because they are solely loaded from changelog file. + // (e.g. with default minDeltasForSnapshot, there is only 1_uuid1.changelog, no 1_uuid1.zip) + // It should end with exactly one version before the change log's version. + changelogWriter = Some(fileManager.getChangeLogWriter( + version + 1, + useColumnFamilies, + sessionStateStoreCkptId, + Some(currVersionLineage))) + } + this } - this -} private def loadWithoutCheckpointId( version: Long, @@ -483,7 +485,6 @@ class RocksDB( * Initialize key metrics based on the metadata loaded from DFS and open local RocksDB. */ private def openLocalRocksDB(metadata: RocksDBCheckpointMetadata): Unit = { - setInitialCFInfo() metadata.columnFamilyMapping.foreach { mapping => colFamilyNameToIdMap.putAll(mapping.asJava) @@ -670,7 +671,6 @@ class RocksDB( * @note This update is not committed to disk until commit() is called. */ def merge(key: Array[Byte], value: Array[Byte]): Unit = { - if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, key) if (oldValue == null) { @@ -701,7 +701,6 @@ class RocksDB( * Get an iterator of all committed and uncommitted key-value pairs. */ def iterator(): Iterator[ByteArrayPair] = { - val iter = db.newIterator() logInfo(log"Getting iterator from version ${MDC(LogKeys.LOADED_VERSION, loadedVersion)}") iter.seekToFirst() @@ -1103,11 +1102,11 @@ class RocksDB( } private def createSnapshot( - checkpointDir: File, - version: Long, - columnFamilyMapping: Map[String, Short], - maxColumnFamilyId: Short, - uniqueId: Option[String] = None): RocksDBSnapshot = { + checkpointDir: File, + version: Long, + columnFamilyMapping: Map[String, Short], + maxColumnFamilyId: Short, + uniqueId: Option[String] = None): RocksDBSnapshot = { val (dfsFileSuffix, immutableFileMapping) = rocksDBFileMapping.createSnapshotFileMapping( fileManager, checkpointDir, version) From d1db5104d080f7eb21bdda2b28ae16a31318afc5 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 13 Dec 2024 07:17:12 -0800 Subject: [PATCH 169/438] [SPARK-50563][BUILD] Upgrade `protobuf-java` to 4.29.1 ### What changes were proposed in this pull request? This pr aims to upgrade `protobuf-java` from 4.28.3 to 4.29.1 ### Why are the changes needed? The new version bring some bug fix like: - CodedOutputStream: avoid updating position to go beyond end of array. (https://github.com/protocolbuffers/protobuf/commit/76ab5f2b109fbd03c160b27a41935ad132287c88) - Fix packed reflection handling bug in edition 2023. (https://github.com/protocolbuffers/protobuf/commit/4c923285a32ebcb53ce2ba03144e50d4dbedaf7b) - Bugfix: Make extensions beyond n=16 immutable. (https://github.com/protocolbuffers/protobuf/commit/ee419f22e0eec21243540d4b7ffe40bd194ed293) Full release notes as follows: - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0-rc1 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0-rc2 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0-rc3 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.1 ### 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 #49169 from LuciferYang/protobuf-java-4.29.1. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 5f8556ce0c211..674b74c979ef2 100644 --- a/pom.xml +++ b/pom.xml @@ -125,7 +125,7 @@ 3.4.1 - 4.28.3 + 4.29.1 3.11.4 3.9.3 5.7.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b060483c89c9b..91eb0aa1612d3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -89,7 +89,7 @@ object BuildCommons { // Google Protobuf version used for generating the protobuf. // SPARK-41247: needs to be consistent with `protobuf.version` in `pom.xml`. - val protoVersion = "4.28.3" + val protoVersion = "4.29.1" // GRPC version used for Spark Connect. val grpcVersion = "1.67.1" } From 3362ec8c3563e2a8262e0ba540f2729346d7b71f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 13 Dec 2024 07:40:04 -0800 Subject: [PATCH 170/438] [SPARK-50567][INFRA] Apply Python 3.11 image in No-ANSI daily build ### What changes were proposed in this pull request? Apply Python 3.11 image in No-ANSI daily build ### Why are the changes needed? switch to the new images ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11", "SPARK_ANSI_SQL_MODE": "false"}' ``` https://github.com/zhengruifeng/spark/actions/runs/12312692530/job/34365351588 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49176 from zhengruifeng/py_311_no_ansi. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- .github/workflows/build_non_ansi.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/build_non_ansi.yml b/.github/workflows/build_non_ansi.yml index 4ac2a589f4f81..32489ea8c858d 100644 --- a/.github/workflows/build_non_ansi.yml +++ b/.github/workflows/build_non_ansi.yml @@ -36,6 +36,8 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-311", + "PYTHON_TO_TEST": "python3.11", "SPARK_ANSI_SQL_MODE": "false", } jobs: >- From fbc061d0c85648fc1cb54733d3f1ebbd2379c4a5 Mon Sep 17 00:00:00 2001 From: cashmand Date: Fri, 13 Dec 2024 10:11:15 -0800 Subject: [PATCH 171/438] [SPARK-48898][SQL] Set nullability correctly in the Variant schema ### What changes were proposed in this pull request? The variantShreddingSchema method converts a human-readable schema for Variant to one that's a valid shredding schema. According to the shredding schema in https://github.com/apache/parquet-format/pull/461, each shredded field in an object should be a required group - i.e. a non-nullable struct. This PR fixes the variantShreddingSchema to mark that struct as non-nullable. ### Why are the changes needed? If we use variantShreddingSchema to construct a schema for Parquet, the schema would be technically non-conformant with the spec by setting the group as optional. I don't think this should really matter to readers, but it would waste a bit of space in the Parquet file by adding an extra definition level. ### Does this PR introduce _any_ user-facing change? No, this code is not used yet. ### How was this patch tested? Added a test to do some minimal validation of the variantShreddingSchema function. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49151 from cashmand/SPARK-48898-nullability-again. Authored-by: cashmand Signed-off-by: Wenchen Fan --- .../parquet/SparkShreddingUtils.scala | 11 +++++-- .../spark/sql/VariantShreddingSuite.scala | 11 ++++++- .../sql/VariantWriteShreddingSuite.scala | 30 +++++++++++++++++++ 3 files changed, 48 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index 41244e20c369f..f38e188ed042c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -73,16 +73,21 @@ case object SparkShreddingUtils { */ def variantShreddingSchema(dataType: DataType, isTopLevel: Boolean = true): StructType = { val fields = dataType match { - case ArrayType(elementType, containsNull) => + case ArrayType(elementType, _) => + // Always set containsNull to false. One of value or typed_value must always be set for + // array elements. val arrayShreddingSchema = - ArrayType(variantShreddingSchema(elementType, false), containsNull) + ArrayType(variantShreddingSchema(elementType, false), containsNull = false) Seq( StructField(VariantValueFieldName, BinaryType, nullable = true), StructField(TypedValueFieldName, arrayShreddingSchema, nullable = true) ) case StructType(fields) => + // The field name level is always non-nullable: Variant null values are represented in the + // "value" columna as "00", and missing values are represented by setting both "value" and + // "typed_value" to null. val objectShreddingSchema = StructType(fields.map(f => - f.copy(dataType = variantShreddingSchema(f.dataType, false)))) + f.copy(dataType = variantShreddingSchema(f.dataType, false), nullable = false))) Seq( StructField(VariantValueFieldName, BinaryType, nullable = true), StructField(TypedValueFieldName, objectShreddingSchema, nullable = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala index 4ff346b957aa0..5d5c441052558 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala @@ -155,7 +155,16 @@ class VariantShreddingSuite extends QueryTest with SharedSparkSession with Parqu Row(metadata(Nil), null, Array(Row(null, null)))) checkException(path, "v", "MALFORMED_VARIANT") // Shredded field must not be null. - writeRows(path, writeSchema(StructType.fromDDL("a int")), + // Construct the schema manually, because SparkShreddingUtils.variantShreddingSchema will make + // `a` non-nullable, which would prevent us from writing the file. + val schema = StructType(Seq(StructField("v", StructType(Seq( + StructField("metadata", BinaryType), + StructField("value", BinaryType), + StructField("typed_value", StructType(Seq( + StructField("a", StructType(Seq( + StructField("value", BinaryType), + StructField("typed_value", BinaryType)))))))))))) + writeRows(path, schema, Row(metadata(Seq("a")), null, Row(null))) checkException(path, "v", "MALFORMED_VARIANT") // `value` must not contain any shredded field. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala index a62c6e4462464..d31bf109af6c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantWriteShreddingSuite.scala @@ -67,6 +67,36 @@ class VariantWriteShreddingSuite extends SparkFunSuite with ExpressionEvalHelper private val emptyMetadata: Array[Byte] = parseJson("null").getMetadata + test("variantShreddingSchema") { + // Validate the schema produced by SparkShreddingUtils.variantShreddingSchema for a few simple + // cases. + // metadata is always non-nullable. + assert(SparkShreddingUtils.variantShreddingSchema(IntegerType) == + StructType(Seq( + StructField("metadata", BinaryType, nullable = false), + StructField("value", BinaryType, nullable = true), + StructField("typed_value", IntegerType, nullable = true)))) + + val fieldA = StructType(Seq( + StructField("value", BinaryType, nullable = true), + StructField("typed_value", TimestampNTZType, nullable = true))) + val arrayType = ArrayType(StructType(Seq( + StructField("value", BinaryType, nullable = true), + StructField("typed_value", StringType, nullable = true))), containsNull = false) + val fieldB = StructType(Seq( + StructField("value", BinaryType, nullable = true), + StructField("typed_value", arrayType, nullable = true))) + val objectType = StructType(Seq( + StructField("a", fieldA, nullable = false), + StructField("b", fieldB, nullable = false))) + val structSchema = DataType.fromDDL("a timestamp_ntz, b array") + assert(SparkShreddingUtils.variantShreddingSchema(structSchema) == + StructType(Seq( + StructField("metadata", BinaryType, nullable = false), + StructField("value", BinaryType, nullable = true), + StructField("typed_value", objectType, nullable = true)))) + } + test("shredding as fixed numeric types") { /* Cast integer to any wider numeric type. */ testWithSchema("1", IntegerType, Row(emptyMetadata, null, 1)) From f5740d0b94e5419494696d44bae1060140ef0a8f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 13 Dec 2024 10:22:44 -0800 Subject: [PATCH 172/438] [SPARK-50571][INFRA] Apply Python 3.11 image in RocksDB as UI Backend daily build ### What changes were proposed in this pull request? Apply Python 3.11 image in RocksDB as UI Backend daily build ### Why are the changes needed? to switch to the new images ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with: ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11", "LIVE_UI_LOCAL_STORE_DIR": "/tmp/kvStore"}' ``` https://github.com/zhengruifeng/spark/actions/runs/12312784922/job/34365520345 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49178 from zhengruifeng/py_311_rock_db. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- .github/workflows/build_rockdb_as_ui_backend.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/build_rockdb_as_ui_backend.yml b/.github/workflows/build_rockdb_as_ui_backend.yml index 96009c41dbbf9..a85edd09256ed 100644 --- a/.github/workflows/build_rockdb_as_ui_backend.yml +++ b/.github/workflows/build_rockdb_as_ui_backend.yml @@ -36,6 +36,8 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "python-311", + "PYTHON_TO_TEST": "python3.11", "LIVE_UI_LOCAL_STORE_DIR": "/tmp/kvStore", } jobs: >- From 42ce6044e15bf09f10ee6bf507e2cbf02776764d Mon Sep 17 00:00:00 2001 From: Julek Sompolski Date: Fri, 13 Dec 2024 10:29:54 -0800 Subject: [PATCH 173/438] [SPARK-50338][CORE] Make LazyTry exceptions less verbose MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? For LazyTry, we want two stack traces: 1. the one from the original execution when the exception happened 2. the one from the current execution, that retrieved the exception. The first one is added as suppressed exception. The second one is stitched into the stack trace of the rethrown exception. The third one (`TRY_WITH_CALLER_STACKTRACE_TRY_STACKTRACE`) doesn't need to be visible to users as a suppressed exception. We only need the depth of the stacktrace until `doTryWithCallerStacktrace`. Use a custom exception `OriginalTryStackTraceException` to store it together with the suppressed first one. Because of the duplicated `TRY_WITH_CALLER_STACKTRACE_TRY_STACKTRACE` exceptions it was actually also doing the stitching incorrectly. #### Before There are 4 suppressed exception. ``` scala> sql("select make_timestamp(2024, 1, 1, 0, 0, 0, 1)").collect() org.apache.spark.SparkDateTimeException: [INVALID_TIMEZONE] The timezone: 1 is invalid. The timezone must be either a region-based zone ID or a zone offset. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in the format '(+|-)HH', '(+|-)HH:mm’ or '(+|-)HH:mm:ss', e.g '-08' , '+01:00' or '-13:33:33', and must be in the range from -18:00 to +18:00. 'Z' and 'UTC' are accepted as synonyms for '+00:00'. SQLSTATE: 22009 at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError(ExecutionErrors.scala:254) at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError$(ExecutionErrors.scala:246) at org.apache.spark.sql.errors.ExecutionErrors$.zoneOffsetError(ExecutionErrors.scala:258) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:52) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1429) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:228) at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:277) at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:324) at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:292) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$8(SQLExecution.scala:148) at org.apache.spark.sql.execution.SQLExecution$.withSessionTagsApplied(SQLExecution.scala:268) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$7(SQLExecution.scala:124) at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94) at org.apache.spark.sql.artifact.ArtifactManager.$anonfun$withResources$1(ArtifactManager.scala:108) at org.apache.spark.sql.artifact.ArtifactManager.withClassLoaderIfNeeded(ArtifactManager.scala:102) at org.apache.spark.sql.artifact.ArtifactManager.withResources(ArtifactManager.scala:107) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:124) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:291) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:123) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId0(SQLExecution.scala:77) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:233) at org.apache.spark.sql.Dataset.withAction(Dataset.scala:2199) at org.apache.spark.sql.Dataset.collect(Dataset.scala:1473) ... 30 elided Caused by: java.time.DateTimeException: Invalid ID for ZoneOffset, invalid format: 1 at java.base/java.time.ZoneOffset.of(ZoneOffset.java:243) at java.base/java.time.ZoneId.of(ZoneId.java:404) at java.base/java.time.ZoneId.of(ZoneId.java:358) at java.base/java.time.ZoneId.of(ZoneId.java:314) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:49) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:192) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:194) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:214) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) ... 50 more Suppressed: java.lang.Exception: Stacktrace under doTryWithCallerStacktrace at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError(ExecutionErrors.scala:254) at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError$(ExecutionErrors.scala:246) at org.apache.spark.sql.errors.ExecutionErrors$.zoneOffsetError(ExecutionErrors.scala:258) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:52) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) Suppressed: java.lang.Exception: Full stacktrace of original doTryWithCallerStacktrace caller at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError(ExecutionErrors.scala:254) at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError$(ExecutionErrors.scala:246) at org.apache.spark.sql.errors.ExecutionErrors$.zoneOffsetError(ExecutionErrors.scala:258) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:52) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:192) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:194) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:214) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) ... 50 more Suppressed: java.lang.Exception: Stacktrace under doTryWithCallerStacktrace at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError(ExecutionErrors.scala:254) at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError$(ExecutionErrors.scala:246) at org.apache.spark.sql.errors.ExecutionErrors$.zoneOffsetError(ExecutionErrors.scala:258) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:52) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1429) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:192) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:194) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:214) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) Suppressed: java.lang.Exception: Full stacktrace of original doTryWithCallerStacktrace caller at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError(ExecutionErrors.scala:254) at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError$(ExecutionErrors.scala:246) at org.apache.spark.sql.errors.ExecutionErrors$.zoneOffsetError(ExecutionErrors.scala:258) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:52) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1429) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:192) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:194) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:214) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1375) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) ... 50 more ``` #### After There is 1 suppressed exception, showing the stack trace of the original caller that triggered the computation of the lazy (it is the same as the caller that reads the lazy, because they are read immediately, but it possibly could be different, and then both the stacktraces are needed for debugging) ``` scala> sql("select make_timestamp(2024, 1, 1, 0, 0, 0, 1)").collect() org.apache.spark.SparkDateTimeException: [INVALID_TIMEZONE] The timezone: 1 is invalid. The timezone must be either a region-based zone ID or a zone offset. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in the format '(+|-)HH', '(+|-)HH:mm’ or '(+|-)HH:mm:ss', e.g '-08' , '+01:00' or '-13:33:33', and must be in the range from -18:00 to +18:00. 'Z' and 'UTC' are accepted as synonyms for '+00:00'. SQLSTATE: 22009 at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError(ExecutionErrors.scala:254) at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError$(ExecutionErrors.scala:246) at org.apache.spark.sql.errors.ExecutionErrors$.zoneOffsetError(ExecutionErrors.scala:258) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:52) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:192) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:194) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:214) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1438) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:228) at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:277) at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:324) at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:292) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$8(SQLExecution.scala:148) at org.apache.spark.sql.execution.SQLExecution$.withSessionTagsApplied(SQLExecution.scala:268) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$7(SQLExecution.scala:124) at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94) at org.apache.spark.sql.artifact.ArtifactManager.$anonfun$withResources$1(ArtifactManager.scala:108) at org.apache.spark.sql.artifact.ArtifactManager.withClassLoaderIfNeeded(ArtifactManager.scala:102) at org.apache.spark.sql.artifact.ArtifactManager.withResources(ArtifactManager.scala:107) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:124) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:291) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:123) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId0(SQLExecution.scala:77) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:233) at org.apache.spark.sql.Dataset.withAction(Dataset.scala:2199) at org.apache.spark.sql.Dataset.collect(Dataset.scala:1473) ... 30 elided Caused by: java.time.DateTimeException: Invalid ID for ZoneOffset, invalid format: 1 at java.base/java.time.ZoneOffset.of(ZoneOffset.java:243) at java.base/java.time.ZoneId.of(ZoneId.java:404) at java.base/java.time.ZoneId.of(ZoneId.java:358) at java.base/java.time.ZoneId.of(ZoneId.java:314) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:49) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:192) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:194) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:214) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) ... 50 more Suppressed: org.apache.spark.util.Utils$OriginalTryStackTraceException: Full stacktrace of original doTryWithCallerStacktrace caller at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError(ExecutionErrors.scala:254) at org.apache.spark.sql.errors.ExecutionErrors.zoneOffsetError$(ExecutionErrors.scala:246) at org.apache.spark.sql.errors.ExecutionErrors$.zoneOffsetError(ExecutionErrors.scala:258) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId(SparkDateTimeUtils.scala:52) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.getZoneId$(SparkDateTimeUtils.scala:43) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.getZoneId(DateTimeUtils.scala:41) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.$anonfun$nullSafeEval$3(datetimeExpressions.scala:2856) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.expressions.MakeTimestamp.nullSafeEval(datetimeExpressions.scala:2856) at org.apache.spark.sql.catalyst.expressions.SeptenaryExpression.eval(Expression.scala:1201) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:82) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:102) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1231) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1230) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:573) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:102) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:106) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:223) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:235) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:241) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:241) at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$5(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:312) at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:246) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:106) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:105) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:436) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:105) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:242) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:231) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:231) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:201) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:201) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$2(QueryExecution.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:258) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:634) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:258) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:706) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:257) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyOptimizedPlan$1(QueryExecution.scala:178) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) at org.apache.spark.util.LazyTry.get(LazyTry.scala:58) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:192) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:194) at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:214) at scala.util.Try$.apply(Try.scala:217) at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46) at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46) ... 50 more ``` ### Why are the changes needed? The suppressed Exception was too verbose. ### Does this PR introduce _any_ user-facing change? Yes. Stacktraces of Exceptions coming from a LazyTry (e.g. QueryExecution plans) should be more readable. ### How was this patch tested? Unit tests were adapted. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48882 from juliuszsompolski/SPARK-50338. Lead-authored-by: Julek Sompolski Co-authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/util/Utils.scala | 37 ++-- .../org/apache/spark/util/UtilsSuite.scala | 166 ++++++++++++++++-- 2 files changed, 173 insertions(+), 30 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 9e7ba6d879aa0..668657c87b39b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1355,8 +1355,10 @@ private[spark] object Utils val TRY_WITH_CALLER_STACKTRACE_FULL_STACKTRACE = "Full stacktrace of original doTryWithCallerStacktrace caller" - val TRY_WITH_CALLER_STACKTRACE_TRY_STACKTRACE = - "Stacktrace under doTryWithCallerStacktrace" + class OriginalTryStackTraceException() + extends Exception(TRY_WITH_CALLER_STACKTRACE_FULL_STACKTRACE) { + var doTryWithCallerStacktraceDepth: Int = 0 + } /** * Use Try with stacktrace substitution for the caller retrieving the error. @@ -1384,14 +1386,19 @@ private[spark] object Utils val commonSuffixLen = origStackTrace.reverse.zip(currentStackTrace.reverse).takeWhile { case (exElem, currentElem) => exElem == currentElem }.length - val belowEx = new Exception(TRY_WITH_CALLER_STACKTRACE_TRY_STACKTRACE) - belowEx.setStackTrace(origStackTrace.dropRight(commonSuffixLen)) - ex.addSuppressed(belowEx) - - // keep the full original stack trace in a suppressed exception. - val fullEx = new Exception(TRY_WITH_CALLER_STACKTRACE_FULL_STACKTRACE) - fullEx.setStackTrace(origStackTrace) - ex.addSuppressed(fullEx) + // Add the full stack trace of the original caller as the suppressed exception. + // It may already be there if it's a nested call to doTryWithCallerStacktrace. + val origEx = ex.getSuppressed.find { e => + e.isInstanceOf[OriginalTryStackTraceException] + }.getOrElse { + val fullEx = new OriginalTryStackTraceException() + fullEx.setStackTrace(origStackTrace) + ex.addSuppressed(fullEx) + fullEx + }.asInstanceOf[OriginalTryStackTraceException] + // Update the depth of the stack of the current doTryWithCallerStacktrace, for stitching + // it with the stack of getTryWithCallerStacktrace. + origEx.doTryWithCallerStacktraceDepth = origStackTrace.size - commonSuffixLen case Success(_) => // nothing } t @@ -1407,7 +1414,7 @@ private[spark] object Utils * Full stack trace of the original doTryWithCallerStacktrace caller can be retrieved with * ``` * ex.getSuppressed.find { e => - * e.getMessage == Utils.TRY_WITH_CALLER_STACKTRACE_FULL_STACKTRACE + * e.isInstanceOf[Utils.OriginalTryStackTraceException] * } * ``` * @@ -1417,13 +1424,15 @@ private[spark] object Utils */ def getTryWithCallerStacktrace[T](t: Try[T]): T = t match { case Failure(ex) => - val belowStacktrace = ex.getSuppressed.find { e => + val originalStacktraceEx = ex.getSuppressed.find { e => // added in doTryWithCallerStacktrace - e.getMessage == TRY_WITH_CALLER_STACKTRACE_TRY_STACKTRACE + e.isInstanceOf[OriginalTryStackTraceException] }.getOrElse { // If we don't have the expected stacktrace information, just rethrow throw ex - }.getStackTrace + }.asInstanceOf[OriginalTryStackTraceException] + val belowStacktrace = originalStacktraceEx.getStackTrace + .take(originalStacktraceEx.doTryWithCallerStacktraceDepth) // We are modifying and throwing the original exception. It would be better if we could // return a copy, but we can't easily clone it and preserve. If this is accessed from // multiple threads that then look at the stack trace, this could break. diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index a6e3345fc600c..febc59e5c5369 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1581,26 +1581,14 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties { // at org.apache.spark.util.UtilsSuite.$anonfun$new$165(UtilsSuite.scala:1658) // ... 56 more // scalastyle:on line.size.limit - val origSt = e1.getSuppressed.find( - _.getMessage == Utils.TRY_WITH_CALLER_STACKTRACE_FULL_STACKTRACE) + val origSt = e1.getSuppressed.find(_.isInstanceOf[Utils.OriginalTryStackTraceException]) assert(origSt.isDefined) assert(origSt.get.getStackTrace.exists(_.getMethodName == "throwException")) assert(origSt.get.getStackTrace.exists(_.getMethodName == "callDoTry")) - // The stack trace under Try should be in the suppressed exceptions. - // Example: - // Suppressed: java.lang.Exception: Stacktrace under doTryWithCallerStacktrace - // at org.apache.spark.util.UtilsSuite.throwException(UtilsSuite.scala: 1640) - // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTry$1(UtilsSuite.scala: 1645) - // at scala.util.Try$.apply(Try.scala: 213) - // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala: 1586) - val trySt = e1.getSuppressed.find( - _.getMessage == Utils.TRY_WITH_CALLER_STACKTRACE_TRY_STACKTRACE) - assert(trySt.isDefined) - // calls under callDoTry should be present. - assert(trySt.get.getStackTrace.exists(_.getMethodName == "throwException")) - // callDoTry should be removed. - assert(!trySt.get.getStackTrace.exists(_.getMethodName == "callDoTry")) + // Should save the depth of the stack trace under doTryWithCallerStacktrace. + assert(origSt.get.asInstanceOf[Utils.OriginalTryStackTraceException] + .doTryWithCallerStacktraceDepth == 4) val e2 = intercept[Exception] { callGetTryAgain(t) @@ -1633,6 +1621,152 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties { assert(st1Again.exists(_.getMethodName == "callGetTryAgain")) assert(!st1Again.exists(_.getMethodName == "callGetTry")) } + + private def callGetTryFromNested(t: Try[String]): String = { + Utils.getTryWithCallerStacktrace(t) + } + + private def callDoTryNested(): Try[String] = { + Utils.doTryWithCallerStacktrace { + val t = callDoTry() + val e = intercept[Exception] { + callGetTryFromNested(t) + } + + // Uncomment for manual inspection + // + // println("\nIntercepted in callDoTryNested:") + // e.printStackTrace() + // + // scalastyle:off line.size.limit + // java.lang.Exception: test + // at org.apache.spark.util.UtilsSuite.throwException(UtilsSuite.scala:1529) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTry$1(UtilsSuite.scala:1534) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1438) + // ----> at org.apache.spark.util.UtilsSuite.callGetTryFromNested(UtilsSuite.scala:1626) <---- STITCHED. + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNested$2(UtilsSuite.scala:1633) + // at org.scalatest.Assertions.intercept(Assertions.scala:749) + // at org.scalatest.Assertions.intercept$(Assertions.scala:746) + // at org.scalatest.funsuite.AnyFunSuite.intercept(AnyFunSuite.scala:1564) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNested$1(UtilsSuite.scala:1632) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.UtilsSuite.callDoTryNested(UtilsSuite.scala:1630) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNestedNested$1(UtilsSuite.scala:1655) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.UtilsSuite.callDoTryNestedNested(UtilsSuite.scala:1654) + // at org.apache.spark.util.UtilsSuite.$anonfun$new$172(UtilsSuite.scala:1674) + // ... + // Suppressed: org.apache.spark.util.Utils$OriginalTryStackTraceException: Full stacktrace of original doTryWithCallerStacktrace caller + // at org.apache.spark.util.UtilsSuite.throwException(UtilsSuite.scala:1529) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTry$1(UtilsSuite.scala:1534) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.UtilsSuite.callDoTry(UtilsSuite.scala:1534) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNested$1(UtilsSuite.scala:1631) + // ... + // scalastyle:on line.size.limit + + assert(e.getStackTrace.exists(_.getMethodName == "callGetTryFromNested")) + assert(!e.getStackTrace.exists(_.getMethodName == "callGetTryFromNestedNested")) + assert(!e.getStackTrace.exists(_.getMethodName == "callGetTry")) + assert(e.getSuppressed.length == 1) + + Utils.getTryWithCallerStacktrace(t) + } + } + + private def callGetTryFromNestedNested(t: Try[String]): String = { + Utils.getTryWithCallerStacktrace(t) + } + + private def callDoTryNestedNested(): Try[String] = { + Utils.doTryWithCallerStacktrace { + val t = callDoTryNested() + val e = intercept[Exception] { + callGetTryFromNestedNested(t) + } + + // Uncomment for manual inspection + // + // println("\nIntercepted in callDoTryNestedNested:") + // e.printStackTrace() + // + // scalastyle:off line.size.limit + // java.lang.Exception: test + // at org.apache.spark.util.UtilsSuite.throwException(UtilsSuite.scala:1529) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTry$1(UtilsSuite.scala:1534) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.UtilsSuite.callDoTry(UtilsSuite.scala:1534) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNested$1(UtilsSuite.scala:1631) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1438) + // ----> at org.apache.spark.util.UtilsSuite.callGetTryFromNestedNested(UtilsSuite.scala:1650) <---- STITCHED. + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNestedNested$2(UtilsSuite.scala:1657) + // at org.scalatest.Assertions.intercept(Assertions.scala:749) + // at org.scalatest.Assertions.intercept$(Assertions.scala:746) + // at org.scalatest.funsuite.AnyFunSuite.intercept(AnyFunSuite.scala:1564) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNestedNested$1(UtilsSuite.scala:1656) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.UtilsSuite.callDoTryNestedNested(UtilsSuite.scala:1654) + // at org.apache.spark.util.UtilsSuite.$anonfun$new$172(UtilsSuite.scala:1674) + // scalastyle:on line.size.limit + + assert(e.getStackTrace.exists(_.getMethodName == "callGetTryFromNestedNested")) + assert(!e.getStackTrace.exists(_.getMethodName == "callGetTryFromNested")) + assert(!e.getStackTrace.exists(_.getMethodName == "callGetTry")) + assert(e.getSuppressed.length == 1) + + Utils.getTryWithCallerStacktrace(t) + } + } + + test("nested doTryWithCallerStacktrace and getTryWithCallerStacktrace") { + val t = callDoTryNestedNested() + + val e = intercept[Exception] { + callGetTry(t) + } + + // Uncomment for manual inspection + // + // println("\nIntercepted in test:") + // e.printStackTrace() + // + // scalastyle:off line.size.limit + // java.lang.Exception: test + // at org.apache.spark.util.UtilsSuite.throwException(UtilsSuite.scala:1529) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTry$1(UtilsSuite.scala:1534) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.UtilsSuite.callDoTry(UtilsSuite.scala:1534) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNested$1(UtilsSuite.scala:1631) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.UtilsSuite.callDoTryNested(UtilsSuite.scala:1630) + // at org.apache.spark.util.UtilsSuite.$anonfun$callDoTryNestedNested$1(UtilsSuite.scala:1655) + // at scala.util.Try$.apply(Try.scala:217) + // at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1377) + // at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1438) + // ----> at org.apache.spark.util.UtilsSuite.callGetTry(UtilsSuite.scala:1539) <---- STITCHED. + // at org.apache.spark.util.UtilsSuite.$anonfun$new$173(UtilsSuite.scala:1677) + // at org.scalatest.Assertions.intercept(Assertions.scala:749) + // at org.scalatest.Assertions.intercept$(Assertions.scala:746) + // at org.scalatest.funsuite.AnyFunSuite.intercept(AnyFunSuite.scala:1564) + // at org.apache.spark.util.UtilsSuite.$anonfun$new$172(UtilsSuite.scala:1676) + // scalastyle:on line.size.limit + + assert(e.getStackTrace.exists(_.getMethodName == "callGetTry")) + assert(!e.getStackTrace.exists(_.getMethodName == "callGetTryFromNested")) + assert(!e.getStackTrace.exists(_.getMethodName == "callGetTryFromNestedNested")) + assert(e.getSuppressed.length == 1) + } } private class SimpleExtension From ebd6b7c5aacd2b3265262c25880f01548e397122 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Fri, 13 Dec 2024 22:32:10 +0300 Subject: [PATCH 174/438] [SPARK-50546][SQL] Add subquery cast support to collation type coercion ### What changes were proposed in this pull request? Adding proper support for implicit casting of subqueries in collation type coercion. Currently I only added support for `Project` and `Aggregate` plans in subquery expression, but we might need to add more later. ### Why are the changes needed? Since `SubqueryExpression` is a special type of expression it is not possible to just cast it into a different type, and this was causing a runtime error. Instead we need to go inside of its plan and cast the plan outputs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49147 from stefankandic/fixSubquery. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../analysis/CollationTypeCoercion.scala | 34 +++++++ .../CollationTypePrecedenceSuite.scala | 97 +++++++++++++++++++ 2 files changed, 131 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index fa5bb2c071527..980b627729ecf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.CollationStrength.{Default, Explicit, Implicit} import org.apache.spark.sql.catalyst.analysis.TypeCoercion.haveSameType import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project} import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType, StructType} @@ -172,6 +173,9 @@ object CollationTypeCoercion { expr match { case lit: Literal => lit.copy(dataType = newDataType) case cast: Cast => cast.copy(dataType = newDataType) + case subquery: SubqueryExpression => + changeTypeInSubquery(subquery, newType) + case _ => Cast(expr, newDataType) } @@ -180,6 +184,36 @@ object CollationTypeCoercion { } } + /** + * Changes the data type of the expression in the subquery to the given `newType`. + * Currently only supports subqueries with [[Project]] and [[Aggregate]] plan. + */ + private def changeTypeInSubquery( + subqueryExpression: SubqueryExpression, + newType: DataType): SubqueryExpression = { + + def transformNamedExpressions(ex: NamedExpression): NamedExpression = { + changeType(ex, newType) match { + case named: NamedExpression => named + case other => Alias(other, ex.name)() + } + } + + val newPlan = subqueryExpression.plan match { + case project: Project => + val newProjectList = project.projectList.map(transformNamedExpressions) + project.copy(projectList = newProjectList) + + case agg: Aggregate => + val newAggregateExpressions = agg.aggregateExpressions.map(transformNamedExpressions) + agg.copy(aggregateExpressions = newAggregateExpressions) + + case other => other + } + + subqueryExpression.withNewPlan(newPlan) + } + /** * If possible, returns the new data type from `inType` by applying * the collation of `castType`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index c0e61de57cd9d..00fcc501fe744 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -223,6 +223,103 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { ) } + test("in subquery expression") { + val tableName = "subquery_tbl" + withTable(tableName) { + sql(s""" + |CREATE TABLE $tableName ( + | c1 STRING COLLATE UTF8_LCASE, + | c2 STRING COLLATE UNICODE + |) USING $dataSource + |""".stripMargin) + + sql(s"INSERT INTO $tableName VALUES ('a', 'A')") + + assertImplicitMismatch( + sql(s""" + |SELECT * FROM $tableName + |WHERE c1 IN (SELECT c2 FROM $tableName) + |""".stripMargin)) + + // this fails since subquery expression collation is implicit by default + assertImplicitMismatch( + sql(s""" + |SELECT * FROM $tableName + |WHERE c1 IN (SELECT c2 collate unicode FROM $tableName) + |""".stripMargin)) + + checkAnswer( + sql(s""" + |SELECT COUNT(*) FROM $tableName + |WHERE c1 collate utf8_lcase IN (SELECT c2 collate unicode FROM $tableName) + |""".stripMargin), + Seq(Row(1))) + + checkAnswer( + sql(s""" + |SELECT COUNT(*) FROM $tableName + |WHERE c1 collate utf8_lcase IN (SELECT c2 FROM $tableName) + |""".stripMargin), + Seq(Row(1))) + + checkAnswer( + sql(s""" + |SELECT COUNT(*) FROM $tableName + |WHERE c1 collate unicode IN (SELECT c2 FROM $tableName) + |""".stripMargin), + Seq(Row(0))) + + checkAnswer( + sql(s""" + |SELECT COUNT(*) FROM $tableName + |WHERE c1 collate unicode IN (SELECT c2 FROM $tableName + | WHERE c2 collate unicode IN (SELECT c1 FROM $tableName)) + |""".stripMargin), + Seq(Row(0))) + } + } + + test("scalar subquery") { + val tableName = "scalar_subquery_tbl" + withTable(tableName) { + sql(s""" + |CREATE TABLE $tableName ( + | c1 STRING COLLATE UTF8_LCASE, + | c2 STRING COLLATE UNICODE + |) USING $dataSource + |""".stripMargin) + + sql(s"INSERT INTO $tableName VALUES ('a', 'A')") + + assertImplicitMismatch( + sql(s""" + |SELECT * FROM $tableName + |WHERE c1 = (SELECT MAX(c2) FROM $tableName) + |""".stripMargin)) + + checkAnswer( + sql(s""" + |SELECT COUNT(*) FROM $tableName + |WHERE c1 collate utf8_lcase = (SELECT MAX(c2) collate unicode FROM $tableName) + |""".stripMargin), + Seq(Row(1))) + + checkAnswer( + sql(s""" + |SELECT COUNT(*) FROM $tableName + |WHERE c1 collate utf8_lcase = (SELECT MAX(c2) FROM $tableName) + |""".stripMargin), + Seq(Row(1))) + + checkAnswer( + sql(s""" + |SELECT COUNT(*) FROM $tableName + |WHERE c1 collate unicode = (SELECT MAX(c2) FROM $tableName) + |""".stripMargin), + Seq(Row(0))) + } + } + test("struct test") { val tableName = "struct_tbl" val c1Collation = "UNICODE_CI" From c1a9fc1026de8ed884b1c2c1414bdbcf6c60c787 Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Fri, 13 Dec 2024 11:37:15 -0800 Subject: [PATCH 175/438] [SPARK-45891][SQL][TESTS] Make VariantSuite test agnostic of the stableDerivedColumnAlias config ### What changes were proposed in this pull request? Make the "group/order/join variant are disabled" test in VariantSuite agnostic of the `spark.sql.stableDerivedColumnAlias.enabled` config. ### Why are the changes needed? Earlier, the suite would fail when `spark.sql.stableDerivedColumnAlias.enabled` was true since the column alias would be different. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49135 from harshmotw-db/harsh-motwani_data/variant_test_fix. Authored-by: Harsh Motwani Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/VariantSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala index f196795989025..09b29b668b134 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala @@ -434,26 +434,26 @@ class VariantSuite extends QueryTest with SharedSparkSession with ExpressionEval checkError( exception = intercept[AnalysisException] { - spark.sql("select parse_json('') order by 1") + spark.sql("select parse_json('') v order by 1") }, condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`sortorder`", "dataType" -> "\"VARIANT\"", - "sqlExpr" -> "\"parse_json() ASC NULLS FIRST\""), - context = ExpectedContext(fragment = "order by 1", start = 22, stop = 31) + "sqlExpr" -> "\"v ASC NULLS FIRST\""), + context = ExpectedContext(fragment = "order by 1", start = 24, stop = 33) ) checkError( exception = intercept[AnalysisException] { - spark.sql("select parse_json('') sort by 1") + spark.sql("select parse_json('') v sort by 1") }, condition = "DATATYPE_MISMATCH.INVALID_ORDERING_TYPE", parameters = Map( "functionName" -> "`sortorder`", "dataType" -> "\"VARIANT\"", - "sqlExpr" -> "\"parse_json() ASC NULLS FIRST\""), - context = ExpectedContext(fragment = "sort by 1", start = 22, stop = 30) + "sqlExpr" -> "\"v ASC NULLS FIRST\""), + context = ExpectedContext(fragment = "sort by 1", start = 24, stop = 32) ) checkError( From 3e7b614a2da6e09cff26848ec644f495a0205750 Mon Sep 17 00:00:00 2001 From: Michael Zhang Date: Fri, 13 Dec 2024 11:45:45 -0800 Subject: [PATCH 176/438] [SPARK_50076] Fix logkeys ### What changes were proposed in this pull request? Fixes some incorrect logkeys and adds new ones. ### Why are the changes needed? Logkeys are needed for correct structured logging. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests pass. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49167 from michaelzhan-db/SPARK-50076. Authored-by: Michael Zhang Signed-off-by: Wenchen Fan --- .../src/main/scala/org/apache/spark/internal/LogKey.scala | 3 +++ .../spark/streaming/kinesis/KinesisRecordProcessor.scala | 4 ++-- .../scala/org/apache/spark/ExecutorAllocationManager.scala | 2 +- .../scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 2 +- .../sql/connect/planner/StreamingForeachBatchHelper.scala | 4 ++-- .../scala/org/apache/spark/streaming/dstream/DStream.scala | 2 +- 6 files changed, 10 insertions(+), 7 deletions(-) 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 740a56cc84ef8..9e681448aaaf2 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 @@ -350,6 +350,7 @@ private[spark] object LogKeys { case object KEYTAB extends LogKey case object KEYTAB_FILE extends LogKey case object KILL_EXECUTORS extends LogKey + case object KINESIS_REASON extends LogKey case object LABEL_COLUMN extends LogKey case object LARGEST_CLUSTER_INDEX extends LogKey case object LAST_ACCESS_TIME extends LogKey @@ -701,6 +702,7 @@ private[spark] object LogKeys { case object RULE_EXECUTOR_NAME extends LogKey case object RULE_NAME extends LogKey case object RUN_ID extends LogKey + case object RUN_ID_STRING extends LogKey case object SCALA_VERSION extends LogKey case object SCALING_DOWN_RATIO extends LogKey case object SCALING_UP_RATIO extends LogKey @@ -884,6 +886,7 @@ private[spark] object LogKeys { case object WRITE_JOB_UUID extends LogKey case object XML_SCHEDULING_MODE extends LogKey case object XSD_PATH extends LogKey + case object YARN_RESOURCE extends LogKey case object YOUNG_GENERATION_GC extends LogKey case object ZERO_TIME extends LogKey } diff --git a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala index aaafb3215d031..cd740f971e484 100644 --- a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala +++ b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -27,7 +27,7 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason import com.amazonaws.services.kinesis.model.Record import org.apache.spark.internal.{Logging, MDC} -import org.apache.spark.internal.LogKeys.{REASON, RETRY_INTERVAL, SHARD_ID, WORKER_URL} +import org.apache.spark.internal.LogKeys.{KINESIS_REASON, RETRY_INTERVAL, SHARD_ID, WORKER_URL} /** * Kinesis-specific implementation of the Kinesis Client Library (KCL) IRecordProcessor. @@ -119,7 +119,7 @@ private[kinesis] class KinesisRecordProcessor[T](receiver: KinesisReceiver[T], w checkpointer: IRecordProcessorCheckpointer, reason: ShutdownReason): Unit = { logInfo(log"Shutdown: Shutting down workerId ${MDC(WORKER_URL, workerId)} " + - log"with reason ${MDC(REASON, reason)}") + log"with reason ${MDC(KINESIS_REASON, reason)}") // null if not initialized before shutdown: if (shardId == null) { logWarning(log"No shardId for workerId ${MDC(WORKER_URL, workerId)}?") diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 1fe02eec3a072..dd131e443135f 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -885,7 +885,7 @@ private[spark] class ExecutorAllocationManager( } else { logWarning(log"Should have exactly one resource profile for stage " + log"${MDC(STAGE_ATTEMPT, stageAttempt)}, but have " + - log"${MDC(RESOURCE_PROFILE_ID, rpForStage)}") + log"${MDC(RESOURCE_PROFILE_IDS, rpForStage)}") } } } 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 c86195d0ef31e..b593d73f85721 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 @@ -529,7 +529,7 @@ private[yarn] class YarnAllocator( log"${MDC(LogKeys.MEMORY_SIZE, resource.getMemorySize)} MB memory." if (resource.getResources.nonEmpty) { requestContainerMessage = requestContainerMessage + - log" with custom resources: ${MDC(LogKeys.RESOURCE, resource)}" + log" with custom resources: ${MDC(LogKeys.YARN_RESOURCE, resource)}" } logInfo(requestContainerMessage) } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala index df883a5c86814..b6f67fe9f02f6 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import org.apache.spark.SparkException import org.apache.spark.api.python.{PythonException, PythonWorkerUtils, SimplePythonFunction, SpecialLengths, StreamingPythonRunner} import org.apache.spark.internal.{Logging, MDC} -import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, QUERY_ID, RUN_ID, SESSION_ID} +import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, QUERY_ID, RUN_ID_STRING, SESSION_ID} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.connect.service.SessionHolder import org.apache.spark.sql.connect.service.SparkConnectService @@ -203,7 +203,7 @@ object StreamingForeachBatchHelper extends Logging { Option(cleanerCache.remove(key)).foreach { cleaner => logInfo( log"Cleaning up runner for queryId ${MDC(QUERY_ID, key.queryId)} " + - log"runId ${MDC(RUN_ID, key.runId)}.") + log"runId ${MDC(RUN_ID_STRING, key.runId)}.") cleaner.close() } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 87d6a4909fdd4..f0f9046c6b623 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -467,7 +467,7 @@ abstract class DStream[T: ClassTag] ( // Explicitly remove blocks of BlockRDD rdd match { case b: BlockRDD[_] => - logInfo(log"Removing blocks of RDD ${MDC(LogKeys.RDD_ID, b)} " + + logInfo(log"Removing blocks of RDD ${MDC(LogKeys.RDD, b)} " + log"of time ${MDC(LogKeys.TIME, time)}") b.removeBlocks() case _ => From 5538d8536e9d7fd027c7724463ff856081702599 Mon Sep 17 00:00:00 2001 From: bogao007 Date: Sat, 14 Dec 2024 10:10:58 +0900 Subject: [PATCH 177/438] [SPARK-50540][PYTHON][SS] Fix string schema for StatefulProcessorHandle ### What changes were proposed in this pull request? Fix string schema for StatefulProcessorHandle, it was throwing an error before when passing the schema as String type because the utility method we used `_parse_datatype_string` requires a SparkContext which is not available on executors. The way we support it is to create a new API `ParseStringSchema` from the client side (Python worker) to server side (JVM). Client passes a string schema to the server, we do the parsing on server side and then send the result back to the client. ### Why are the changes needed? This is to fix an issue/bug with the existing code. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Updated existing test cases to include string schemas. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49138 from bogao007/parse-schema. Authored-by: bogao007 Signed-off-by: Jungtaek Lim --- .../sql/streaming/list_state_client.py | 34 ++-- .../pyspark/sql/streaming/map_state_client.py | 12 +- .../sql/streaming/proto/StateMessage_pb2.py | 162 +++++++++--------- .../sql/streaming/proto/StateMessage_pb2.pyi | 77 +++++++++ .../sql/streaming/stateful_processor.py | 22 +-- .../stateful_processor_api_client.py | 39 ++++- .../sql/streaming/value_state_client.py | 20 ++- .../test_pandas_transform_with_state.py | 8 +- .../execution/streaming/StateMessage.proto | 17 ++ ...ransformWithStateInPandasStateServer.scala | 32 +++- ...ormWithStateInPandasStateServerSuite.scala | 12 +- 11 files changed, 303 insertions(+), 132 deletions(-) diff --git a/python/pyspark/sql/streaming/list_state_client.py b/python/pyspark/sql/streaming/list_state_client.py index d2152842819a5..cb618d1a691b3 100644 --- a/python/pyspark/sql/streaming/list_state_client.py +++ b/python/pyspark/sql/streaming/list_state_client.py @@ -14,10 +14,10 @@ # See the License for the specific language governing permissions and # limitations under the License. # -from typing import Dict, Iterator, List, Union, cast, Tuple +from typing import Dict, Iterator, List, Union, Tuple from pyspark.sql.streaming.stateful_processor_api_client import StatefulProcessorApiClient -from pyspark.sql.types import StructType, TYPE_CHECKING, _parse_datatype_string +from pyspark.sql.types import StructType, TYPE_CHECKING from pyspark.errors import PySparkRuntimeError import uuid @@ -28,8 +28,16 @@ class ListStateClient: - def __init__(self, stateful_processor_api_client: StatefulProcessorApiClient) -> None: + def __init__( + self, + stateful_processor_api_client: StatefulProcessorApiClient, + schema: Union[StructType, str], + ) -> None: self._stateful_processor_api_client = stateful_processor_api_client + if isinstance(schema, str): + self.schema = self._stateful_processor_api_client._parse_string_schema(schema) + else: + self.schema = schema # A dictionary to store the mapping between list state name and a tuple of pandas DataFrame # and the index of the last row that was read. self.pandas_df_dict: Dict[str, Tuple["PandasDataFrameLike", int]] = {} @@ -105,12 +113,10 @@ def get(self, state_name: str, iterator_id: str) -> Tuple: pandas_row = pandas_df.iloc[index] return tuple(pandas_row) - def append_value(self, state_name: str, schema: Union[StructType, str], value: Tuple) -> None: + def append_value(self, state_name: str, value: Tuple) -> None: import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage - if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) - bytes = self._stateful_processor_api_client._serialize_to_bytes(schema, value) + bytes = self._stateful_processor_api_client._serialize_to_bytes(self.schema, value) append_value_call = stateMessage.AppendValue(value=bytes) list_state_call = stateMessage.ListStateCall( stateName=state_name, appendValue=append_value_call @@ -125,13 +131,9 @@ def append_value(self, state_name: str, schema: Union[StructType, str], value: T # TODO(SPARK-49233): Classify user facing errors. raise PySparkRuntimeError(f"Error updating value state: " f"{response_message[1]}") - def append_list( - self, state_name: str, schema: Union[StructType, str], values: List[Tuple] - ) -> None: + def append_list(self, state_name: str, values: List[Tuple]) -> None: import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage - if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) append_list_call = stateMessage.AppendList() list_state_call = stateMessage.ListStateCall( stateName=state_name, appendList=append_list_call @@ -141,18 +143,16 @@ def append_list( self._stateful_processor_api_client._send_proto_message(message.SerializeToString()) - self._stateful_processor_api_client._send_arrow_state(schema, values) + self._stateful_processor_api_client._send_arrow_state(self.schema, values) response_message = self._stateful_processor_api_client._receive_proto_message() status = response_message[0] if status != 0: # TODO(SPARK-49233): Classify user facing errors. raise PySparkRuntimeError(f"Error updating value state: " f"{response_message[1]}") - def put(self, state_name: str, schema: Union[StructType, str], values: List[Tuple]) -> None: + def put(self, state_name: str, values: List[Tuple]) -> None: import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage - if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) put_call = stateMessage.ListStatePut() list_state_call = stateMessage.ListStateCall(stateName=state_name, listStatePut=put_call) state_variable_request = stateMessage.StateVariableRequest(listStateCall=list_state_call) @@ -160,7 +160,7 @@ def put(self, state_name: str, schema: Union[StructType, str], values: List[Tupl self._stateful_processor_api_client._send_proto_message(message.SerializeToString()) - self._stateful_processor_api_client._send_arrow_state(schema, values) + self._stateful_processor_api_client._send_arrow_state(self.schema, values) response_message = self._stateful_processor_api_client._receive_proto_message() status = response_message[0] if status != 0: diff --git a/python/pyspark/sql/streaming/map_state_client.py b/python/pyspark/sql/streaming/map_state_client.py index 6ec7448b48634..c4761ddd48a16 100644 --- a/python/pyspark/sql/streaming/map_state_client.py +++ b/python/pyspark/sql/streaming/map_state_client.py @@ -14,10 +14,10 @@ # See the License for the specific language governing permissions and # limitations under the License. # -from typing import Dict, Iterator, Union, cast, Tuple, Optional +from typing import Dict, Iterator, Union, Tuple, Optional from pyspark.sql.streaming.stateful_processor_api_client import StatefulProcessorApiClient -from pyspark.sql.types import StructType, TYPE_CHECKING, _parse_datatype_string +from pyspark.sql.types import StructType, TYPE_CHECKING from pyspark.errors import PySparkRuntimeError import uuid @@ -36,11 +36,15 @@ def __init__( ) -> None: self._stateful_processor_api_client = stateful_processor_api_client if isinstance(user_key_schema, str): - self.user_key_schema = cast(StructType, _parse_datatype_string(user_key_schema)) + self.user_key_schema = self._stateful_processor_api_client._parse_string_schema( + user_key_schema + ) else: self.user_key_schema = user_key_schema if isinstance(value_schema, str): - self.value_schema = cast(StructType, _parse_datatype_string(value_schema)) + self.value_schema = self._stateful_processor_api_client._parse_string_schema( + value_schema + ) else: self.value_schema = value_schema # Dictionaries to store the mapping between iterator id and a tuple of pandas DataFrame diff --git a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py index 0a54690513a39..bcd8e0fc68f5c 100644 --- a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py +++ b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py @@ -40,7 +40,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n;org/apache/spark/sql/execution/streaming/StateMessage.proto\x12.org.apache.spark.sql.execution.streaming.state"\xa0\x04\n\x0cStateRequest\x12\x18\n\x07version\x18\x01 \x01(\x05R\x07version\x12}\n\x15statefulProcessorCall\x18\x02 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.StatefulProcessorCallH\x00R\x15statefulProcessorCall\x12z\n\x14stateVariableRequest\x18\x03 \x01(\x0b\x32\x44.org.apache.spark.sql.execution.streaming.state.StateVariableRequestH\x00R\x14stateVariableRequest\x12\x8c\x01\n\x1aimplicitGroupingKeyRequest\x18\x04 \x01(\x0b\x32J.org.apache.spark.sql.execution.streaming.state.ImplicitGroupingKeyRequestH\x00R\x1aimplicitGroupingKeyRequest\x12\x62\n\x0ctimerRequest\x18\x05 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.TimerRequestH\x00R\x0ctimerRequestB\x08\n\x06method"i\n\rStateResponse\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x0cR\x05value"x\n\x1cStateResponseWithLongTypeVal\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x03R\x05value"\xa0\x05\n\x15StatefulProcessorCall\x12h\n\x0esetHandleState\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetHandleStateH\x00R\x0esetHandleState\x12h\n\rgetValueState\x18\x02 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\rgetValueState\x12\x66\n\x0cgetListState\x18\x03 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0cgetListState\x12\x64\n\x0bgetMapState\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0bgetMapState\x12o\n\x0etimerStateCall\x18\x05 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.TimerStateCallCommandH\x00R\x0etimerStateCall\x12j\n\x0e\x64\x65leteIfExists\x18\x06 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0e\x64\x65leteIfExistsB\x08\n\x06method"\xd5\x02\n\x14StateVariableRequest\x12h\n\x0evalueStateCall\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.ValueStateCallH\x00R\x0evalueStateCall\x12\x65\n\rlistStateCall\x18\x02 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.ListStateCallH\x00R\rlistStateCall\x12\x62\n\x0cmapStateCall\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.MapStateCallH\x00R\x0cmapStateCallB\x08\n\x06method"\x83\x02\n\x1aImplicitGroupingKeyRequest\x12h\n\x0esetImplicitKey\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetImplicitKeyH\x00R\x0esetImplicitKey\x12q\n\x11removeImplicitKey\x18\x02 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.RemoveImplicitKeyH\x00R\x11removeImplicitKeyB\x08\n\x06method"\x81\x02\n\x0cTimerRequest\x12q\n\x11timerValueRequest\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.TimerValueRequestH\x00R\x11timerValueRequest\x12t\n\x12\x65xpiryTimerRequest\x18\x02 \x01(\x0b\x32\x42.org.apache.spark.sql.execution.streaming.state.ExpiryTimerRequestH\x00R\x12\x65xpiryTimerRequestB\x08\n\x06method"\xf6\x01\n\x11TimerValueRequest\x12s\n\x12getProcessingTimer\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.GetProcessingTimeH\x00R\x12getProcessingTimer\x12\x62\n\x0cgetWatermark\x18\x02 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.GetWatermarkH\x00R\x0cgetWatermarkB\x08\n\x06method"B\n\x12\x45xpiryTimerRequest\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs"\x13\n\x11GetProcessingTime"\x0e\n\x0cGetWatermark"\xc7\x01\n\x10StateCallCommand\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12\x16\n\x06schema\x18\x02 \x01(\tR\x06schema\x12\x30\n\x13mapStateValueSchema\x18\x03 \x01(\tR\x13mapStateValueSchema\x12K\n\x03ttl\x18\x04 \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.TTLConfigR\x03ttl"\xa7\x02\n\x15TimerStateCallCommand\x12[\n\x08register\x18\x01 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.RegisterTimerH\x00R\x08register\x12U\n\x06\x64\x65lete\x18\x02 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.DeleteTimerH\x00R\x06\x64\x65lete\x12P\n\x04list\x18\x03 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.ListTimersH\x00R\x04listB\x08\n\x06method"\x92\x03\n\x0eValueStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12G\n\x03get\x18\x03 \x01(\x0b\x32\x33.org.apache.spark.sql.execution.streaming.state.GetH\x00R\x03get\x12n\n\x10valueStateUpdate\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.ValueStateUpdateH\x00R\x10valueStateUpdate\x12M\n\x05\x63lear\x18\x05 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xdf\x04\n\rListStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12\x62\n\x0clistStateGet\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStateGetH\x00R\x0clistStateGet\x12\x62\n\x0clistStatePut\x18\x04 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStatePutH\x00R\x0clistStatePut\x12_\n\x0b\x61ppendValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.AppendValueH\x00R\x0b\x61ppendValue\x12\\\n\nappendList\x18\x06 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.AppendListH\x00R\nappendList\x12M\n\x05\x63lear\x18\x07 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xc2\x06\n\x0cMapStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12V\n\x08getValue\x18\x03 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.GetValueH\x00R\x08getValue\x12_\n\x0b\x63ontainsKey\x18\x04 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.ContainsKeyH\x00R\x0b\x63ontainsKey\x12_\n\x0bupdateValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.UpdateValueH\x00R\x0bupdateValue\x12V\n\x08iterator\x18\x06 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.IteratorH\x00R\x08iterator\x12J\n\x04keys\x18\x07 \x01(\x0b\x32\x34.org.apache.spark.sql.execution.streaming.state.KeysH\x00R\x04keys\x12P\n\x06values\x18\x08 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ValuesH\x00R\x06values\x12Y\n\tremoveKey\x18\t \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.RemoveKeyH\x00R\tremoveKey\x12M\n\x05\x63lear\x18\n \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method""\n\x0eSetImplicitKey\x12\x10\n\x03key\x18\x01 \x01(\x0cR\x03key"\x13\n\x11RemoveImplicitKey"\x08\n\x06\x45xists"\x05\n\x03Get"=\n\rRegisterTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs";\n\x0b\x44\x65leteTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs",\n\nListTimers\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x10ValueStateUpdate\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x07\n\x05\x43lear".\n\x0cListStateGet\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"\x0e\n\x0cListStatePut"#\n\x0b\x41ppendValue\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x0c\n\nAppendList"$\n\x08GetValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"\'\n\x0b\x43ontainsKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"=\n\x0bUpdateValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey\x12\x14\n\x05value\x18\x02 \x01(\x0cR\x05value"*\n\x08Iterator\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"&\n\x04Keys\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x06Values\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"%\n\tRemoveKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"c\n\x0eSetHandleState\x12Q\n\x05state\x18\x01 \x01(\x0e\x32;.org.apache.spark.sql.execution.streaming.state.HandleStateR\x05state"+\n\tTTLConfig\x12\x1e\n\ndurationMs\x18\x01 \x01(\x05R\ndurationMs*`\n\x0bHandleState\x12\x0b\n\x07\x43REATED\x10\x00\x12\x0f\n\x0bINITIALIZED\x10\x01\x12\x12\n\x0e\x44\x41TA_PROCESSED\x10\x02\x12\x13\n\x0fTIMER_PROCESSED\x10\x03\x12\n\n\x06\x43LOSED\x10\x04\x62\x06proto3' + b'\n;org/apache/spark/sql/execution/streaming/StateMessage.proto\x12.org.apache.spark.sql.execution.streaming.state"\x84\x05\n\x0cStateRequest\x12\x18\n\x07version\x18\x01 \x01(\x05R\x07version\x12}\n\x15statefulProcessorCall\x18\x02 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.StatefulProcessorCallH\x00R\x15statefulProcessorCall\x12z\n\x14stateVariableRequest\x18\x03 \x01(\x0b\x32\x44.org.apache.spark.sql.execution.streaming.state.StateVariableRequestH\x00R\x14stateVariableRequest\x12\x8c\x01\n\x1aimplicitGroupingKeyRequest\x18\x04 \x01(\x0b\x32J.org.apache.spark.sql.execution.streaming.state.ImplicitGroupingKeyRequestH\x00R\x1aimplicitGroupingKeyRequest\x12\x62\n\x0ctimerRequest\x18\x05 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.TimerRequestH\x00R\x0ctimerRequest\x12\x62\n\x0cutilsRequest\x18\x06 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.UtilsRequestH\x00R\x0cutilsRequestB\x08\n\x06method"i\n\rStateResponse\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x0cR\x05value"x\n\x1cStateResponseWithLongTypeVal\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x03R\x05value"z\n\x1eStateResponseWithStringTypeVal\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\tR\x05value"\xa0\x05\n\x15StatefulProcessorCall\x12h\n\x0esetHandleState\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetHandleStateH\x00R\x0esetHandleState\x12h\n\rgetValueState\x18\x02 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\rgetValueState\x12\x66\n\x0cgetListState\x18\x03 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0cgetListState\x12\x64\n\x0bgetMapState\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0bgetMapState\x12o\n\x0etimerStateCall\x18\x05 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.TimerStateCallCommandH\x00R\x0etimerStateCall\x12j\n\x0e\x64\x65leteIfExists\x18\x06 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0e\x64\x65leteIfExistsB\x08\n\x06method"\xd5\x02\n\x14StateVariableRequest\x12h\n\x0evalueStateCall\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.ValueStateCallH\x00R\x0evalueStateCall\x12\x65\n\rlistStateCall\x18\x02 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.ListStateCallH\x00R\rlistStateCall\x12\x62\n\x0cmapStateCall\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.MapStateCallH\x00R\x0cmapStateCallB\x08\n\x06method"\x83\x02\n\x1aImplicitGroupingKeyRequest\x12h\n\x0esetImplicitKey\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetImplicitKeyH\x00R\x0esetImplicitKey\x12q\n\x11removeImplicitKey\x18\x02 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.RemoveImplicitKeyH\x00R\x11removeImplicitKeyB\x08\n\x06method"\x81\x02\n\x0cTimerRequest\x12q\n\x11timerValueRequest\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.TimerValueRequestH\x00R\x11timerValueRequest\x12t\n\x12\x65xpiryTimerRequest\x18\x02 \x01(\x0b\x32\x42.org.apache.spark.sql.execution.streaming.state.ExpiryTimerRequestH\x00R\x12\x65xpiryTimerRequestB\x08\n\x06method"\xf6\x01\n\x11TimerValueRequest\x12s\n\x12getProcessingTimer\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.GetProcessingTimeH\x00R\x12getProcessingTimer\x12\x62\n\x0cgetWatermark\x18\x02 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.GetWatermarkH\x00R\x0cgetWatermarkB\x08\n\x06method"B\n\x12\x45xpiryTimerRequest\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs"\x13\n\x11GetProcessingTime"\x0e\n\x0cGetWatermark"\x8b\x01\n\x0cUtilsRequest\x12q\n\x11parseStringSchema\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.ParseStringSchemaH\x00R\x11parseStringSchemaB\x08\n\x06method"+\n\x11ParseStringSchema\x12\x16\n\x06schema\x18\x01 \x01(\tR\x06schema"\xc7\x01\n\x10StateCallCommand\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12\x16\n\x06schema\x18\x02 \x01(\tR\x06schema\x12\x30\n\x13mapStateValueSchema\x18\x03 \x01(\tR\x13mapStateValueSchema\x12K\n\x03ttl\x18\x04 \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.TTLConfigR\x03ttl"\xa7\x02\n\x15TimerStateCallCommand\x12[\n\x08register\x18\x01 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.RegisterTimerH\x00R\x08register\x12U\n\x06\x64\x65lete\x18\x02 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.DeleteTimerH\x00R\x06\x64\x65lete\x12P\n\x04list\x18\x03 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.ListTimersH\x00R\x04listB\x08\n\x06method"\x92\x03\n\x0eValueStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12G\n\x03get\x18\x03 \x01(\x0b\x32\x33.org.apache.spark.sql.execution.streaming.state.GetH\x00R\x03get\x12n\n\x10valueStateUpdate\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.ValueStateUpdateH\x00R\x10valueStateUpdate\x12M\n\x05\x63lear\x18\x05 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xdf\x04\n\rListStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12\x62\n\x0clistStateGet\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStateGetH\x00R\x0clistStateGet\x12\x62\n\x0clistStatePut\x18\x04 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStatePutH\x00R\x0clistStatePut\x12_\n\x0b\x61ppendValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.AppendValueH\x00R\x0b\x61ppendValue\x12\\\n\nappendList\x18\x06 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.AppendListH\x00R\nappendList\x12M\n\x05\x63lear\x18\x07 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xc2\x06\n\x0cMapStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12V\n\x08getValue\x18\x03 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.GetValueH\x00R\x08getValue\x12_\n\x0b\x63ontainsKey\x18\x04 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.ContainsKeyH\x00R\x0b\x63ontainsKey\x12_\n\x0bupdateValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.UpdateValueH\x00R\x0bupdateValue\x12V\n\x08iterator\x18\x06 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.IteratorH\x00R\x08iterator\x12J\n\x04keys\x18\x07 \x01(\x0b\x32\x34.org.apache.spark.sql.execution.streaming.state.KeysH\x00R\x04keys\x12P\n\x06values\x18\x08 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ValuesH\x00R\x06values\x12Y\n\tremoveKey\x18\t \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.RemoveKeyH\x00R\tremoveKey\x12M\n\x05\x63lear\x18\n \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method""\n\x0eSetImplicitKey\x12\x10\n\x03key\x18\x01 \x01(\x0cR\x03key"\x13\n\x11RemoveImplicitKey"\x08\n\x06\x45xists"\x05\n\x03Get"=\n\rRegisterTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs";\n\x0b\x44\x65leteTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs",\n\nListTimers\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x10ValueStateUpdate\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x07\n\x05\x43lear".\n\x0cListStateGet\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"\x0e\n\x0cListStatePut"#\n\x0b\x41ppendValue\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x0c\n\nAppendList"$\n\x08GetValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"\'\n\x0b\x43ontainsKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"=\n\x0bUpdateValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey\x12\x14\n\x05value\x18\x02 \x01(\x0cR\x05value"*\n\x08Iterator\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"&\n\x04Keys\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x06Values\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"%\n\tRemoveKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"c\n\x0eSetHandleState\x12Q\n\x05state\x18\x01 \x01(\x0e\x32;.org.apache.spark.sql.execution.streaming.state.HandleStateR\x05state"+\n\tTTLConfig\x12\x1e\n\ndurationMs\x18\x01 \x01(\x05R\ndurationMs*`\n\x0bHandleState\x12\x0b\n\x07\x43REATED\x10\x00\x12\x0f\n\x0bINITIALIZED\x10\x01\x12\x12\n\x0e\x44\x41TA_PROCESSED\x10\x02\x12\x13\n\x0fTIMER_PROCESSED\x10\x03\x12\n\n\x06\x43LOSED\x10\x04\x62\x06proto3' ) _globals = globals() @@ -50,82 +50,88 @@ ) if not _descriptor._USE_C_DESCRIPTORS: DESCRIPTOR._loaded_options = None - _globals["_HANDLESTATE"]._serialized_start = 5997 - _globals["_HANDLESTATE"]._serialized_end = 6093 + _globals["_HANDLESTATE"]._serialized_start = 6408 + _globals["_HANDLESTATE"]._serialized_end = 6504 _globals["_STATEREQUEST"]._serialized_start = 112 - _globals["_STATEREQUEST"]._serialized_end = 656 - _globals["_STATERESPONSE"]._serialized_start = 658 - _globals["_STATERESPONSE"]._serialized_end = 763 - _globals["_STATERESPONSEWITHLONGTYPEVAL"]._serialized_start = 765 - _globals["_STATERESPONSEWITHLONGTYPEVAL"]._serialized_end = 885 - _globals["_STATEFULPROCESSORCALL"]._serialized_start = 888 - _globals["_STATEFULPROCESSORCALL"]._serialized_end = 1560 - _globals["_STATEVARIABLEREQUEST"]._serialized_start = 1563 - _globals["_STATEVARIABLEREQUEST"]._serialized_end = 1904 - _globals["_IMPLICITGROUPINGKEYREQUEST"]._serialized_start = 1907 - _globals["_IMPLICITGROUPINGKEYREQUEST"]._serialized_end = 2166 - _globals["_TIMERREQUEST"]._serialized_start = 2169 - _globals["_TIMERREQUEST"]._serialized_end = 2426 - _globals["_TIMERVALUEREQUEST"]._serialized_start = 2429 - _globals["_TIMERVALUEREQUEST"]._serialized_end = 2675 - _globals["_EXPIRYTIMERREQUEST"]._serialized_start = 2677 - _globals["_EXPIRYTIMERREQUEST"]._serialized_end = 2743 - _globals["_GETPROCESSINGTIME"]._serialized_start = 2745 - _globals["_GETPROCESSINGTIME"]._serialized_end = 2764 - _globals["_GETWATERMARK"]._serialized_start = 2766 - _globals["_GETWATERMARK"]._serialized_end = 2780 - _globals["_STATECALLCOMMAND"]._serialized_start = 2783 - _globals["_STATECALLCOMMAND"]._serialized_end = 2982 - _globals["_TIMERSTATECALLCOMMAND"]._serialized_start = 2985 - _globals["_TIMERSTATECALLCOMMAND"]._serialized_end = 3280 - _globals["_VALUESTATECALL"]._serialized_start = 3283 - _globals["_VALUESTATECALL"]._serialized_end = 3685 - _globals["_LISTSTATECALL"]._serialized_start = 3688 - _globals["_LISTSTATECALL"]._serialized_end = 4295 - _globals["_MAPSTATECALL"]._serialized_start = 4298 - _globals["_MAPSTATECALL"]._serialized_end = 5132 - _globals["_SETIMPLICITKEY"]._serialized_start = 5134 - _globals["_SETIMPLICITKEY"]._serialized_end = 5168 - _globals["_REMOVEIMPLICITKEY"]._serialized_start = 5170 - _globals["_REMOVEIMPLICITKEY"]._serialized_end = 5189 - _globals["_EXISTS"]._serialized_start = 5191 - _globals["_EXISTS"]._serialized_end = 5199 - _globals["_GET"]._serialized_start = 5201 - _globals["_GET"]._serialized_end = 5206 - _globals["_REGISTERTIMER"]._serialized_start = 5208 - _globals["_REGISTERTIMER"]._serialized_end = 5269 - _globals["_DELETETIMER"]._serialized_start = 5271 - _globals["_DELETETIMER"]._serialized_end = 5330 - _globals["_LISTTIMERS"]._serialized_start = 5332 - _globals["_LISTTIMERS"]._serialized_end = 5376 - _globals["_VALUESTATEUPDATE"]._serialized_start = 5378 - _globals["_VALUESTATEUPDATE"]._serialized_end = 5418 - _globals["_CLEAR"]._serialized_start = 5420 - _globals["_CLEAR"]._serialized_end = 5427 - _globals["_LISTSTATEGET"]._serialized_start = 5429 - _globals["_LISTSTATEGET"]._serialized_end = 5475 - _globals["_LISTSTATEPUT"]._serialized_start = 5477 - _globals["_LISTSTATEPUT"]._serialized_end = 5491 - _globals["_APPENDVALUE"]._serialized_start = 5493 - _globals["_APPENDVALUE"]._serialized_end = 5528 - _globals["_APPENDLIST"]._serialized_start = 5530 - _globals["_APPENDLIST"]._serialized_end = 5542 - _globals["_GETVALUE"]._serialized_start = 5544 - _globals["_GETVALUE"]._serialized_end = 5580 - _globals["_CONTAINSKEY"]._serialized_start = 5582 - _globals["_CONTAINSKEY"]._serialized_end = 5621 - _globals["_UPDATEVALUE"]._serialized_start = 5623 - _globals["_UPDATEVALUE"]._serialized_end = 5684 - _globals["_ITERATOR"]._serialized_start = 5686 - _globals["_ITERATOR"]._serialized_end = 5728 - _globals["_KEYS"]._serialized_start = 5730 - _globals["_KEYS"]._serialized_end = 5768 - _globals["_VALUES"]._serialized_start = 5770 - _globals["_VALUES"]._serialized_end = 5810 - _globals["_REMOVEKEY"]._serialized_start = 5812 - _globals["_REMOVEKEY"]._serialized_end = 5849 - _globals["_SETHANDLESTATE"]._serialized_start = 5851 - _globals["_SETHANDLESTATE"]._serialized_end = 5950 - _globals["_TTLCONFIG"]._serialized_start = 5952 - _globals["_TTLCONFIG"]._serialized_end = 5995 + _globals["_STATEREQUEST"]._serialized_end = 756 + _globals["_STATERESPONSE"]._serialized_start = 758 + _globals["_STATERESPONSE"]._serialized_end = 863 + _globals["_STATERESPONSEWITHLONGTYPEVAL"]._serialized_start = 865 + _globals["_STATERESPONSEWITHLONGTYPEVAL"]._serialized_end = 985 + _globals["_STATERESPONSEWITHSTRINGTYPEVAL"]._serialized_start = 987 + _globals["_STATERESPONSEWITHSTRINGTYPEVAL"]._serialized_end = 1109 + _globals["_STATEFULPROCESSORCALL"]._serialized_start = 1112 + _globals["_STATEFULPROCESSORCALL"]._serialized_end = 1784 + _globals["_STATEVARIABLEREQUEST"]._serialized_start = 1787 + _globals["_STATEVARIABLEREQUEST"]._serialized_end = 2128 + _globals["_IMPLICITGROUPINGKEYREQUEST"]._serialized_start = 2131 + _globals["_IMPLICITGROUPINGKEYREQUEST"]._serialized_end = 2390 + _globals["_TIMERREQUEST"]._serialized_start = 2393 + _globals["_TIMERREQUEST"]._serialized_end = 2650 + _globals["_TIMERVALUEREQUEST"]._serialized_start = 2653 + _globals["_TIMERVALUEREQUEST"]._serialized_end = 2899 + _globals["_EXPIRYTIMERREQUEST"]._serialized_start = 2901 + _globals["_EXPIRYTIMERREQUEST"]._serialized_end = 2967 + _globals["_GETPROCESSINGTIME"]._serialized_start = 2969 + _globals["_GETPROCESSINGTIME"]._serialized_end = 2988 + _globals["_GETWATERMARK"]._serialized_start = 2990 + _globals["_GETWATERMARK"]._serialized_end = 3004 + _globals["_UTILSREQUEST"]._serialized_start = 3007 + _globals["_UTILSREQUEST"]._serialized_end = 3146 + _globals["_PARSESTRINGSCHEMA"]._serialized_start = 3148 + _globals["_PARSESTRINGSCHEMA"]._serialized_end = 3191 + _globals["_STATECALLCOMMAND"]._serialized_start = 3194 + _globals["_STATECALLCOMMAND"]._serialized_end = 3393 + _globals["_TIMERSTATECALLCOMMAND"]._serialized_start = 3396 + _globals["_TIMERSTATECALLCOMMAND"]._serialized_end = 3691 + _globals["_VALUESTATECALL"]._serialized_start = 3694 + _globals["_VALUESTATECALL"]._serialized_end = 4096 + _globals["_LISTSTATECALL"]._serialized_start = 4099 + _globals["_LISTSTATECALL"]._serialized_end = 4706 + _globals["_MAPSTATECALL"]._serialized_start = 4709 + _globals["_MAPSTATECALL"]._serialized_end = 5543 + _globals["_SETIMPLICITKEY"]._serialized_start = 5545 + _globals["_SETIMPLICITKEY"]._serialized_end = 5579 + _globals["_REMOVEIMPLICITKEY"]._serialized_start = 5581 + _globals["_REMOVEIMPLICITKEY"]._serialized_end = 5600 + _globals["_EXISTS"]._serialized_start = 5602 + _globals["_EXISTS"]._serialized_end = 5610 + _globals["_GET"]._serialized_start = 5612 + _globals["_GET"]._serialized_end = 5617 + _globals["_REGISTERTIMER"]._serialized_start = 5619 + _globals["_REGISTERTIMER"]._serialized_end = 5680 + _globals["_DELETETIMER"]._serialized_start = 5682 + _globals["_DELETETIMER"]._serialized_end = 5741 + _globals["_LISTTIMERS"]._serialized_start = 5743 + _globals["_LISTTIMERS"]._serialized_end = 5787 + _globals["_VALUESTATEUPDATE"]._serialized_start = 5789 + _globals["_VALUESTATEUPDATE"]._serialized_end = 5829 + _globals["_CLEAR"]._serialized_start = 5831 + _globals["_CLEAR"]._serialized_end = 5838 + _globals["_LISTSTATEGET"]._serialized_start = 5840 + _globals["_LISTSTATEGET"]._serialized_end = 5886 + _globals["_LISTSTATEPUT"]._serialized_start = 5888 + _globals["_LISTSTATEPUT"]._serialized_end = 5902 + _globals["_APPENDVALUE"]._serialized_start = 5904 + _globals["_APPENDVALUE"]._serialized_end = 5939 + _globals["_APPENDLIST"]._serialized_start = 5941 + _globals["_APPENDLIST"]._serialized_end = 5953 + _globals["_GETVALUE"]._serialized_start = 5955 + _globals["_GETVALUE"]._serialized_end = 5991 + _globals["_CONTAINSKEY"]._serialized_start = 5993 + _globals["_CONTAINSKEY"]._serialized_end = 6032 + _globals["_UPDATEVALUE"]._serialized_start = 6034 + _globals["_UPDATEVALUE"]._serialized_end = 6095 + _globals["_ITERATOR"]._serialized_start = 6097 + _globals["_ITERATOR"]._serialized_end = 6139 + _globals["_KEYS"]._serialized_start = 6141 + _globals["_KEYS"]._serialized_end = 6179 + _globals["_VALUES"]._serialized_start = 6181 + _globals["_VALUES"]._serialized_end = 6221 + _globals["_REMOVEKEY"]._serialized_start = 6223 + _globals["_REMOVEKEY"]._serialized_end = 6260 + _globals["_SETHANDLESTATE"]._serialized_start = 6262 + _globals["_SETHANDLESTATE"]._serialized_end = 6361 + _globals["_TTLCONFIG"]._serialized_start = 6363 + _globals["_TTLCONFIG"]._serialized_end = 6406 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi b/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi index 52f66928294cb..03ede5d25b2b7 100644 --- a/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi +++ b/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi @@ -79,6 +79,7 @@ class StateRequest(google.protobuf.message.Message): STATEVARIABLEREQUEST_FIELD_NUMBER: builtins.int IMPLICITGROUPINGKEYREQUEST_FIELD_NUMBER: builtins.int TIMERREQUEST_FIELD_NUMBER: builtins.int + UTILSREQUEST_FIELD_NUMBER: builtins.int version: builtins.int @property def statefulProcessorCall(self) -> global___StatefulProcessorCall: ... @@ -88,6 +89,8 @@ class StateRequest(google.protobuf.message.Message): def implicitGroupingKeyRequest(self) -> global___ImplicitGroupingKeyRequest: ... @property def timerRequest(self) -> global___TimerRequest: ... + @property + def utilsRequest(self) -> global___UtilsRequest: ... def __init__( self, *, @@ -96,6 +99,7 @@ class StateRequest(google.protobuf.message.Message): stateVariableRequest: global___StateVariableRequest | None = ..., implicitGroupingKeyRequest: global___ImplicitGroupingKeyRequest | None = ..., timerRequest: global___TimerRequest | None = ..., + utilsRequest: global___UtilsRequest | None = ..., ) -> None: ... def HasField( self, @@ -110,6 +114,8 @@ class StateRequest(google.protobuf.message.Message): b"statefulProcessorCall", "timerRequest", b"timerRequest", + "utilsRequest", + b"utilsRequest", ], ) -> builtins.bool: ... def ClearField( @@ -125,6 +131,8 @@ class StateRequest(google.protobuf.message.Message): b"statefulProcessorCall", "timerRequest", b"timerRequest", + "utilsRequest", + b"utilsRequest", "version", b"version", ], @@ -137,6 +145,7 @@ class StateRequest(google.protobuf.message.Message): "stateVariableRequest", "implicitGroupingKeyRequest", "timerRequest", + "utilsRequest", ] | None ): ... @@ -193,6 +202,31 @@ class StateResponseWithLongTypeVal(google.protobuf.message.Message): global___StateResponseWithLongTypeVal = StateResponseWithLongTypeVal +class StateResponseWithStringTypeVal(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + STATUSCODE_FIELD_NUMBER: builtins.int + ERRORMESSAGE_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + statusCode: builtins.int + errorMessage: builtins.str + value: builtins.str + def __init__( + self, + *, + statusCode: builtins.int = ..., + errorMessage: builtins.str = ..., + value: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "errorMessage", b"errorMessage", "statusCode", b"statusCode", "value", b"value" + ], + ) -> None: ... + +global___StateResponseWithStringTypeVal = StateResponseWithStringTypeVal + class StatefulProcessorCall(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -492,6 +526,49 @@ class GetWatermark(google.protobuf.message.Message): global___GetWatermark = GetWatermark +class UtilsRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PARSESTRINGSCHEMA_FIELD_NUMBER: builtins.int + @property + def parseStringSchema(self) -> global___ParseStringSchema: ... + def __init__( + self, + *, + parseStringSchema: global___ParseStringSchema | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "method", b"method", "parseStringSchema", b"parseStringSchema" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "method", b"method", "parseStringSchema", b"parseStringSchema" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["method", b"method"] + ) -> typing_extensions.Literal["parseStringSchema"] | None: ... + +global___UtilsRequest = UtilsRequest + +class ParseStringSchema(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SCHEMA_FIELD_NUMBER: builtins.int + schema: builtins.str + def __init__( + self, + *, + schema: builtins.str = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["schema", b"schema"]) -> None: ... + +global___ParseStringSchema = ParseStringSchema + class StateCallCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/streaming/stateful_processor.py b/python/pyspark/sql/streaming/stateful_processor.py index 9caa9304d6a87..b04bb955488ab 100644 --- a/python/pyspark/sql/streaming/stateful_processor.py +++ b/python/pyspark/sql/streaming/stateful_processor.py @@ -45,12 +45,9 @@ class ValueState: .. versionadded:: 4.0.0 """ - def __init__( - self, value_state_client: ValueStateClient, state_name: str, schema: Union[StructType, str] - ) -> None: + def __init__(self, value_state_client: ValueStateClient, state_name: str) -> None: self._value_state_client = value_state_client self._state_name = state_name - self.schema = schema def exists(self) -> bool: """ @@ -68,7 +65,7 @@ def update(self, new_value: Tuple) -> None: """ Update the value of the state. """ - self._value_state_client.update(self._state_name, self.schema, new_value) + self._value_state_client.update(self._state_name, new_value) def clear(self) -> None: """ @@ -127,12 +124,9 @@ class ListState: .. versionadded:: 4.0.0 """ - def __init__( - self, list_state_client: ListStateClient, state_name: str, schema: Union[StructType, str] - ) -> None: + def __init__(self, list_state_client: ListStateClient, state_name: str) -> None: self._list_state_client = list_state_client self._state_name = state_name - self.schema = schema def exists(self) -> bool: """ @@ -150,19 +144,19 @@ def put(self, new_state: List[Tuple]) -> None: """ Update the values of the list state. """ - self._list_state_client.put(self._state_name, self.schema, new_state) + self._list_state_client.put(self._state_name, new_state) def append_value(self, new_state: Tuple) -> None: """ Append a new value to the list state. """ - self._list_state_client.append_value(self._state_name, self.schema, new_state) + self._list_state_client.append_value(self._state_name, new_state) def append_list(self, new_state: List[Tuple]) -> None: """ Append a list of new values to the list state. """ - self._list_state_client.append_list(self._state_name, self.schema, new_state) + self._list_state_client.append_list(self._state_name, new_state) def clear(self) -> None: """ @@ -275,7 +269,7 @@ def getValueState( If ttl is not specified the state will never expire. """ self.stateful_processor_api_client.get_value_state(state_name, schema, ttl_duration_ms) - return ValueState(ValueStateClient(self.stateful_processor_api_client), state_name, schema) + return ValueState(ValueStateClient(self.stateful_processor_api_client, schema), state_name) def getListState( self, state_name: str, schema: Union[StructType, str], ttl_duration_ms: Optional[int] = None @@ -299,7 +293,7 @@ def getListState( If ttl is not specified the state will never expire. """ self.stateful_processor_api_client.get_list_state(state_name, schema, ttl_duration_ms) - return ListState(ListStateClient(self.stateful_processor_api_client), state_name, schema) + return ListState(ListStateClient(self.stateful_processor_api_client, schema), state_name) def getMapState( self, diff --git a/python/pyspark/sql/streaming/stateful_processor_api_client.py b/python/pyspark/sql/streaming/stateful_processor_api_client.py index 53704188081c3..79bb63d81d79f 100644 --- a/python/pyspark/sql/streaming/stateful_processor_api_client.py +++ b/python/pyspark/sql/streaming/stateful_processor_api_client.py @@ -15,16 +15,16 @@ # limitations under the License. # from enum import Enum +import json import os import socket -from typing import Any, Dict, List, Union, Optional, cast, Tuple, Iterator +from typing import Any, Dict, List, Union, Optional, Tuple, Iterator from pyspark.serializers import write_int, read_int, UTF8Deserializer from pyspark.sql.pandas.serializers import ArrowStreamSerializer from pyspark.sql.types import ( StructType, TYPE_CHECKING, - _parse_datatype_string, Row, ) from pyspark.sql.pandas.types import convert_pandas_using_numpy_type @@ -129,7 +129,7 @@ def get_value_state( import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) + schema = self._parse_string_schema(schema) state_call_command = stateMessage.StateCallCommand() state_call_command.stateName = state_name @@ -152,7 +152,7 @@ def get_list_state( import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) + schema = self._parse_string_schema(schema) state_call_command = stateMessage.StateCallCommand() state_call_command.stateName = state_name @@ -290,9 +290,9 @@ def get_map_state( import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage if isinstance(user_key_schema, str): - user_key_schema = cast(StructType, _parse_datatype_string(user_key_schema)) + user_key_schema = self._parse_string_schema(user_key_schema) if isinstance(value_schema, str): - value_schema = cast(StructType, _parse_datatype_string(value_schema)) + value_schema = self._parse_string_schema(value_schema) state_call_command = stateMessage.StateCallCommand() state_call_command.stateName = state_name @@ -393,6 +393,15 @@ def _receive_proto_message_with_long_value(self) -> Tuple[int, str, int]: message.ParseFromString(bytes) return message.statusCode, message.errorMessage, message.value + def _receive_proto_message_with_string_value(self) -> Tuple[int, str, str]: + import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage + + length = read_int(self.sockfile) + bytes = self.sockfile.read(length) + message = stateMessage.StateResponseWithStringTypeVal() + message.ParseFromString(bytes) + return message.statusCode, message.errorMessage, message.value + def _receive_str(self) -> str: return self.utf8_deserializer.loads(self.sockfile) @@ -436,6 +445,24 @@ def _send_arrow_state(self, schema: StructType, state: List[Tuple]) -> None: def _read_arrow_state(self) -> Any: return self.serializer.load_stream(self.sockfile) + # Parse a string schema into a StructType schema. This method will perform an API call to + # JVM side to parse the schema string. + def _parse_string_schema(self, schema: str) -> StructType: + import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage + + parse_string_schema_call = stateMessage.ParseStringSchema(schema=schema) + utils_request = stateMessage.UtilsRequest(parseStringSchema=parse_string_schema_call) + message = stateMessage.StateRequest(utilsRequest=utils_request) + + self._send_proto_message(message.SerializeToString()) + response_message = self._receive_proto_message_with_string_value() + status = response_message[0] + if status != 0: + # TODO(SPARK-49233): Classify user facing errors. + raise PySparkRuntimeError(f"Error parsing string schema: " f"{response_message[1]}") + else: + return StructType.fromJson(json.loads(response_message[2])) + class ListTimerIterator: def __init__(self, stateful_processor_api_client: StatefulProcessorApiClient): diff --git a/python/pyspark/sql/streaming/value_state_client.py b/python/pyspark/sql/streaming/value_state_client.py index fd783af7931da..532a89cf92d22 100644 --- a/python/pyspark/sql/streaming/value_state_client.py +++ b/python/pyspark/sql/streaming/value_state_client.py @@ -14,18 +14,26 @@ # See the License for the specific language governing permissions and # limitations under the License. # -from typing import Union, cast, Tuple, Optional +from typing import Union, Tuple, Optional from pyspark.sql.streaming.stateful_processor_api_client import StatefulProcessorApiClient -from pyspark.sql.types import StructType, _parse_datatype_string +from pyspark.sql.types import StructType from pyspark.errors import PySparkRuntimeError __all__ = ["ValueStateClient"] class ValueStateClient: - def __init__(self, stateful_processor_api_client: StatefulProcessorApiClient) -> None: + def __init__( + self, + stateful_processor_api_client: StatefulProcessorApiClient, + schema: Union[StructType, str], + ) -> None: self._stateful_processor_api_client = stateful_processor_api_client + if isinstance(schema, str): + self.schema = self._stateful_processor_api_client._parse_string_schema(schema) + else: + self.schema = schema def exists(self, state_name: str) -> bool: import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage @@ -69,12 +77,10 @@ def get(self, state_name: str) -> Optional[Tuple]: # TODO(SPARK-49233): Classify user facing errors. raise PySparkRuntimeError(f"Error getting value state: " f"{response_message[1]}") - def update(self, state_name: str, schema: Union[StructType, str], value: Tuple) -> None: + def update(self, state_name: str, value: Tuple) -> None: import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage - if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) - bytes = self._stateful_processor_api_client._serialize_to_bytes(schema, value) + bytes = self._stateful_processor_api_client._serialize_to_bytes(self.schema, value) update_call = stateMessage.ValueStateUpdate(value=bytes) value_state_call = stateMessage.ValueStateCall( stateName=state_name, valueStateUpdate=update_call diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 15089f2cb0d6d..5405a0ae6765c 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -1061,8 +1061,9 @@ class SimpleStatefulProcessor(StatefulProcessor, unittest.TestCase): batch_id = 0 def init(self, handle: StatefulProcessorHandle) -> None: + # Test both string type and struct type schemas + self.num_violations_state = handle.getValueState("numViolations", "value int") state_schema = StructType([StructField("value", IntegerType(), True)]) - self.num_violations_state = handle.getValueState("numViolations", state_schema) self.temp_state = handle.getValueState("tempState", state_schema) handle.deleteIfExists("tempState") @@ -1253,9 +1254,8 @@ def init(self, handle: StatefulProcessorHandle) -> None: class MapStateProcessor(StatefulProcessor): def init(self, handle: StatefulProcessorHandle): - key_schema = StructType([StructField("name", StringType(), True)]) - value_schema = StructType([StructField("count", IntegerType(), True)]) - self.map_state = handle.getMapState("mapState", key_schema, value_schema) + # Test string type schemas + self.map_state = handle.getMapState("mapState", "name string", "count int") def handleInputRows(self, key, rows, timer_values) -> Iterator[pd.DataFrame]: count = 0 diff --git a/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto b/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto index 4b0477290c8f7..e69727a260a90 100644 --- a/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto +++ b/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto @@ -26,6 +26,7 @@ message StateRequest { StateVariableRequest stateVariableRequest = 3; ImplicitGroupingKeyRequest implicitGroupingKeyRequest = 4; TimerRequest timerRequest = 5; + UtilsRequest utilsRequest = 6; } } @@ -41,6 +42,12 @@ message StateResponseWithLongTypeVal { int64 value = 3; } +message StateResponseWithStringTypeVal { + int32 statusCode = 1; + string errorMessage = 2; + string value = 3; +} + message StatefulProcessorCall { oneof method { SetHandleState setHandleState = 1; @@ -91,6 +98,16 @@ message GetProcessingTime { message GetWatermark { } +message UtilsRequest { + oneof method { + ParseStringSchema parseStringSchema = 1; + } +} + +message ParseStringSchema { + string schema = 1; +} + message StateCallCommand { string stateName = 1; string schema = 2; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala index 2957f4b387580..d03c75620df8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala @@ -33,8 +33,9 @@ import org.apache.spark.sql.api.python.PythonSQLUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleState, StateVariableType} -import org.apache.spark.sql.execution.streaming.state.StateMessage.{HandleState, ImplicitGroupingKeyRequest, ListStateCall, MapStateCall, StatefulProcessorCall, StateRequest, StateResponse, StateResponseWithLongTypeVal, StateVariableRequest, TimerRequest, TimerStateCallCommand, TimerValueRequest, ValueStateCall} +import org.apache.spark.sql.execution.streaming.state.StateMessage.{HandleState, ImplicitGroupingKeyRequest, ListStateCall, MapStateCall, StatefulProcessorCall, StateRequest, StateResponse, StateResponseWithLongTypeVal, StateResponseWithStringTypeVal, StateVariableRequest, TimerRequest, TimerStateCallCommand, TimerValueRequest, UtilsRequest, ValueStateCall} import org.apache.spark.sql.streaming.{ListState, MapState, TTLConfig, ValueState} import org.apache.spark.sql.types.{BinaryType, LongType, StructField, StructType} import org.apache.spark.sql.util.ArrowUtils @@ -186,6 +187,19 @@ class TransformWithStateInPandasStateServer( handleStateVariableRequest(message.getStateVariableRequest) case StateRequest.MethodCase.TIMERREQUEST => handleTimerRequest(message.getTimerRequest) + case StateRequest.MethodCase.UTILSREQUEST => + handleUtilsRequest(message.getUtilsRequest) + case _ => + throw new IllegalArgumentException("Invalid method call") + } + } + + private[sql] def handleUtilsRequest(message: UtilsRequest): Unit = { + message.getMethodCase match { + case UtilsRequest.MethodCase.PARSESTRINGSCHEMA => + val stringSchema = message.getParseStringSchema.getSchema + val schema = CatalystSqlParser.parseTableSchema(stringSchema) + sendResponseWithStringVal(0, null, schema.json) case _ => throw new IllegalArgumentException("Invalid method call") } @@ -690,6 +704,22 @@ class TransformWithStateInPandasStateServer( outputStream.write(responseMessageBytes) } + def sendResponseWithStringVal( + status: Int, + errorMessage: String = null, + stringVal: String): Unit = { + val responseMessageBuilder = StateResponseWithStringTypeVal.newBuilder().setStatusCode(status) + if (status != 0 && errorMessage != null) { + responseMessageBuilder.setErrorMessage(errorMessage) + } + responseMessageBuilder.setValue(stringVal) + val responseMessage = responseMessageBuilder.build() + val responseMessageBytes = responseMessage.toByteArray + val byteLength = responseMessageBytes.length + outputStream.writeInt(byteLength) + outputStream.write(responseMessageBytes) + } + def sendIteratorAsArrowBatches[T]( iter: Iterator[T], outputSchema: StructType, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServerSuite.scala index e05264825f773..c3d4541bac29c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServerSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.execution.streaming.{StatefulProcessorHandleImpl, StatefulProcessorHandleState} import org.apache.spark.sql.execution.streaming.state.StateMessage -import org.apache.spark.sql.execution.streaming.state.StateMessage.{AppendList, AppendValue, Clear, ContainsKey, DeleteTimer, Exists, ExpiryTimerRequest, Get, GetProcessingTime, GetValue, GetWatermark, HandleState, Keys, ListStateCall, ListStateGet, ListStatePut, ListTimers, MapStateCall, RegisterTimer, RemoveKey, SetHandleState, StateCallCommand, StatefulProcessorCall, TimerRequest, TimerStateCallCommand, TimerValueRequest, UpdateValue, Values, ValueStateCall, ValueStateUpdate} +import org.apache.spark.sql.execution.streaming.state.StateMessage.{AppendList, AppendValue, Clear, ContainsKey, DeleteTimer, Exists, ExpiryTimerRequest, Get, GetProcessingTime, GetValue, GetWatermark, HandleState, Keys, ListStateCall, ListStateGet, ListStatePut, ListTimers, MapStateCall, ParseStringSchema, RegisterTimer, RemoveKey, SetHandleState, StateCallCommand, StatefulProcessorCall, TimerRequest, TimerStateCallCommand, TimerValueRequest, UpdateValue, UtilsRequest, Values, ValueStateCall, ValueStateUpdate} import org.apache.spark.sql.streaming.{ListState, MapState, TTLConfig, ValueState} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} @@ -574,6 +574,16 @@ class TransformWithStateInPandasStateServerSuite extends SparkFunSuite with Befo verify(arrowStreamWriter).finalizeCurrentArrowBatch() } + test("utils request - parse string schema") { + val message = UtilsRequest.newBuilder().setParseStringSchema( + ParseStringSchema.newBuilder().setSchema( + "value int" + ).build() + ).build() + stateServer.handleUtilsRequest(message) + verify(outputStream).writeInt(argThat((x: Int) => x > 0)) + } + private def getIntegerRow(value: Int): Row = { new GenericRowWithSchema(Array(value), stateSchema) } From 3db46bf1fb48e93d55b7f766e444e6d4a396981c Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Sat, 14 Dec 2024 10:47:16 +0900 Subject: [PATCH 178/438] [SPARK-50526][SS] Add store encoding format conf into offset log and block non supported stateful operators from using avro ### What changes were proposed in this pull request? Add store encoding format conf into offset log and block non supported stateful operators from using avro ### Why are the changes needed? Changes are needed to ensure that encoding format info is stored in offset log and that trying to use avro in a query that has stateful operators that don't support this encoding format yet will result in query failure. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Added unit tests ``` [info] Run completed in 2 seconds, 547 milliseconds. [info] Total number of tests run: 6 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 6, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. [success] Total time: 13 s, completed Dec 9, 2024, 6:15:25 PM ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #49121 from anishshri-db/task/SPARK-50526. Authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim --- .../UnsupportedOperationChecker.scala | 39 ++++ .../sql/execution/streaming/OffsetSeq.scala | 5 +- .../commits/.0.crc | Bin 0 -> 12 bytes .../commits/.1.crc | Bin 0 -> 12 bytes .../commits/0 | 2 + .../commits/1 | 2 + .../metadata | 1 + .../offsets/.0.crc | Bin 0 -> 16 bytes .../offsets/.1.crc | Bin 0 -> 16 bytes .../offsets/0 | 3 + .../offsets/1 | 3 + .../state/0/0/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/0/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/0/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/0/1.changelog | Bin 0 -> 63 bytes .../state/0/0/1.zip | Bin 0 -> 3782 bytes .../state/0/0/2.changelog | Bin 0 -> 63 bytes ...1ee246-6831-4c62-9fd7-7741cb534368.sst.crc | Bin 0 -> 20 bytes ...8-7b1ee246-6831-4c62-9fd7-7741cb534368.sst | Bin 0 -> 1205 bytes .../state/0/1/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/1/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/1/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/1/1.changelog | Bin 0 -> 47 bytes .../state/0/1/1.zip | Bin 0 -> 3609 bytes .../state/0/1/2.changelog | Bin 0 -> 63 bytes .../state/0/2/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/2/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/2/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/2/1.changelog | Bin 0 -> 47 bytes .../state/0/2/1.zip | Bin 0 -> 3610 bytes .../state/0/2/2.changelog | Bin 0 -> 47 bytes .../state/0/3/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/3/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/3/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/3/1.changelog | Bin 0 -> 47 bytes .../state/0/3/1.zip | Bin 0 -> 3610 bytes .../state/0/3/2.changelog | Bin 0 -> 47 bytes .../state/0/4/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/4/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/4/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/4/1.changelog | Bin 0 -> 47 bytes .../state/0/4/1.zip | Bin 0 -> 3610 bytes .../state/0/4/2.changelog | Bin 0 -> 47 bytes .../state/0/_metadata/v2/.0.crc | Bin 0 -> 16 bytes .../state/0/_metadata/v2/0 | 2 + ...0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0.crc | Bin 0 -> 12 bytes .../0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0 | Bin 0 -> 284 bytes .../commits/.0.crc | Bin 0 -> 12 bytes .../commits/.1.crc | Bin 0 -> 12 bytes .../commits/0 | 2 + .../commits/1 | 2 + .../metadata | 1 + .../offsets/.0.crc | Bin 0 -> 16 bytes .../offsets/.1.crc | Bin 0 -> 16 bytes .../offsets/0 | 3 + .../offsets/1 | 3 + .../state/0/0/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/0/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/0/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/0/1.changelog | Bin 0 -> 88 bytes .../state/0/0/1.zip | Bin 0 -> 3785 bytes .../state/0/0/2.changelog | Bin 0 -> 88 bytes ...6e23ce-e7de-4df8-b320-2b0378b53e52.sst.crc | Bin 0 -> 20 bytes ...8-9b6e23ce-e7de-4df8-b320-2b0378b53e52.sst | Bin 0 -> 1228 bytes .../state/0/1/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/1/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/1/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/1/1.changelog | Bin 0 -> 47 bytes .../state/0/1/1.zip | Bin 0 -> 3610 bytes .../state/0/1/2.changelog | Bin 0 -> 88 bytes .../state/0/2/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/2/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/2/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/2/1.changelog | Bin 0 -> 47 bytes .../state/0/2/1.zip | Bin 0 -> 3610 bytes .../state/0/2/2.changelog | Bin 0 -> 47 bytes .../state/0/3/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/3/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/3/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/3/1.changelog | Bin 0 -> 47 bytes .../state/0/3/1.zip | Bin 0 -> 3610 bytes .../state/0/3/2.changelog | Bin 0 -> 47 bytes .../state/0/4/.1.changelog.crc | Bin 0 -> 12 bytes .../state/0/4/.1.zip.crc | Bin 0 -> 40 bytes .../state/0/4/.2.changelog.crc | Bin 0 -> 12 bytes .../state/0/4/1.changelog | Bin 0 -> 47 bytes .../state/0/4/1.zip | Bin 0 -> 3609 bytes .../state/0/4/2.changelog | Bin 0 -> 47 bytes .../state/0/_metadata/v2/.0.crc | Bin 0 -> 16 bytes .../state/0/_metadata/v2/0 | 2 + ...0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751.crc | Bin 0 -> 12 bytes .../0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751 | Bin 0 -> 284 bytes .../streaming/OffsetSeqLogSuite.scala | 67 ++++++ .../streaming/StreamingAggregationSuite.scala | 20 ++ .../StreamingDeduplicationSuite.scala | 15 ++ .../TransformWithStateChainingSuite.scala | 195 ++++++++++-------- 96 files changed, 278 insertions(+), 89 deletions(-) create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/commits/.0.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/commits/.1.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/commits/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/commits/1 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/metadata create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/offsets/.0.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/offsets/.1.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/offsets/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/offsets/1 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/SSTs/.000008-7b1ee246-6831-4c62-9fd7-7741cb534368.sst.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/SSTs/000008-7b1ee246-6831-4c62-9fd7-7741cb534368.sst create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_metadata/v2/.0.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_metadata/v2/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_stateSchema/default/.0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_stateSchema/default/0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/commits/.0.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/commits/.1.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/commits/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/commits/1 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/metadata create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/offsets/.0.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/offsets/.1.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/offsets/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/offsets/1 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/SSTs/.000008-9b6e23ce-e7de-4df8-b320-2b0378b53e52.sst.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/SSTs/000008-9b6e23ce-e7de-4df8-b320-2b0378b53e52.sst create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.1.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.1.zip.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.2.changelog.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/1.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/1.zip create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/2.changelog create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_metadata/v2/.0.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_metadata/v2/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_stateSchema/default/.0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751.crc create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_stateSchema/default/0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 5b7583c763c06..f7ab41bd6f96c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import java.util.Locale + import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.{ANALYSIS_ERROR, QUERY_PLAN} import org.apache.spark.sql.AnalysisException @@ -140,6 +142,38 @@ object UnsupportedOperationChecker extends Logging { } } + private def checkAvroSupportForStatefulOperator(p: LogicalPlan): Option[String] = p match { + // TODO: remove operators from this list as support for avro encoding is added + case s: Aggregate if s.isStreaming => Some("aggregation") + // Since the Distinct node will be replaced to Aggregate in the optimizer rule + // [[ReplaceDistinctWithAggregate]], here we also need to check all Distinct node by + // assuming it as Aggregate. + case d @ Distinct(_: LogicalPlan) if d.isStreaming => Some("distinct") + case _ @ Join(left, right, _, _, _) if left.isStreaming && right.isStreaming => Some("join") + case f: FlatMapGroupsWithState if f.isStreaming => Some("flatMapGroupsWithState") + case f: FlatMapGroupsInPandasWithState if f.isStreaming => + Some("applyInPandasWithState") + case d: Deduplicate if d.isStreaming => Some("dropDuplicates") + case d: DeduplicateWithinWatermark if d.isStreaming => Some("dropDuplicatesWithinWatermark") + case _ => None + } + + // Rule to check that avro encoding format is not supported in case any + // non-transformWithState stateful streaming operators are present in the query. + def checkSupportedStoreEncodingFormats(plan: LogicalPlan): Unit = { + val storeEncodingFormat = SQLConf.get.stateStoreEncodingFormat + if (storeEncodingFormat.toLowerCase(Locale.ROOT) == "avro") { + plan.foreach { subPlan => + val operatorOpt = checkAvroSupportForStatefulOperator(subPlan) + if (operatorOpt.isDefined) { + val errorMsg = "State store encoding format as avro is not supported for " + + s"operator=${operatorOpt.get} used within the query" + throwError(errorMsg)(plan) + } + } + } + } + def checkForStreaming(plan: LogicalPlan, outputMode: OutputMode): Unit = { if (!plan.isStreaming) { throwError( @@ -199,6 +233,11 @@ object UnsupportedOperationChecker extends Logging { "DataFrames/Datasets")(plan) } + // check to see that if store encoding format is set to true, then we have no stateful + // operators in the query or only variants of operators that support avro encoding such as + // transformWithState. + checkSupportedStoreEncodingFormats(plan) + val aggregates = collectStreamingAggregates(plan) // Disallow some output mode outputMode match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index e1e5b3a7ef88e..a599f3bc66118 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -102,7 +102,7 @@ object OffsetSeqMetadata extends Logging { FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION, STREAMING_AGGREGATION_STATE_FORMAT_VERSION, STREAMING_JOIN_STATE_FORMAT_VERSION, STATE_STORE_COMPRESSION_CODEC, STATE_STORE_ROCKSDB_FORMAT_VERSION, STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION, - PRUNE_FILTERS_CAN_PRUNE_STREAMING_SUBPLAN + PRUNE_FILTERS_CAN_PRUNE_STREAMING_SUBPLAN, STREAMING_STATE_STORE_ENCODING_FORMAT ) /** @@ -125,7 +125,8 @@ object OffsetSeqMetadata extends Logging { SymmetricHashJoinStateManager.legacyVersion.toString, STATE_STORE_COMPRESSION_CODEC.key -> CompressionCodec.LZ4, STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "false", - PRUNE_FILTERS_CAN_PRUNE_STREAMING_SUBPLAN.key -> "true" + PRUNE_FILTERS_CAN_PRUNE_STREAMING_SUBPLAN.key -> "true", + STREAMING_STATE_STORE_ENCODING_FORMAT.key -> "unsaferow" ) def apply(json: String): OffsetSeqMetadata = Serialization.read[OffsetSeqMetadata](json) diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/commits/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/commits/.0.crc new file mode 100644 index 0000000000000000000000000000000000000000..dd09db7ad216c6b594c83f8e23549103549cccb9 GIT binary patch literal 12 TcmYc;N@ieSU}Bir|Fs#8^?!)$dG*-VhAZ~22*0D>|rq0X-t%zY(v&8SxS?AXJpTwUG}w*>}x}m zHT#q;gpa)x8COY!+4&$2}f}Tt2oLMXFWVEFkkHKTIUb=O%i_x!&R^<)E)vYO0u) zsm&RG=J+BF#4BQAo#`}NFzu^X{C1|vq{rfLr-LX|S4X&up7oNM3f?@NnX&O7a#)`Z zQG{Z;ElZ>KoHpz%skHUcUI_c0ZBpBsB96pzcFit07g2s-Ssq!HGHuT+t-fmAFjT`* z7i~~v6~nOBHD@GLpl`aZs=uhd>7HkgEDjB#BMl)4W#d!_jvx_bt1a8!9?WNn&nbS_otF9^12ph3>chXXct=N>l zN`u+A3reE(q^2TSRKo4J+O!$-P0#QTxzc&1rsaVHn-N1l=#b$ zvQJ}mYtC9|F>s!J?Wxy>UAq3NY<9Y5;G)jJ1Ac5eZtSQ%c=sE%`dgsW4)c^)Z9Dof ze~bC%c}f)|1Woo4+amulx5F1Yk~h#vE#qlQ1}?dTnGX<`Ews`#*8bR#JDizc#{Jmy zz@+}*Xl!GeX8GXB>>ODBWko*Kz=fwmL)Y)4)9;4YY4ZcAuJQD(55tnp425C(@OZOU zdd`d~Q6EUqoc}D;ov*rT|r?|-P@M^)QjJHW!u$< z^MQ^u`@2*s0yMHx_g)|0k!HXtX4^JzbsQ)NreT7oEn)C`A*w{qEVWK16XAG}Z`pzY zN|jb%5-ig}`y+HsIh#F<%Buq_#Q_t%Xa9uE`yuThiOEyTmkZv0lwpo^@DOZ<-Z*mH zxbudA{XtXKrT65Vo~$|*eFqTtm?FZIaRExWB;%I!72 zZ=M&Q6PSsMeax`K7VrMbeg0zaBz1U>XaeYwI5h7q_+ng-A%u|m5I&h0aY1rN5N~xh z|3bX7dCtKZojP(EL_DMgrwbH{eya2q$o%LlFT+o1us;Uo??#ML=*$gR{@&im6Ut$j z#R9zZ?7QL;I81r#%?_#g>kjT$BT{Mt%5}W^S7zkq7)*=36={iH_rB$A+jChQ-Je%zEA4$*nR3&g1r>_Zdx%b-v~BWprA5 zy;Cwr>Se!mrR;qAbVj=x36nKSlcA8{>Pja^7E5tHyY7r~m14>^HZa!c)@lpZ_xLI(Y<;&_gD?|?26FNC~~Nk>=VpcRTjt2Yo*;a5S{BF%Vx7v z!V-&jkGQc+5kMW5Z~KvKTlqP!nji?8sqN=CevUOWu;nF}klOHCMDnVEL`Afx}Kies^(`>m^%csdMurq4()nHhZ=$D&L@sdxcp+DpZDF#pxSJ4K%gedJ{y_ zUichBs!Gd6sr23#f5#wh)s=JcH&iv;R84a7WnNLdmg6*9Py&Tum6_c5GcG0mRI0|x zs4k|R6Tcw6yh_-jR4-W7T+N>7lC##+`(|=fx>%J z(|EJLgLb%69&i=adi(RsT9{~k(YfLGEuvbr#1Ok04N!~Fc`27IV14qw%f3?aSa#T* z@eRY?MU;I+l+@#ZMws+Fd*`<{9@Vudhqy=5sJM9*m8$Whu6eEldh8Q~acq(F^S!k- z&)AHmB|m2TX9iiGY*nCD1c`rYt}xc?M#gv)Ig2it62~M+tgA?~s_~QGnDk1|wx^my zdP+vpS~L8bc$#E)ob;n^zbxtjVOQ(&aDVW-2uXS&4OD{v-H9DMA%?Z8;sxM{4@4zc z=gcWha3Q_N0_1{XWhNs6{a0llQ(=Qg@j1}^c>v(f?8SRGYC(%1#Bff#?GH3~Qk;*v zM`Rz3Yw)+!m&*B-lQHF$J`Yt~XHbRG(?j*ys3twSA=e7U;*`e6tAXE|R4y?_K@L>N zekiX{mt4f%hADI~8-KG$T&Y?|O|^5qk}Hkssm4lUi{4p8w+&(f*F{aM(QaPVQ#)XE z6x~~~u3LPZK&>?`@mABT3PscOM6U+2OVwJeq5$THPnt{$MFccyBnDM<#vym=3t3u( z5dCk?^4Ijwc_(o-Y8=s+*OY1)XkR&_t0Q?+%;o*dcy2LP_c;Xu#M_if%XBW9*j zjyxF`y{^m1mA@YW{WxQj;HGmoXe^A2?$Eqb-1DJKlVhUbaCJc*?66U!F&ZY!IW*qzC-`&V$jJ?Kr@|9?_;b(Uy+&s8v(q)Q+OKTxI zs%HdlR><^IqSeeZ^}x6DauVb?U;m19!9>r(!^TEbQBGwDW@EIvU?q8^2l`OS!cC=c?9mFjr$wYRv~ z`QD{)xiregw)$Pj&q?l(R}nR<5oV8EX<6h=zPVfmq=Qec={>u?IjLq`GF`M(#yNK zt!Eh2ZnL`S%#rK7>)Q&;zVjIBrAoE-(tL@ytc*LRO*O9CV1Hk{>tsqDm!1M|Zk+i! zJLM1Ww@AL`Mo<865eVG(Lch&O#&C$?!!8Mcmp~E$_`J`n{6M<}1rL7tg@{Uk&nMi3 z*iAij`t8BK&1l2r=yFKz8oOGCQ~o}_4YZi=(2Eemmv_>9ICtj-r-H?9dcU(U^u~vv zK{^fuqMVI^(JqHZF4nxG`ZLNikpL`U8O?=K95jb5@3n@>CvvwcL^@uxifNz|AYk3t zl}wI$dWh7~97?=;BR5OWX%XEu(J3xy$t9p@E41Q?PXg8L%*+eomGTd{F(C4lK7)#Z zNq#Lf$m-C|sTwEiZtYQHBJzg4J&sRC)D?c*JUCV|>(F$#mL!^uPw#RJ?TDWzTLJ^%^=zyAvmO0cfetb(IaRsN2=ccdJeb6G=#Gxkc;~Aaz!5JK0@xV7^td4T zDpzQ3hRh|gcWzwFZ)GFKUZq#0hf#dwnrR;KK-0{-49SB6k=MiFGM{opQP$LQ?&gGB z7~BtN@C|GZUvjZ3qSYZwSHM?4FZJ3k#pRAs?fKB)5b50U&(5Dic(>!?0ywm`!ibWh z;ryE;z9a~B3Q9pvZ|fZ@0H6^B03iOcVm=7~{Hm0%iP$+*{9 zJNT-%dHiddnWj$Y>wlP+zQ5Fv(%#y6EoAZmld}waA zTD~`(-rU?N7#nRImlPe9wTLu|GEOy0Sx$M0iZF~ZjKT~rHFy39N)!kU@{fGxAM_vP|07AcS05-9=4?gx0 zcX+BJWS+pp)`Bc!+^CJa-zyZUkvf3$5w3rncgQ%yt{UxK!{eAKmnTF@RwkY48e2lR zV<~=~@F_jT;6^Qi;=-tQ*~O9>k00XU_U#U*oXArHaC#`??{mI-Zoxf#w;`eSkFQ+*ER>_I!Z*czI i{AUJ!{Wkw|x19fdkqGs(^1QLua3_dWGKrS2r07KUZN&o-= literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/SSTs/.000008-7b1ee246-6831-4c62-9fd7-7741cb534368.sst.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/SSTs/.000008-7b1ee246-6831-4c62-9fd7-7741cb534368.sst.crc new file mode 100644 index 0000000000000000000000000000000000000000..2026a6dcab3bf5c91b68cf90c2c4dc1d84b22fce GIT binary patch literal 20 bcmYc;N@ieSU}7)}4sqS&B(N&}z?97ZGtCBZ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/SSTs/000008-7b1ee246-6831-4c62-9fd7-7741cb534368.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/0/SSTs/000008-7b1ee246-6831-4c62-9fd7-7741cb534368.sst new file mode 100644 index 0000000000000000000000000000000000000000..c4f2175a47371c98e894a432b8e148c639c154de GIT binary patch literal 1205 zcmah}&2Jk;6rXXDdhNIyJ8lc|B|el3l1K4J>PAr{5|rXfLFMDp1|e(K^VS)4c6ON= z$97M-AobXA=H5#=zy%2*fmDere*p*h2e^PZ!kcyCz=@T1{pP)SzxVZfCLyJSkOI$t zyo4|DDzb7>X^>4asQJ^mw^R#(H~53s1*A$y6NO8M^B;5h>hlg7|0ZwZ@z-xIUXvT% zrNBwU+r3PV4u1Nc++8*4=&2PEixdhh;uZvRJW{}<7BFLOo`JvKsM5?pEN3j1%7Kx3 zyxpkN>1m2rpThiF-43dBl&VRhS+_QLY>`l8hVfQlsDu6vjWC zJD5rf@1|FlN^!-2wKCPmC%t-bL5FGP@MNc$Svnm?(%Jo1ySLZOG4{Ki{=NQQ(>lb3 z@qB-KubJ@KP{@%RMu={`=9zV3=+$^j>T5k{U$gkH(E`#lZD`-`^;_MYdr_-5>hxQ?oqp8n@VPgCob zK7->2gX0f-@9jRRRd3lv^-JTEljFxHgO46Q9<;k3bl&go?6e<-#kEv3Clh$_!|&hS z2uf6N$C;JSV5_uFL4m`8;(RO#HrH@;z(#ql|Mprhuj52nmMS@#BN5VMG2@L@>Vb~R zA)PHym;ya^r=)a~rY7Nzoq_QaBEfCC;ESh%v0=RP@~>SaWe*Ng@w;_N`sIlfC`poV7L~R)K_jHT|Rc6c?VerT0HV5)CC(T|R>NLmqO${sM%ZB)qZ-V1e zG3&T-ERACeZp-gozN=q<@%l^hRv||GMJ?=VZ4W27jZz=_p*_eKbL)s~F1I=}c_!G0 pbNaIA@!rNa|Cjmmm#@OrSHUB>>DB-9tfBp{pa1y$+k@Yp{{!K!W)J`X literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.1.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.1.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..0b52e7b4922a0cd57accac957e78d6a3e42619cd GIT binary patch literal 40 wcmYc;N@ieSU}7jp)b8N@r;{5#t9-A{kK3CgS)QtjaD5f9`@r>n(y{&@03It4$^ZZW literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..889a96a47f0a364fcae2fcbab9539c65e1a213ed GIT binary patch literal 12 TcmYc;N@ieSU}DHhjWY)T5lRA| literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..3986c42e7e5ef19e2f11e217f4cbead03d29e8c7 GIT binary patch literal 3609 zcmaJ^XHb*d77azfAOYzuAcW9+2@pj{kdn|sM=&&{3ep17R62@?NE3u8UAiDmdhdkZ z1Qajz(ovd#zyt4lPw$<1Z=IPl-^||Yoc*nJe(a6WAR(m(P*6|+B&#ED0)7MN>{|z8 zfYQ;{7ZE>)?840%k0}`K!+wi22MaUf7@Mx5Flx`CUQb?UxM$ILRUZ`@H{v26?U%q* zXPM#C<9ciW55LfIoyC-cHL=OsbC-rsp$#by<7 zF64{ZxfCWdbZhwYf8Q=BD@u4c1`|AW95X;yI4&CGPWMhFDdr^kCU`2dm9Wv4EjN_S z$W>=gJaK)N3g#9ycgS#?%b$5t_J#gD}vIFWGVf`@9d%MJUs|wkYN?EnLkQ`fzgDZ*#6)7{$jItVQ4t2v- zOf|8_h4yiD>s|9V1@euoc2P!48r$Bv&IZMz=3~CgW<)N2tJ>D%@lbqf599XeI>8p) z@3YDkN4h=hFE<(eqVTO^dFX4w{;_Shsy|m27hd_7!A+lmwUg`YsqtcKv0b&R2DN4D zUKvu8kvnykMrndRdZFI#lR|%FhHtDAY)fwySop%@J3w+~rnc=2H<^1nse$6F zjwNqX5zejrQdmQo&o7uzQT&-Nk!)!xrA1)z#{JC$h-d)K=)ko8n)n;ufBahH$xb_qBnR4ic5VZ*Mxf3o!(Ahi{Vw2uo(Fs;|+R35V zUFZ4gd*$gQ|~t+k#j zg3Tn838eYto60gIN^R%E9l|uav`i1Zl ze6{IxVbPKhH)Ez3r)5E&ovW(YM5O%qFs0InI9Z}~pI5uW8ePajvj#s=D5hS zIPMiQ#pAuBudR>=QAXw6{Ybj2_LLjV4~A#x1_Vr8WX=d_dcz@r9$Al0UNe>~8wg;z zR6<r+O>gsY(mnBydT_V+T~5eCTngS~Q*& zr8%)_(zj&h933O`FsL3O+wSb%=IB#dZRV12U)C&PL0!FK;-qVV9+-52c!KK0L?!5JgE}vL4D3jq z^t;+BW$~5B8wjNiM$@m(7~zTyv(K#@Zxu>n2$eWlTw%LyY|D^E$cC6zCDzNY^79@9 z8$;73-lfdL4$@iIk!ZFOQ7W9F-SVp=<*n3V76UNe3vMtk5arXRk{nXkn}BK77BDpl zUfhigYg$3FQzxKol3!q$fw|0Zl0Ct`2`4J+T$}Dht6Vb z3}F;$UBK`|vf0}A_0gkF!B?=;JR@P*{3*iXz505y!tC;Ji_Njh{MF=90=)Mik|x@;ULD#CR$C?#{w`U(5*)duq?@o|Ft@~K}+gpEtb77yDfu~FF{3XOM z^({YFNTsN6sojVDnC1+B8(p;)ZGGR9nn}S*+gCw26k}H8{C?;}dTTn=DA1xM2Pw&< zv$#pCYHk zuNz|1ZVdHr$C|9fmcnw@S<&fkd58EG@KT;jA4VKs+DX-WQS%wQlI?!lfV&X74LCcfXP<-jmrk7 zp*mQw{jrx@Wu&~fgHM&Y=m*x;1RgmtPvmLi(0K8jOT+Q{E3qtmT9<1?$IE%Empd0) zQq)2}({m51R3QM~4lYBvIh&cr+cmqcHxs=!%U^8lq-OR|zx!@{nLeo92(D<;i~D1^ z^65dPGT$I%Bg_8wT*rV2$VHARyCTOOL)O;o!yvv#b8?c(y--01iGGt9#6tH6!NY*3 ze3j~Y`1~u@&dn=%%`98-BD5;B2=b2{vyG!ZSgHk&VMTC=!PVzTxlh?*W)74J-ZqoU z7LniKVb^fkJjq2UOtVX-KHnpw+}G0kej1uV=*7oe{7f}~R3wMdF;4JU_XzrzJYi{gpZEW6BQ(Na_ zBZ2UdNU>4=_1BSYJ|doH_2B+*#pmKU+u*G3SmCUMq~`OC1Vu&NdU{bJM(-n{Ms>S| z2)o*ar0$n+aWUH-Xpik_S`R4?!3T7dK*{Z)_=3E^<}4p*6Z8o*7k$=7!O>Mu82Z!& z74(&wf67*eo~qV4yTa^e5k$oFfM0{o?}nZaI)BIC#~zHv-?`_x|FQJk@?Vgo{QUv^ zllQxg=Z61+(;4sI_WzUgyOQS)|3dK@=^uXoll;2^KVRa1+18hSNsrOEKz1IuaJJZq L0Dz^l7y$4uhLkNQ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/1/2.changelog new file mode 100644 index 0000000000000000000000000000000000000000..13ec6c9cdd84377d2556a9a0f986b11611a7c809 GIT binary patch literal 63 vcmeZ?GI7euPtFz{0SQJX1|OJ8AQuh*GU^BF literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.1.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.1.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..6e5b8098e6e4020a08c36a4b3ca4a4c16613ade0 GIT binary patch literal 40 wcmYc;N@ieSU}7jp)b8N@r;{5#t9-A{kK3CgS)QtH&0ZeS9L$sEqH*R903qfO8~^|S literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/2/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..59318e5f734e964e29e399bf271bf85765b410ce GIT binary patch literal 3610 zcmaJ^cQjn>`W+?6V2IvE8>9C!m_(QnJs4d~MoUBq5e(6i7&Syh2_Z~$(L=Q8HG1zs z@Tt)yS`b|F``w#--&*(Xv({N>t>@YA+3VTwANxUR5fRe?$jHb5l2zfi0Dk~<{;i8L z)X>#25D~wC?BcB%w@DbylRk@7dkeHlv`t50FqQjYw>z&B+`VwDvX_#C8*!PBW)_#? zBr|k&Qit*5;TKw|wV1TGCNy1t>0JLMs6Oej+*gm_vxRlWM2<<>1F1<*r=I-b*edV6 z#XK=v=YmA~PAy;lA3OP_g>g?tVS;B4qlTL04oikPQ{9sZirER?aqh~j#jG@?EA^!_ za#dO5p$}iAfVoA@?bBW6^JbnJ6@HqlH}AQ3vfs8XP+dK78#C`CHyyllGB@Xd8*<*9 z2~mMtblDci9J*{d+5+{AF+M1#{XN3_l?AK`B`i7}NcL^TfmKDr@}wCj23f6j``V#O z#_AZ80=rkV8yyR`1oDin_B4!_wRSvnoD7RZ%}2dg&;-stYdY3ru~2+U7sJlTM$bKP zpVu1KE8?9wU%83MsDgKjWkGKR`$l(MD!;BSExz$Bg`2$q>m=6NQQ^hbV>)Wq3~Nd^ zJkq5m!gp&ejZ+1^^n*NICxrgW2-#fgxmR*K-@+Rn+Xj+DpH+M*`z)z$L> z6s^BDb!#2nkhD$Z1wQD)(3b<_dtGr7%V^(@~5X5&Rjik*ujnC52$|hJ&p`h^Qa+vw$FW&G)dp$+bNPh=WRs{DdM$Q^p}v&w=*!)`&^qnoSyrE zT$sQ0z@&nG#5B<<(a_H)jOEf+b^s2pdg?$SKsC^BGKAZ2w{wT zZQVq}mNqT+7#6gEn+JYj57{~Uv5DWVOS|zQ%Z~ZPRsv)?*eZ0uFGOf?!?vG%`FpQ? zi`H;1=pp6N0Z^TfQeMXL{n<@f+HjT3`whEo$4dMu7Qr*N2&83*#x`4qW;?yP&};D1 z4@)L!4Jy7Vh+G@hkFX83OqK|sPa9T-6~S-m6hQ3zfNGGy{F!a?lCK|G#6xOi2sX`V z+;H5i{hEo>aeW4FCkdN3lYVL6G0gK-!9?pp$tD#3rq?ydA9UxSLJ3 zx54wxoAdbu!}f9?1LCC7x4-NNTIl;m3z;Pr4}K&8&-n!5jO#IlO{70SP9;QMk=p0S z+g-}N@>zGKtnnVuu8rB$Y01}9KruqrQ@aQWy?N3<<&H{n+*p60e&-P#a-crXE z#%h|u2)Y^iU1bFlp|<;RpU~!g8^^m58BIR5YVIT9IfVr}eIo@3*ua>Qg-rdfW|VL! zzRGN>pm14;n<2x4!!keD_MxiSc)0x75V_K@I7z&9uScuGI&I)$_!0aLot2sXw=ABt zc6*;zRU1R2%unuQ?a!Xg=~Wma>mG%CLoAbw#_&&}W*Nm}z(Ew6JpN zQ;#Pqfp1sE+)baQ%o@X+^aap)Rt#sKX@WzA1@=b`hnk4LAY3)%!&$iP^g0G&vYq9b z9rlYDWAUDm*H%gW$s=;_e3G34`*Tw3|XbCCslD?GnFTSR@F$L(RC;vv;rTBc`xdhzYC?H2o?;Q%h)~qu1OU zFPie=@d>QFxI_$S)LHo50;ODC!Vax1uf4AE#^ou)nu>Im%V=H^9EMe+_kgDHCj3;b zz{+VXr(Bk}BD=aiu}h|vx306EIoY9LZ)Eh%{IqzdT)5w$l9xD0Yorv7fHpcFcYAUlKkoaeN>Rs~+}77uqhXt}G+mE?Nb zH6s6LTpQY0Q>@@uLQJ1i{5(u$lU4&xLjyNv22Oc(!KCxW<5b7TD?s1s)p_ZnVaMvk zKh)O9i#Wq?Ae7n|%)U9Hgv&S4(=F`p6pEvJDzLKHg4TO6&4U(!n_^ZK7!RL{>3s+$ zn);J?hcXWvNOwb5qRC1`sbGd?+ozV8w?dam48ZUppx!)Rluw6Ja!_4=9CouNpRrK@ z)&KDlw6cG}_YHfU_9>-JWwEx2o-mPtzSMPbx6b6(9O6u#3rZ6(Un}}0|4<=1i@7od zlEmp>58PMd^zSz+Z+$O>JS?l>!uz&%txtJgZX+i;$ivw`o>(-KCi5 zo)M&VKK)P0CTs7v$4}b@-@wjtjfG|NCVQ41)YYL2vdTg%wni)R))Gg0;N6GueC}^G zHQycKMxr03a5=0dM1(YHmX2XXMYwMHCuPiAb0VRo`x*t>2j&2tu_;3rYXIr;z$=&i zBqZ2XStc$YXKQiY6xs^Yv2Eih^LqItD_WcA{V5Wt{Rm@sXZtU34(tmP@ND_LuY@>7 z@5)Ps6tcSJngiI+DUOhLk(KL_){oq&7!|B^ycL9lQ0PLZ4}+)D+fzZtIE&_Nq$H#6 z*0OD+&&Qe{)Q7t&^^K|Xq%tqI?@JAT*TuwcrmJS~5;W3t^^@XTikvZu1JU|~f(O}6 zM6{YWm^}0sNc3C#b|W%xK7spa05_6tR<>8w!q2GQnw4*{bQT`CSdqu2CLtT@=6=pk z0#&k8N1eU$QqM7EBdxUQWFXuYG4{+O$>7qP-I)KzWR#*>KY1hKSX}rU=CZjwYq3hthVhYN`1qupTU` z=BGV9+vG89_|}DiHurd*{KN?5zxwo$9@Zx8gmv%Az)RQ!=`!-v2PULg(LV3|F-2#{M-_ z@%*qtnQs8HnQ3=tzO7#bo1I$if?z_ASobCg$x}5gHX51nFn?xrPxh4CSKRkRmwH@ajvX+?Omdv^}|kr_F@2 zMfeYR@HK1}Phz14s>wOSfbXes&RgjNQo*)Sy~VKL5ZUbUS=ZSR?!CA;e^%Yyh;1pc zm(ZP&rvxZ1QZjxvU;BL^08j@808sxmF`qX8ekEmTQAcN6qVrBtvWq}Xc=0;Ne`k`O zSCx(Q^;L8XcieY(ux^x~0T}>+f#|O@sw#qasm?p7|K}MO9oh&T4OJBb!~YBHGCQ87 zsit^b@c;ta!86d`-5{+ZJHk1jKh`qEtuZD!(vKSJZ|WBwX_1rpEGH3agly&Z0C_O>GGGQ4$s77DT2cEKPC2}M3iWQv2{KY)=*+aOfZ{$+CbmT-3S|ajh4scnr zBSm-;MYp3c#V^U0mEkfT=Yp{RQ}V@hoNTaGcdf8iLQ)I4#)6`vE?wOkBE}!WB1ZH& zg?jdM3W!}Pu&<(b+%(;GrfA%x+yo!fk^?1o24nMcaZQ)ts!-J^uyzmk0tvI>7Hu=TAp3Hl4rY?|TnQ>+jr)-hX>~;rVY!lK*)F z{>l5(#|y`Q!||N=|Nj4z^yefmEdGt6bJ9Ny|0ns+3H-W>|6$*u_&s`*)+LgQ!lm007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.1.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.1.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..4899bd7e696cb9eccfcb5ccdb28471745d371405 GIT binary patch literal 40 wcmYc;N@ieSU}7jp)b8N@r;{5#t9-A{kK3CgS)QtHDf!R6`9ht&i^iEd04dB7CjbBd literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..29a07a94b5dbc4adf4952e4a60889882f7b8c933 GIT binary patch literal 3610 zcmaJ^c{G%N`yNYZFk~-=7-Qd;!9>i+9>y-y5G6|_4B0AUElHM$nC!A=mwn$&_NAft zTC*!#2rvD9?_1yRIq!Qt=RD^*=f1DcbDjJ8{BhkV9Wrtz02LJ#K&m?Y2H+2XPQUe0 z##;J%MxqjDkek0TR^R7i?;15459NF?DgPthIZ~RlY)H*lU%E7W4sJ+#B>&Yj!9K{-3YC(c8ay_lV$Y^kAa zO1?U4{MmySDPSHkONVsV*}SPICWUP?4VHwvN4p)Hf;Bb7o0wT&`7a?`M>8{yxFMJI zsZcezRgYb9%)aY}lO52&6yu9>-rXU+S5?5CP8%p#|=>QFaS z#aa_%R$%{%ajk3ahG3ql^^TV5qRy6Aj0#L#StHy9 z_j#{yzarn7@slS;MisnME)RYy)Hk~2TJ?2#Vg8L@8QkIpSTC{8o(?a*8q-y~Vq9Cc z=9w-{4BxK1WtuAFZ5Zt3Mil-lBXoU*aJTekzLgI=wgV)OKB@d%{z*#Hgg%r@^d$Jv zDB5sk^2RE;|8@$(pVXgqi`3~SS3xp1XHmbJwQ}5B+(oHFAt*Oxp-(c(MqytK< zI2ON4K{&SxNMlT7zeKTWMF?cPMzW_Sl@@{}8uvE#A!7dYPXmKFwco??CRTPFRo9KF z@Ip}vPh*X$&fV2#=a?(vf;oTC^j{G?6@^#?;EvF8_0E+<%>jh zE9NM7o8>AEm8LNaL;eZdr2Hwn;|Y8uXP}c>-p868Qp9I9A0VlaZ*OE_@To3)I6b$7 z>#@(FW$oed*v1s?@?qfY97H*}EEhO%;i=%zmD`xqn=fk&pdg?Oci;LjBGJZN7-5Qh zZPU!ak@iLW5iEEPHw*mC8M<}yV;#R)pLVSz%l_h#og~P7uw8ghK$z6%nq5Ej;`fgV ztvbWGpa--Edq7QoS_Ro#?@zAFF@~#U-fP_MI8+fxu?m^8Lm+R3YHf04Xm>JO3cm(F zDOoT>YtivfLgYK>euS-QWL}B@`gUMt*%1P_oCC@I?$ZsDSU$B&Uhwm$ig-Ye48^9I zj2n+zbSj!TA2ww0byINouo;&19m2d`6%dn$q#v1vlsjS5g`6!@Ahv1U6`f#Jr|oR2 zoi*NXK9`>pSvD8@SP(}|e*NVK&_cgAI>;>Xc<@6>cupJSavZ@NMohntoJ@$kAiXPq zw?CJA;kAZs)*+c;4TU`FHLNMz2qYNuRJ{$v^6)DU<4-vV&I*C^+>BP~%nn%o-dfKc z#%`X$3cCL6yV?>YLSy^GE~)MN4z712vfBI_H9Q9*GfHzzh9*i7u#qY4B`VE3+EF6U z@YNQR1%->kJS-WWT(|Ob?H;I$kB2La4N{X&GYj1wy;gQ^ zL;CSV74XgKm^dyh44!}_6A)8G1)E( z7aeztSYz>Ck&4Td0n`z>_db#DXguf9768N34gCGb&$FcmH6?QjY7ehPCa#)EmG%2y zq$wt?ZViL-A9b2Ty(BHK6m65gotq~KyG_r!McBDp{sB|?QJ4*^2{ivI$xugXrf1Od zFy7IZmK8ObEZ5PzA~+1I!R!f5<4gFd zUWt|0T1+`Fc|mS@mAFl%leen3nmN&>E z1<^`Yf3?4ZZn#q!Ac}6jHl19J5U(vbKiu6Uu3xQto*ZW5%Cb>9ZnCLXvSs27f9 zMqD4?F#ouSc8-jeeH>7akZX5#YjgCjtVX-UJ(NSo&1-5_j30N+a~?8a15p;S1#-{# z*VcSu)0UR}S@6@03f!4mAp1xX+}mtntgm9)cr*p85xF}1Bv_)WK(C_yv;Uaf3Sq}b z+a)zA?af*vQbsaGx+hNVVUK?X^?!GK$ZF=gtEF*Jm%^Qmh6q_DY1i?zP_0dIIatAsx4OXFC}NrDSjTNw$7*pXJCMvUIb2h_rPTGCF0b_$16eK8Z`NsqG5-c z0tFm0xxt zm}vSoi7r)M4v_wuzGSnts7k>U!=`T?IbWqdn>c{weqe)Tz8JqAt<<2V;W+GiZ9Z$0 zAgcevIcQb?oZlPHdfj7M+p1z+GXoJaBSYz{67Jo}uel`HyyjGhFh6VNg@9+mOja}H zN)(CVG3!Q5oVf>)@J}<2@t%e^gU2E`>5ptXC4KJ8H#|rX7_Q9AK^)ba^izBNbP|33 zBHU-;eoNL`_hJhm$V%6eC5zS%8Hhg*iJIKNbgBd@BcFq3czBm5=jI`VN%t@8ocaqf z(S#ACO+NEasb(9Ww}($Uh2FqUa!p0#@+Jri_v`D?1zF{xRvV+0c`Jz{1bFX$Jio_V zZS8jlxRL0GDcp|B2@#>q+GS%{F;VUt0ZAFNHkXmmvR$nL-91YH@7ScVs||p1ap0Bf zZW0pgrYakkkF&G7Y7T9O>DhH~m3zN@oE5E0_Wl?N)P0DtzrFbvI0yEb4S2Hn-cM43 z=Ht>!r4*|AmfAho&q=P(cac@Akv0!K=vbAk^?Z~>f>G!~=aRu=na#;yQ=C;xHd2aJ ze`C=u()UB{5BmLWwT7nDSxVU#oA;!Lzw2XS*VENA_(+t}w> ze!(I8?+7EL?YL%ux~Tcuh=E<5q$iRC9qTgK4*Fr3UXRd@rEidM3w^{eGZ?H zxN3q)y*Ajl6=S{}QwqyoyQH1wntOn60Wapdd_+m$OFLlUGwckw`uGkwq zNY9Q!m0o0Ia>}ArinZ;ioJRX35J7pYquFrE!^Vi^{pJYe1g>V4s0T9kuj=UeiCE7| z>Xyd@!X|YLyE>1_$kpQW`kP*Qm?U?M^b$zC8D19nC?0q%J^g}Yx$=E3D=@`MpIO(X$v_C`DtpW5{xtmE~Z_3Q0( zElC=|Uzm6X)T$5wF9(;woa~JZv#pw)2RGt9H_D^dw^K5D=-&S@yTBCCZVFen?Zy5z zRQY_rQk8!IvYu&wd$yxr6yze$npKhQhN5We^=6jXr9VDS;hC>sghVFC2V7+Qi{Ng| zTfRbjHFWOHrOu7Zxy=_hV?`O%7!Z`7IAo;&3=P%vjJG_tx3KQCpaEF`iILc^GpZ{?w&_kg=>O*#XC1l-JuP)LBjf)I?6NgK z+Db5ZT=4(`I?~wOC^8^oGA1!HsX4~`Y53;YpbX*qm{gyxrIRG1>s^HdP_e@Z@^j=>KXG# zO(7*CmF8a}>a-`S^-q6+{w0EhlnL;A)A`fUvrXsk`1{_2()m00toPrZo_YQoQq+Im zfPeD-^zqE`-*7tR{lEYJB>g$bGmC$t=#=yi!~aSCa{|Av;(yq;X?~9$rE`wrtZ?r1 O;UED3zMj?qfPVqs5H1z~ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/3/2.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.1.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.1.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..846c94b6dfcfcd1ec41f45db09851b51ffde2827 GIT binary patch literal 40 ycmV+@0N4Lxa$^7h00IDTVJnCY{w!@*ns>b{_}jBt1mz`{<*(^n%0=x&D9qe3ClVn5 literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..38e6a75814585b33b6f9ec4db24f3a3e40829245 GIT binary patch literal 3610 zcmaJ^cQjn>`W+?6h#`6#!WcC=gGq!L(MI&qGlWD9(T3>Jg@`C2+K3)KM2j9Jdhadx z)aVi|2rl{k?oGaLt$X)b>#VcZ^X&KR_3ZbL{b;BV5K;q3NJs!8)!_z!KLEP;R@Xo) ztE*}A3td8D$zaxb3P$PEXOd=ZVx$*s-cb}x?lRcz!f6M0DH^NlB_m=-T;-yi^Gmf8 zA38s+$M|w^^R3pIOj(=Z>u3XGkG7g;3+v1yn8x7Z{s&)zcm{o z4>#$uEQvXC+_tp@YU*Mpd2w_X|us;IVBWNu%@1FXa;=N;(uFto-Le-lIlZ z>(d4sMxT;*$lbw*T^O1QV4UXwE~}9DByhWG?{_0_zG-Q%!9`sseeJ3!t*M*aBheW+~gUfBP8N7DJsSxw@&Wd)hob6r?$^ItC zH;)@HCTVw8`e+fSjb8ob$Iv3LL=|MVU_AJ#Fg*7&&mI`NNx6^+Vz43s^TC@B>L>U0);H_?!*DP@7yRn*;4PIP=myDl%5?KqlW42NMAXG$G&b_Th2-X%}9qw!tRIlC%v8_@CH}YH)cisclCLKE;D-?}oMcfHX%i|iO@oCx3e_s=zxa+xtoQ7DpzPC< z(-Sv8A;pAKMZ4l8o_6_Wk`C~>SeL%?gS1A9Qu3%8#rxf#I>1f7rmrY_2{_{b1G#IQ zm?b8i@gLBESm0R6>BvC84JjWWHh2`54bNKw0PZc^cyPNKywWWc?s(YpNR=(w>8xu+ z>gl*Dw6V5C+P9RDCb#58nEV!%GMthUuFC+NcJG3T6$r&CjE`4>zBMRuQb)s1ln8$) zu9Ftu2)~PvZKE~(W~XtpV#{czh2@=eNpw#oRsvhtY7x^sXcD+3Xj+MJ^{kvZfMB92 zJ_~ioaWI3_H`RrkO!;LCXDN3)>j*h3)#(KRv`+#Wj0*&~)W}2zm9)lT_i7918hJGO zKVE@W^)GrQvec`dk(pPOsOo9nB+%9py(8q@`SuN~5WU-?>?F*~lxEpKl#kkEzFe9p zDLiINo0=u>I1)ZFZxipTbw6k6vz(VPa8Vj*0FW zL7ElN{1j<2^GG>)*3O#E|}dwLdy=63snz|0UTq~Xh$;u@yft!$Ae@f z*hx-2uE5XIbH6{cp{##-+F$|pNomEipu5~%tVWA$+7FK{mG3qA0B<-M1%(6!#x zSJJ5@_06@1u%FYcA@3rqHX_ZQx{%XJo2q$8-we_)Dzf`9cqXQfZ{N>JL(c$mvnAoihg-lMoMjEz$Vq8nHJw|3AS__}|FsF%t z3U!y>Rcnz*tF>=0BI}+H+*292`PO`OXH7BuoIJ&_Vw>mu20dGRu+W0Pip1Q$lxi zFll!O`*vgW*J4UxIh%~AbjQ47Tr+qj&%Ref2v^!p)_wKfOJ-S%!?b=UKKSkKpg{^I z4UJL;Dr(zoGU-_Jw#qppkAz83KHX>zocQEZ#M)6)giHc!lWdfOnAPh#3a&}4E2Dz( zSx?UnX$+GByUxg+lB?=FZfclhXN>47NU#ZB7T_HZyql46UASE439AX1XuVIbY+xFy zjsaVpx;j>dOSxIQR~ZX@WNeA!kQ8)5o_`t~D_*c~INeMX%)+I0IE1ymSu}mKccnR5 zF=&RGeL%hn0dTXnAI#0!&eYqj*>^CAcik?J+S*Ia>>_{vL+?7Zf2%HB#=IN**HGn) zqe?lh0mxRC)x(9hetwX>Bwcnzj*|w_=Wcfzp#zGuvsCt_3MxqC+jxHls=s=i(H!OL zWOqUq6B*mLZ{#&G?8Nd@%2OhUCs^h`jksgTmYj!Vz=7ynuaJ^ovIUK-Nu}M)C*@4S zf53xpW3xGuij*~)>@&5wp6TYMh#eC1wvB2og$0L5Yd2jFQ7{5qq$GI)>tqJ!dpo^jcsicnKlkk>~4UtkwMaSU@$ zffGE;2nY-HO9<6W&y6Zdi;7Ap3M~oEEi#Yxt12=@8Ky@iAQCmT@}T(@!J-5N8My)8 zA@~H4478e!fsGm}Iz`)D*~Z<%G0)lO3BpX%OxZ{C9RXy4&LV}ImPmn-bU=whBOuYiDKSGO|1?uW355zS7% zo_)1KLZ@rk*U`JqDCga2N@r1LUT-Q=pvdlEY(bu1QG*HhUhw|k|9_JHoaCj&zfpWa`iJ5FB>y>qUsv%z?1$HWk6uIN3ejcZ%EiNs M2LOD%r~v@~0>8s1W&i*H literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/4/2.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_metadata/v2/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_metadata/v2/.0.crc new file mode 100644 index 0000000000000000000000000000000000000000..9850f2cdfc9b6f600a3f98f0741f8d2f5845c267 GIT binary patch literal 16 XcmYc;N@ieSU}8vE{AWS~=bi@uCkzGe literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_metadata/v2/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_metadata/v2/0 new file mode 100644 index 0000000000000..5c53036530462 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_metadata/v2/0 @@ -0,0 +1,2 @@ +v2 +{"operatorInfo":{"operatorId":0,"operatorName":"transformWithStateExec"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5,"stateSchemaFilePath":"file:/Users/anish.shrigondekar/spark/spark/target/tmp/spark-dcaeba6f-ff09-4f91-ba1b-4d14fe53cc9f/state/0/_stateSchema/default/0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0"}],"operatorPropertiesJson":"{\"timeMode\":\"NoTime\",\"outputMode\":\"Update\",\"stateVariables\":[{\"stateName\":\"countState\",\"stateVariableType\":\"ValueState\",\"ttlEnabled\":false}]}"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_stateSchema/default/.0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_stateSchema/default/.0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0.crc new file mode 100644 index 0000000000000000000000000000000000000000..3c16c8244a3b72687206dd01d2f80cef08136e0e GIT binary patch literal 12 TcmYc;N@ieSU}Et2e=Z0B6N>|) literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_stateSchema/default/0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-avro/state/0/_stateSchema/default/0_6b12d3c5-57e6-4001-8321-3ae63d6be7a0 new file mode 100644 index 0000000000000000000000000000000000000000..cd3e8f6d96bf532900124feb48fa5fe6ef2d384c GIT binary patch literal 284 zcmbV{F$=;#424ez2Y-cR?j|nXodgl;q&UQDD#i4!Tyw>8+<*6M2T^b_+v7{#%Tx1y zt(2OnrMBKkAt*HJa{w7yf*d+=UJGPU)x>27c|r!87wVBcCGy^^%`Iu;neobLCEZd6 z4GESg2ziL{632kk9|3H?>?`E%hyNzPorbZ{)@WOjJ5GJ}_TZ?hAG=Rlw4`BrZ4p*! F;uE=USz!PG literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/commits/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/commits/.0.crc new file mode 100644 index 0000000000000000000000000000000000000000..dd09db7ad216c6b594c83f8e23549103549cccb9 GIT binary patch literal 12 TcmYc;N@ieSU}Bir|FsV-D%Kb_q0S>=0me%#(1$?{ZnyR`w2hW<06u9Dga7~l literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..4ad793ade578292d6ac2952f82e1794fc259ba24 GIT binary patch literal 12 TcmYc;N@ieSU}CuVdm1YM6y^h> literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..21cc1e7055f479ef509200c08d83cf3c9e368b83 GIT binary patch literal 88 zcmeZ?GI7euPtI1=V_;yg0^-&c)@&0Pn97V8fgEX824*G(25yF60U$3?fWZ++39>OT TFfmL3$%BCp)Cf_S0Eh+vNwx~b literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..2c6fbb713b4365d8f96a5ce2229992687dc3d6d0 GIT binary patch literal 3785 zcmaKv2T&93_QpdIFi7u0KnWeC6CjEqp@W1Lx&c9{(g_Fwl#XIV1ZhH$E?qj(i%1da zJv60CM`;2A7ruYd>&(5+&hG5ad!Bc8&-v}lIXapoq>KOx3JO4PHO>U^hp5l)wRKSH z+DQHDV&{@sFq!d~fY3kcwMetKFf)p_?kozU@f`T<$>#+1EE=uup(5jkU*xC%5}4{F zJ$QQDfDYgh5MHjgn6S4ZHeP$-(l`~`nCvh6%`5D5evKuGYeHsUa>CoGyI?4`2D`hE zFKX*jn8fr+(@)^%c0qYj+@leQ(5b@+N~6+Y5tTdnc_I;(lZcJ;RAet z8EVmGTM~2Nw&`dK)HOu=>NxG~5Q$Y%AJcLTvh}UYuevD0E-0BJ_!!x)OZKUpIF^>P~1RK;Duhw zJ&Dz?zF?!xz;WRn*5CtX`NrGQxtZ?%D|-F+c`<3Y(UaD&{qK~TAF18;n5V^RTG7XO zJIuE(QK+FHXwq>^Gi*Gk{V{Ynx4(l@7HdfgF6OgX2ojeou+uly9k0(B%E&9_dW1bP zuRA&!-JGFXISQVe2gBZ!=K=fao(K-!xQ9-=^P*Olml`N_wRd9(o@8Y#3^zo?TeZ-0 zq)&_bLqg{R=YUh3;oGM_H}G2x=}M*9c5KJC;?%|iZNmEk!o>PYwtbXKKYHX^HHY%3 zU8xTDfol9za?*F-o!*vVz$s@vXxeE%QV>YB2%E8mBkqQ)Z*gR5bTF9<$Acc1E*hDs z)9_D%W!q_fMy#u5U5*6$wqvB(;R1J^f=T_HX$FYQpV+=x^b4SfbfravW6}-AP-CVY zH;tT*8Z-Gmk#S&I^~!sXAl|PE32z1^{SCt^95ER}PUfj#>-0~R9Uw)=og9kYb)N6o zE6)hbTT8vn@Z)B`zKTQMBEQ#~h-}dW&_i)(?nm&IxNc(zA;TFlnHWVUxhH_PyO2j0 zuWFrrM50$qCaV(MI z>f29$C@+H}Rd?R+5nI1&=XyIVt--Ha%YArlR(_sQ&p;jw(l?~KOrdsPr3dK}v-oF|q`!9F41)<;i@jwerMb*oTtIg?abatY4PG%{3?G{@5$?q;6SOj)KzrdnOdR`#oUv|~vM zpgT1&_cOjwWsl;GdV?9gtA=t;H9$|V3GR)c23ts{;O^>jxXavjx}E(oIWBT+4tvEc zv3T#On=9l&l#zK4#z}WopK)slfS?(=0RdwdSu;YK-*5_Q46R2ctrpr z{}@tPQYH#C_*C@6LPxQ-jFY#%vi_F(Yq!VDtIAT@ZX@}{PzXkq$%{9gFY%X36-HKl zDfOZ_oy^J_VTVF9e+{{oHPI<=Z(#7<{G?>N@>J^mdvffTS*#j z^|jLsb-)1E&03Vcys3eU))ihH`qV6{U9%PLSgi$W7Q7_wu|r*#bm(!YQZ$+sd3$Wr zxM#`CDJoj}QBVV1rp?Lyqk~UXjhRc_Lm9KU1vRzGv6IdP&LetEu#RbLq0F;`^>u7) z`touBGyV&M+|?{~YP%?+z?2+ejPK3#v1l?DeNq+nNsw4)A+oYzDqvJ*wR;z<;gXh| z{(8L$AtjzF*%c@Auqz;wvR~NKzU)OHxGhSOUQo*{A@I(`9-i=uwX*y<;DiSR6sUFQ zkRf;w-({iZgkoeTqe24LPCC~rH_WD6Ip4~cM0ZzVWH5zoHZdOtEJ8L!EvwL8zE#tEV01L? zN3l*t9u8{lb#3t$%j*h-GxS@&^`v}N+N`1gX6N8W^8yimB$dQ~n%)@Xc3lBWv!G7j z`wP6)ee-^=IUBT2sI03?w2XAGk?8A5-V*cp^d_E5jMaNyfdKKdWLgY*D$HmxTOm)D zgp1kGXXMN~jDn8OIwW}M-3c9yq-%_8f;@ zbpDXN{%Pq0AjCq;oH?7y4-t&N2!1)aiSAGch9RDTX1RG*Cg&Hx-IE^E?3~(*G11+_ z2&)37UlJ`=*p#Ek9YU`mr+J3gWb!Av7o8g#%nGwB!YwvOs`6KphP$Dk4-)u2Q#3T* z9tI9aKTN&qu#y-V-l9=HiV?Yf)g&l6bI$4tg13B6y-;i49KbU=iE^_7kT3PWa@$Kr zfZP?O;|c<8Ep8d}wn31#?OYW;FCJw_YmvM=K>)QLqV4W&eFf!0rdWZeOYi)|#V++M zzmQL*X!uaK5BW9875+A=dM(Q8p(hQCyd@GVe=StUtjMW!;6!R`GSo28;zJHXf<=3C z$u`ROeceymgB|6@=CnC->E~MyB!_-zqhmKRR5JO9nwhTlk>gv7UC^-pXgy+~{hSsO z1`Q=vFTHs(y|&(+$gJCspuXzB^*7ebTPv!#Q<@ah%FWB4iuT|-Cp&@44X)x&Sx3Pfs!9JMXnsQM8XoeS`=QoO4+@tr{yPLye_Ml zpLBO`QO2;Va2pKYD!Hh=<&8uqd!QwksYP3$<-z_5K&6ZfI`ImaGnWO3Y_->@ynm8c z8x68M_HwJj$$8uRRGW*uzuX$fBP;5OIBgmjEuM2}JYIh-nuSm6bd6|_pSO(Px$q%b zHFTPhyI;8)4)C^j8OY7q%rx4r-E}od@Y<|+xv`U)*+uj2rx6`vP@5qXX8jrSb+GE$ zL6stZKX@a{?%rH`-*sviS(fa|9CsbEkDq;*#P(=UPExrSDjC30ZxVvo7`}FUpm-`) zscwbOzrNhDc_pufZ7cRVy)r$Ve4KN(Y1ju%wcs%b1BIY&ygl;6e+lQ_jf)Fn*WQWTk`#Tx zyFL7vNJo>LLV&~1eh&x$G=KmAo&VS|p9KK^>6E2J99?Wl&LYVv&Kqjt^UIn3vy=R6 zRnb6CPZ^2Y_T1UVcu-OIO9P1XMgG~NrYdZQ<}8BtzwU7!p#?{(t0?QE{-%suq(l?Z6G1@hQ?Xm9NLFbauApFauVLn6 z8(Eq+8{0UW?3m(q>GqGVZEw;Zi87_c;hZ(J``*%v{R6h+U$>vhvduUS<}CqgBV(S0 zhSL3$6LN=N_eRgYn33qe<@G%MPSzO9`<56>gXbl{7BNQUhp@Ts=jBf;N-_{|rTEC% zD5-b!Y6A$plDLyvD(JVVX`gUku5kzAO~ouKtiU^lgd#e=6c$T54Yhkev2It0h7S+H zr~aKt=a;2q7ny;bDiz6UD*6cPPe2imFu7wx@8gzZ)>%^(!8a}!-#4eY;;sZ{O9p$! z787u`{riNev}ER+?{&!O;`$wtO%Y39+i&JQnK?(LYphNZfA+%NXQz%P5iuj+_h9xX zfainRzw6(}w7FRp)^{;bP6k-t@ZX8McI kzs-MU;GaL||J|*Zeiu?l^8(rV!iBSgJ^%m! literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/2.changelog new file mode 100644 index 0000000000000000000000000000000000000000..2375a971fdc202d2f24469613cfc1d17765b315d GIT binary patch literal 88 zcmeZ?GI7euPtI1=V_;yg0^;M$dwC}?FqIiG0y)yG49rXn4BQOC0zh7(0D~it5@cgw TU}Bg66o&vGs1c$t0T2xUQ|b!r literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/SSTs/.000008-9b6e23ce-e7de-4df8-b320-2b0378b53e52.sst.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/SSTs/.000008-9b6e23ce-e7de-4df8-b320-2b0378b53e52.sst.crc new file mode 100644 index 0000000000000000000000000000000000000000..d72d0acb543e10ca353a5964af5a7d205cecc784 GIT binary patch literal 20 bcmYc;N@ieSU}D&CQ}#LU%^8`$TfJ=nL~RGQ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/SSTs/000008-9b6e23ce-e7de-4df8-b320-2b0378b53e52.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/0/SSTs/000008-9b6e23ce-e7de-4df8-b320-2b0378b53e52.sst new file mode 100644 index 0000000000000000000000000000000000000000..bed4218de7ece456cf6b12d98567f003c66b1ef2 GIT binary patch literal 1228 zcmah}UyEBs6rb76nw#u4n{3)z)}n#7D2o}AO|sj#EVip#S6Jz4DYRQkn0xQJ*|BqH zt}~NOQhex(;G?48NAL>>f*=Tf0N(_C^b7bR_@kYj76OcmwPo6B~vKI^08-)O+CtgFz z8y>9H@B*FyUH>JQ@b=H$@U7SP$tynO*UyFg0`a{c0{wpZ;~u$MFgki<6yA6RB5E;C{--ssgP49 zVsf!iR6K;S?ms-N?e#csZFSnssNZQd zyKxB3>s`*9TbupAj%t%~$4pGCnM=%rM@pT%@O5j*wQC#UL?9P?3%yra_H*C&|qauoV_QQJMx7ngS}A&cJXL zTT)*$q`DCy52O;rGt(D*5ixr$3Qv3kl-S zYGKdTc5j5+DAb-G+O2#sH}=Wee5*5)XM%nBhJIW0P#!e@oc#9M!^OhS!EN`dSHIva Qq5bb){`&0un}2-%AILCg>Hq)$ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.1.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.1.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..2e4c44069438a2d4b71cf06cb16ef0776d7f78e7 GIT binary patch literal 40 wcmYc;N@ieSU}8A@>V-D%Kb_q0S>=0me%#(1$?{aSSeLU&xO_*2i^iEd06*jq)&Kwi literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..57fb3ea9d4a6746f0f0f17b5a71113616aabbb3e GIT binary patch literal 12 TcmYc;N@ieSU}E^|weL0n6sZIo literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/1/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..85388bed9dc404f094366148b859254e10b0cecc GIT binary patch literal 3610 zcmaJ^cQ9Oi8(kvEiY0nmqO2O7#U_FkJ^HGfwMf(`!4fTrE<{AMXsZ)FM2lXd_nsw) z8eO6V!AHK5yzk8WojY^q&OFcg-80Ym{c#>GwMzt40AgZdKz~)35#SGi&c8La5GtA) zy8J>HkXSUDahZftJncuP*`SdIkrtgrL1eB&J+7SgFxR5-syF4sO1cI`pKCIlkfgE60YCbCRQ9EeW3+xHfX#8i9kE#?ba zITj{Tcd2=E|JW%gD~f$O2IW1o9Yd&A*e)S*r+OyiWpd&@V_oGLOBgB2RvOA?B&)M0 zLLR+L1+xj5*km}(=g&OTEBZLsVA5-OvfsYVQ&WSxhnn}2oDSMKnVYlq9d_KD308oi zyRAy14xP5_tbp43C@(Gh{XP7LRfUZ4r3@OKaOQ29!BrVVMbeBtjfC2|P2F%6ZB3Lx zp>;InM(2VNPrkm{o{Ijm+Kzj!J)&5^WXy8~iRb9GreQuF1Hqw6O8h)}VuLBx)Z)PWv>|Le;IPle}G; z+gFK|5l|GtCrp#fr=0d@u+iMX4iZUEGXnNvPV}OmuvCGyuCex~x}1@Wy!R|mJ&#Rl zk59+9W=L0${pT0hWm3xWfP>eb^9BHZcH&ZaB zPYXVQ1}^x{13xnd@0|VE#BSH8-+7;HO@Cq~3^E*Q<2&Hy!`HoIH9)fby-%uDZ6pu$ zi1g?HsLVwwC2soe?5+f5m_pXW#@+T~S?*MH(2SKF+%#Ban<-PZgW81e4fxsnB?F`i z8P^oMWINf9&<&+5hH#))J4T#Qj@#7UpTOIJYzWWfxmC)Nw-0gnBXW2!CS7j=F=5y@N>K=Z+7zY>;)!~$_z^_&_$wRbPYHHkw4ATocBE`aj^FC{)_T@ZM#D^6 z(A|*l3M=g4O1mHS@h#r9v%DP@SLITwVLQ4pC%r(WqbJP{*3~CvAXa{$`uau)w%T~A zuxOc&jV9BL#k3&L>XD-0M3~h0Fp2Dl5Mi8opIe*sI%U9O*b(eLm6@^5w``8|4jZpF zMGJ&p)<;+3j_1$kv?~#EQbwtgL?XMUaeWA$#0Rz_AG@`*%Q z@V)A&2N|=Z+2dG)et#Zl7*9}C<27KtRmf%;n zhC;YbIt(H1!X`J1cL|ag7V$#wlhf|>?pc<9Ko#}z(SenLhF^s#>hKLTv|IY(1X5o< zIe}J`lnMg%x{AJ|wd89`nIUx*b+=U#oSxCFDTrk|jpY}^pco};H%K~X{7=P7jHJqP z>J{N@607UDU1GKTb&d6`$xdk-J-u%xrzJZTH%bpjzjMBaZE4&ger7vjK|#OkdC)(K z5kFcK%u5e=GUe`Vt2EbKZw3I^4%oiz*OXk<)S})b# z9%v^U>5u{NBb)Ecrc}!b))rnF>1q{~|KM?{K0^{bbYXtQ_zXzNi~jdYBCEP;$&R93E-IPF|yKBmC^?!Ol`jv1X}&tRwJzlXLhmUboUGA_-}A2^1Npz(Spc8Wr`Qea0o$diOk49n+H1 z6E+&*V#2AS-LVpnyL~cA2KiiVN?-Z1w?&9j@Teo>eD6)}V{y@R6=g30ryO7)ca1ZX z1kMHDlorGc!$?j=1o*B?Jq2Qd#<1D2yhQ-u?)-JrTh-v@9-%O&gVqP?tVzzN-J?>E zC)6QLwI$L%r3BQuB`-o1HYrtL6cjLhdf=2tH&m=ZC{}S|q7wA2L79^(5_+sm@Iz^h zr1*N+9XZ){8sl&FS~n^-k<+crZ>38jdn+*#n8G&8sFopgz^0&CCCbgKa(bT~6-oY4 zs8gPU38cB9Dco$vFIzZ6vF%kyz*(tDCkUW%@NY0F5a7}v6&X_2nSkD{Eud}U(Hi)0 z8B#T{;GMu+uYO8uQB|UDpnc<#u8!z!A(yU{H!MPQ?hCRwsJ9vQl3xfP6?(2*nlLde zYEzerIqxU}_G!*G&Q0fD;CMJQ`H4k`u&0A$!=rfak;?pBxs!Um0TTD0cKk11hIuYI zv}A8|Ew=yy(CQ{M*`(fZf9w_Z*Hc@l4q1O0_zUnH8^`L@!XkU`l*=?Dv*uD%WbY{4 zynyGDy!{ke0XrPYS}nOfS=XKFDY~0{5l*`wy#pCeqaLN7@tBonFENH2cw<#li*-y zdGXi+Un}%&Lr5D`!>XO7+~d{L>`3)X?@r-B^~Wgd``cf@xzNvaz_aCd-oipx`&M2_ zrxMq<)E+>8PO${Pji_3WFn{byMk{Tm;VFG1PzzaP|9oAGS)^w?4wO$jmIEU?ZVX>NYz~)+XK9vw{YcC@x{jO+hjC+2G&$|< z-6n}*RAkc|y%L;!^-@h!~ySQWLy(2mvLZ0g9+FA4ax?mAetz! z^@*EPWtfz^jYpMNE;GqcMF_6I_w85 z=oTiMBe6(DtJyJAm+P5+ZnD?`5pVmL_F`yIutd(ptn+Lz+g@y}AEV}O__nCvE6C31 zGdwLdB4TbPZ<~D}08kGG0JQ!wF`qX8ekElw0Xs*lOXrZ(I5M4yA*-)( z;%ln8S+repLY!@SqOFZ-`a|;w%Yuhy<_@`uaj~`zHc1K5(dIUdd4*qk8XK)wO=r>? z8#}mTqKsn`q9QYw5k`^5$wo=5Nh!z(!)U`u^vFtM$B)2xuFyc=*CD=vUtWddG@*ow z8ibZm8p2DXIf%pRCw%=knX^7MHH1f05iRQUDZhlY=8ck%V^ zX%rGTU&Ta6?zpJB>`YO(h`R7Tp(Fu{>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..03b4665cea1450bf8e048192cba78499dbbb47b6 GIT binary patch literal 40 wcmYc;N@ieSU}8A@>V-D%Kb_q0S>=0me%#(1$?{aylB>l%HNAhii^iEd06=>Y0RR91 literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..bf6c8277bc195cec5ae3f8f6c412b9f5eb3c472f GIT binary patch literal 3610 zcmaJ^cQjn>`W+?6V2Iv^Fh=iXFo`fDdh~8GghZ4O!4NeuY9b;^2w|d&9->9>y?0@P zPmM0og5Z+h@80D5*1C6}waz+gJUm0!&_yeHxZyjAj zbscR3VX+IyESk-@O~GiM^joIdTcS*&Z8{4>sN9Eo+EW}JI37k2M8ETX}EE(qDd!`Z;vJ<@H+?801S!qgF>Pu&2 ztFk7ZKYE!0<`S{6Pj{Kmn|W$f*g98l(R=@7zkQp(x_a_9dfrENI%MZ$Zq5Na?7TS> zstmX6wk?i1blGyW1?m~2eRQ4n_Xr|WDubS{|j=NVh=sT(hA?s(=n85W6HjCrr12%LS^w5`Wup}3T8hMm!k-uvKw zuQkrs#5;4ovXhZf1@9Hgg5L@BkL|creqCK!eCt~ZH-8D%POP<~!ilcObk?jH)|75| zq)SeQ@7CTiP8IOd5B7AO6#OeAbaSoue#z~8OK*5=J4hCFR`I3mvxJ%vbts4MS+L(2 zN`DP+wvK8@+NSaXA9SPX%Ykt|gSgBBf#aa9%H6}%2h=uHGYaY(%9Mg;Ox+z+K(RH4 z;`b>Crxrd*w2}056r*|sU&b3GYid$SAy};8VCxVf;!pi7FqmEA11xW9ZO=hz(~t}& z5GD64)}ZpzeH~i1OCP+AnlUTa-hZ5*=^ebPKX?y{Nev%AZ3#K}My}Zka@l8?7OQGO zpXBZ`+@K&+GlZduKVupdK4-T-g^%V8c96?@TMi76)zql+fMZ(Y(q7ZB?9Qvj*(_X@ZE6=B=&tsHAGnxBQwD;yL|{F(qZK={16IGc)NzKf znr1M9Zax35yaI_(-TkyrX!D_+nCf-wh!KJ|mhNtWK`%heEN)KvGX6M%!*@#SBH<2hE@Moqd zCah`+gYuqqm_j|pEv^;q62DtmBnZ1p&A8LMcfafty0A}>39JS*{VGmVOK764*W4E` zlJe5;1Xf;LA__F>D*SG#t5jXW4y`S(y`lcrmiA4urw$`FG|ao+ z2mPZM>7zx_+_WG+yr-{&>ReNu6#!tLt!Q<|WE~_vs|R6rpbMhT6oz|QFVdP9&#xg` zi0W<(v{Q|AC;)^}O*dzgs}Q0!1(!#<8bx)gwnH5&wZM)16w+?Hpqj)ZwbbqJX@C)ZX7uZk*^bKGMYRNSJPTKU9j=OX(t4JJ_6JhnjQ#o@+= zcWm0qia!HxmR624QypX%Nq~KqEr{{Co;DFp!e~IO!ioorbrxus*M0FHms#uG^VV=q zO-g&a(SVc^Pm%17lX=|jpFut-=x$%~3JYnAl%(O;Ld9cmPwnF-Uo(}Lz66|dgMob2 zu52=sZiIIjLF{mhEIu*_yDs+xhzS|PWx;b70f1ZcSMOY}0x$Q7g}WTI+|%Mray{)H zm3us)1#PS;miI3qrq3yU5vIIJs}853fg3Xe@m}39seG|Gm5GT8(6@Ru9=d4Qu^RCY z)iv^>tKl~hitP;M-<))X$~RHdE$r{*i=%rhFfy2ew)-*7LzY3CqE;1X51)$ZeF!?5 zx>c-GiJJ|iv!Ns2WF@RvFhjHLQ%lTKp~EB!V0aiy;t4%2}q^3UsyH%6V*vPLt z@aYn?a$v#tEqk5TDWy$iv6hLR5RrktT)ENzsd;WA3e(^Hg zd+A|w)<)NIGa$%P%Yq?`(ia(sy9|lKZ=pLB0~L@jz;j&OtN4XQNH5-Pnw4E=DJHsi z6ltAL|5KvL+WXz{(++{Ru(MoaA(_0X-ld0ib*O@@vQW#dv5LI4#L-@O&tW{T`#TMd z_ea>#=*KCX4yy?fp-md4;}{WPPP2fdjCt#;NNDN4dV$t~1%P`TZ|GtTAYC4O?XsVQ z1iLCp$K_*fEpM1Y+hE$Z?HpxZubyN@YY}}oMFO=RqwVf){{_y0ePIHgEr0M87o+G~ zc_p7hR@Yo}0Q-sO2z?(}xgKf#*qw?|-b&kBUMN@>Rp|6_=u~POA8d@ZY|chXFzReA z+eZ3)s`){ExT{>>m^x1?{c`((BR%H;DXyi+8LcoFtxqU$kljQ? zt8tUbLw|upzpZ~aBJFJuKQ5rE5?U5A1kmlVCA3QLh?0KYHcQ(`xa z(5W|v`gdYXS7Sx6wf`_PC zb#+UaY3UrZDCJ{q+AC&J-U*YSJjStXIO%ai#Oh&FghB#GlVa2(DZAIT)Vz}z4;B@R z)85`~@)%YXE~C*K#g}!qJ+;wEZfMCBkZ2RUG|(>|cr!iyig=mALk>$Y$y&cj=^!4e zg9h83c(_!A%X!*+Ra%IAVrhxvmKAkJo;3`O7tK4@pKQDp&BUd4J_>7pvtadR_fm6` zYVb52*PwDG0^n)yJd~5Ym0_||z4yp0-eapQYI8Rwqnql(50fi&0d2-`1)CnsU&9qI z4l9&+2O*o8c6aC72ZTY+vW!{f*{-@Itvz1!V*Avmrzu>E<+PB<yd5h{qfCP!{meesFzSV-Tyz^&00$Xfdxez!k|m0=CztoMnN+e2 z{{auVj>+OqEL7KRa?UW|eQKQZPU?VEpnXhlF)SohCVOJmbvBf1FD@>CRcAM1TT=8D zbZ7J_fvzSg86TUk{XP%?r~?B4y8ksXpEm%0C1oiQM`v52^G;H-i$G0y@jAzUXOf;* zm5lWDm9-6b+;?{{Zj_)wX#jzN$geZ1DnfRt&O508=NT6rS_o}*6=eg%{|oHW*OrcD z)Nt71K}2iQLF!Y%HvSkEF{;-k z*t@4)K!@x>HHmk-+Od5|IWSW{kNwVp8tjf`JXr7 zpS(YPym0(C9M5_G@BcqZe@^nk;@>DbC;h|lf0F;4z^|+LANE~}-=o*nyhL(QxOD!o N5dZ*R&uaj{zW~p}Gcy1H literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/2/2.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.1.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.1.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..a28994f96defcdd8fef1e229a979ede42e1e23a0 GIT binary patch literal 40 ycmV+@0N4Lxa$^7h00IER>gX#C{w!@*ns>b{_}jBt1mz{Rz^8RtcfDFfD9qeY$P#J* literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/3/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..f4b734fa36955d8b77d70ce188f8073013b0512f GIT binary patch literal 3610 zcmaJ^c{G%N`yNYZWXLXt7-P-OU`ou$G9%jX;#J?A{`zw6)3_qS*WlQ6kl;4$^^m6Vl9!jk9 z*`6 z`gDXE+_KZIJYmmu!^sY)Z-n#JbKcn|zf)VnnNo2JNUYJkytAv|92J58N+tU*yRukIau3+C* zu6gFjOhl9FZyIHZcw@r6+$Kc-%8giG>9(!7QEcf0PizG#m><`Csv4KpG-Qb25jzg^ zA2G+QOqs5lzf0ew_Xh8F;#jJINxuDryb_Uv(2ZKsUe+B38~SNw)S4Qds407A8$D2R z#j*TFCc?QzNCsyp_c@LY6(f}U49S_5UQr5`e7C!?2N4fsco-7KjeZF$nq1j-R9(l? z5Jch>A0`^qp0U+s;yUxv$M8LV>GF$Lv(w%E7cu>}q4=!m(W92|-EXv7A3&}u};`*$iDU}-=5>pP6}i)@KJPENR-^*nq435;`biK z7Of#7=pNnvE>KgDPEqdW%j4_vOwnq2cixd&4^)IQEyJhn5XhSms7=rizx zR}04GD0;yuh(asWbGUtiTewA3$E3z;vG48AV~FZ=+xnAB|oo5;C~oJxs3FS8>= zus=gQ|4hRs|9}GXhDt&28LTPV03@97Q2hgl_5N2rrl0bVAWKA$=VrW0TS4&h_m&3U zC{B}HHqiCR?`lhs7!A_v9detOtvoM=<iJU5GpcK6gJ#-bHR2WeG?B&m|EdptiXtulqqNAJULFkKJRJXwz=6&#>X?yr^M!yD&P&Ca{pqA!t&!WPX%5goW63i!Hq-9ZY2oC> zFpQ2@k(HHpa_1pUs3S1O9 z9Cymt5(!?hSC*-RX=8|Y#woWo9`T`t!0;UXz`(Jy>^Y%L&$)%sLu;{VtH#omeSsY3 z%E{|mqM(9@Z6;7JDT~WxB+87rd9tWm3~XE7+qPA&aiu+?>|jlx$yX`HdU9hO{r5e| z;+c>A4`J2i6%s(hj?(Xzda7?KxS{pc^;c1;t`AsO)MWErM~ceeFuVqfCp23i<)?ZL zUIDe3c~o$9yku!FmjN5 zc6*2Ma{KcV#OzT2DK9@qjhW^KD*(WuK*j2q-8xk2i#~+IkvWthPYmvDJZ@%{Bc^yLHZOPf8jwT7+y3GiuT5WKX@Hshm66kH(zWcs<>1bZe z^|1|;o<(!#*m${s;0A>JM`yPWj@~tO<}OM1<;|1kH8rcpj@swB4;b+wdS;0w@{jh` z)_fAPmzDxq3165LdGk;p`&hD|i~>=-@0IMacq%pnN_EaDuw;9QPIbelz)|^??rk5m zOICVz>e@S`tW>5$4YcUa12wc1cFN?zY_4=1q4y z>KsO^}jhX_?@vs9; z${!jlv}G5guOU=gSs_i|H~-wi{X(fczPko5k1zRXoA7?XGIU+Sss`ujTl0Ab zf{SPPAla_U&jr$5)0Jwr5>qLeX593xrxd8sWtRZ3-VJHAC>9sgp_3la#EikNzb$5K z64vW`eFj?FH|Lki-JpF$XH#3QZLEKZ!T=+4RnooV`7<6#cCR^=37DT1%R+FZD6{2E zl@e82bi%p;GdFQR7Ct`XnCywU5jGmb&2VVbCgpQiq48dd&`?cLA>y#Xu#eX3r<2&D z$I(6uci-o)bu7LIgj#A_u;$bGAwvjfA#qb1xHgp#W#l993?KjU)Z9Fzd&>PYC%5iG zLVWiy(z=-Cr*yNmPsYK6Hjz}=G12Ife9>h0!rg`j^OF3k2+NI;nxd7o;cj@>Ub3J^ z1{(chKWI4qekQNua!O1@GrDpVFD}Mw8l0XxYkd(3t=vJCXzyA8_(!L(uGRqR#r~(R zJLyQUo2pz=agd$mRTJn(n2ud5PnGwRfc$uEikC-7p!R*7{jJTvz=g0+?7-v2mwr-` z=X#c&C}q+#ynnk3`#HrE@glZ%HP-sR2R)mTm5z_nr7%77Qs-9#N3xq!VMalg?+cL9 zY`PnZcCo&%-~M3OBdIkuWzAB{J>I+{GxS{-m$;szo-07s#KPN0O=u}|!72C0W5`8z z3z{jI(AU^KF>_RykG-Uryz2pQUlefdxy{n%vPSeVeTG@}#)Xd3T~{mGq^xx0yM~#c zv!8>IeYR<`o)an%(!>PM&8TlPGEX@r2{8hKkR`BmFro0vDiq|jpyCZxUWly%_Ux0C+m599O9!5@v_$~Im2 zzXod_?bWCX_Cwb5>~GDs_KAU96xi~s3*7XmK6H7rNbWEk9cA*(S2ICkpC<=%F#Xl- zj^(dfp}QI}mwKUX<07${V>3~VQH>EnJr!FaG zQ%2$mbZhtlnVuFkjS!cg!wwJtXaEBMdjB;spELk|C1qK0Cl@=4lTK=y(?Csr`Z~dX zXHuV3RShv1H684h2WboMP6z6j1CSYr|2m_tCY(fn(!uaQ&p7SSM(Ci_)eNxz7ue;} zwsQ`%;S{%WLeQ93aY6dTl(vdcPavdPA!j&6E3O_m!Y7@ zDGc$CAg6%jV09c#9d*$083r~eM{iqKqI!*Zo>XcvYn@kh3APN{Qs1EIvr;lywz#@3ngasDk^4J zzeBWpTc?E5?HvAT{FXb~eQS!*UB+F+pNSSIy)}?nOblwy^M*D-BcX-plN<#{*G97R zXo@Hqs5Ji)QIDRi(>?ix`7aS<8a0Jhk{Y%1%iCF#Mn7KPT|(D*lIk>)h|r>uH^#IxU007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.1.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.1.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.1.zip.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.1.zip.crc new file mode 100644 index 0000000000000000000000000000000000000000..80ad867da34f3016593e98b888fa8c2de829e7f0 GIT binary patch literal 40 wcmYc;N@ieSU}8A@>V-D%Kb_q0S>=0me%#(1$?{Z{E!vzvNAs-Eq+|U*06bj~*8l(j literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.2.changelog.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/.2.changelog.crc new file mode 100644 index 0000000000000000000000000000000000000000..22e87bcdbe20116deb642decc0401d73bcd2be9f GIT binary patch literal 12 TcmYc;N@ieSU}9Ly>}&!65cdLN literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/1.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/1.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/1.zip new file mode 100644 index 0000000000000000000000000000000000000000..e91c9a67416139d90e2e7c7bce0c57b4315d36d7 GIT binary patch literal 3609 zcmaJ^c{J2-`yLV+&5(T?VvIG6Z7?N_ee7e6hA3GgVaQh56_RWbQ}*nUE&G~%-$VFX zv#S||@Y3)1zV-c{^S;+}&T~HJ+}HIy_vgNzKki#k^BfsHfRd6D&{r981MmmHr{B7I zh8ns$2Ey<&NH5-)ahpWYKJK$jwYM}kiMHt|454-(>~`mMLb?}@RrZ3&xfL()(a!p( zI7tni9Mxg{c=&}@YAq-2tw~MSUpUu)39e6mB=glH8>{ zBU6<%{_Mf?6bQG7g?+lqeBR6xqry*f^%gyMk9OO(1*)qjZer(sWTrz1M{{!y{zJ~2 zGodO-%P!mEn0=QmM_Zu2G1f=VX?KV8US$DWVhO8G2a01`eqdGJusnIjiBVc}-M)6H zlBqhzq`)qgZlh!2hCrUN)sBYovL?YZ$H}lr#A4KY#hk?1XHCa?EDnZG>0%^|Z1miP z^m(mu#gY-`d}Su0UKYHQFAIJn*f&aWsrM~xR_>ep2zJ?6bJK5ltwk@JaBa zQFF}N)QxrXhU9H(FUVdOmZ2ON?=yhUED$^l+Nvb(r{1Hnp`MY~*iZop-C*u&rv}2; z9E#tiC_1(9OJI$pre88?MDl06Lb0VLmlQ(a4SQSrP!T_xr-8v7==X@c$+aB^rAdOr*Q_A=kMy$v7dkMZPbigx%%$I{7lclWz4`G7%nwp?6@Ul?;DloC$P&d<21ag z1$&fBWW0WnQr!@NCHstPl>eOF_5?YSGtf>Y<84I-E#kFY3=osew=*!)|6H3roSyrE z^Rf4#Ma|*y*wzea^)PUL0V@BtG#5B<>8Zfb)!W$An=h*MVPK#nSKsEaVv@C~kfJf_ zm30#>d)l<qUIVf1P=PB4p|~`o zal>)5_G>0ihxHk}o#gD^%$U-?Lxg8+!Nl7kiATmEWsbOXK_`n8s7+dDc{@bOk(f=n zv%&Mt`*QdM3&K>{Vf&!J+`A2+Oq>zzqizJ zg|V4tFoCt6eOFn5Mye4%?vmQPZ{vJ7B8BEttL8qqGAFk{k1>*iLJW*Std#0^&@Zn% z!&jM26%;NDaWiIkaNf$#wSAx}IvycAHbkW`3@1;p?)7MuTc-qbXwTB zFf`*y3Xq#sF?Z5uL0MyXlfFQD&x+yf6Ex)66@lGR!=Wa!FN&@jvJtG@cKRIyG1<_db*DsD*Q*`60-3eLuhP3(V<3jc+*w(8C*1N$V!!rTu;^ z7mG=&TEbv_NA0FCPce(DMMSbU3yUOSw`rINJv(>HK4J@dg_t4gK+~^cw6&xrI{MAM z2_h-a9~~jei%Ud-MxBM-}xi!|n0_Ve=;C*|$}SqBR8JPD#{qSU(ydOepB%g@s?43^A4;3YFRH7Tj~{m|avaj)0`<(|3Z%pL zH#WTE(pFad81b`ovRs)OV7n+1|2Nq}IG<~2fW!yYG2;lWntY~4 z0lof@=V6uo3%;*8>a>qRHkHNNCi+*-8DJ!?!`(XHzT$*4doCzUAbhPDmI9s$(Ob@y z$&n{T#B3VSbL1XGAwSPKBzRzM29HH@&>Y#ci+SIdseh2jKU|TQqj*$j)KBI4(@{A5 zd4%`U{pPHV&gEu6kfpW-V;0C46^Oq8eL1y-ZC41CM}wJcv;!W1xZw{Zd3%*92O6(i^X+>Q!{2qWahvI?8N4Ko3|##b_?9APto%SUhE#Ab zyXhPqTAA4cvp|k%?IT8JYCT5!XaG0f+N^A^szsbozcDM{V(l#4bFrd|PfbQO)Xn{z zpY})f-%XPAm{5QcCnh*=g?*coh-HK0BYF9tD-iJje9r7T4D7h1;02RkiYfy<`Vuh- zzixz0RUYgk#F(zelpwMuxtLY4+TN&qUSr(Y5)lfTbt2_awWGbtUI zg6U!*c1IpA6%n$Y_Fk11A|F{>;(265-BBkEgJVVW&h98~4{x zMfiS&65jxHGt=(&d|SUT*ja`tt32CPkNi`&7Xy5k=J+^;d$F7j8ud0IfQ9a_9yddt zvNh24(1q8m?OT^~n^?BvglSc16)8S*%r%U7VL^*-L-LRy!>cb)GGDSp&F!h=JZ&bF zEF*p(L$2Ync#;Y=^qQPA4EUZH=e&{JqY!Kx)n5z?36;(spLLxL<=%;p4`9Qe^I|2*T2p{=N+p{inF_DOxuPH^E1AR6udUU|fE#e^aIxtP%DMmV-WRqmbyzXAHgS zf^r55O}~<;MNd}go?c=8D+v-(dcf~N=TAe=2A#j-?_-ai=HInv+<#koX8CW3Q~mh> z{;B)Z#xuiz!|_!2|Mve=^k*f{9R7`>Q_(;C{-^lQ3jBJB|500C{9SrI&GY1Ef%B(} Nodf__K8*nY{{juzG0gw~ literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/4/2.changelog new file mode 100644 index 0000000000000000000000000000000000000000..85a6a13b976fce208668e6a9f2b1f512d5fda877 GIT binary patch literal 47 lcmeZ?GI7euPtF!)Wnf?c;aLi9ObkqAMhreM8K58>007!G2WtQT literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_metadata/v2/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_metadata/v2/.0.crc new file mode 100644 index 0000000000000000000000000000000000000000..257c878a1611bfd2c009589cc194ea52bc81ec7b GIT binary patch literal 16 XcmYc;N@ieSU}8vT^7z=mx#s}@Ac_S7 literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_metadata/v2/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_metadata/v2/0 new file mode 100644 index 0000000000000..313c13df69acb --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_metadata/v2/0 @@ -0,0 +1,2 @@ +v2 +{"operatorInfo":{"operatorId":0,"operatorName":"transformWithStateExec"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5,"stateSchemaFilePath":"file:/Users/anish.shrigondekar/spark/spark/target/tmp/spark-ae28252a-e696-4653-a9a5-7a9a0766f4c1/state/0/_stateSchema/default/0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751"}],"operatorPropertiesJson":"{\"timeMode\":\"NoTime\",\"outputMode\":\"Update\",\"stateVariables\":[{\"stateName\":\"countState\",\"stateVariableType\":\"ValueState\",\"ttlEnabled\":false}]}"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_stateSchema/default/.0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_stateSchema/default/.0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751.crc new file mode 100644 index 0000000000000000000000000000000000000000..3c16c8244a3b72687206dd01d2f80cef08136e0e GIT binary patch literal 12 TcmYc;N@ieSU}Et2e=Z0B6N>|) literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_stateSchema/default/0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0-tws-unsaferow/state/0/_stateSchema/default/0_2e8e6b52-e3c3-4184-b8ef-8d391b75d751 new file mode 100644 index 0000000000000000000000000000000000000000..cd3e8f6d96bf532900124feb48fa5fe6ef2d384c GIT binary patch literal 284 zcmbV{F$=;#424ez2Y-cR?j|nXodgl;q&UQDD#i4!Tyw>8+<*6M2T^b_+v7{#%Tx1y zt(2OnrMBKkAt*HJa{w7yf*d+=UJGPU)x>27c|r!87wVBcCGy^^%`Iu;neobLCEZd6 z4GESg2ziL{632kk9|3H?>?`E%hyNzPorbZ{)@WOjJ5GJ}_TZ?hAG=Rlw4`BrZ4p*! F;uE=USz!PG literal 0 HcmV?d00001 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala index e5dfa33164903..5f7205eaf4bbe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.streaming import java.io.File +import org.apache.commons.io.FileUtils + import org.apache.spark.sql.catalyst.util.stringToFile import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -129,4 +131,69 @@ class OffsetSeqLogSuite extends SharedSparkSession { val log = new OffsetSeqLog(spark, input.toString) log.getLatest().get } + + // SPARK-50526 - sanity tests to ensure that values are set correctly for state store + // encoding format within OffsetSeqMetadata + test("offset log records defaults to unsafeRow for store encoding format") { + val offsetSeqMetadata = OffsetSeqMetadata.apply(batchWatermarkMs = 0, batchTimestampMs = 0, + spark.conf) + assert(offsetSeqMetadata.conf.get(SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT.key) === + Some("unsaferow")) + } + + test("offset log uses the store encoding format set in the conf") { + val offsetSeqMetadata = OffsetSeqMetadata.apply(batchWatermarkMs = 0, batchTimestampMs = 0, + Map(SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT.key -> "avro")) + assert(offsetSeqMetadata.conf.get(SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT.key) === + Some("avro")) + } + + // Verify whether entry exists within the offset log and has the right value or that we pick up + // the correct default values when populating the session conf. + private def verifyOffsetLogEntry( + checkpointDir: String, + entryExists: Boolean, + encodingFormat: String): Unit = { + val log = new OffsetSeqLog(spark, s"$checkpointDir/offsets") + val latestBatchId = log.getLatestBatchId() + assert(latestBatchId.isDefined, "No offset log entries found in the checkpoint location") + + // Read the latest offset log + val offsetSeq = log.get(latestBatchId.get).get + val offsetSeqMetadata = offsetSeq.metadata.get + + if (entryExists) { + val encodingFormatOpt = offsetSeqMetadata.conf.get( + SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT.key) + assert(encodingFormatOpt.isDefined, "No store encoding format found in the offset log entry") + assert(encodingFormatOpt.get == encodingFormat) + } + + val clonedSqlConf = spark.sessionState.conf.clone() + OffsetSeqMetadata.setSessionConf(offsetSeqMetadata, clonedSqlConf) + assert(clonedSqlConf.stateStoreEncodingFormat == encodingFormat) + } + + // verify that checkpoint created with different store encoding formats are read correctly + Seq("unsaferow", "avro").foreach { storeEncodingFormat => + test(s"verify format values from checkpoint loc - $storeEncodingFormat") { + withTempDir { checkpointDir => + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-4.0.0-tws-" + storeEncodingFormat + "/").toURI + FileUtils.copyDirectory(new File(resourceUri), checkpointDir.getCanonicalFile) + verifyOffsetLogEntry(checkpointDir.getAbsolutePath, entryExists = true, + storeEncodingFormat) + } + } + } + + test("verify format values from old checkpoint with Spark version 3.5.1") { + withTempDir { checkpointDir => + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-3.5.1-streaming-deduplication/").toURI + FileUtils.copyDirectory(new File(resourceUri), checkpointDir.getCanonicalFile) + verifyOffsetLogEntry(checkpointDir.getAbsolutePath, entryExists = false, + "unsaferow") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index bcf0d4ac46655..0f382f4ed77de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -874,6 +874,26 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { ) } + testWithAllStateVersions("test that avro encoding is not supported") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + val ex = intercept[Exception] { + withSQLConf(SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT.key -> "avro") { + testStream(aggregated, Update)( + AddData(inputData, 3), + ProcessAllAvailable() + ) + } + } + assert(ex.getMessage.contains("State store encoding format as avro is not supported")) + } + private def prepareTestForChangingSchemaOfState( tempDir: File): (MemoryStream[Int], DataFrame) = { val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index ab9df9a1e5a6f..040b99e55cb01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -574,6 +574,21 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { matchPVals = true ) } + + test("test that avro encoding is not supported") { + val inputData = MemoryStream[String] + val result = inputData.toDS().dropDuplicates() + + val ex = intercept[Exception] { + withSQLConf(SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT.key -> "avro") { + testStream(result, Append)( + AddData(inputData, "a"), + ProcessAllAvailable() + ) + } + } + assert(ex.getMessage.contains("State store encoding format as avro is not supported")) + } } @SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala index 6888fcba45f3e..0e963bec41b4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.{SparkRuntimeException, SparkThrowable} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} -import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.functions.window import org.apache.spark.sql.internal.SQLConf @@ -103,47 +103,53 @@ case class AggEventRow( window: Window, count: Long) -class TransformWithStateChainingSuite extends StreamTest { +class TransformWithStateChainingSuite extends StreamTest + with AlsoTestWithRocksDBFeatures + with AlsoTestWithEncodingTypes { import testImplicits._ + private def isAvroEnabled: Boolean = SQLConf.get.stateStoreEncodingFormat == "avro" + test("watermark is propagated correctly for next stateful operator" + " after transformWithState") { - withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> - classOf[RocksDBStateStoreProvider].getName) { - val inputData = MemoryStream[InputEventRow] + if (!isAvroEnabled) { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName) { + val inputData = MemoryStream[InputEventRow] - val result = inputData.toDS() - .withWatermark("eventTime", "1 minute") - .groupByKey(x => x.key) - .transformWithState[OutputRow]( - new TestStatefulProcessor(), - "outputEventTime", - OutputMode.Append()) - .groupBy(window($"outputEventTime", "1 minute")) - .count() - .as[AggEventRow] + val result = inputData.toDS() + .withWatermark("eventTime", "1 minute") + .groupByKey(x => x.key) + .transformWithState[OutputRow]( + new TestStatefulProcessor(), + "outputEventTime", + OutputMode.Append()) + .groupBy(window($"outputEventTime", "1 minute")) + .count() + .as[AggEventRow] - testStream(result, OutputMode.Append())( - AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 00:00:00"), "e1")), - // watermark should be 1 minute behind `2024-01-01 00:00:00`, nothing is - // emitted as all records have timestamp > epoch - CheckNewAnswer(), - Execute("assertWatermarkEquals") { q => - assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00")) - }, - AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1")), - // global watermark should now be 1 minute behind `2024-02-01 00:00:00`. - CheckNewAnswer(AggEventRow( - Window(timestamp("2024-01-01 00:00:00"), timestamp("2024-01-01 00:01:00")), 1) - ), - Execute("assertWatermarkEquals") { q => - assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00")) - }, - AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 00:00:00"), "e1")), - CheckNewAnswer(AggEventRow( - Window(timestamp("2024-02-01 00:00:00"), timestamp("2024-02-01 00:01:00")), 1) + testStream(result, OutputMode.Append())( + AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 00:00:00"), "e1")), + // watermark should be 1 minute behind `2024-01-01 00:00:00`, nothing is + // emitted as all records have timestamp > epoch + CheckNewAnswer(), + Execute("assertWatermarkEquals") { q => + assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00")) + }, + AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1")), + // global watermark should now be 1 minute behind `2024-02-01 00:00:00`. + CheckNewAnswer(AggEventRow( + Window(timestamp("2024-01-01 00:00:00"), timestamp("2024-01-01 00:01:00")), 1) + ), + Execute("assertWatermarkEquals") { q => + assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00")) + }, + AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 00:00:00"), "e1")), + CheckNewAnswer(AggEventRow( + Window(timestamp("2024-02-01 00:00:00"), timestamp("2024-02-01 00:01:00")), 1) + ) ) - ) + } } } @@ -166,33 +172,35 @@ class TransformWithStateChainingSuite extends StreamTest { } } - test("missing eventTime column to transformWithState fails the query if" + - " another stateful operator is added") { - withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> - classOf[RocksDBStateStoreProvider].getName) { - val inputData = MemoryStream[InputEventRow] + test("missing eventTime column to transformWithState fails the query if " + + "another stateful operator is added") { + if (!isAvroEnabled) { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName) { + val inputData = MemoryStream[InputEventRow] - val result = inputData.toDS() - .withWatermark("eventTime", "1 minute") - .groupByKey(x => x.key) - .transformWithState[OutputRow]( - new TestStatefulProcessor(), - TimeMode.None(), - OutputMode.Append()) - .groupBy(window($"outputEventTime", "1 minute")) - .count() - - checkError( - exception = intercept[AnalysisException] { - testStream(result, OutputMode.Append())( - StartStream() - ) - }, - condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", - sqlState = "42KDE", - parameters = Map( - "outputMode" -> "append", - "operation" -> "streaming aggregations without watermark")) + val result = inputData.toDS() + .withWatermark("eventTime", "1 minute") + .groupByKey(x => x.key) + .transformWithState[OutputRow]( + new TestStatefulProcessor(), + TimeMode.None(), + OutputMode.Append()) + .groupBy(window($"outputEventTime", "1 minute")) + .count() + + checkError( + exception = intercept[AnalysisException] { + testStream(result, OutputMode.Append())( + StartStream() + ) + }, + condition = "STREAMING_OUTPUT_MODE.UNSUPPORTED_OPERATION", + sqlState = "42KDE", + parameters = Map( + "outputMode" -> "append", + "operation" -> "streaming aggregations without watermark")) + } } } @@ -234,25 +242,27 @@ class TransformWithStateChainingSuite extends StreamTest { test("dropDuplicateWithWatermark after transformWithState operator" + " fails if watermark column is not provided") { - withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> - classOf[RocksDBStateStoreProvider].getName) { - val inputData = MemoryStream[InputEventRow] - val result = inputData.toDS() - .withWatermark("eventTime", "1 minute") - .groupByKey(x => x.key) - .transformWithState[OutputRow]( - new TestStatefulProcessor(), - TimeMode.None(), - OutputMode.Append()) - .dropDuplicatesWithinWatermark() + if (!isAvroEnabled) { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName) { + val inputData = MemoryStream[InputEventRow] + val result = inputData.toDS() + .withWatermark("eventTime", "1 minute") + .groupByKey(x => x.key) + .transformWithState[OutputRow]( + new TestStatefulProcessor(), + TimeMode.None(), + OutputMode.Append()) + .dropDuplicatesWithinWatermark() - val ex = intercept[ExtendedAnalysisException] { - testStream(result, OutputMode.Append())( - StartStream() - ) + val ex = intercept[ExtendedAnalysisException] { + testStream(result, OutputMode.Append())( + StartStream() + ) + } + assert(ex.getMessage.contains("dropDuplicatesWithinWatermark is not supported on" + + " streaming DataFrames/DataSets without watermark")) } - assert(ex.getMessage.contains("dropDuplicatesWithinWatermark is not supported on" + - " streaming DataFrames/DataSets without watermark")) } } @@ -269,14 +279,25 @@ class TransformWithStateChainingSuite extends StreamTest { OutputMode.Append()) .dropDuplicatesWithinWatermark() - testStream(result, OutputMode.Append())( - AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1"), - InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1")), - CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 2)), - Execute("assertWatermarkEquals") { q => - assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00")) + if (!isAvroEnabled) { + testStream(result, OutputMode.Append())( + AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1"), + InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1")), + CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 2)), + Execute("assertWatermarkEquals") { q => + assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00")) + } + ) + } else { + val ex = intercept[Exception] { + testStream(result, OutputMode.Append())( + AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1"), + InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1")), + ProcessAllAvailable() + ) } - ) + assert(ex.getMessage.contains("State store encoding format as avro is not supported")) + } } } @@ -340,7 +361,7 @@ class TransformWithStateChainingSuite extends StreamTest { val inputData = MemoryStream[InputEventRow] inputData.toDS() .withWatermark("eventTime", "1 minute") - .createTempView("tempViewWithWatermark") + .createOrReplaceTempView("tempViewWithWatermark") val result = spark.readStream.table("tempViewWithWatermark") .as[InputEventRow] @@ -365,7 +386,7 @@ class TransformWithStateChainingSuite extends StreamTest { classOf[RocksDBStateStoreProvider].getName) { val inputData = MemoryStream[InputEventRow] inputData.toDS() - .createTempView("tempViewWithoutWatermark") + .createOrReplaceTempView("tempViewWithoutWatermark") val ex = intercept[AnalysisException] { val result = spark.readStream.table("tempViewWithoutWatermark") From f8de6c7b74b895e0fa00a678120dd64c3aa1db3d Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Sat, 14 Dec 2024 10:56:51 +0900 Subject: [PATCH 179/438] [SPARK-50565][SS][TESTS] Add transformWithState correctness test ### What changes were proposed in this pull request? As title ### Why are the changes needed? Test coverage ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Test only addition ### Was this patch authored or co-authored using generative AI tooling? No Closes #49173 from WweiL/tws-followup. Authored-by: Wei Liu Signed-off-by: Jungtaek Lim --- ...sDBStateStoreCheckpointFormatV2Suite.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala index 24c99bffd8bab..5725ebaf727bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala @@ -542,4 +542,35 @@ class RocksDBStateStoreCheckpointFormatV2Suite extends StreamTest } validateCheckpointInfo(6, 1, Set(2, 4, 6)) } + + test("checkpointFormatVersion2 validate transformWithState") { + withTempDir { checkpointDir => + val inputData = MemoryStream[String] + val result = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessor(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result, Update())( + StartStream(checkpointLocation = checkpointDir.getAbsolutePath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + Execute { q => + assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0) + assert(q.lastProgress.stateOperators(0).customMetrics.get("numRegisteredTimers") == 0) + }, + AddData(inputData, "a", "b"), + CheckNewAnswer(("a", "2"), ("b", "1")), + StopStream, + StartStream(checkpointLocation = checkpointDir.getAbsolutePath), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckNewAnswer(("b", "2")), + StopStream, + StartStream(checkpointLocation = checkpointDir.getAbsolutePath), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckNewAnswer(("a", "1"), ("c", "1")) + ) + } + } } From 2b9eb08a019f9f5bdacefca9c0ad8ee01b8975f5 Mon Sep 17 00:00:00 2001 From: Prashanth Menon Date: Fri, 13 Dec 2024 20:10:26 -0800 Subject: [PATCH 180/438] [SPARK-50515][CORE] Add read-only interface to `SparkConf` ### What changes were proposed in this pull request? This PR lifts read-only APIs of `SparkConf` into a new `ReadOnlySparkConf`. `SparkContext` now exposes a new read-only API to the conf through `SparkContext.getReadOnlyConf`, which can be used by clients outside the `spark` package if they require only read-only access. The new API avoids copying the entire (potentially large) conf as in `SparkContext.getConf`. This PR also changes all appropriate call sites to use the new API. ### Why are the changes needed? Cloning the entire conf adds unnecessary CPU overhead due to copying, and GC overhead due to cleanup. Both affect tail latencies on certain workloads. ### Does this PR introduce _any_ user-facing change? It adds a new public API `SparkContext.getReadOnlyConf`. ### How was this patch tested? It is a refactoring PR, so we rely on existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49100 from pmenon/read-only-confs. Authored-by: Prashanth Menon Signed-off-by: Wenchen Fan --- .../kafka010/DirectKafkaInputDStream.scala | 2 +- .../scala/org/apache/spark/SparkConf.scala | 413 ++++++++++-------- .../scala/org/apache/spark/SparkContext.scala | 3 + .../spark/api/java/JavaSparkContext.scala | 3 + .../scala/org/apache/spark/util/Utils.scala | 4 +- .../org/apache/spark/graphx/Pregel.scala | 2 +- .../mllib/tree/model/DecisionTreeModel.scala | 2 +- .../mllib/tree/model/treeEnsembleModels.scala | 2 +- 8 files changed, 239 insertions(+), 192 deletions(-) diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index cefaa3de182a5..f7bea064d2d6c 100644 --- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -57,7 +57,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( ppc: PerPartitionConfig ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging with CanCommitOffsets { - private val initialRate = context.sparkContext.getConf.getLong( + private val initialRate = context.sparkContext.getReadOnlyConf.getLong( "spark.streaming.backpressure.initialRate", 0) val executorKafkaParams = { diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 74abd5cbe0ae5..ae6ef1ee55608 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -35,6 +35,228 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils +trait ReadOnlySparkConf { + /** Get a parameter; throws a NoSuchElementException if it's not set */ + def get(key: String): String = { + getOption(key).getOrElse(throw new NoSuchElementException(key)) + } + + /** Get a parameter, falling back to a default if not set */ + def get(key: String, defaultValue: String): String = { + getOption(key).getOrElse(defaultValue) + } + + /** + * Retrieves the value of a pre-defined configuration entry. + * + * - This is an internal Spark API. + * - The return type if defined by the configuration entry. + * - This will throw an exception is the config is not optional and the value is not set. + */ + private[spark] def get[T](entry: ConfigEntry[T]): T + + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + * + * @throws java.util.NoSuchElementException If the time parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as seconds + */ + def getTimeAsSeconds(key: String): Long = catchIllegalValue(key) { + Utils.timeStringAsSeconds(get(key)) + } + + /** + * Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + * + * @throws NumberFormatException If the value cannot be interpreted as seconds + */ + def getTimeAsSeconds(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.timeStringAsSeconds(get(key, defaultValue)) + } + + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + * + * @throws java.util.NoSuchElementException If the time parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as milliseconds + */ + def getTimeAsMs(key: String): Long = catchIllegalValue(key) { + Utils.timeStringAsMs(get(key)) + } + + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + * + * @throws NumberFormatException If the value cannot be interpreted as milliseconds + */ + def getTimeAsMs(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.timeStringAsMs(get(key, defaultValue)) + } + + /** + * Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then bytes are assumed. + * + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as bytes + */ + def getSizeAsBytes(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsBytes(get(key)) + } + + /** + * Get a size parameter as bytes, falling back to a default if not set. If no + * suffix is provided then bytes are assumed. + * + * @throws NumberFormatException If the value cannot be interpreted as bytes + */ + def getSizeAsBytes(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsBytes(get(key, defaultValue)) + } + + /** + * Get a size parameter as bytes, falling back to a default if not set. + * + * @throws NumberFormatException If the value cannot be interpreted as bytes + */ + def getSizeAsBytes(key: String, defaultValue: Long): Long = catchIllegalValue(key) { + Utils.byteStringAsBytes(get(key, s"${defaultValue}B")) + } + + /** + * Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Kibibytes are assumed. + * + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as Kibibytes + */ + def getSizeAsKb(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsKb(get(key)) + } + + /** + * Get a size parameter as Kibibytes, falling back to a default if not set. If no + * suffix is provided then Kibibytes are assumed. + * + * @throws NumberFormatException If the value cannot be interpreted as Kibibytes + */ + def getSizeAsKb(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsKb(get(key, defaultValue)) + } + + /** + * Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Mebibytes are assumed. + * + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as Mebibytes + */ + def getSizeAsMb(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsMb(get(key)) + } + + /** + * Get a size parameter as Mebibytes, falling back to a default if not set. If no + * suffix is provided then Mebibytes are assumed. + * + * @throws NumberFormatException If the value cannot be interpreted as Mebibytes + */ + def getSizeAsMb(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsMb(get(key, defaultValue)) + } + + /** + * Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Gibibytes are assumed. + * + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as Gibibytes + */ + def getSizeAsGb(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsGb(get(key)) + } + + /** + * Get a size parameter as Gibibytes, falling back to a default if not set. If no + * suffix is provided then Gibibytes are assumed. + * + * @throws NumberFormatException If the value cannot be interpreted as Gibibytes + */ + def getSizeAsGb(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsGb(get(key, defaultValue)) + } + + /** Get a parameter as an Option */ + def getOption(key: String): Option[String] + + /** Get all parameters as a list of pairs */ + def getAll: Array[(String, String)] + + /** + * Get a parameter as an integer, falling back to a default if not set + * + * @throws NumberFormatException If the value cannot be interpreted as an integer + */ + def getInt(key: String, defaultValue: Int): Int = catchIllegalValue(key) { + getOption(key).map(_.toInt).getOrElse(defaultValue) + } + + /** + * Get a parameter as a long, falling back to a default if not set + * + * @throws NumberFormatException If the value cannot be interpreted as a long + */ + def getLong(key: String, defaultValue: Long): Long = catchIllegalValue(key) { + getOption(key).map(_.toLong).getOrElse(defaultValue) + } + + /** + * Get a parameter as a double, falling back to a default if not ste + * + * @throws NumberFormatException If the value cannot be interpreted as a double + */ + def getDouble(key: String, defaultValue: Double): Double = catchIllegalValue(key) { + getOption(key).map(_.toDouble).getOrElse(defaultValue) + } + + /** + * Get a parameter as a boolean, falling back to a default if not set + * + * @throws IllegalArgumentException If the value cannot be interpreted as a boolean + */ + def getBoolean(key: String, defaultValue: Boolean): Boolean = catchIllegalValue(key) { + getOption(key).map(_.toBoolean).getOrElse(defaultValue) + } + + /** Does the configuration contain a given parameter? */ + def contains(key: String): Boolean + + /** Does the configuration have the typed config entry? */ + def contains(entry: ConfigEntry[_]): Boolean = contains(entry.key) + + /** + * Wrapper method for get() methods which require some specific value format. This catches + * any [[NumberFormatException]] or [[IllegalArgumentException]] and re-raises it with the + * incorrectly configured key in the exception message. + */ + protected def catchIllegalValue[T](key: String)(getValue: => T): T = { + try { + getValue + } catch { + case e: NumberFormatException => + // NumberFormatException doesn't have a constructor that takes a cause for some reason. + throw new NumberFormatException(s"Illegal value for config key $key: ${e.getMessage}") + .initCause(e) + case e: IllegalArgumentException => + throw new IllegalArgumentException(s"Illegal value for config key $key: ${e.getMessage}", e) + } + } +} + /** * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. * @@ -53,7 +275,11 @@ import org.apache.spark.util.Utils * @note Once a SparkConf object is passed to Spark, it is cloned and can no longer be modified * by the user. Spark does not support modifying the configuration at runtime. */ -class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Serializable { +class SparkConf(loadDefaults: Boolean) + extends ReadOnlySparkConf + with Cloneable + with Logging + with Serializable { import SparkConf._ @@ -242,16 +468,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria remove(entry.key) } - /** Get a parameter; throws a NoSuchElementException if it's not set */ - def get(key: String): String = { - getOption(key).getOrElse(throw new NoSuchElementException(key)) - } - - /** Get a parameter, falling back to a default if not set */ - def get(key: String, defaultValue: String): String = { - getOption(key).getOrElse(defaultValue) - } - /** * Retrieves the value of a pre-defined configuration entry. * @@ -263,128 +479,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria entry.readFrom(reader) } - /** - * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no - * suffix is provided then seconds are assumed. - * @throws java.util.NoSuchElementException If the time parameter is not set - * @throws NumberFormatException If the value cannot be interpreted as seconds - */ - def getTimeAsSeconds(key: String): Long = catchIllegalValue(key) { - Utils.timeStringAsSeconds(get(key)) - } - - /** - * Get a time parameter as seconds, falling back to a default if not set. If no - * suffix is provided then seconds are assumed. - * @throws NumberFormatException If the value cannot be interpreted as seconds - */ - def getTimeAsSeconds(key: String, defaultValue: String): Long = catchIllegalValue(key) { - Utils.timeStringAsSeconds(get(key, defaultValue)) - } - - /** - * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no - * suffix is provided then milliseconds are assumed. - * @throws java.util.NoSuchElementException If the time parameter is not set - * @throws NumberFormatException If the value cannot be interpreted as milliseconds - */ - def getTimeAsMs(key: String): Long = catchIllegalValue(key) { - Utils.timeStringAsMs(get(key)) - } - - /** - * Get a time parameter as milliseconds, falling back to a default if not set. If no - * suffix is provided then milliseconds are assumed. - * @throws NumberFormatException If the value cannot be interpreted as milliseconds - */ - def getTimeAsMs(key: String, defaultValue: String): Long = catchIllegalValue(key) { - Utils.timeStringAsMs(get(key, defaultValue)) - } - - /** - * Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no - * suffix is provided then bytes are assumed. - * @throws java.util.NoSuchElementException If the size parameter is not set - * @throws NumberFormatException If the value cannot be interpreted as bytes - */ - def getSizeAsBytes(key: String): Long = catchIllegalValue(key) { - Utils.byteStringAsBytes(get(key)) - } - - /** - * Get a size parameter as bytes, falling back to a default if not set. If no - * suffix is provided then bytes are assumed. - * @throws NumberFormatException If the value cannot be interpreted as bytes - */ - def getSizeAsBytes(key: String, defaultValue: String): Long = catchIllegalValue(key) { - Utils.byteStringAsBytes(get(key, defaultValue)) - } - - /** - * Get a size parameter as bytes, falling back to a default if not set. - * @throws NumberFormatException If the value cannot be interpreted as bytes - */ - def getSizeAsBytes(key: String, defaultValue: Long): Long = catchIllegalValue(key) { - Utils.byteStringAsBytes(get(key, s"${defaultValue}B")) - } - - /** - * Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no - * suffix is provided then Kibibytes are assumed. - * @throws java.util.NoSuchElementException If the size parameter is not set - * @throws NumberFormatException If the value cannot be interpreted as Kibibytes - */ - def getSizeAsKb(key: String): Long = catchIllegalValue(key) { - Utils.byteStringAsKb(get(key)) - } - - /** - * Get a size parameter as Kibibytes, falling back to a default if not set. If no - * suffix is provided then Kibibytes are assumed. - * @throws NumberFormatException If the value cannot be interpreted as Kibibytes - */ - def getSizeAsKb(key: String, defaultValue: String): Long = catchIllegalValue(key) { - Utils.byteStringAsKb(get(key, defaultValue)) - } - - /** - * Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no - * suffix is provided then Mebibytes are assumed. - * @throws java.util.NoSuchElementException If the size parameter is not set - * @throws NumberFormatException If the value cannot be interpreted as Mebibytes - */ - def getSizeAsMb(key: String): Long = catchIllegalValue(key) { - Utils.byteStringAsMb(get(key)) - } - - /** - * Get a size parameter as Mebibytes, falling back to a default if not set. If no - * suffix is provided then Mebibytes are assumed. - * @throws NumberFormatException If the value cannot be interpreted as Mebibytes - */ - def getSizeAsMb(key: String, defaultValue: String): Long = catchIllegalValue(key) { - Utils.byteStringAsMb(get(key, defaultValue)) - } - - /** - * Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no - * suffix is provided then Gibibytes are assumed. - * @throws java.util.NoSuchElementException If the size parameter is not set - * @throws NumberFormatException If the value cannot be interpreted as Gibibytes - */ - def getSizeAsGb(key: String): Long = catchIllegalValue(key) { - Utils.byteStringAsGb(get(key)) - } - - /** - * Get a size parameter as Gibibytes, falling back to a default if not set. If no - * suffix is provided then Gibibytes are assumed. - * @throws NumberFormatException If the value cannot be interpreted as Gibibytes - */ - def getSizeAsGb(key: String, defaultValue: String): Long = catchIllegalValue(key) { - Utils.byteStringAsGb(get(key, defaultValue)) - } - /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { Option(settings.get(key)).orElse(getDeprecatedConfig(key, settings)) @@ -408,38 +502,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria .map { case (k, v) => (k.substring(prefix.length), v) } } - /** - * Get a parameter as an integer, falling back to a default if not set - * @throws NumberFormatException If the value cannot be interpreted as an integer - */ - def getInt(key: String, defaultValue: Int): Int = catchIllegalValue(key) { - getOption(key).map(_.toInt).getOrElse(defaultValue) - } - - /** - * Get a parameter as a long, falling back to a default if not set - * @throws NumberFormatException If the value cannot be interpreted as a long - */ - def getLong(key: String, defaultValue: Long): Long = catchIllegalValue(key) { - getOption(key).map(_.toLong).getOrElse(defaultValue) - } - - /** - * Get a parameter as a double, falling back to a default if not ste - * @throws NumberFormatException If the value cannot be interpreted as a double - */ - def getDouble(key: String, defaultValue: Double): Double = catchIllegalValue(key) { - getOption(key).map(_.toDouble).getOrElse(defaultValue) - } - - /** - * Get a parameter as a boolean, falling back to a default if not set - * @throws IllegalArgumentException If the value cannot be interpreted as a boolean - */ - def getBoolean(key: String, defaultValue: Boolean): Boolean = catchIllegalValue(key) { - getOption(key).map(_.toBoolean).getOrElse(defaultValue) - } - /** Get all executor environment variables set on this SparkConf */ def getExecutorEnv: Seq[(String, String)] = { getAllWithPrefix("spark.executorEnv.").toImmutableArraySeq @@ -457,8 +519,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria configsWithAlternatives.get(key).toSeq.flatten.exists { alt => contains(alt.key) } } - private[spark] def contains(entry: ConfigEntry[_]): Boolean = contains(entry.key) - /** Copy this object */ override def clone: SparkConf = { val cloned = new SparkConf(false) @@ -474,25 +534,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria */ private[spark] def getenv(name: String): String = System.getenv(name) - /** - * Wrapper method for get() methods which require some specific value format. This catches - * any [[NumberFormatException]] or [[IllegalArgumentException]] and re-raises it with the - * incorrectly configured key in the exception message. - */ - private def catchIllegalValue[T](key: String)(getValue: => T): T = { - try { - getValue - } catch { - case e: NumberFormatException => - // NumberFormatException doesn't have a constructor that takes a cause for some reason. - throw new NumberFormatException(s"Illegal value for config key $key: ${e.getMessage}") - .initCause(e) - case e: IllegalArgumentException => - throw new IllegalArgumentException(s"Illegal value for config key $key: ${e.getMessage}", e) - } - } - - /** * Checks for illegal or deprecated config settings. Throws an exception for the former. Not * idempotent - may mutate this conf object to convert deprecated settings to supported ones. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 20180e142f57f..5d3a9c2690c41 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -252,6 +252,9 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] def conf: SparkConf = _conf + /** Get a read-only reference to the spark conf. This is preferred version over [[getConf]]. */ + def getReadOnlyConf: ReadOnlySparkConf = _conf + /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be * changed at runtime. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 9de350bc3130f..7311ef296363d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -630,6 +630,9 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { */ def getConf: SparkConf = sc.getConf + /** Return a read-only version of the spark conf. */ + def getReadOnlyConf: ReadOnlySparkConf = sc.getReadOnlyConf + /** * Pass-through to SparkContext.setCallSite. For API support only. */ 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 668657c87b39b..e7b65bf1a4eff 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2522,7 +2522,7 @@ private[spark] object Utils * * @return whether it is local mode */ - def isLocalMaster(conf: SparkConf): Boolean = { + def isLocalMaster(conf: ReadOnlySparkConf): Boolean = { val master = conf.get("spark.master", "") master == "local" || master.startsWith("local[") } @@ -2606,7 +2606,7 @@ private[spark] object Utils /** * Return whether dynamic allocation is enabled in the given conf. */ - def isDynamicAllocationEnabled(conf: SparkConf): Boolean = { + def isDynamicAllocationEnabled(conf: ReadOnlySparkConf): Boolean = { val dynamicAllocationEnabled = conf.get(DYN_ALLOCATION_ENABLED) dynamicAllocationEnabled && (!isLocalMaster(conf) || conf.get(DYN_ALLOCATION_TESTING)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 1493d8114c699..823143f9b9abb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -126,7 +126,7 @@ object Pregel extends Logging { require(maxIterations > 0, s"Maximum number of iterations must be greater than 0," + s" but got ${maxIterations}") - val checkpointInterval = graph.vertices.sparkContext.getConf + val checkpointInterval = graph.vertices.sparkContext.getReadOnlyConf .getInt("spark.graphx.pregel.checkpointInterval", -1) var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)) val graphCheckpointer = new PeriodicGraphCheckpointer[VD, ED]( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index b45211c1689c7..2acc49e218f2d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -204,7 +204,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { // TODO: Fix this issue for real. val memThreshold = 768 if (sc.isLocal) { - val driverMemory = sc.getConf.getOption("spark.driver.memory") + val driverMemory = sc.getReadOnlyConf.getOption("spark.driver.memory") .orElse(Option(System.getenv("SPARK_DRIVER_MEMORY"))) .map(Utils.memoryStringToMb) .getOrElse(Utils.DEFAULT_DRIVER_MEM_MB) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index 7251dfd07a1fa..af922dda13f6b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -402,7 +402,7 @@ private[tree] object TreeEnsembleModel extends Logging { // TODO: Fix this issue for real. val memThreshold = 768 if (sc.isLocal) { - val driverMemory = sc.getConf.getOption("spark.driver.memory") + val driverMemory = sc.getReadOnlyConf.getOption("spark.driver.memory") .orElse(Option(System.getenv("SPARK_DRIVER_MEMORY"))) .map(Utils.memoryStringToMb) .getOrElse(Utils.DEFAULT_DRIVER_MEM_MB) From 976192af22bacad91329099f34e0e9e48e7810e7 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Sat, 14 Dec 2024 18:50:02 +0300 Subject: [PATCH 181/438] [SPARK-50559][SQL] Store Except, Intersect and Union's outputs as lazy vals ### What changes were proposed in this pull request? Store `Except`, `Intersect` and `Union`'s outputs as lazy vals. ### Why are the changes needed? Currently `Union`'s (same is for `Except` and `Intersect`) `output` is a `def`. This creates performance issues for queries with large number of stacked `UNION`s because of rules like `WidenSetOperationTypes` that traverse the logical plan and call `output` on each `Union` node. This has quadratic complexity: O(number_of_unions * (1 + 2 + 3 + ... + number_of_unions)). Profile: ![image](https://github.com/user-attachments/assets/97192bf3-c38e-47dd-81ac-49ee9a546525) ![image](https://github.com/user-attachments/assets/68ed13d7-b108-4c8d-b156-1dcec07c324b) [flamegraph.tar.gz](https://github.com/user-attachments/files/18118260/flamegraph.tar.gz) The improvement in parsing + analysis wall-clock time for a query with 500 UNIONs over 30 columns each is 13x (5.5s -> 400ms): ![image](https://github.com/user-attachments/assets/a824c693-0a6b-4c6b-8a90-a783a3c44d6d) Repro: ``` def genValues(num: Int) = s"VALUES (${(0 until num).mkString(", ")})" def genUnions(numUnions: Int, numValues: Int) = (0 until numUnions).map(_ => genValues(numValues)).mkString(" UNION ALL ") spark.time { spark.sql(s"SELECT * FROM ${genUnions(numUnions = 500, numValues = 30)}").queryExecution.analyzed } ``` For `EXCEPT` the perf difference is not that noticeable. Perhaps because it reuses the same `Seq` (it just calls `left.output`). ### Does this PR introduce _any_ user-facing change? No, this is an optimization. ### How was this patch tested? - Ran the async-profiler - Ran the benchmark in spark-shell. - Existing tests. ### Was this patch authored or co-authored using generative AI tooling? copilot.nvim. Closes #49166 from vladimirg-db/vladimirg-db/store-union-output-as-lazy-val. Authored-by: Vladimir Golubev Signed-off-by: Max Gekk --- .../plans/logical/basicLogicalOperators.scala | 64 +++++++++---- .../apache/spark/sql/internal/SQLConf.scala | 13 +++ .../SetOperationsBenchmark-jdk21-results.txt | 13 +++ .../SetOperationsBenchmark-results.txt | 13 +++ .../benchmark/SetOperationsBenchmark.scala | 90 +++++++++++++++++++ 5 files changed, 177 insertions(+), 16 deletions(-) create mode 100644 sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt create mode 100644 sql/core/benchmarks/SetOperationsBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SetOperationsBenchmark.scala 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 f2a46b57b478f..4636ca59d4e86 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 @@ -376,10 +376,13 @@ case class Intersect( final override val nodePatterns: Seq[TreePattern] = Seq(INTERSECT) - override def output: Seq[Attribute] = - left.output.zip(right.output).map { case (leftAttr, rightAttr) => - leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) + override def output: Seq[Attribute] = { + if (conf.getConf(SQLConf.LAZY_SET_OPERATOR_OUTPUT)) { + lazyOutput + } else { + computeOutput() } + } override def metadataOutput: Seq[Attribute] = Nil @@ -396,6 +399,14 @@ case class Intersect( override protected def withNewChildrenInternal( newLeft: LogicalPlan, newRight: LogicalPlan): Intersect = copy(left = newLeft, right = newRight) + + private lazy val lazyOutput: Seq[Attribute] = computeOutput() + + /** We don't use right.output because those rows get excluded from the set. */ + private def computeOutput(): Seq[Attribute] = + left.output.zip(right.output).map { case (leftAttr, rightAttr) => + leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) + } } case class Except( @@ -403,8 +414,14 @@ case class Except( right: LogicalPlan, isAll: Boolean) extends SetOperation(left, right) { override def nodeName: String = getClass.getSimpleName + ( if ( isAll ) " All" else "" ) - /** We don't use right.output because those rows get excluded from the set. */ - override def output: Seq[Attribute] = left.output + + override def output: Seq[Attribute] = { + if (conf.getConf(SQLConf.LAZY_SET_OPERATOR_OUTPUT)) { + lazyOutput + } else { + computeOutput() + } + } override def metadataOutput: Seq[Attribute] = Nil @@ -416,6 +433,11 @@ case class Except( override protected def withNewChildrenInternal( newLeft: LogicalPlan, newRight: LogicalPlan): Except = copy(left = newLeft, right = newRight) + + private lazy val lazyOutput: Seq[Attribute] = computeOutput() + + /** We don't use right.output because those rows get excluded from the set. */ + private def computeOutput(): Seq[Attribute] = left.output } /** Factory for constructing new `Union` nodes. */ @@ -479,18 +501,11 @@ case class Union( AttributeSet.fromAttributeSets(children.map(_.outputSet)).size } - // updating nullability to make all the children consistent override def output: Seq[Attribute] = { - children.map(_.output).transpose.map { attrs => - val firstAttr = attrs.head - val nullable = attrs.exists(_.nullable) - val newDt = attrs.map(_.dataType).reduce(StructType.unionLikeMerge) - if (firstAttr.dataType == newDt) { - firstAttr.withNullability(nullable) - } else { - AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( - firstAttr.exprId, firstAttr.qualifier) - } + if (conf.getConf(SQLConf.LAZY_SET_OPERATOR_OUTPUT)) { + lazyOutput + } else { + computeOutput() } } @@ -509,6 +524,23 @@ case class Union( children.length > 1 && !(byName || allowMissingCol) && childrenResolved && allChildrenCompatible } + private lazy val lazyOutput: Seq[Attribute] = computeOutput() + + // updating nullability to make all the children consistent + private def computeOutput(): Seq[Attribute] = { + children.map(_.output).transpose.map { attrs => + val firstAttr = attrs.head + val nullable = attrs.exists(_.nullable) + val newDt = attrs.map(_.dataType).reduce(StructType.unionLikeMerge) + if (firstAttr.dataType == newDt) { + firstAttr.withNullability(nullable) + } else { + AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( + firstAttr.exprId, firstAttr.qualifier) + } + } + } + /** * Maps the constraints containing a given (original) sequence of attributes to those with a * given (reference) sequence of attributes. Given the nature of union, we expect that the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6b430f167ca8b..c50b7b1063863 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -5306,6 +5306,19 @@ object SQLConf { .booleanConf .createWithDefault(true) + val LAZY_SET_OPERATOR_OUTPUT = buildConf("spark.sql.lazySetOperatorOutput.enabled") + .internal() + .doc( + "When set to true, Except/Intersect/Union operator's output will be a lazy val. It " + + "is a performance optimization for querires with a large number of stacked set operators. " + + "This is because of rules like WidenSetOperationTypes that traverse the logical plan tree " + + "and call output on each Except/Intersect/Union node. Such traversal has quadratic " + + "complexity: O(number_of_nodes * (1 + 2 + 3 + ... + number_of_nodes))." + ) + .version("4.0.0") + .booleanConf + .createWithDefault(true) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt b/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt new file mode 100644 index 0000000000000..31c29c93fb9e5 --- /dev/null +++ b/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt @@ -0,0 +1,13 @@ +================================================================================================ +Set Operations Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 21.0.5+11-Ubuntu-1ubuntu120.04 on Linux 5.4.0-1131-aws-fips +Intel(R) Xeon(R) Platinum 8375C CPU @ 2.90GHz +Parsing + Analysis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UNION ALL 342 358 22 0.0 22784.2 1.0X +EXCEPT ALL 310 351 44 0.0 20675.4 1.1X +INTERSECT ALL 305 309 5 0.0 20301.6 1.1X + + diff --git a/sql/core/benchmarks/SetOperationsBenchmark-results.txt b/sql/core/benchmarks/SetOperationsBenchmark-results.txt new file mode 100644 index 0000000000000..56bd366579e82 --- /dev/null +++ b/sql/core/benchmarks/SetOperationsBenchmark-results.txt @@ -0,0 +1,13 @@ +================================================================================================ +Set Operations Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 17.0.12+7-Ubuntu-1ubuntu220.04 on Linux 5.4.0-1131-aws-fips +Intel(R) Xeon(R) Platinum 8375C CPU @ 2.90GHz +Parsing + Analysis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +UNION ALL 360 423 70 0.0 24019.4 1.0X +EXCEPT ALL 322 328 5 0.0 21463.2 1.1X +INTERSECT ALL 327 360 33 0.0 21777.2 1.1X + + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SetOperationsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SetOperationsBenchmark.scala new file mode 100644 index 0000000000000..379e31ead2f31 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SetOperationsBenchmark.scala @@ -0,0 +1,90 @@ +/* + * 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.benchmark + +import org.apache.spark.benchmark.Benchmark + +/** + * Benchmark to measure performance for set operations. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class + * --jars , + * 2. build/sbt "sql/Test/runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/Test/runMain " + * Results will be written to "benchmarks/SetOperationsBenchmark-results.txt". + * }}} + */ +object SetOperationsBenchmark extends SqlBasedBenchmark { + private val setOperations = Seq("UNION ALL", "EXCEPT ALL", "INTERSECT ALL") + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("Set Operations Benchmark") { + val numOperations = 500 + val numValues = 30 + + val benchmark = + new Benchmark( + "Parsing + Analysis", + valuesPerIteration = numOperations * numValues, + output = output + ) + + for (operation <- setOperations) { + benchmark.addCase(operation) { _ => + spark + .sql( + generateQuery( + operation = operation, + numOperations = numOperations, + numValues = numValues + ) + ) + .queryExecution + .analyzed + () + } + } + + benchmark.run() + } + } + + private def generateQuery(operation: String, numOperations: Int, numValues: Int) = { + s""" + SELECT + * + FROM + ${generateOperations( + operation = operation, + numOperations = numOperations, + numValues = numValues + )} + """ + } + + private def generateOperations(operation: String, numOperations: Int, numValues: Int) = { + (0 until numOperations).map(_ => generateValues(numValues)).mkString(s" ${operation} ") + } + + private def generateValues(num: Int) = { + s"VALUES (${(0 until num).mkString(", ")})" + } +} From d2965ae99aa758511dc03ed374a6a7abf8a7350f Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Sat, 14 Dec 2024 21:16:14 -0800 Subject: [PATCH 182/438] [SPARK-49839][SQL] SPJ: Skip shuffles if possible for sorts ### What changes were proposed in this pull request? This is a proposal for skipping shuffles for ORDER BY or other sort operations, if on partition columns. ### Why are the changes needed? This could potentially optimize many jobs, where today all data is shuffled even if we have all the partition values and can sort them. This is a common scenario, for example Iceberg often requests data to be sorted by partition before write , to avoid small file issue. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add test in KeyGroupedPartitioningSuite ### Was this patch authored or co-authored using generative AI tooling? No Closes #48303 from szehon-ho/SPARK-49839. Authored-by: Szehon Ho Signed-off-by: Chao Sun --- .../plans/physical/partitioning.scala | 10 ++++ .../apache/spark/sql/internal/SQLConf.scala | 13 +++++ .../exchange/EnsureRequirements.scala | 19 ++++++- .../KeyGroupedPartitioningSuite.scala | 56 +++++++++++++++++++ 4 files changed, 97 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 30e223c3c3c87..6e19a1d6bbc8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -176,6 +176,13 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { override def createPartitioning(numPartitions: Int): Partitioning = { RangePartitioning(ordering, numPartitions) } + + def areAllClusterKeysMatched(expressions: Seq[Expression]): Boolean = { + expressions.length == ordering.length && + expressions.zip(ordering).forall { + case (x, o) => x.semanticEquals(o.child) + } + } } /** @@ -394,6 +401,9 @@ case class KeyGroupedPartitioning( } } + case o @ OrderedDistribution(_) if SQLConf.get.v2BucketingAllowSorting => + o.areAllClusterKeysMatched(expressions) + case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c50b7b1063863..8bdb8b1addb00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1679,6 +1679,16 @@ object SQLConf { .booleanConf .createWithDefault(false) + val V2_BUCKETING_SORTING_ENABLED = + buildConf("spark.sql.sources.v2.bucketing.sorting.enabled") + .doc(s"When turned on, Spark will recognize the specific distribution reported by " + + s"a V2 data source through SupportsReportPartitioning, and will try to avoid a shuffle " + + s"if possible when sorting by those columns. This config requires " + + s"${V2_BUCKETING_ENABLED.key} to be enabled.") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + val BUCKETING_MAX_BUCKETS = buildConf("spark.sql.sources.bucketing.maxBuckets") .doc("The maximum number of buckets allowed.") .version("2.4.0") @@ -5896,6 +5906,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def v2BucketingAllowCompatibleTransforms: Boolean = getConf(SQLConf.V2_BUCKETING_ALLOW_COMPATIBLE_TRANSFORMS) + def v2BucketingAllowSorting: Boolean = + getConf(SQLConf.V2_BUCKETING_SORTING_ENABLED) + def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 8ec903f8e61da..de5c3aaa4fe4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -64,7 +64,7 @@ case class EnsureRequirements( // Ensure that the operator's children satisfy their output distribution requirements. var children = originalChildren.zip(requiredChildDistributions).map { case (child, distribution) if child.outputPartitioning.satisfies(distribution) => - child + ensureOrdering(child, distribution) case (child, BroadcastDistribution(mode)) => BroadcastExchangeExec(mode, child) case (child, distribution) => @@ -290,6 +290,23 @@ case class EnsureRequirements( } } + private def ensureOrdering(plan: SparkPlan, distribution: Distribution) = { + (plan.outputPartitioning, distribution) match { + case (p @ KeyGroupedPartitioning(expressions, _, partitionValues, _), + d @ OrderedDistribution(ordering)) if p.satisfies(d) => + val attrs = expressions.flatMap(_.collectLeaves()).map(_.asInstanceOf[Attribute]) + val partitionOrdering: Ordering[InternalRow] = { + RowOrdering.create(ordering, attrs) + } + // Sort 'commonPartitionValues' and use this mechanism to ensure BatchScan's + // output partitions are ordered + val sorted = partitionValues.sorted(partitionOrdering) + populateCommonPartitionInfo(plan, sorted.map((_, 1)), + None, None, applyPartialClustering = false, replicatePartitions = false) + case _ => plan + } + } + /** * Recursively reorders the join keys based on partitioning. It starts reordering the * join keys to match HashPartitioning on either side, followed by PartitioningCollection. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index 152896499010c..c24f52bd93070 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -370,6 +370,62 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { checkAnswer(df.sort("res"), Seq(Row(10.0), Row(15.5), Row(41.0))) } + test("SPARK-48655: order by on partition keys should not introduce additional shuffle") { + val items_partitions = Array(identity("price"), identity("id")) + createTable(items, itemsColumns, items_partitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(null, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(3, 'cc', null, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { sortingEnabled => + withSQLConf(SQLConf.V2_BUCKETING_SORTING_ENABLED.key -> sortingEnabled.toString) { + + def verifyShuffle(cmd: String, answer: Seq[Row]): Unit = { + val df = sql(cmd) + if (sortingEnabled) { + assert(collectAllShuffles(df.queryExecution.executedPlan).isEmpty, + "should contain no shuffle when sorting by partition values") + } else { + assert(collectAllShuffles(df.queryExecution.executedPlan).size == 1, + "should contain one shuffle when optimization is disabled") + } + checkAnswer(df, answer) + }: Unit + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items ORDER BY price ASC, id ASC", + Seq(Row(null, 3), Row(10.0, 2), Row(15.5, null), + Row(15.5, 3), Row(40.0, 1), Row(41.0, 1))) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items " + + s"ORDER BY price ASC NULLS LAST, id ASC NULLS LAST", + Seq(Row(10.0, 2), Row(15.5, 3), Row(15.5, null), + Row(40.0, 1), Row(41.0, 1), Row(null, 3))) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items ORDER BY price DESC, id ASC", + Seq(Row(41.0, 1), Row(40.0, 1), Row(15.5, null), + Row(15.5, 3), Row(10.0, 2), Row(null, 3))) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items ORDER BY price DESC, id DESC", + Seq(Row(41.0, 1), Row(40.0, 1), Row(15.5, 3), + Row(15.5, null), Row(10.0, 2), Row(null, 3))) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items " + + s"ORDER BY price DESC NULLS FIRST, id DESC NULLS FIRST", + Seq(Row(null, 3), Row(41.0, 1), Row(40.0, 1), + Row(15.5, null), Row(15.5, 3), Row(10.0, 2))); + } + } + } + test("SPARK-49179: Fix v2 multi bucketed inner joins throw AssertionError") { val cols = Array( Column.create("id", LongType), From 769e569b7916467a9310a784048782c9df13f336 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 16 Dec 2024 09:22:09 +0800 Subject: [PATCH 183/438] [SPARK-49954][SQL][FOLLOWUP] Move states to SchemaOfJsonEvaluator ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/48473. We should follow the same approach of `schema_of_csv` and create `SchemaOfJsonEvaluator` to keep the states and the `schema_of_json` function can leverage it using `Invoke`. ### Why are the changes needed? code cleanup ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49184 from cloud-fan/json. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../json/JsonExpressionEvalUtils.scala | 62 ++++++++++--------- .../expressions/jsonExpressions.scala | 29 +++------ .../function_schema_of_json.explain | 2 +- ...nction_schema_of_json_with_options.explain | 2 +- 4 files changed, 42 insertions(+), 53 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala index 7a399aba4382c..cc2bc19c4f779 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala @@ -18,8 +18,6 @@ package org.apache.spark.sql.catalyst.expressions.json import java.io.CharArrayWriter -import com.fasterxml.jackson.core.JsonFactory - import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils @@ -31,34 +29,6 @@ import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, St import org.apache.spark.unsafe.types.{UTF8String, VariantVal} import org.apache.spark.util.Utils -object JsonExpressionEvalUtils { - - def schemaOfJson( - jsonFactory: JsonFactory, - jsonOptions: JSONOptions, - jsonInferSchema: JsonInferSchema, - json: UTF8String): UTF8String = { - val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => - parser.nextToken() - // To match with schema inference from JSON datasource. - jsonInferSchema.inferField(parser) match { - case st: StructType => - jsonInferSchema.canonicalizeType(st, jsonOptions).getOrElse(StructType(Nil)) - case at: ArrayType if at.elementType.isInstanceOf[StructType] => - jsonInferSchema - .canonicalizeType(at.elementType, jsonOptions) - .map(ArrayType(_, containsNull = at.containsNull)) - .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull)) - case other: DataType => - jsonInferSchema.canonicalizeType(other, jsonOptions).getOrElse( - SQLConf.get.defaultStringType) - } - } - - UTF8String.fromString(dt.sql) - } -} - case class JsonToStructsEvaluator( options: Map[String, String], nullableSchema: DataType, @@ -160,3 +130,35 @@ case class StructsToJsonEvaluator( converter(value) } } + +case class SchemaOfJsonEvaluator(options: Map[String, String]) { + @transient + private lazy val jsonOptions = new JSONOptions(options, "UTC") + + @transient + private lazy val jsonFactory = jsonOptions.buildJsonFactory() + + @transient + private lazy val jsonInferSchema = new JsonInferSchema(jsonOptions) + + final def evaluate(json: UTF8String): Any = { + val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => + parser.nextToken() + // To match with schema inference from JSON datasource. + jsonInferSchema.inferField(parser) match { + case st: StructType => + jsonInferSchema.canonicalizeType(st, jsonOptions).getOrElse(StructType(Nil)) + case at: ArrayType if at.elementType.isInstanceOf[StructType] => + jsonInferSchema + .canonicalizeType(at.elementType, jsonOptions) + .map(ArrayType(_, containsNull = at.containsNull)) + .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull)) + case other: DataType => + jsonInferSchema.canonicalizeType(other, jsonOptions).getOrElse( + SQLConf.get.defaultStringType) + } + } + + UTF8String.fromString(dt.sql) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index affc8261dc883..bb8cbdd3e6f18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, CodegenFallback, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper -import org.apache.spark.sql.catalyst.expressions.json.{JsonExpressionEvalUtils, JsonExpressionUtils, JsonToStructsEvaluator, StructsToJsonEvaluator} +import org.apache.spark.sql.catalyst.expressions.json.{JsonExpressionUtils, JsonToStructsEvaluator, SchemaOfJsonEvaluator, StructsToJsonEvaluator} import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.trees.TreePattern.{JSON_TO_STRUCT, RUNTIME_REPLACEABLE, TreePattern} @@ -849,15 +849,6 @@ case class SchemaOfJson( override def nullable: Boolean = false - @transient - private lazy val jsonOptions = new JSONOptions(options, "UTC") - - @transient - private lazy val jsonFactory = jsonOptions.buildJsonFactory() - - @transient - private lazy val jsonInferSchema = new JsonInferSchema(jsonOptions) - @transient private lazy val json = child.eval().asInstanceOf[UTF8String] @@ -878,19 +869,15 @@ case class SchemaOfJson( } } - @transient private lazy val jsonFactoryObjectType = ObjectType(classOf[JsonFactory]) - @transient private lazy val jsonOptionsObjectType = ObjectType(classOf[JSONOptions]) - @transient private lazy val jsonInferSchemaObjectType = ObjectType(classOf[JsonInferSchema]) + @transient + private lazy val evaluator: SchemaOfJsonEvaluator = SchemaOfJsonEvaluator(options) - override def replacement: Expression = StaticInvoke( - JsonExpressionEvalUtils.getClass, + override def replacement: Expression = Invoke( + Literal.create(evaluator, ObjectType(classOf[SchemaOfJsonEvaluator])), + "evaluate", dataType, - "schemaOfJson", - Seq(Literal(jsonFactory, jsonFactoryObjectType), - Literal(jsonOptions, jsonOptionsObjectType), - Literal(jsonInferSchema, jsonInferSchemaObjectType), - child), - Seq(jsonFactoryObjectType, jsonOptionsObjectType, jsonInferSchemaObjectType, child.dataType), + Seq(child), + Seq(child.dataType), returnNullable = false) override def prettyName: String = "schema_of_json" diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain index b400aeeca5af2..d75545d8766d0 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain @@ -1,2 +1,2 @@ -Project [static_invoke(JsonExpressionEvalUtils.schemaOfJson(com.fasterxml.jackson.core.JsonFactory, org.apache.spark.sql.catalyst.json.JSONOptions, org.apache.spark.sql.catalyst.json.JsonInferSchema, [{"col":01}])) AS schema_of_json([{"col":01}])#0] +Project [invoke(SchemaOfJsonEvaluator(Map()).evaluate([{"col":01}])) AS schema_of_json([{"col":01}])#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain index b400aeeca5af2..37321af1deed4 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain @@ -1,2 +1,2 @@ -Project [static_invoke(JsonExpressionEvalUtils.schemaOfJson(com.fasterxml.jackson.core.JsonFactory, org.apache.spark.sql.catalyst.json.JSONOptions, org.apache.spark.sql.catalyst.json.JsonInferSchema, [{"col":01}])) AS schema_of_json([{"col":01}])#0] +Project [invoke(SchemaOfJsonEvaluator(Map(allowNumericLeadingZeros -> true)).evaluate([{"col":01}])) AS schema_of_json([{"col":01}])#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] From 3df17696258f917de6aba10777e6acb5d8f94474 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 15 Dec 2024 18:34:32 -0800 Subject: [PATCH 184/438] [SPARK-50574][DOCS] Upgrade `rexml` to 3.3.9 ### What changes were proposed in this pull request? This PR aims to upgrade `rexml` to 3.3.9. ### Why are the changes needed? To use the latest bug fixed version in Apache Spark documentation generation. As a side note, this will resolve the following doc-generation Dependabot alerts. - https://github.com/apache/spark/security/dependabot/92 (Fixed in 3.2.7+) - https://github.com/apache/spark/security/dependabot/96 (Fixed in 3.3.2+) - https://github.com/apache/spark/security/dependabot/97 (Fixed in 3.3.3+) - https://github.com/apache/spark/security/dependabot/98 (Fixed in 3.3.3+) - https://github.com/apache/spark/security/dependabot/100 (Fixed in 3.3.6+) - https://github.com/apache/spark/security/dependabot/107 (Fixed in 3.3.9+) ### Does this PR introduce _any_ user-facing change? No, there is no behavior change. This is a doc-generation-related change. ### How was this patch tested? ``` $ cd docs $ SKIP_API=1 bundle exec jekyll build ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49181 from dongjoon-hyun/SPARK-50574. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/Gemfile | 1 + docs/Gemfile.lock | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/Gemfile b/docs/Gemfile index db1ba08204f67..68727dee9e1fb 100644 --- a/docs/Gemfile +++ b/docs/Gemfile @@ -27,3 +27,4 @@ gem "jekyll-redirect-from", "~> 0.16" # This resolves a build issue on Apple Silicon. # See: https://issues.apache.org/jira/browse/SPARK-38488 gem "ffi", "~> 1.15" +gem "rexml", "~> 3.3.9" diff --git a/docs/Gemfile.lock b/docs/Gemfile.lock index 286c8fe97948d..7709f07a1ceea 100644 --- a/docs/Gemfile.lock +++ b/docs/Gemfile.lock @@ -53,7 +53,7 @@ GEM rb-fsevent (0.11.2) rb-inotify (0.10.1) ffi (~> 1.0) - rexml (3.2.6) + rexml (3.3.9) rouge (4.5.1) safe_yaml (1.0.5) sass-embedded (1.63.6) @@ -71,6 +71,7 @@ DEPENDENCIES ffi (~> 1.15) jekyll (~> 4.3) jekyll-redirect-from (~> 0.16) + rexml (~> 3.3.9) BUNDLED WITH 2.4.22 From 7d7711de86c91fa53d1629a43e3bad1794667b60 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sun, 15 Dec 2024 18:35:34 -0800 Subject: [PATCH 185/438] [SPARK-50576][BUILD] Upgrade `common-text` to `1.13.0` ### What changes were proposed in this pull request? This pr aims to upgrade `common-text` from `1.12.0` to `1.13.0`. ### Why are the changes needed? The full release notes as follows: - https://github.com/apache/commons-text/blob/rel/commons-text-1.13.0/RELEASE-NOTES.txt ### 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 #49185 from panbingkun/SPARK-50576. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- 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 4721941937a2d..53ab9ade891d7 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -49,7 +49,7 @@ commons-lang/2.6//commons-lang-2.6.jar commons-lang3/3.17.0//commons-lang3-3.17.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.12.0//commons-text-1.12.0.jar +commons-text/1.13.0//commons-text-1.13.0.jar compress-lzf/1.1.2//compress-lzf-1.1.2.jar curator-client/5.7.1//curator-client-5.7.1.jar curator-framework/5.7.1//curator-framework-5.7.1.jar diff --git a/pom.xml b/pom.xml index 674b74c979ef2..36d9a3dcb1298 100644 --- a/pom.xml +++ b/pom.xml @@ -631,7 +631,7 @@ org.apache.commons commons-text - 1.12.0 + 1.13.0 commons-lang From e7a2e4b8f4cf73ffe246c418990c96ce9b36606a Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 15 Dec 2024 18:38:58 -0800 Subject: [PATCH 186/438] [MINOR][DOCS] Modify a link in running-on-yarn.md ### What changes were proposed in this pull request? This PR fixes a minor issue that a link in `running-on-yarn.md`. The linked page should be about building with YARN profile. ### Why are the changes needed? To make the doc better. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Built the doc and generated html files, then confirmed the modified link worked. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49192 from sarutak/minor-yarn-doc. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b6f847ff533f5..465f3a9d075a2 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -853,7 +853,7 @@ will include a list of all tokens obtained, and their expiry details To start the Spark Shuffle Service on each `NodeManager` in your YARN cluster, follow these instructions: -1. Build Spark with the [YARN profile](building-spark.html). Skip this step if you are using a +1. Build Spark with the [YARN profile](building-spark.html#specifying-the-hadoop-version-and-enabling-yarn). Skip this step if you are using a pre-packaged distribution. 1. Locate the `spark--yarn-shuffle.jar`. This should be under `$SPARK_HOME/common/network-yarn/target/scala-` if you are building Spark yourself, and under From 279d5eefef322507c5bee6c9e8c2c6c4af1bd512 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Sun, 15 Dec 2024 18:40:10 -0800 Subject: [PATCH 187/438] [SPARK-50564][PYTHON] Upgrade `protobuf` Python package to 5.29.1 ### What changes were proposed in this pull request? This PR aims to upgrade protobuf Python package from 5.28.3 to 5.29.1. ### Why are the changes needed? To bring the latest bug fixes, such as - Fix a python bug when assign an empty Struct at creation. (https://github.com/protocolbuffers/protobuf/pull/18977) (https://github.com/protocolbuffers/protobuf/commit/bb0d57512d7731164837f57274e23eb411f80ccf), this one only introduced since 5.28.x - Fix cord handling in DynamicMessage and oneofs. (https://github.com/protocolbuffers/protobuf/commit/9e8b30c2135944d9d3a6e53b70e534581e2dbe2b) - Fix packed reflection handling bug in edition 2023. (https://github.com/protocolbuffers/protobuf/commit/4c923285a32ebcb53ce2ba03144e50d4dbedaf7b) - Fix ContainerBase::DeepCopy to not modify the source object. (https://github.com/protocolbuffers/protobuf/commit/9fa1f4f9bb74776ee65847ca960a1f023eac3ebe) Full release notes as follows: - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0-rc1 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0-rc2 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0-rc3 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.0 - https://github.com/protocolbuffers/protobuf/releases/tag/v29.1 - https://pypi.org/project/protobuf/5.29.1/ ### 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 #49170 from LuciferYang/protobuf-python-5.29.1. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 4 ++-- .github/workflows/build_python_connect35.yml | 2 +- .github/workflows/maven_test.yml | 2 +- .github/workflows/python_macos_test.yml | 2 +- dev/create-release/spark-rm/Dockerfile | 2 +- dev/infra/Dockerfile | 2 +- dev/requirements.txt | 2 +- dev/spark-test-image/python-309/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 900c2a7453202..533581ce4b6d2 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -340,7 +340,7 @@ jobs: - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') run: | - python3.11 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.28.3' + python3.11 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' python3.11 -m pip list # Run the tests. - name: Run tests @@ -726,7 +726,7 @@ jobs: python-version: '3.11' - name: Install dependencies for Python CodeGen check run: | - python3.11 -m pip install 'black==23.9.1' 'protobuf==5.28.3' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' + python3.11 -m pip install 'black==23.9.1' 'protobuf==5.29.1' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' python3.11 -m pip list - name: Python CodeGen check for branch-3.5 if: inputs.branch == 'branch-3.5' diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml index ad250d95fb844..b292553f99098 100644 --- a/.github/workflows/build_python_connect35.yml +++ b/.github/workflows/build_python_connect35.yml @@ -70,7 +70,7 @@ jobs: pip install 'numpy==1.25.1' 'pyarrow==12.0.1' 'pandas<=2.0.3' scipy unittest-xml-reporting plotly>=4.8 'mlflow>=2.3.1' coverage 'matplotlib==3.7.2' openpyxl 'memory-profiler==0.60.0' 'scikit-learn==1.1.*' # Add Python deps for Spark Connect. - pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.28.3' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' + pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' # Add torch as a testing dependency for TorchDistributor pip install 'torch==2.0.1' 'torchvision==0.15.2' torcheval diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 6965fb4968af3..206806a7a0ed7 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -178,7 +178,7 @@ jobs: - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql#core')) || contains(matrix.modules, 'connect') run: | - python3.11 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.28.3' + python3.11 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' python3.11 -m pip list # Run the tests. - name: Run tests diff --git a/.github/workflows/python_macos_test.yml b/.github/workflows/python_macos_test.yml index cca133dab541a..231816750236b 100644 --- a/.github/workflows/python_macos_test.yml +++ b/.github/workflows/python_macos_test.yml @@ -134,7 +134,7 @@ jobs: python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' python${{matrix.python}} -m pip install numpy 'pyarrow>=15.0.0' 'six==1.16.0' 'pandas==2.2.3' scipy 'plotly>=4.8' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ - python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.28.3' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ + python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ python${{matrix.python}} -m pip cache purge && \ python${{matrix.python}} -m pip list # Run the tests. diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 8a9890bf68dde..ae6b7cfea6f22 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -102,7 +102,7 @@ RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.3' scipy coverage matp ARG BASIC_PIP_PKGS="numpy pyarrow>=15.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 2817818cbc4e3..99cd887c6b6ea 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -96,7 +96,7 @@ RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.3' scipy coverage matp ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/requirements.txt b/dev/requirements.txt index a9874f77113ab..04cab4cbfcc3c 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -61,7 +61,7 @@ py grpcio>=1.67.0 grpcio-status>=1.67.0 googleapis-common-protos>=1.65.0 -protobuf==5.28.3 +protobuf==5.29.1 # Spark Connect python proto generation plugin (optional) mypy-protobuf==3.3.0 diff --git a/dev/spark-test-image/python-309/Dockerfile b/dev/spark-test-image/python-309/Dockerfile index c2104dca95fc1..2b4d5bc258012 100644 --- a/dev/spark-test-image/python-309/Dockerfile +++ b/dev/spark-test-image/python-309/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.9 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index b86ce88b9807f..26748dd26b060 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -65,7 +65,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 6287caca648f2..33956b05b2d31 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index ecfb1ab07123c..29ec14290c4b7 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.12 at the last stage to avoid breaking the existing Python installations RUN apt-get update && apt-get install -y \ diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index 86232da71097c..50d634b72602d 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -68,7 +68,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0 graphviz==0.20.3" # Install Python 3.13 packages From a6f82c64e3e482a575a07c22b99c8e51d6099fd4 Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Mon, 16 Dec 2024 12:32:07 +0900 Subject: [PATCH 188/438] [SPARK-49461][SS][TESTS][FOLLOWUP] Move related resource files to correct place ### What changes were proposed in this pull request? I created a new resource location `sql/core/src/test/{scala/org/apache/spark/sql/streaming/resources` before, moving it to the correct structured-streaming resource folder. ### Why are the changes needed? Better code structure ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #49172 from WweiL/SPARK-49461-followup. Authored-by: Wei Liu Signed-off-by: Jungtaek Lim --- .../testCommitLogV1/testCommitLog | 0 .../testCommitLogV2/testCommitLog | 0 .../spark/sql/streaming/CommitLogSuite.scala | 14 ++------------ 3 files changed, 2 insertions(+), 12 deletions(-) rename sql/core/src/test/{scala/org/apache/spark/sql/streaming/resources => resources/structured-streaming}/testCommitLogV1/testCommitLog (100%) rename sql/core/src/test/{scala/org/apache/spark/sql/streaming/resources => resources/structured-streaming}/testCommitLogV2/testCommitLog (100%) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/resources/testCommitLogV1/testCommitLog b/sql/core/src/test/resources/structured-streaming/testCommitLogV1/testCommitLog similarity index 100% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/resources/testCommitLogV1/testCommitLog rename to sql/core/src/test/resources/structured-streaming/testCommitLogV1/testCommitLog diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/resources/testCommitLogV2/testCommitLog b/sql/core/src/test/resources/structured-streaming/testCommitLogV2/testCommitLog similarity index 100% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/resources/testCommitLogV2/testCommitLog rename to sql/core/src/test/resources/structured-streaming/testCommitLogV2/testCommitLog diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala index bdf4163c72068..068f56839e6e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala @@ -39,13 +39,8 @@ class CommitLogSuite extends SparkFunSuite with SharedSparkSession { "core", "src", "test", - "scala", - "org", - "apache", - "spark", - "sql", - "streaming", "resources", + "structured-streaming", "testCommitLogV2" ) } @@ -56,13 +51,8 @@ class CommitLogSuite extends SparkFunSuite with SharedSparkSession { "core", "src", "test", - "scala", - "org", - "apache", - "spark", - "sql", - "streaming", "resources", + "structured-streaming", "testCommitLogV1" ) } From ef37f9a8e202812ea6b710bfb084ce176ff5e63d Mon Sep 17 00:00:00 2001 From: Dima Date: Mon, 16 Dec 2024 12:34:13 +0900 Subject: [PATCH 189/438] [SPARK-50568][SS][TESTS] Fix `KafkaMicroBatchSourceSuite` to cover `KafkaOffsetReaderConsumer` ### What changes were proposed in this pull request? This was missed when the default value of configuration is changed via the following. - https://github.com/apache/spark/pull/38306 KafkaMicroBatchSourceSuite consists of set of different suites, where KafkaMicroBatchSourceSuiteBase based suite is defined. There are 4 implementations of this abstract class for now: 1. `KafkaMicroBatchV1SourceSuite` - V1 source that supposes to use `KafkaOffsetReaderConsumer` as `KafkaOffsetReader`. 2. `KafkaMicroBatchV2SourceSuite` - V2 source that supposes to use `KafkaOffsetReaderConsumer` as `KafkaOffsetReader`. 3. `KafkaMicroBatchV1SourceWithAdminSuite` - V1 source that uses `KafkaOffsetReaderAdmin` as `KafkaOffsetReader`. 4. `KafkaMicroBatchV2SourceWithAdminSuite` - V2 source that uses `KafkaOffsetReaderAdmin` as `KafkaOffsetReader`. But `KafkaMicroBatchV1SourceSuite` and `KafkaMicroBatchV2SourceSuite` are still running based on `KafkaOffsetReaderAdmin`, as `USE_DEPRECATED_KAFKA_OFFSET_FETCHING` is `false` be default. By switching it to `true` in `beforeAll`, we make sure that corresponding `KafkaOffsetReader` is in use. ### Why are the changes needed? To improve unit tests coverage for `KafkaOffsetReaderConsumer` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit Tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49164 from ostronaut/hotifx/KafkaMicroBatchSourceSuite-cover-KafkaOffsetReaderConsumer. Authored-by: Dima Signed-off-by: Jungtaek Lim --- .../kafka010/KafkaMicroBatchSourceSuite.scala | 48 ++++++++++++------- 1 file changed, 31 insertions(+), 17 deletions(-) diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 1d119de43970f..22eeae97874b1 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -1591,22 +1591,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase with } } - -class KafkaMicroBatchV1SourceWithAdminSuite extends KafkaMicroBatchV1SourceSuite { - override def beforeAll(): Unit = { - super.beforeAll() - spark.conf.set(SQLConf.USE_DEPRECATED_KAFKA_OFFSET_FETCHING.key, "false") - } -} - -class KafkaMicroBatchV2SourceWithAdminSuite extends KafkaMicroBatchV2SourceSuite { - override def beforeAll(): Unit = { - super.beforeAll() - spark.conf.set(SQLConf.USE_DEPRECATED_KAFKA_OFFSET_FETCHING.key, "false") - } -} - -class KafkaMicroBatchV1SourceSuite extends KafkaMicroBatchSourceSuiteBase { +abstract class KafkaMicroBatchV1SourceSuite extends KafkaMicroBatchSourceSuiteBase { override def beforeAll(): Unit = { super.beforeAll() spark.conf.set( @@ -1637,7 +1622,7 @@ class KafkaMicroBatchV1SourceSuite extends KafkaMicroBatchSourceSuiteBase { } } -class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { +abstract class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { test("V2 Source is used by default") { val topic = newTopic() @@ -1870,6 +1855,35 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { } } +class KafkaMicroBatchV1SourceWithAdminSuite extends KafkaMicroBatchV1SourceSuite { + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set(SQLConf.USE_DEPRECATED_KAFKA_OFFSET_FETCHING.key, "false") + } +} + +class KafkaMicroBatchV1SourceWithConsumerSuite extends KafkaMicroBatchV1SourceSuite { + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set(SQLConf.USE_DEPRECATED_KAFKA_OFFSET_FETCHING.key, "true") + } +} + +class KafkaMicroBatchV2SourceWithAdminSuite extends KafkaMicroBatchV2SourceSuite { + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set(SQLConf.USE_DEPRECATED_KAFKA_OFFSET_FETCHING.key, "false") + } +} + +class KafkaMicroBatchV2SourceWithConsumerSuite extends KafkaMicroBatchV2SourceSuite { + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set(SQLConf.USE_DEPRECATED_KAFKA_OFFSET_FETCHING.key, "true") + } +} + + abstract class KafkaSourceSuiteBase extends KafkaSourceTest { import testImplicits._ From 15dcd21323fa0cc015e965c6443ff3875108342a Mon Sep 17 00:00:00 2001 From: Zhihong Yu Date: Mon, 16 Dec 2024 12:38:07 +0900 Subject: [PATCH 190/438] [SPARK-50360][SS][FOLLOWUP][MINOR] Make readVersion lazy value ### What changes were proposed in this pull request? This is followup to https://github.com/apache/spark/pull/48880 `readVersion` is made a `lazy` value. After `input` is closed, its value is not changed to `null` because `input` is only used by `readVersion`. ### Why are the changes needed? `StateStoreChangelogReaderFactory` is able to produce more than one reader. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing test suite. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49091 from tedyu/null-input. Authored-by: Zhihong Yu Signed-off-by: Jungtaek Lim --- .../sql/execution/streaming/state/StateStoreChangelog.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala index f6787a37bc80d..b4fbb5560f2f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala @@ -353,7 +353,7 @@ class StateStoreChangelogReaderFactory( } protected val input: DataInputStream = decompressStream(sourceStream) - private def readVersion(): Short = { + private lazy val changeLogVersion: Short = { try { val versionStr = input.readUTF() // Versions in the first line are prefixed with "v", e.g. "v2" @@ -379,7 +379,7 @@ class StateStoreChangelogReaderFactory( def constructChangelogReader(): StateStoreChangelogReader = { var reader: StateStoreChangelogReader = null try { - reader = readVersion() match { + reader = changeLogVersion match { case 1 => new StateStoreChangelogReaderV1(fm, fileToRead, compressionCodec) case 2 => new StateStoreChangelogReaderV2(fm, fileToRead, compressionCodec) case 3 => new StateStoreChangelogReaderV3(fm, fileToRead, compressionCodec) @@ -389,6 +389,7 @@ class StateStoreChangelogReaderFactory( } finally { if (input != null) { input.close() + // input is not set to null because it is effectively lazy. } } reader From 3f8e3959108dd734aca531992c9a22f4ddf79ba9 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 16 Dec 2024 13:18:57 +0900 Subject: [PATCH 191/438] [SPARK-50583][INFRA] Apply Python 3.11 image in PR build ### What changes were proposed in this pull request? Apply Python 3.11 image in PR build ### Why are the changes needed? for `master`: - after this PR, all workflows should only use the new images; - a new workflow should specify `PYSPARK_IMAGE_TO_TEST` and `PYSPARK_TO_TEST` if `pyspark` job is enabled; for `branch-3.5`: - it still uses the old image `dev/infra/Dockerfile` ### 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 #49193 from zhengruifeng/py_311_pr_builder. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .github/workflows/build_and_test.yml | 38 ++++++++++++++------- .github/workflows/build_branch35.yml | 1 + .github/workflows/build_branch35_python.yml | 1 + 3 files changed, 28 insertions(+), 12 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 533581ce4b6d2..174a8aa6bacb0 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -41,7 +41,7 @@ on: description: Additional environment variables to set when running the tests. Should be in JSON format. required: false type: string - default: '{}' + default: '{"PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11"}' jobs: description: >- Jobs to run, and should be in JSON format. The values should be matched with the job's key defined @@ -55,7 +55,6 @@ jobs: runs-on: ubuntu-latest env: GITHUB_PREV_SHA: ${{ github.event.before }} - PYSPARK_IMAGE_TO_TEST: '' outputs: required: ${{ steps.set-outputs.outputs.required }} image_url: ${{ steps.infra-image-outputs.outputs.image_url }} @@ -135,6 +134,28 @@ jobs: precondition="${precondition//$'\n'/}" echo "required=$precondition" >> $GITHUB_OUTPUT fi + - name: Check envs + id: check-envs + if: inputs.branch != 'branch-3.5' + env: ${{ fromJSON(inputs.envs) }} + run: | + if [[ "${{ fromJson(steps.set-outputs.outputs.required).pyspark }}" == 'true' ]]; then + if [[ "${{ env.PYSPARK_IMAGE_TO_TEST }}" == "" ]]; then + echo "PYSPARK_IMAGE_TO_TEST is required when pyspark is enabled." + exit 1 + fi + PYSPARK_IMAGE_PATH="dev/spark-test-image/${{ env.PYSPARK_IMAGE_TO_TEST }}/Dockerfile" + if [ -f $PYSPARK_IMAGE_PATH ]; then + echo "Dockerfile $PYSPARK_IMAGE_PATH exists." + else + echo "Dockerfile $PYSPARK_IMAGE_PATH does NOT exist." + exit 1 + fi + if [[ "${{ env.PYTHON_TO_TEST }}" == "" ]]; then + echo "PYTHON_TO_TEST is required when pyspark is enabled." + exit 1 + fi + fi - name: Generate infra image URL id: infra-image-outputs run: | @@ -192,11 +213,7 @@ jobs: echo "image_docs_url_link=${{ steps.infra-image-docs-outputs.outputs.image_docs_url }}" >> $GITHUB_OUTPUT echo "image_lint_url_link=${{ steps.infra-image-lint-outputs.outputs.image_lint_url }}" >> $GITHUB_OUTPUT echo "image_sparkr_url_link=${{ steps.infra-image-sparkr-outputs.outputs.image_sparkr_url }}" >> $GITHUB_OUTPUT - if [[ "${{ env.PYSPARK_IMAGE_TO_TEST }}" != "" ]]; then - echo "image_pyspark_url_link=${{ steps.infra-image-pyspark-outputs.outputs.image_pyspark_url }}" >> $GITHUB_OUTPUT - else - echo "image_pyspark_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT - fi + echo "image_pyspark_url_link=${{ steps.infra-image-pyspark-outputs.outputs.image_pyspark_url }}" >> $GITHUB_OUTPUT fi # Build: build Spark and run the tests for specified modules. @@ -380,8 +397,6 @@ jobs: runs-on: ubuntu-latest permissions: packages: write - env: - PYSPARK_IMAGE_TO_TEST: '' steps: - name: Login to GitHub Container Registry uses: docker/login-action@v3 @@ -450,8 +465,8 @@ jobs: ${{ needs.precondition.outputs.image_sparkr_url }} # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ inputs.branch }} - - name: Build and push (PySpark ${{ env.PYSPARK_IMAGE_TO_TEST }}) - if: ${{ env.PYSPARK_IMAGE_TO_TEST }} + - name: Build and push (PySpark with ${{ env.PYSPARK_IMAGE_TO_TEST }}) + if: (inputs.branch != 'branch-3.5') && (${{ env.PYSPARK_IMAGE_TO_TEST }} != '') id: docker_build_pyspark env: ${{ fromJSON(inputs.envs) }} uses: docker/build-push-action@v6 @@ -511,7 +526,6 @@ jobs: - modules: ${{ fromJson(needs.precondition.outputs.required).pyspark-pandas != 'true' && 'pyspark-pandas-connect-part3' }} env: MODULES_TO_TEST: ${{ matrix.modules }} - PYTHON_TO_TEST: 'python3.11' HADOOP_PROFILE: ${{ inputs.hadoop }} HIVE_PROFILE: hive2.3 # GitHub Actions' default miniconda to use in pip packaging test. diff --git a/.github/workflows/build_branch35.yml b/.github/workflows/build_branch35.yml index 2ec080d5722c1..ad46b6ce32387 100644 --- a/.github/workflows/build_branch35.yml +++ b/.github/workflows/build_branch35.yml @@ -37,6 +37,7 @@ jobs: envs: >- { "SCALA_PROFILE": "scala2.13", + "PYSPARK_IMAGE_TO_TEST": "", "PYTHON_TO_TEST": "", "ORACLE_DOCKER_IMAGE_NAME": "gvenzl/oracle-xe:21.3.0" } diff --git a/.github/workflows/build_branch35_python.yml b/.github/workflows/build_branch35_python.yml index 1585534d33ba9..ed83f8c4cdf88 100644 --- a/.github/workflows/build_branch35_python.yml +++ b/.github/workflows/build_branch35_python.yml @@ -36,6 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { + "PYSPARK_IMAGE_TO_TEST": "", "PYTHON_TO_TEST": "" } jobs: >- From d293ba66a0b12cd8a959978ed3b62fe7054a5e4f Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Mon, 16 Dec 2024 13:27:06 +0900 Subject: [PATCH 192/438] [SPARK-50511][PYTHON] Avoid wrapping Python data source error messages ### What changes were proposed in this pull request? This PR removes the extra try catch during Python data source creation and planning. This is to avoid the redundancy of having `During handling of the above exception, another exception occurred`: ``` AnalysisException: [PYTHON_DATA_SOURCE_ERROR] Failed to create Python data source instance: Traceback (most recent call last): File "/Users/allison.wang/oss/spark/python/pyspark/sql/worker/create_data_source.py", line 145, in main raise PySparkRuntimeError( ^^^^^^^^^^^^^^^^^^ File "/Users/allison.wang/oss/spark/python/pyspark/sql/datasource.py", line 118, in schema raise PySparkNotImplementedError( pyspark.errors.exceptions.base.PySparkNotImplementedError: [NOT_IMPLEMENTED] schema is not implemented. During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/Users/allison.wang/oss/spark/python/pyspark/sql/worker/create_data_source.py", line 151, in main pyspark.errors.exceptions.base.PySparkRuntimeError: [NOT_IMPLEMENTED] DataSource.schema is not implemented. SQLSTATE: 38000 ``` ### Why are the changes needed? To improve error messages ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #49092 from allisonwang-db/spark-50511-pyds-err-msg. Authored-by: Allison Wang Signed-off-by: Hyukjin Kwon --- python/docs/source/development/debugging.rst | 2 +- python/pyspark/errors/error-conditions.json | 5 ---- .../pyspark/sql/worker/create_data_source.py | 26 +++++-------------- .../python/UserDefinedPythonDataSource.scala | 8 +++--- 4 files changed, 12 insertions(+), 29 deletions(-) diff --git a/python/docs/source/development/debugging.rst b/python/docs/source/development/debugging.rst index 9510fe0abde1e..0aa2426cf862d 100644 --- a/python/docs/source/development/debugging.rst +++ b/python/docs/source/development/debugging.rst @@ -669,7 +669,7 @@ Stack Traces There are Spark configurations to control stack traces: -- ``spark.sql.execution.pyspark.udf.simplifiedTraceback.enabled`` is true by default to simplify traceback from Python UDFs. +- ``spark.sql.execution.pyspark.udf.simplifiedTraceback.enabled`` is true by default to simplify traceback from Python UDFs and Data Sources. - ``spark.sql.pyspark.jvmStacktrace.enabled`` is false by default to hide JVM stacktrace and to show a Python-friendly exception only. diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index dbb4bb0087e06..c4ad3f8d5feba 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -189,11 +189,6 @@ "Remote client cannot create a SparkContext. Create SparkSession instead." ] }, - "DATA_SOURCE_CREATE_ERROR": { - "message": [ - "Failed to create python data source instance, error: ." - ] - }, "DATA_SOURCE_INVALID_RETURN_TYPE": { "message": [ "Unsupported return type ('') from Python data source ''. Expected types: ." diff --git a/python/pyspark/sql/worker/create_data_source.py b/python/pyspark/sql/worker/create_data_source.py index ef70876fc32c5..f74c1555e6e9e 100644 --- a/python/pyspark/sql/worker/create_data_source.py +++ b/python/pyspark/sql/worker/create_data_source.py @@ -20,7 +20,7 @@ from typing import IO from pyspark.accumulators import _accumulatorRegistry -from pyspark.errors import PySparkAssertionError, PySparkRuntimeError, PySparkTypeError +from pyspark.errors import PySparkAssertionError, PySparkTypeError from pyspark.serializers import ( read_bool, read_int, @@ -127,13 +127,7 @@ def main(infile: IO, outfile: IO) -> None: options[key] = value # Instantiate a data source. - try: - data_source = data_source_cls(options=options) # type: ignore - except Exception as e: - raise PySparkRuntimeError( - errorClass="DATA_SOURCE_CREATE_ERROR", - messageParameters={"error": str(e)}, - ) + data_source = data_source_cls(options=options) # type: ignore # Get the schema of the data source. # If user_specified_schema is not None, use user_specified_schema. @@ -141,17 +135,11 @@ def main(infile: IO, outfile: IO) -> None: # Throw exception if the data source does not implement schema(). is_ddl_string = False if user_specified_schema is None: - try: - schema = data_source.schema() - if isinstance(schema, str): - # Here we cannot use _parse_datatype_string to parse the DDL string schema. - # as it requires an active Spark session. - is_ddl_string = True - except NotImplementedError: - raise PySparkRuntimeError( - errorClass="NOT_IMPLEMENTED", - messageParameters={"feature": "DataSource.schema"}, - ) + schema = data_source.schema() + if isinstance(schema, str): + # Here we cannot use _parse_datatype_string to parse the DDL string schema. + # as it requires an active Spark session. + is_ddl_string = True else: schema = user_specified_schema # type: ignore diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala index 241d8087fc3c2..b3fd8479bda0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala @@ -307,7 +307,7 @@ case class PythonDataSourceReadInfo( /** * Send information to a Python process to plan a Python data source read. * - * @param func an Python data source instance + * @param func a Python data source instance * @param inputSchema input schema to the data source read from its child plan * @param outputSchema output schema of the Python data source */ @@ -342,7 +342,7 @@ private class UserDefinedPythonDataSourceReadRunner( if (length == SpecialLengths.PYTHON_EXCEPTION_THROWN) { val msg = PythonWorkerUtils.readUTF(dataIn) throw QueryCompilationErrors.pythonDataSourceError( - action = "plan", tpe = "read", msg = msg) + action = "initialize", tpe = "reader", msg = msg) } // Receive the pickled 'read' function. @@ -354,7 +354,7 @@ private class UserDefinedPythonDataSourceReadRunner( if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) { val msg = PythonWorkerUtils.readUTF(dataIn) throw QueryCompilationErrors.pythonDataSourceError( - action = "plan", tpe = "read", msg = msg) + action = "generate", tpe = "read partitions", msg = msg) } for (_ <- 0 until numPartitions) { val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn) @@ -420,7 +420,7 @@ private class UserDefinedPythonDataSourceWriteRunner( if (length == SpecialLengths.PYTHON_EXCEPTION_THROWN) { val msg = PythonWorkerUtils.readUTF(dataIn) throw QueryCompilationErrors.pythonDataSourceError( - action = "plan", tpe = "write", msg = msg) + action = "initialize", tpe = "writer", msg = msg) } // Receive the pickled data source write function. From 0faf9d5da06dd44e8138883dd1c4fbde1fbd3f7c Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 16 Dec 2024 15:04:46 +0900 Subject: [PATCH 193/438] [SPARK-50579][SQL] Fix `truncatedString` ### What changes were proposed in this pull request? In the PR, I propose to respect the `maxFields` argument of `SparkStringUtils.truncatedString` in all cases independently from the size of the another argument `seq`. If size of `seq` is grater or equal to `maxFields`, output exact `maxFields` elements of `seq`. Also, if the number of printed elements is zero, don't print the redundant comma `,` like `[, ... 100 more fields]`, and don't overflow when `maxFields` is `Int.MinValue`. ### Why are the changes needed? To make output consistent for the same `maxFields`. For example, if `maxFields` is 2: ```scala truncatedString(Seq(1, 2), "[", ", ", "]", maxFields = 2) -> "[1, 2]" ``` but for more elements in the input sequence, it prints only one element: ```scala truncatedString(Seq(1, 2, 3), "[", ", ", "]", maxFields = 2) -> "[1, ... 2 more fields]" ``` though the expected output is **"[1, 2, ... 1 more fields]"**. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By existing tests from `UtilSuite` that are moved to `StringUtilsSuite` and additional checks: ``` $ build/sbt "test:testOnly *StringUtilsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49187 from MaxGekk/fix-truncatedString. Authored-by: Max Gekk Signed-off-by: Hyukjin Kwon --- .../spark/sql/catalyst/util/StringUtils.scala | 11 ++++--- .../sql/catalyst/util/StringUtilsSuite.scala | 15 +++++++++ .../org/apache/spark/sql/util/UtilSuite.scala | 31 ------------------- .../explain-results/function_lit.explain | 2 +- .../explain-results/function_typedLit.explain | 2 +- .../analyzer-results/group-analytics.sql.out | 2 +- .../analyzer-results/postgreSQL/join.sql.out | 2 +- .../typeCoercion/native/concat.sql.out | 20 ++++++------ .../typeCoercion/native/mapconcat.sql.out | 12 +++---- .../udf/postgreSQL/udf-join.sql.out | 2 +- .../analyzer-results/udf/udf-window.sql.out | 8 ++--- .../sql-tests/analyzer-results/window.sql.out | 8 ++--- 12 files changed, 50 insertions(+), 65 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index 0608322be13b3..e8c50be9f5513 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -73,7 +73,7 @@ object SparkStringUtils extends Logging { /** * Format a sequence with semantics similar to calling .mkString(). Any elements beyond - * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. + * `maxFields` will be dropped and replaced by a "... N more fields" placeholder. * * @return * the trimmed and formatted string. @@ -90,10 +90,11 @@ object SparkStringUtils extends Logging { "Truncated the string representation of a plan since it was too large. This " + s"behavior can be adjusted by setting 'spark.sql.debug.maxToStringFields'.") } - val numFields = math.max(0, maxFields - 1) - seq - .take(numFields) - .mkString(start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) + val numFields = math.max(0, maxFields) + val restNum = seq.length - numFields + val ending = (if (numFields == 0) "" else sep) + + (if (restNum == 0) "" else s"... $restNum more fields") + end + seq.take(numFields).mkString(start, sep, ending) } else { seq.mkString(start, sep, end) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala index 385850376d147..fb4053964a841 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala @@ -136,4 +136,19 @@ class StringUtilsSuite extends SparkFunSuite with SQLHelper { val expectedOutput = Seq("`c1`", "`v2.c2`", "`v1`.`c2`") assert(orderSuggestedIdentifiersBySimilarity(baseString, testStrings) === expectedOutput) } + + test("SPARK-50579: truncated string") { + assert(truncatedString(Seq.empty, ", ", -1) === "") + assert(truncatedString(Seq("a"), ", ", -1) === "... 1 more fields") + assert(truncatedString(Seq("B"), "(", ", ", ")", -1) === "(... 1 more fields)") + assert(truncatedString(Seq.empty, ", ", 0) === "") + assert(truncatedString(Seq.empty, "[", ", ", "]", 0) === "[]") + assert(truncatedString(Seq("a", "b"), ", ", 0) === "... 2 more fields") + assert(truncatedString(Seq.empty, ",", 1) === "") + assert(truncatedString(Seq("a"), ",", 1) === "a") + assert(truncatedString(Seq("a", "b"), ", ", 1) === "a, ... 1 more fields") + assert(truncatedString(Seq("a", "b"), ", ", 2) === "a, b") + assert(truncatedString(Seq("a", "b", "c"), ", ", Int.MaxValue) === "a, b, c") + assert(truncatedString(Seq("a", "b", "c"), ", ", Int.MinValue) === "... 3 more fields") + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala deleted file mode 100644 index d95de71e897a2..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.util - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.truncatedString - -class UtilSuite extends SparkFunSuite { - test("truncatedString") { - assert(truncatedString(Nil, "[", ", ", "]", 2) == "[]") - assert(truncatedString(Seq(1, 2), "[", ", ", "]", 2) == "[1, 2]") - assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", 2) == "[1, ... 2 more fields]") - assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", -5) == "[, ... 3 more fields]") - assert(truncatedString(Seq(1, 2, 3), ", ", 10) == "1, 2, 3") - } -} diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain index 928dd0bf85cc7..4491b6166afae 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain @@ -1,2 +1,2 @@ -Project [id#0L, id#0L, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, 2023-02-23 AS DATE '2023-02-23'#0, ... 3 more fields] +Project [id#0L, id#0L, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, 2023-02-23 AS DATE '2023-02-23'#0, INTERVAL '0 00:03:20' DAY TO SECOND AS INTERVAL '0 00:03:20' DAY TO SECOND#0, ... 2 more fields] +- LocalRelation , [id#0L, a#0, b#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain index 606bb694bad47..6d854da250fcc 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain @@ -1,2 +1,2 @@ -Project [id#0L, id#0L, 1 AS 1#0, null AS NULL#0, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, ... 19 more fields] +Project [id#0L, id#0L, 1 AS 1#0, null AS NULL#0, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, ... 18 more fields] +- LocalRelation , [id#0L, a#0, b#0] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out index cdb6372bec099..8b2e55f126287 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out @@ -630,7 +630,7 @@ Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, coun SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ()) -- !query analysis Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] -+- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, b#x, 0, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, b#x, 0, 7], [a#x, b#x, a#x, b#x, 0, 8], [a#x, b#x, a#x, b#x, 0, 9], [a#x, b#x, a#x, b#x, 0, 10], [a#x, b#x, a#x, b#x, 0, 11], [a#x, b#x, a#x, b#x, 0, 12], [a#x, b#x, a#x, null, 1, 13], [a#x, b#x, a#x, null, 1, 14], [a#x, b#x, a#x, b#x, 0, 15], [a#x, b#x, a#x, null, 1, 16], [a#x, b#x, a#x, null, 1, 17], [a#x, b#x, a#x, b#x, 0, 18], [a#x, b#x, a#x, b#x, 0, 19], [a#x, b#x, a#x, b#x, 0, 20], [a#x, b#x, a#x, b#x, 0, 21], [a#x, b#x, a#x, b#x, 0, 22], [a#x, b#x, a#x, b#x, 0, 23], ... 12 more fields], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, b#x, 0, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, b#x, 0, 7], [a#x, b#x, a#x, b#x, 0, 8], [a#x, b#x, a#x, b#x, 0, 9], [a#x, b#x, a#x, b#x, 0, 10], [a#x, b#x, a#x, b#x, 0, 11], [a#x, b#x, a#x, b#x, 0, 12], [a#x, b#x, a#x, null, 1, 13], [a#x, b#x, a#x, null, 1, 14], [a#x, b#x, a#x, b#x, 0, 15], [a#x, b#x, a#x, null, 1, 16], [a#x, b#x, a#x, null, 1, 17], [a#x, b#x, a#x, b#x, 0, 18], [a#x, b#x, a#x, b#x, 0, 19], [a#x, b#x, a#x, b#x, 0, 20], [a#x, b#x, a#x, b#x, 0, 21], [a#x, b#x, a#x, b#x, 0, 22], [a#x, b#x, a#x, b#x, 0, 23], [a#x, b#x, a#x, b#x, 0, 24], ... 11 more fields], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] +- SubqueryAlias testdata +- View (`testData`, [a#x, b#x]) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out index 37ec8291c4e4b..144fb69812ee6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out @@ -1326,7 +1326,7 @@ Aggregate [count(1) AS count(1)#xL] +- Filter unique1#x IN (list#x []) : +- Project [unique1#x] : +- Filter (unique2#x = 42) - : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, ... 7 more fields] + : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, ... 6 more fields] : +- Join Inner, (unique1#x = unique1#x) : :- SubqueryAlias b : : +- SubqueryAlias spark_catalog.default.tenk1 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out index 6f3bc9ccb66f3..4a35fffe3191b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out @@ -226,9 +226,9 @@ CreateViewCommand `various_arrays`, SELECT * FROM VALUES ( struct_array1, struct_array2, map_array1, map_array2 ), false, false, LocalTempView, UNSUPPORTED, true - +- Project [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields] + +- Project [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields] +- SubqueryAlias various_arrays - +- LocalRelation [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields] + +- LocalRelation [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields] -- !query @@ -251,11 +251,11 @@ FROM various_arrays -- !query analysis Project [concat(boolean_array1#x, boolean_array2#x) AS boolean_array#x, concat(tinyint_array1#x, tinyint_array2#x) AS tinyint_array#x, concat(smallint_array1#x, smallint_array2#x) AS smallint_array#x, concat(int_array1#x, int_array2#x) AS int_array#x, concat(bigint_array1#x, bigint_array2#x) AS bigint_array#x, concat(decimal_array1#x, decimal_array2#x) AS decimal_array#x, concat(double_array1#x, double_array2#x) AS double_array#x, concat(float_array1#x, float_array2#x) AS float_array#x, concat(date_array1#x, data_array2#x) AS data_array#x, concat(timestamp_array1#x, timestamp_array2#x) AS timestamp_array#x, concat(string_array1#x, string_array2#x) AS string_array#x, concat(array_array1#x, array_array2#x) AS array_array#x, concat(struct_array1#x, struct_array2#x) AS struct_array#x, concat(map_array1#x, map_array2#x) AS map_array#x] +- SubqueryAlias various_arrays - +- View (`various_arrays`, [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields]) - +- Project [cast(boolean_array1#x as array) AS boolean_array1#x, cast(boolean_array2#x as array) AS boolean_array2#x, cast(tinyint_array1#x as array) AS tinyint_array1#x, cast(tinyint_array2#x as array) AS tinyint_array2#x, cast(smallint_array1#x as array) AS smallint_array1#x, cast(smallint_array2#x as array) AS smallint_array2#x, cast(int_array1#x as array) AS int_array1#x, cast(int_array2#x as array) AS int_array2#x, cast(bigint_array1#x as array) AS bigint_array1#x, cast(bigint_array2#x as array) AS bigint_array2#x, cast(decimal_array1#x as array) AS decimal_array1#x, cast(decimal_array2#x as array) AS decimal_array2#x, cast(double_array1#x as array) AS double_array1#x, cast(double_array2#x as array) AS double_array2#x, cast(float_array1#x as array) AS float_array1#x, cast(float_array2#x as array) AS float_array2#x, cast(date_array1#x as array) AS date_array1#x, cast(data_array2#x as array) AS data_array2#x, cast(timestamp_array1#x as array) AS timestamp_array1#x, cast(timestamp_array2#x as array) AS timestamp_array2#x, cast(string_array1#x as array) AS string_array1#x, cast(string_array2#x as array) AS string_array2#x, cast(array_array1#x as array>) AS array_array1#x, cast(array_array2#x as array>) AS array_array2#x, ... 4 more fields] - +- Project [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields] + +- View (`various_arrays`, [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields]) + +- Project [cast(boolean_array1#x as array) AS boolean_array1#x, cast(boolean_array2#x as array) AS boolean_array2#x, cast(tinyint_array1#x as array) AS tinyint_array1#x, cast(tinyint_array2#x as array) AS tinyint_array2#x, cast(smallint_array1#x as array) AS smallint_array1#x, cast(smallint_array2#x as array) AS smallint_array2#x, cast(int_array1#x as array) AS int_array1#x, cast(int_array2#x as array) AS int_array2#x, cast(bigint_array1#x as array) AS bigint_array1#x, cast(bigint_array2#x as array) AS bigint_array2#x, cast(decimal_array1#x as array) AS decimal_array1#x, cast(decimal_array2#x as array) AS decimal_array2#x, cast(double_array1#x as array) AS double_array1#x, cast(double_array2#x as array) AS double_array2#x, cast(float_array1#x as array) AS float_array1#x, cast(float_array2#x as array) AS float_array2#x, cast(date_array1#x as array) AS date_array1#x, cast(data_array2#x as array) AS data_array2#x, cast(timestamp_array1#x as array) AS timestamp_array1#x, cast(timestamp_array2#x as array) AS timestamp_array2#x, cast(string_array1#x as array) AS string_array1#x, cast(string_array2#x as array) AS string_array2#x, cast(array_array1#x as array>) AS array_array1#x, cast(array_array2#x as array>) AS array_array2#x, cast(struct_array1#x as array>) AS struct_array1#x, ... 3 more fields] + +- Project [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields] +- SubqueryAlias various_arrays - +- LocalRelation [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields] + +- LocalRelation [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields] -- !query @@ -273,8 +273,8 @@ FROM various_arrays -- !query analysis Project [concat(cast(tinyint_array1#x as array), smallint_array2#x) AS ts_array#x, concat(cast(smallint_array1#x as array), int_array2#x) AS si_array#x, concat(cast(int_array1#x as array), bigint_array2#x) AS ib_array#x, concat(cast(bigint_array1#x as array), cast(decimal_array2#x as array)) AS bd_array#x, concat(cast(decimal_array1#x as array), double_array2#x) AS dd_array#x, concat(double_array1#x, cast(float_array2#x as array)) AS df_array#x, concat(cast(string_array1#x as array), data_array2#x) AS std_array#x, concat(timestamp_array1#x, cast(string_array2#x as array)) AS tst_array#x, concat(cast(string_array1#x as array), cast(int_array2#x as array)) AS sti_array#x] +- SubqueryAlias various_arrays - +- View (`various_arrays`, [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields]) - +- Project [cast(boolean_array1#x as array) AS boolean_array1#x, cast(boolean_array2#x as array) AS boolean_array2#x, cast(tinyint_array1#x as array) AS tinyint_array1#x, cast(tinyint_array2#x as array) AS tinyint_array2#x, cast(smallint_array1#x as array) AS smallint_array1#x, cast(smallint_array2#x as array) AS smallint_array2#x, cast(int_array1#x as array) AS int_array1#x, cast(int_array2#x as array) AS int_array2#x, cast(bigint_array1#x as array) AS bigint_array1#x, cast(bigint_array2#x as array) AS bigint_array2#x, cast(decimal_array1#x as array) AS decimal_array1#x, cast(decimal_array2#x as array) AS decimal_array2#x, cast(double_array1#x as array) AS double_array1#x, cast(double_array2#x as array) AS double_array2#x, cast(float_array1#x as array) AS float_array1#x, cast(float_array2#x as array) AS float_array2#x, cast(date_array1#x as array) AS date_array1#x, cast(data_array2#x as array) AS data_array2#x, cast(timestamp_array1#x as array) AS timestamp_array1#x, cast(timestamp_array2#x as array) AS timestamp_array2#x, cast(string_array1#x as array) AS string_array1#x, cast(string_array2#x as array) AS string_array2#x, cast(array_array1#x as array>) AS array_array1#x, cast(array_array2#x as array>) AS array_array2#x, ... 4 more fields] - +- Project [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields] + +- View (`various_arrays`, [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields]) + +- Project [cast(boolean_array1#x as array) AS boolean_array1#x, cast(boolean_array2#x as array) AS boolean_array2#x, cast(tinyint_array1#x as array) AS tinyint_array1#x, cast(tinyint_array2#x as array) AS tinyint_array2#x, cast(smallint_array1#x as array) AS smallint_array1#x, cast(smallint_array2#x as array) AS smallint_array2#x, cast(int_array1#x as array) AS int_array1#x, cast(int_array2#x as array) AS int_array2#x, cast(bigint_array1#x as array) AS bigint_array1#x, cast(bigint_array2#x as array) AS bigint_array2#x, cast(decimal_array1#x as array) AS decimal_array1#x, cast(decimal_array2#x as array) AS decimal_array2#x, cast(double_array1#x as array) AS double_array1#x, cast(double_array2#x as array) AS double_array2#x, cast(float_array1#x as array) AS float_array1#x, cast(float_array2#x as array) AS float_array2#x, cast(date_array1#x as array) AS date_array1#x, cast(data_array2#x as array) AS data_array2#x, cast(timestamp_array1#x as array) AS timestamp_array1#x, cast(timestamp_array2#x as array) AS timestamp_array2#x, cast(string_array1#x as array) AS string_array1#x, cast(string_array2#x as array) AS string_array2#x, cast(array_array1#x as array>) AS array_array1#x, cast(array_array2#x as array>) AS array_array2#x, cast(struct_array1#x as array>) AS struct_array1#x, ... 3 more fields] + +- Project [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields] +- SubqueryAlias various_arrays - +- LocalRelation [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields] + +- LocalRelation [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, struct_array1#x, ... 3 more fields] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out index b1a3dc46dabac..6fee49a07667b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out @@ -71,9 +71,9 @@ CreateViewCommand `various_maps`, SELECT * FROM VALUES ( string_int_map1, string_int_map2, int_string_map1, int_string_map2 ), false, false, LocalTempView, UNSUPPORTED, true - +- Project [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields] + +- Project [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, struct_map1#x, ... 5 more fields] +- SubqueryAlias various_maps - +- LocalRelation [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields] + +- LocalRelation [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, struct_map1#x, ... 5 more fields] -- !query @@ -97,11 +97,11 @@ FROM various_maps -- !query analysis Project [map_concat(boolean_map1#x, boolean_map2#x) AS boolean_map#x, map_concat(tinyint_map1#x, tinyint_map2#x) AS tinyint_map#x, map_concat(smallint_map1#x, smallint_map2#x) AS smallint_map#x, map_concat(int_map1#x, int_map2#x) AS int_map#x, map_concat(bigint_map1#x, bigint_map2#x) AS bigint_map#x, map_concat(decimal_map1#x, decimal_map2#x) AS decimal_map#x, map_concat(float_map1#x, float_map2#x) AS float_map#x, map_concat(double_map1#x, double_map2#x) AS double_map#x, map_concat(date_map1#x, date_map2#x) AS date_map#x, map_concat(timestamp_map1#x, timestamp_map2#x) AS timestamp_map#x, map_concat(string_map1#x, string_map2#x) AS string_map#x, map_concat(array_map1#x, array_map2#x) AS array_map#x, map_concat(struct_map1#x, struct_map2#x) AS struct_map#x, map_concat(string_int_map1#x, string_int_map2#x) AS string_int_map#x, map_concat(int_string_map1#x, int_string_map2#x) AS int_string_map#x] +- SubqueryAlias various_maps - +- View (`various_maps`, [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields]) - +- Project [cast(boolean_map1#x as map) AS boolean_map1#x, cast(boolean_map2#x as map) AS boolean_map2#x, cast(tinyint_map1#x as map) AS tinyint_map1#x, cast(tinyint_map2#x as map) AS tinyint_map2#x, cast(smallint_map1#x as map) AS smallint_map1#x, cast(smallint_map2#x as map) AS smallint_map2#x, cast(int_map1#x as map) AS int_map1#x, cast(int_map2#x as map) AS int_map2#x, cast(bigint_map1#x as map) AS bigint_map1#x, cast(bigint_map2#x as map) AS bigint_map2#x, cast(decimal_map1#x as map) AS decimal_map1#x, cast(decimal_map2#x as map) AS decimal_map2#x, cast(double_map1#x as map) AS double_map1#x, cast(double_map2#x as map) AS double_map2#x, cast(float_map1#x as map) AS float_map1#x, cast(float_map2#x as map) AS float_map2#x, cast(date_map1#x as map) AS date_map1#x, cast(date_map2#x as map) AS date_map2#x, cast(timestamp_map1#x as map) AS timestamp_map1#x, cast(timestamp_map2#x as map) AS timestamp_map2#x, cast(string_map1#x as map) AS string_map1#x, cast(string_map2#x as map) AS string_map2#x, cast(array_map1#x as map,array>) AS array_map1#x, cast(array_map2#x as map,array>) AS array_map2#x, ... 6 more fields] - +- Project [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields] + +- View (`various_maps`, [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, struct_map1#x, ... 5 more fields]) + +- Project [cast(boolean_map1#x as map) AS boolean_map1#x, cast(boolean_map2#x as map) AS boolean_map2#x, cast(tinyint_map1#x as map) AS tinyint_map1#x, cast(tinyint_map2#x as map) AS tinyint_map2#x, cast(smallint_map1#x as map) AS smallint_map1#x, cast(smallint_map2#x as map) AS smallint_map2#x, cast(int_map1#x as map) AS int_map1#x, cast(int_map2#x as map) AS int_map2#x, cast(bigint_map1#x as map) AS bigint_map1#x, cast(bigint_map2#x as map) AS bigint_map2#x, cast(decimal_map1#x as map) AS decimal_map1#x, cast(decimal_map2#x as map) AS decimal_map2#x, cast(double_map1#x as map) AS double_map1#x, cast(double_map2#x as map) AS double_map2#x, cast(float_map1#x as map) AS float_map1#x, cast(float_map2#x as map) AS float_map2#x, cast(date_map1#x as map) AS date_map1#x, cast(date_map2#x as map) AS date_map2#x, cast(timestamp_map1#x as map) AS timestamp_map1#x, cast(timestamp_map2#x as map) AS timestamp_map2#x, cast(string_map1#x as map) AS string_map1#x, cast(string_map2#x as map) AS string_map2#x, cast(array_map1#x as map,array>) AS array_map1#x, cast(array_map2#x as map,array>) AS array_map2#x, cast(struct_map1#x as map,struct>) AS struct_map1#x, ... 5 more fields] + +- Project [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, struct_map1#x, ... 5 more fields] +- SubqueryAlias various_maps - +- LocalRelation [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields] + +- LocalRelation [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, struct_map1#x, ... 5 more fields] -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out index c74124402c554..e7bd8a9535eb1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out @@ -1330,7 +1330,7 @@ Aggregate [cast(udf(cast(cast(udf(cast(count(1) as string)) as bigint) as string +- Filter cast(udf(cast(cast(udf(cast(unique1#x as string)) as int) as string)) as int) IN (list#x []) : +- Project [cast(udf(cast(unique1#x as string)) as int) AS udf(unique1)#x] : +- Filter (cast(udf(cast(cast(udf(cast(unique2#x as string)) as int) as string)) as int) = cast(udf(cast(42 as string)) as int)) - : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, ... 7 more fields] + : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, ... 6 more fields] : +- Join Inner, (unique1#x = unique1#x) : :- SubqueryAlias b : : +- SubqueryAlias spark_catalog.default.tenk1 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out index c10988310c0a9..1d0d8974e97ca 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out @@ -385,11 +385,11 @@ FROM testData WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val)) ORDER BY cate, udf(val) -- !query analysis -Project [udf(val)#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, ... 11 more fields] +Project [udf(val)#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, var_pop#x, ... 10 more fields] +- Sort [cate#x ASC NULLS FIRST, cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST], true - +- Project [udf(val)#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, ... 12 more fields] - +- Project [udf(val)#x, cate#x, _w0#x, _w1#x, _w2#x, _w3#x, _w4#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, ... 50 more fields] - +- Window [max(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS max#x, min(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, min(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, count(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count#xL, sum(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum#xL, avg(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg#x, stddev(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS stddev#x, first_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, first_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_ignore_null#x, first_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_contain_null#x, any_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, any_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_ignore_null#x, any_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_contain_null#x, last_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x, last_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_ignore_null#x, last_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_contain_null#x, rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#x, dense_rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS dense_rank#x, cume_dist() windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS cume_dist#x, percent_rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS percent_rank#x, ntile(2) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ntile#x, row_number() windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number#x, var_pop(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_pop#x, var_samp(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_samp#x, ... 9 more fields], [_w1#x], [_w0#x ASC NULLS FIRST] + +- Project [udf(val)#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, var_pop#x, ... 11 more fields] + +- Project [udf(val)#x, cate#x, _w0#x, _w1#x, _w2#x, _w3#x, _w4#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, ... 49 more fields] + +- Window [max(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS max#x, min(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, min(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, count(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count#xL, sum(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum#xL, avg(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg#x, stddev(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS stddev#x, first_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, first_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_ignore_null#x, first_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_contain_null#x, any_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, any_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_ignore_null#x, any_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_contain_null#x, last_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x, last_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_ignore_null#x, last_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_contain_null#x, rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#x, dense_rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS dense_rank#x, cume_dist() windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS cume_dist#x, percent_rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS percent_rank#x, ntile(2) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ntile#x, row_number() windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number#x, var_pop(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_pop#x, var_samp(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_samp#x, approx_count_distinct(_w0#x, 0.05, 0, 0) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS approx_count_distinct#xL, ... 8 more fields], [_w1#x], [_w0#x ASC NULLS FIRST] +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, cast(udf(cast(val#x as string)) as int) AS _w0#x, cast(udf(cast(cate#x as string)) as string) AS _w1#x, cast(cast(udf(cast(val#x as string)) as int) as double) AS _w2#x, cast(cast(udf(cast(val_long#xL as string)) as bigint) as double) AS _w3#x, cast(udf(cast(val_double#x as string)) as double) AS _w4#x, val#x] +- SubqueryAlias testdata +- View (`testData`, [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x]) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out index a1a2b76006378..77dc2f3ba2736 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out @@ -583,9 +583,9 @@ WINDOW w AS (PARTITION BY cate ORDER BY val) ORDER BY cate, val -- !query analysis Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true -+- Project [val#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, ... 11 more fields] - +- Project [val#x, cate#x, _w0#x, _w1#x, val_double#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, ... 47 more fields] - +- Window [max(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS max#x, min(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, min(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, count(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count#xL, sum(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum#xL, avg(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg#x, stddev(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS stddev#x, first_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, first_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_ignore_null#x, first_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_contain_null#x, any_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, any_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_ignore_null#x, any_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_contain_null#x, last_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x, last_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_ignore_null#x, last_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_contain_null#x, rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#x, dense_rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS dense_rank#x, cume_dist() windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS cume_dist#x, percent_rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS percent_rank#x, ntile(2) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ntile#x, row_number() windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number#x, var_pop(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_pop#x, var_samp(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_samp#x, ... 9 more fields], [cate#x], [val#x ASC NULLS FIRST] ++- Project [val#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, var_pop#x, ... 10 more fields] + +- Project [val#x, cate#x, _w0#x, _w1#x, val_double#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ... 46 more fields] + +- Window [max(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS max#x, min(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, min(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, count(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count#xL, sum(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum#xL, avg(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg#x, stddev(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS stddev#x, first_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, first_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_ignore_null#x, first_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_contain_null#x, any_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, any_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_ignore_null#x, any_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_contain_null#x, last_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x, last_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_ignore_null#x, last_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_contain_null#x, rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#x, dense_rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS dense_rank#x, cume_dist() windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS cume_dist#x, percent_rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS percent_rank#x, ntile(2) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ntile#x, row_number() windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number#x, var_pop(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_pop#x, var_samp(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_samp#x, approx_count_distinct(val#x, 0.05, 0, 0) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS approx_count_distinct#xL, ... 8 more fields], [cate#x], [val#x ASC NULLS FIRST] +- Project [val#x, cate#x, cast(val#x as double) AS _w0#x, cast(val_long#xL as double) AS _w1#x, val_double#x] +- SubqueryAlias testdata +- View (`testData`, [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x]) @@ -1009,7 +1009,7 @@ ORDER BY id -- !query analysis Sort [id#x ASC NULLS FIRST], true +- Project [content#x, id#x, v#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, lag_3#x, lag_plus_3#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] - +- Project [content#x, id#x, v#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, lag_3#x, lag_plus_3#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, ... 9 more fields] + +- Project [content#x, id#x, v#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, lag_3#x, lag_plus_3#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, ... 8 more fields] +- Window [lead(v#x, 0, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 0, 0)) AS lead_0#x, lead(v#x, 1, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 1, 1)) AS lead_1#x, lead(v#x, 2, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 2, 2)) AS lead_2#x, lead(v#x, 3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 3, 3)) AS lead_3#x, lag(v#x, 0, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 0, 0)) AS lag_0#x, lag(v#x, -1, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, -1)) AS lag_1#x, lag(v#x, -2, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, -2)) AS lag_2#x, lag(v#x, -3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -3, -3)) AS lag_3#x, lag(v#x, -3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -3, -3)) AS lag_plus_3#x, nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x], [id#x ASC NULLS FIRST] +- Project [content#x, id#x, v#x] +- SubqueryAlias test_ignore_null From 44ab3492455e71ebc46d545ce2323be41c03633f Mon Sep 17 00:00:00 2001 From: tomsisso Date: Mon, 16 Dec 2024 20:36:45 +0900 Subject: [PATCH 194/438] [SPARK-50581][SQL] fix support for UDAF in Dataset.observe() ### What changes were proposed in this pull request? Mark inputProjection field as transient in org.apache.spark.sql.execution.aggregate.ScalaAggregator. ### Why are the changes needed? To support UDAF in Dataset.observe() which currently fails due to serialization exception. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added test. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49190 from tomsisso/support-udaf-in-observe. Authored-by: tomsisso Signed-off-by: Hyukjin Kwon --- .../spark/sql/execution/aggregate/udaf.scala | 16 +++---- .../sql/util/DataFrameCallbackSuite.scala | 48 ++++++++++++++++++- 2 files changed, 55 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 09d9915022a65..1197a16a35e9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -501,17 +501,17 @@ case class ScalaAggregator[IN, BUF, OUT]( with Logging { // input and buffer encoders are resolved by ResolveEncodersInScalaAgg - private[this] lazy val inputDeserializer = inputEncoder.createDeserializer() - private[this] lazy val bufferSerializer = bufferEncoder.createSerializer() - private[this] lazy val bufferDeserializer = bufferEncoder.createDeserializer() - private[this] lazy val outputEncoder = encoderFor(agg.outputEncoder) - private[this] lazy val outputSerializer = outputEncoder.createSerializer() + @transient private[this] lazy val inputDeserializer = inputEncoder.createDeserializer() + @transient private[this] lazy val bufferSerializer = bufferEncoder.createSerializer() + @transient private[this] lazy val bufferDeserializer = bufferEncoder.createDeserializer() + @transient private[this] lazy val outputEncoder = encoderFor(agg.outputEncoder) + @transient private[this] lazy val outputSerializer = outputEncoder.createSerializer() def dataType: DataType = outputEncoder.objSerializer.dataType def inputTypes: Seq[DataType] = inputEncoder.schema.map(_.dataType) - override lazy val deterministic: Boolean = isDeterministic + @transient override lazy val deterministic: Boolean = isDeterministic def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ScalaAggregator[IN, BUF, OUT] = copy(mutableAggBufferOffset = newMutableAggBufferOffset) @@ -519,7 +519,7 @@ case class ScalaAggregator[IN, BUF, OUT]( def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ScalaAggregator[IN, BUF, OUT] = copy(inputAggBufferOffset = newInputAggBufferOffset) - private[this] lazy val inputProjection = UnsafeProjection.create(children) + @transient private[this] lazy val inputProjection = UnsafeProjection.create(children) def createAggregationBuffer(): BUF = agg.zero @@ -533,7 +533,7 @@ case class ScalaAggregator[IN, BUF, OUT]( if (outputEncoder.isSerializedAsStruct) row else row.get(0, dataType) } - private[this] lazy val bufferRow = new UnsafeRow(bufferEncoder.namedExpressions.length) + @transient private[this] lazy val bufferRow = new UnsafeRow(bufferEncoder.namedExpressions.length) def serialize(agg: BUF): Array[Byte] = bufferSerializer(agg).asInstanceOf[UnsafeRow].getBytes() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index be91f5e789e2c..7e6f10bcc46f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -22,7 +22,7 @@ import java.lang.{Long => JLong} import scala.collection.mutable.ArrayBuffer import org.apache.spark._ -import org.apache.spark.sql.{functions, Dataset, QueryTest, Row, SparkSession} +import org.apache.spark.sql.{functions, Dataset, Encoder, Encoders, QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, LeafRunnableCommand} import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StringType @@ -339,6 +340,51 @@ class DataFrameCallbackSuite extends QueryTest } } + test("SPARK-50581: support observe with udaf") { + withUserDefinedFunction(("someUdaf", true)) { + spark.udf.register("someUdaf", functions.udaf(new Aggregator[JLong, JLong, JLong] { + def zero: JLong = 0L + def reduce(b: JLong, a: JLong): JLong = a + b + def merge(b1: JLong, b2: JLong): JLong = b1 + b2 + def finish(r: JLong): JLong = r + def bufferEncoder: Encoder[JLong] = Encoders.LONG + def outputEncoder: Encoder[JLong] = Encoders.LONG + })) + + val df = spark.range(100) + + val metricMaps = ArrayBuffer.empty[Map[String, Row]] + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + if (qe.observedMetrics.nonEmpty) { + metricMaps += qe.observedMetrics + } + } + + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { + // No-op + } + } + try { + spark.listenerManager.register(listener) + + // udaf usage in observe is not working (serialization exception) + df.observe( + name = "my_metrics", + expr("someUdaf(id)").as("agg") + ) + .collect() + + sparkContext.listenerBus.waitUntilEmpty() + assert(metricMaps.size === 1) + assert(metricMaps.head("my_metrics") === Row(4950L)) + + } finally { + spark.listenerManager.unregister(listener) + } + } + } + private def validateObservedMetrics(df: Dataset[JLong]): Unit = { val metricMaps = ArrayBuffer.empty[Map[String, Row]] val listener = new QueryExecutionListener { From 868c58730e56dd80276f0bebc6fb3090eaa59f27 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 16 Dec 2024 22:03:32 +0800 Subject: [PATCH 195/438] [SPARK-50583][INFRA][FOLLOW-UP] Fix 3.5 daily build MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Fix 3.5 daily build ``` if: (inputs.branch != 'branch-3.5') && (${{ env.PYSPARK_IMAGE_TO_TEST }} != '') ``` the condition returns true for branch-3.5 ### Why are the changes needed? Fix 3.5 daily build ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? manually check 1, `${{ inputs.branch != 'master' && env.PYSPARK_IMAGE_TO_TEST != '' }}` skipped in PR build ![Uploading image.png…]() https://github.com/zhengruifeng/spark/actions/runs/12354167258/job/34474934451 2, `${{ inputs.branch != 'branch-3.5' && env.PYSPARK_IMAGE_TO_TEST != '' }}` runned in PR build ### Was this patch authored or co-authored using generative AI tooling? no Closes #49206 from zhengruifeng/fix_daily_35. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 174a8aa6bacb0..f827cb9a63731 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -466,7 +466,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ inputs.branch }} - name: Build and push (PySpark with ${{ env.PYSPARK_IMAGE_TO_TEST }}) - if: (inputs.branch != 'branch-3.5') && (${{ env.PYSPARK_IMAGE_TO_TEST }} != '') + if: ${{ inputs.branch != 'branch-3.5' && env.PYSPARK_IMAGE_TO_TEST != '' }} id: docker_build_pyspark env: ${{ fromJSON(inputs.envs) }} uses: docker/build-push-action@v6 From 576caec1da85c4451fe63e2a5923f2dbf136e278 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 16 Dec 2024 08:00:58 -0800 Subject: [PATCH 196/438] [SPARK-50580][BUILD] Upgrade log4j2 to 2.24.3 ### What changes were proposed in this pull request? The pr aims to upgrade log4j2 from `2.24.2` to `2.24.3`. ### Why are the changes needed? - The full release notes: https://github.com/apache/logging-log4j2/releases/tag/rel%2F2.24.3 ### 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 #49189 from panbingkun/SPARK-50580. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 10 +++++----- pom.xml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 53ab9ade891d7..a1b9d66f66579 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -188,11 +188,11 @@ leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.16.0//libthrift-0.16.0.jar listenablefuture/9999.0-empty-to-avoid-conflict-with-guava//listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar -log4j-1.2-api/2.24.2//log4j-1.2-api-2.24.2.jar -log4j-api/2.24.2//log4j-api-2.24.2.jar -log4j-core/2.24.2//log4j-core-2.24.2.jar -log4j-layout-template-json/2.24.2//log4j-layout-template-json-2.24.2.jar -log4j-slf4j2-impl/2.24.2//log4j-slf4j2-impl-2.24.2.jar +log4j-1.2-api/2.24.3//log4j-1.2-api-2.24.3.jar +log4j-api/2.24.3//log4j-api-2.24.3.jar +log4j-core/2.24.3//log4j-core-2.24.3.jar +log4j-layout-template-json/2.24.3//log4j-layout-template-json-2.24.3.jar +log4j-slf4j2-impl/2.24.3//log4j-slf4j2-impl-2.24.3.jar logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.8.0//lz4-java-1.8.0.jar metrics-core/4.2.29//metrics-core-4.2.29.jar diff --git a/pom.xml b/pom.xml index 36d9a3dcb1298..bfbfdeebd1eb2 100644 --- a/pom.xml +++ b/pom.xml @@ -121,7 +121,7 @@ spark 9.7.1 2.0.16 - 2.24.2 + 2.24.3 3.4.1 From 1c3d580eaa0034ff8ea8aa31e3a8009d97736e8a Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 16 Dec 2024 16:20:41 -0800 Subject: [PATCH 197/438] [SPARK-50592][INFRA] Make 3.5 daily build able to manually trigger ### What changes were proposed in this pull request? Make 3.5 daily build able to manually trigger https://docs.github.com/en/actions/managing-workflow-runs-and-deployments/managing-workflow-runs/manually-running-a-workflow ``` workflow_dispatch: ``` should be a valid syntax, see [example](https://docs.github.com/en/actions/writing-workflows/workflow-syntax-for-github-actions#example-of-jobsjob_idsecretsinherit) ### Why are the changes needed? for ease of debugging ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? will manually check ### Was this patch authored or co-authored using generative AI tooling? no Closes #49207 from zhengruifeng/make_35_daily_manually_trigger. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- .github/workflows/build_branch35.yml | 1 + .github/workflows/build_branch35_python.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/build_branch35.yml b/.github/workflows/build_branch35.yml index ad46b6ce32387..4757ca3c574f5 100644 --- a/.github/workflows/build_branch35.yml +++ b/.github/workflows/build_branch35.yml @@ -22,6 +22,7 @@ name: "Build (branch-3.5, Scala 2.13, Hadoop 3, JDK 8)" on: schedule: - cron: '0 11 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_branch35_python.yml b/.github/workflows/build_branch35_python.yml index ed83f8c4cdf88..452a55f3bc2c1 100644 --- a/.github/workflows/build_branch35_python.yml +++ b/.github/workflows/build_branch35_python.yml @@ -22,6 +22,7 @@ name: "Build / Python-only (branch-3.5)" on: schedule: - cron: '0 11 * * *' + workflow_dispatch: jobs: run-build: From 4c0f7b581243edcd9bd6d265d7ef120d6a963a4c Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 17 Dec 2024 12:19:28 +0800 Subject: [PATCH 198/438] [SPARK-50594][BUILD][INFRA] Align the gRPC-related Python package version and pin protobuf version in `page.yml` and `docs/Dockerfile` ### What changes were proposed in this pull request? This pr specifies the gRPC-related Python packages version as 1.67.0 in both `page.yml` and `docs/Dockerfile`, and also sets the protobuf version to 5.29.1 to maintain consistency with other testing envs. ### Why are the changes needed? Unify the Python test dependency versions for grpc and protobuf in test envs ### 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 #49205 from LuciferYang/page-and-docs-grpc-pb. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .github/workflows/pages.yml | 2 +- dev/spark-test-image/docs/Dockerfile | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml index 8729012c2b8d2..637abb86b36cd 100644 --- a/.github/workflows/pages.yml +++ b/.github/workflows/pages.yml @@ -63,7 +63,7 @@ jobs: pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow 'pandas==2.2.3' 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.62.0' 'grpcio-status==1.62.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' - name: Install Ruby for documentation generation uses: ruby/setup-ruby@v1 diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index 2db7e0717cdfd..0f3559fa2d4cf 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -86,6 +86,6 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 RUN python3.9 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ - 'pandas-stubs==1.2.0.53' 'grpcio==1.62.0' 'grpcio-status==1.62.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ + 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' \ && python3.9 -m pip cache purge From 79026ad70380d6e92503ea27b1f61f484c63dc8b Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 17 Dec 2024 17:31:10 +0900 Subject: [PATCH 199/438] [SPARK-50588][BUILD] `build-docs` skips building R docs on host when `SKIP_RDOC=1` ### What changes were proposed in this pull request? `build-docs` skips calling `bundle` command when `SKIP_RDOC=1`, so that devs do not need to install `bundle` on the host. ### Why are the changes needed? Skip unnecessary steps when `SKIP_RDOC=1` ### 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 #49200 from pan3793/SPARK-50588. Authored-by: Cheng Pan Signed-off-by: Hyukjin Kwon --- dev/spark-test-image-util/docs/build-docs | 30 ++++++++++++----------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/dev/spark-test-image-util/docs/build-docs b/dev/spark-test-image-util/docs/build-docs index 6ff9c7cd9455c..ca59769f24231 100755 --- a/dev/spark-test-image-util/docs/build-docs +++ b/dev/spark-test-image-util/docs/build-docs @@ -49,20 +49,22 @@ docker run \ --interactive --tty "${IMG_URL}" \ /bin/bash -c "sh ${BUILD_DOCS_SCRIPT_PATH}" -# 4.Build docs on host: `r doc`. -# -# Why does `r` document need to be compiled outside the container? -# Because when compiling inside the container, the permission of the directory -# `/__w/spark/spark/R/pkg/docs` automatically generated by `RScript` is `dr-xr--r-x`, -# and when writing to subsequent files, will throw an error as: -# `! [EACCES] Failed to copy '/usr/local/lib/R/site-library/pkgdown/BS5/assets/katex-auto.js' -# to '/__w/spark/spark/R/pkg/docs/katex-auto.js': permission denied` -export SKIP_ERRORDOC=1 -export SKIP_SCALADOC=1 -export SKIP_PYTHONDOC=1 -export SKIP_SQLDOC=1 -cd docs -bundle exec jekyll build +if [[ "$SKIP_RDOC" != "1" ]]; then + # 4.Build docs on host: `r doc`. + # + # Why does `r` document need to be compiled outside the container? + # Because when compiling inside the container, the permission of the directory + # `/__w/spark/spark/R/pkg/docs` automatically generated by `RScript` is `dr-xr--r-x`, + # and when writing to subsequent files, will throw an error as: + # `! [EACCES] Failed to copy '/usr/local/lib/R/site-library/pkgdown/BS5/assets/katex-auto.js' + # to '/__w/spark/spark/R/pkg/docs/katex-auto.js': permission denied` + export SKIP_ERRORDOC=1 + export SKIP_SCALADOC=1 + export SKIP_PYTHONDOC=1 + export SKIP_SQLDOC=1 + cd docs + bundle exec jekyll build +fi # 5.Remove container image. IMG_ID=$(docker images | grep "${IMG_TAG}" | awk '{print $3}') From 63c7ca4df2970d12574ad3b542ec17eb5276ef86 Mon Sep 17 00:00:00 2001 From: Anton Lykov Date: Tue, 17 Dec 2024 22:39:11 +0800 Subject: [PATCH 200/438] [SPARK-50597][SQL] Refactor batch construction in Optimizer.scala and SparkOptimizer.scala ### What changes were proposed in this pull request? See description. Previously, it was a pain to reorder batches and guard behavior of certain batches / sequences of batches by a flag. This was primarily due to ample usage of `::`, `:::`, and `:+` to juggle rules and batches around which imposed syntactic limitations. After this change, we keep a single sequence `allBatches`, that can contain either `Batch` or `Seq[Batch]` elements to allow further groupings, which is later flattened into a single `Seq[Batch]`. We avoid any usage of `::`, `:::`, and `:+`. To add/replace a flag-guarded batch of sequence of batches, write a function that returns either `Batch` of `Seq[Batch]` with desired behavior, and add/replace in the relevant place in the `allBatches` list. ### Why are the changes needed? This simplifies further restructuring and reordering of batches. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No tests. ### Was this patch authored or co-authored using generative AI tooling? Closes #49208 from anton5798/batch-refactor. Authored-by: Anton Lykov Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 113 ++++++++++-------- .../spark/sql/execution/SparkOptimizer.scala | 63 +++++----- 2 files changed, 98 insertions(+), 78 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 31c1f89177632..b141d2be04c32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -73,6 +73,21 @@ abstract class Optimizer(catalogManager: CatalogManager) conf.optimizerMaxIterations, maxIterationsSetting = SQLConf.OPTIMIZER_MAX_ITERATIONS.key) + /** + * A helper method that takes as input a Seq of Batch or Seq[Batch], and flattens it out. + */ + def flattenBatches(nestedBatchSequence: Seq[Any]): Seq[Batch] = { + assert(nestedBatchSequence.forall { + case _: Batch => true + case s: Seq[_] => s.forall(_.isInstanceOf[Batch]) + case _ => false + }) + nestedBatchSequence.flatMap { + case batches: Seq[Batch @unchecked] => batches + case batch: Batch => Seq(batch) + } + } + /** * Defines the default rule batches in the Optimizer. * @@ -143,39 +158,38 @@ abstract class Optimizer(catalogManager: CatalogManager) PushdownPredicatesAndPruneColumnsForCTEDef) ++ extendedOperatorOptimizationRules - val operatorOptimizationBatch: Seq[Batch] = { + val operatorOptimizationBatch: Seq[Batch] = Seq( Batch("Operator Optimization before Inferring Filters", fixedPoint, - operatorOptimizationRuleSet: _*) :: + operatorOptimizationRuleSet: _*), Batch("Infer Filters", Once, InferFiltersFromGenerate, - InferFiltersFromConstraints) :: + InferFiltersFromConstraints), Batch("Operator Optimization after Inferring Filters", fixedPoint, - operatorOptimizationRuleSet: _*) :: + operatorOptimizationRuleSet: _*), Batch("Push extra predicate through join", fixedPoint, PushExtraPredicateThroughJoin, - PushDownPredicates) :: Nil - } + PushDownPredicates)) - val batches = ( - Batch("Finish Analysis", FixedPoint(1), FinishAnalysis) :: + val batches: Seq[Batch] = flattenBatches(Seq( + Batch("Finish Analysis", FixedPoint(1), FinishAnalysis), // We must run this batch after `ReplaceExpressions`, as `RuntimeReplaceable` expression // may produce `With` expressions that need to be rewritten. - Batch("Rewrite With expression", fixedPoint, RewriteWithExpression) :: + Batch("Rewrite With expression", fixedPoint, RewriteWithExpression), ////////////////////////////////////////////////////////////////////////////////////////// // Optimizer rules start here ////////////////////////////////////////////////////////////////////////////////////////// - Batch("Eliminate Distinct", Once, EliminateDistinct) :: + Batch("Eliminate Distinct", Once, EliminateDistinct), // - Do the first call of CombineUnions before starting the major Optimizer rules, // since it can reduce the number of iteration and the other rules could add/move // extra operators between two adjacent Union operators. // - Call CombineUnions again in Batch("Operator Optimizations"), // since the other rules might make two separate Unions operators adjacent. Batch("Inline CTE", Once, - InlineCTE()) :: + InlineCTE()), Batch("Union", fixedPoint, RemoveNoopOperators, CombineUnions, - RemoveNoopUnion) :: + RemoveNoopUnion), // Run this once earlier. This might simplify the plan and reduce cost of optimizer. // For example, a query such as Filter(LocalRelation) would go through all the heavy // optimizer rules that are triggered when there is a filter @@ -186,16 +200,16 @@ abstract class Optimizer(catalogManager: CatalogManager) PropagateEmptyRelation, // PropagateEmptyRelation can change the nullability of an attribute from nullable to // non-nullable when an empty relation child of a Union is removed - UpdateAttributeNullability) :: + UpdateAttributeNullability), Batch("Pullup Correlated Expressions", Once, OptimizeOneRowRelationSubquery, PullOutNestedDataOuterRefExpressions, - PullupCorrelatedPredicates) :: + PullupCorrelatedPredicates), // Subquery batch applies the optimizer rules recursively. Therefore, it makes no sense // to enforce idempotence on it and we change this batch from Once to FixedPoint(1). Batch("Subquery", FixedPoint(1), OptimizeSubqueries, - OptimizeOneRowRelationSubquery) :: + OptimizeOneRowRelationSubquery), Batch("Replace Operators", fixedPoint, RewriteExceptAll, RewriteIntersectAll, @@ -203,48 +217,48 @@ abstract class Optimizer(catalogManager: CatalogManager) ReplaceExceptWithFilter, ReplaceExceptWithAntiJoin, ReplaceDistinctWithAggregate, - ReplaceDeduplicateWithAggregate) :: + ReplaceDeduplicateWithAggregate), Batch("Aggregate", fixedPoint, RemoveLiteralFromGroupExpressions, - RemoveRepetitionFromGroupExpressions) :: Nil ++ - operatorOptimizationBatch) :+ - Batch("Clean Up Temporary CTE Info", Once, CleanUpTempCTEInfo) :+ + RemoveRepetitionFromGroupExpressions), + operatorOptimizationBatch, + Batch("Clean Up Temporary CTE Info", Once, CleanUpTempCTEInfo), // This batch rewrites plans after the operator optimization and // before any batches that depend on stats. - Batch("Pre CBO Rules", Once, preCBORules: _*) :+ + Batch("Pre CBO Rules", Once, preCBORules: _*), // This batch pushes filters and projections into scan nodes. Before this batch, the logical // plan may contain nodes that do not report stats. Anything that uses stats must run after // this batch. - Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*) :+ - Batch("Update CTE Relation Stats", Once, UpdateCTERelationStats) :+ + Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*), + Batch("Update CTE Relation Stats", Once, UpdateCTERelationStats), // Since join costs in AQP can change between multiple runs, there is no reason that we have an // idempotence enforcement on this batch. We thus make it FixedPoint(1) instead of Once. Batch("Join Reorder", FixedPoint(1), - CostBasedJoinReorder) :+ + CostBasedJoinReorder), Batch("Eliminate Sorts", Once, EliminateSorts, - RemoveRedundantSorts) :+ + RemoveRedundantSorts), Batch("Decimal Optimizations", fixedPoint, - DecimalAggregates) :+ + DecimalAggregates), // This batch must run after "Decimal Optimizations", as that one may change the // aggregate distinct column Batch("Distinct Aggregate Rewrite", Once, - RewriteDistinctAggregates) :+ + RewriteDistinctAggregates), Batch("Object Expressions Optimization", fixedPoint, EliminateMapObjects, CombineTypedFilters, ObjectSerializerPruning, - ReassignLambdaVariableID) :+ + ReassignLambdaVariableID), Batch("LocalRelation", fixedPoint, ConvertToLocalRelation, PropagateEmptyRelation, // PropagateEmptyRelation can change the nullability of an attribute from nullable to // non-nullable when an empty relation child of a Union is removed - UpdateAttributeNullability) :+ - Batch("Optimize One Row Plan", fixedPoint, OptimizeOneRowPlan) :+ + UpdateAttributeNullability), + Batch("Optimize One Row Plan", fixedPoint, OptimizeOneRowPlan), // The following batch should be executed after batch "Join Reorder" and "LocalRelation". Batch("Check Cartesian Products", Once, - CheckCartesianProducts) :+ + CheckCartesianProducts), Batch("RewriteSubquery", Once, RewritePredicateSubquery, PushPredicateThroughJoin, @@ -252,10 +266,10 @@ abstract class Optimizer(catalogManager: CatalogManager) ColumnPruning, CollapseProject, RemoveRedundantAliases, - RemoveNoopOperators) :+ + RemoveNoopOperators), // This batch must be executed after the `RewriteSubquery` batch, which creates joins. - Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) :+ - Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression) + Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers), + Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression))) // remove any batches with no rules. this may happen when subclasses do not add optional rules. batches.filter(_.rules.nonEmpty) @@ -270,22 +284,23 @@ abstract class Optimizer(catalogManager: CatalogManager) * (defaultBatches - (excludedRules - nonExcludableRules)). */ def nonExcludableRules: Seq[String] = - FinishAnalysis.ruleName :: - RewriteDistinctAggregates.ruleName :: - ReplaceDeduplicateWithAggregate.ruleName :: - ReplaceIntersectWithSemiJoin.ruleName :: - ReplaceExceptWithFilter.ruleName :: - ReplaceExceptWithAntiJoin.ruleName :: - RewriteExceptAll.ruleName :: - RewriteIntersectAll.ruleName :: - ReplaceDistinctWithAggregate.ruleName :: - PullupCorrelatedPredicates.ruleName :: - RewriteCorrelatedScalarSubquery.ruleName :: - RewritePredicateSubquery.ruleName :: - NormalizeFloatingNumbers.ruleName :: - ReplaceUpdateFieldsExpression.ruleName :: - RewriteLateralSubquery.ruleName :: - OptimizeSubqueries.ruleName :: Nil + Seq( + FinishAnalysis.ruleName, + RewriteDistinctAggregates.ruleName, + ReplaceDeduplicateWithAggregate.ruleName, + ReplaceIntersectWithSemiJoin.ruleName, + ReplaceExceptWithFilter.ruleName, + ReplaceExceptWithAntiJoin.ruleName, + RewriteExceptAll.ruleName, + RewriteIntersectAll.ruleName, + ReplaceDistinctWithAggregate.ruleName, + PullupCorrelatedPredicates.ruleName, + RewriteCorrelatedScalarSubquery.ruleName, + RewritePredicateSubquery.ruleName, + NormalizeFloatingNumbers.ruleName, + ReplaceUpdateFieldsExpression.ruleName, + RewriteLateralSubquery.ruleName, + OptimizeSubqueries.ruleName) /** * Apply finish-analysis rules for the entire plan including all subqueries. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 6173703ef3cd9..6ceb363b41aef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -36,38 +36,41 @@ class SparkOptimizer( override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = // TODO: move SchemaPruning into catalyst - Seq(SchemaPruning) :+ - GroupBasedRowLevelOperationScanPlanning :+ - V1Writes :+ - V2ScanRelationPushDown :+ - V2ScanPartitioningAndOrdering :+ - V2Writes :+ - PruneFileSourcePartitions + Seq( + SchemaPruning, + GroupBasedRowLevelOperationScanPlanning, + V1Writes, + V2ScanRelationPushDown, + V2ScanPartitioningAndOrdering, + V2Writes, + PruneFileSourcePartitions) override def preCBORules: Seq[Rule[LogicalPlan]] = - OptimizeMetadataOnlyDeleteFromTable :: Nil + Seq(OptimizeMetadataOnlyDeleteFromTable) - override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ - Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ + override def defaultBatches: Seq[Batch] = flattenBatches(Seq( + preOptimizationBatches, + super.defaultBatches, + Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)), Batch("PartitionPruning", Once, PartitionPruning, // We can't run `OptimizeSubqueries` in this batch, as it will optimize the subqueries // twice which may break some optimizer rules that can only be applied once. The rule below // only invokes `OptimizeSubqueries` to optimize newly added subqueries. - new RowLevelOperationRuntimeGroupFiltering(OptimizeSubqueries)) :+ + new RowLevelOperationRuntimeGroupFiltering(OptimizeSubqueries)), Batch("InjectRuntimeFilter", FixedPoint(1), - InjectRuntimeFilter) :+ + InjectRuntimeFilter), Batch("MergeScalarSubqueries", Once, MergeScalarSubqueries, - RewriteDistinctAggregates) :+ + RewriteDistinctAggregates), Batch("Pushdown Filters from PartitionPruning", fixedPoint, - PushDownPredicates) :+ + PushDownPredicates), Batch("Cleanup filters that cannot be pushed down", Once, CleanupDynamicPruningFilters, // cleanup the unnecessary TrueLiteral predicates BooleanSimplification, - PruneFilters)) ++ - postHocOptimizationBatches :+ + PruneFilters), + postHocOptimizationBatches, Batch("Extract Python UDFs", Once, ExtractPythonUDFFromJoinCondition, // `ExtractPythonUDFFromJoinCondition` can convert a join to a cartesian product. @@ -84,25 +87,27 @@ class SparkOptimizer( LimitPushDown, PushPredicateThroughNonJoin, PushProjectionThroughLimit, - RemoveNoopOperators) :+ + RemoveNoopOperators), Batch("Infer window group limit", Once, InferWindowGroupLimit, LimitPushDown, LimitPushDownThroughWindow, EliminateLimits, - ConstantFolding) :+ - Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) :+ - Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition) + ConstantFolding), + Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*), + Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition))) - override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+ - ExtractPythonUDFFromJoinCondition.ruleName :+ - ExtractPythonUDFFromAggregate.ruleName :+ ExtractGroupingPythonUDFFromAggregate.ruleName :+ - ExtractPythonUDFs.ruleName :+ - GroupBasedRowLevelOperationScanPlanning.ruleName :+ - V2ScanRelationPushDown.ruleName :+ - V2ScanPartitioningAndOrdering.ruleName :+ - V2Writes.ruleName :+ - ReplaceCTERefWithRepartition.ruleName + override def nonExcludableRules: Seq[String] = super.nonExcludableRules ++ + Seq( + ExtractPythonUDFFromJoinCondition.ruleName, + ExtractPythonUDFFromAggregate.ruleName, + ExtractGroupingPythonUDFFromAggregate.ruleName, + ExtractPythonUDFs.ruleName, + GroupBasedRowLevelOperationScanPlanning.ruleName, + V2ScanRelationPushDown.ruleName, + V2ScanPartitioningAndOrdering.ruleName, + V2Writes.ruleName, + ReplaceCTERefWithRepartition.ruleName) /** * Optimization batches that are executed before the regular optimization batches (also before From accde839ead894cf4878965d42565cca9a7ed2c2 Mon Sep 17 00:00:00 2001 From: Milan Cupac Date: Wed, 18 Dec 2024 02:17:35 +0800 Subject: [PATCH 201/438] [SPARK-50598][SQL] An initial, no-op PR which adds new parameters to already existing classes UnresolvedWith, CTERelationRef and CTERelationDef to enable later implementation of recursive CTEs ### What changes were proposed in this pull request? PR adds new parameters to already existing classes UnresolvedWith, CTERelationRef and CTERelationDef to enable later implementation of recursive CTEs. - Additional parameters are added in pattern matching cases in other files of mentioned classes as well. - A large number of trivial changes in tests was required, due to new parameters introduced to the classes, which are also addressed in this PR. - More information for reviewers can be found here: https://docs.google.com/document/d/1qcEJxqoXcr5cSt6HgIQjWQSqhfkSaVYkoDHsg5oxXp4/edit ### Why are the changes needed? Support for the recursive CTE. ### Does this PR introduce _any_ user-facing change? No. RECURSIVE keyword is not introduced in this PR. ### How was this patch tested? The tests failing after the initial change were all in `SQLQueryTestSuite`, so this patch was tested by running this Test Suite. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49180 from milanisvet/milanrcte2. Authored-by: Milan Cupac Signed-off-by: Wenchen Fan --- .../catalyst/analysis/CTESubstitution.scala | 6 +- ...wnPredicatesAndPruneColumnsForCTEDef.scala | 6 +- .../plans/logical/basicLogicalOperators.scala | 17 +- .../analyzer-results/cte-command.sql.out | 20 +-- .../analyzer-results/cte-nested.sql.out | 162 +++++++++--------- .../analyzer-results/cte-nonlegacy.sql.out | 92 +++++----- .../sql-tests/analyzer-results/cte.sql.out | 134 +++++++-------- .../double-quoted-identifiers-enabled.sql.out | 8 +- .../identifier-clause.sql.out | 16 +- .../analyzer-results/join-lateral.sql.out | 8 +- .../non-excludable-rule.sql.out | 4 +- .../postgreSQL/window_part3.sql.out | 16 +- .../analyzer-results/postgreSQL/with.sql.out | 26 +-- .../sql-session-variables.sql.out | 4 +- .../exists-subquery/exists-cte.sql.out | 30 ++-- .../in-subquery/in-multiple-columns.sql.out | 6 +- .../subquery/in-subquery/in-with-cte.sql.out | 84 ++++----- .../scalar-subquery-select.sql.out | 38 ++-- .../analyzer-results/transform.sql.out | 6 +- .../analyzer-results/using-join.sql.out | 8 +- 20 files changed, 351 insertions(+), 340 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index d75e7d528d5b3..84f6d04eeb1a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -123,7 +123,7 @@ object CTESubstitution extends Rule[LogicalPlan] { startOfQuery: Boolean = true): Unit = { val resolver = conf.resolver plan match { - case UnresolvedWith(child, relations) => + case UnresolvedWith(child, relations, _) => val newNames = ArrayBuffer.empty[String] newNames ++= outerCTERelationNames relations.foreach { @@ -149,7 +149,7 @@ object CTESubstitution extends Rule[LogicalPlan] { plan: LogicalPlan, cteDefs: ArrayBuffer[CTERelationDef]): LogicalPlan = { plan.resolveOperatorsUp { - case UnresolvedWith(child, relations) => + case UnresolvedWith(child, relations, _) => val resolvedCTERelations = resolveCTERelations(relations, isLegacy = true, forceInline = false, Seq.empty, cteDefs) substituteCTE(child, alwaysInline = true, resolvedCTERelations) @@ -202,7 +202,7 @@ object CTESubstitution extends Rule[LogicalPlan] { var firstSubstituted: Option[LogicalPlan] = None val newPlan = plan.resolveOperatorsDownWithPruning( _.containsAnyPattern(UNRESOLVED_WITH, PLAN_EXPRESSION)) { - case UnresolvedWith(child: LogicalPlan, relations) => + case UnresolvedWith(child: LogicalPlan, relations, _) => val resolvedCTERelations = resolveCTERelations(relations, isLegacy = false, forceInline, outerCTEDefs, cteDefs) ++ outerCTEDefs diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala index aa13e6a67c510..838827a0e2e1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala @@ -122,7 +122,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { private def pushdownPredicatesAndAttributes( plan: LogicalPlan, cteMap: CTEMap): LogicalPlan = plan.transformWithSubqueries { - case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _) => + case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _, _, _) => val (_, _, newPreds, newAttrSet) = cteMap(id) val originalPlan = originalPlanWithPredicates.map(_._1).getOrElse(child) val preds = originalPlanWithPredicates.map(_._2).getOrElse(Seq.empty) @@ -141,7 +141,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { cteDef } - case cteRef @ CTERelationRef(cteId, _, output, _, _) => + case cteRef @ CTERelationRef(cteId, _, output, _, _, _) => val (cteDef, _, _, newAttrSet) = cteMap(cteId) if (needsPruning(cteDef.child, newAttrSet)) { val indices = newAttrSet.toSeq.map(cteDef.output.indexOf) @@ -170,7 +170,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { object CleanUpTempCTEInfo extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(CTE)) { - case cteDef @ CTERelationDef(_, _, Some(_), _) => + case cteDef @ CTERelationDef(_, _, Some(_), _, _, _) => cteDef.copy(originalPlanWithPredicates = None) } } 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 4636ca59d4e86..a6405f18ad444 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 @@ -833,10 +833,12 @@ object View { * @param child The final query of this CTE. * @param cteRelations A sequence of pair (alias, the CTE definition) that this CTE defined * Each CTE can see the base tables and the previously defined CTEs only. + * @param allowRecursion A boolean flag if recursion is allowed. */ case class UnresolvedWith( child: LogicalPlan, - cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { + cteRelations: Seq[(String, SubqueryAlias)], + allowRecursion: Boolean = false) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_WITH) override def output: Seq[Attribute] = child.output @@ -862,12 +864,19 @@ case class UnresolvedWith( * pushdown to help ensure rule idempotency. * @param underSubquery If true, it means we don't need to add a shuffle for this CTE relation as * subquery reuse will be applied to reuse CTE relation output. + * @param recursive If true, then this CTE Definition is recursive - it contains a self-reference. + * @param recursionAnchor A helper plan node that temporary stores the anchor term of recursive + * definitions. In the beginning of recursive resolution the `ResolveWithCTE` + * rule updates this parameter and once it is resolved the same rule resolves + * the recursive [[CTERelationRef]] references and removes this parameter. */ case class CTERelationDef( child: LogicalPlan, id: Long = CTERelationDef.newId, originalPlanWithPredicates: Option[(LogicalPlan, Seq[Expression])] = None, - underSubquery: Boolean = false) extends UnaryNode { + underSubquery: Boolean = false, + recursive: Boolean = false, + recursionAnchor: Option[LogicalPlan] = None) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) @@ -891,13 +900,15 @@ object CTERelationDef { * de-duplication. * @param statsOpt The optional statistics inferred from the corresponding CTE * definition. + * @param recursive If this is a recursive reference. */ case class CTERelationRef( cteId: Long, _resolved: Boolean, override val output: Seq[Attribute], override val isStreaming: Boolean, - statsOpt: Option[Statistics] = None) extends LeafNode with MultiInstanceRelation { + statsOpt: Option[Statistics] = None, + recursive: Boolean = false) extends LeafNode with MultiInstanceRelation { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out index 0b539267e720f..926e3cd003046 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out @@ -4,13 +4,13 @@ CREATE TABLE cte_tbl USING csv AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`cte_tbl`, ErrorIfExists, [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -26,13 +26,13 @@ CREATE TEMPORARY VIEW cte_view AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateViewCommand `cte_view`, WITH s AS (SELECT 42 AS col) SELECT * FROM s, false, false, LocalTempView, UNSUPPORTED, true +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -43,13 +43,13 @@ Project [col#x] +- View (`cte_view`, [col#x]) +- Project [cast(col#x as int) AS col#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -58,13 +58,13 @@ INSERT INTO cte_tbl SELECT * FROM S -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [43 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias S - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query @@ -80,13 +80,13 @@ INSERT INTO cte_tbl WITH s AS (SELECT 44 AS col) SELECT * FROM s -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias s : +- Project [44 AS col#x] : +- OneRowRelation +- Project [col#x] +- SubqueryAlias s - +- CTERelationRef xxxx, true, [col#x], false + +- CTERelationRef xxxx, true, [col#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out index 3a9fc5ea1297f..0365c4dcd8e09 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out @@ -7,18 +7,18 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -30,14 +30,14 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -48,13 +48,13 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -66,7 +66,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [outer(id#xL)] : : +- OneRowRelation @@ -83,7 +83,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -128,22 +128,22 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [2#x], false +: +- CTERelationRef xxxx, true, [2#x], false, false +- Project [2#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -160,29 +160,29 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false +: : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false +: : +- CTERelationRef xxxx, true, [c#x], false, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false -- !query @@ -199,31 +199,31 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3 AS 3#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [3#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [3#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [3#x], false +: +- CTERelationRef xxxx, true, [3#x], false, false +- Project [3#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [3#x], false + +- CTERelationRef xxxx, true, [3#x], false, false -- !query @@ -234,12 +234,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -248,7 +248,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -261,12 +261,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -277,7 +277,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -291,17 +291,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -312,7 +312,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -323,19 +323,19 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [2#x], false + : +- CTERelationRef xxxx, true, [2#x], false, false +- OneRowRelation @@ -349,20 +349,20 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false + : : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [2#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [2#x], false + : : +- CTERelationRef xxxx, true, [2#x], false, false : +- OneRowRelation +- OneRowRelation @@ -378,25 +378,25 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false + : : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [3 AS 3#x] : : : +- OneRowRelation : : +- Project [3#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [3#x], false + : : +- CTERelationRef xxxx, true, [3#x], false, false : +- OneRowRelation +- OneRowRelation @@ -410,7 +410,7 @@ WHERE c IN ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] @@ -418,16 +418,16 @@ WithCTE +- Project [c#x] +- Filter c#x IN (list#x []) : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [2#x AS c#x] : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [c#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [c#x], false + : +- CTERelationRef xxxx, true, [c#x], false, false +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -440,22 +440,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -468,22 +468,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias aBc : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x] : +- SubqueryAlias aBC -: +- CTERelationRef xxxx, true, [2#x], false +: +- CTERelationRef xxxx, true, [2#x], false, false +- Project [2#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -494,19 +494,19 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias aBc : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias aBC - : +- CTERelationRef xxxx, true, [2#x], false + : +- CTERelationRef xxxx, true, [2#x], false, false +- OneRowRelation @@ -522,23 +522,23 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -553,20 +553,20 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -586,27 +586,27 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out index e8640c3cbb6bd..53dcd46361dc1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out @@ -7,18 +7,18 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -30,14 +30,14 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -48,13 +48,13 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -66,7 +66,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [outer(id#xL)] : : +- OneRowRelation @@ -83,7 +83,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias unreferenced : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -153,29 +153,29 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false +: : :- CTERelationDef xxxx, false, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x], false +: : +- CTERelationRef xxxx, true, [c#x], false, false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x], false + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false, false -- !query @@ -211,12 +211,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -225,7 +225,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -238,12 +238,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -254,7 +254,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -268,17 +268,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -289,7 +289,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x], false + +- CTERelationRef xxxx, true, [c#x], false, false -- !query @@ -384,22 +384,22 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -454,23 +454,23 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -485,20 +485,20 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -518,27 +518,27 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [1#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index 9b86c4df62c4d..9a221f6a699cd 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -62,7 +62,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t AS (SELECT 1 FROM t) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- SubqueryAlias t @@ -73,7 +73,7 @@ WithCTE : +- LocalRelation [id#x] +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x], false + +- CTERelationRef xxxx, true, [1#x], false, false -- !query @@ -100,7 +100,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [id#x] : +- SubqueryAlias t2 @@ -109,17 +109,17 @@ WithCTE : +- Project [id#x] : +- SubqueryAlias t : +- LocalRelation [id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [id#x], false +: +- CTERelationRef xxxx, true, [id#x], false, false +- Project [id#x, 2#x] +- Join Cross :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [id#x], false + : +- CTERelationRef xxxx, true, [id#x], false, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x], false + +- CTERelationRef xxxx, true, [2#x], false, false -- !query @@ -134,7 +134,7 @@ FROM CTE1 t1 CROSS JOIN CTE1 t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias CTE1 : +- Project [id#x AS id#x] : +- Join Cross @@ -157,10 +157,10 @@ WithCTE +- Join Cross :- SubqueryAlias t1 : +- SubqueryAlias CTE1 - : +- CTERelationRef xxxx, true, [id#x], false + : +- CTERelationRef xxxx, true, [id#x], false, false +- SubqueryAlias t2 +- SubqueryAlias CTE1 - +- CTERelationRef xxxx, true, [id#x], false + +- CTERelationRef xxxx, true, [id#x], false, false -- !query @@ -168,7 +168,7 @@ WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE x = 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x] : +- Project [1 AS 1#x] @@ -176,7 +176,7 @@ WithCTE +- Project [x#x] +- Filter (x#x = 1) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -184,7 +184,7 @@ WITH t(x, y) AS (SELECT 1, 2) SELECT * FROM t WHERE x = 1 AND y = 2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -192,7 +192,7 @@ WithCTE +- Project [x#x, y#x] +- Filter ((x#x = 1) AND (y#x = 2)) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, y#x], false + +- CTERelationRef xxxx, true, [x#x, y#x], false, false -- !query @@ -200,14 +200,14 @@ WITH t(x, x) AS (SELECT 1, 2) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS x#x] : +- Project [1 AS 1#x, 2 AS 2#x] : +- OneRowRelation +- Project [x#x, x#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, x#x], false + +- CTERelationRef xxxx, true, [x#x, x#x], false, false -- !query @@ -300,56 +300,56 @@ WITH w1(c1) AS SELECT * FROM w1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w8 : +- Project [1#x AS c8#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w7 : +- Project [c8#x AS c7#x] : +- Project [c8#x] : +- SubqueryAlias w8 -: +- CTERelationRef xxxx, true, [c8#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c8#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w6 : +- Project [c7#x AS c6#x] : +- Project [c7#x] : +- SubqueryAlias w7 -: +- CTERelationRef xxxx, true, [c7#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c7#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w5 : +- Project [c6#x AS c5#x] : +- Project [c6#x] : +- SubqueryAlias w6 -: +- CTERelationRef xxxx, true, [c6#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c6#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w4 : +- Project [c5#x AS c4#x] : +- Project [c5#x] : +- SubqueryAlias w5 -: +- CTERelationRef xxxx, true, [c5#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c5#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w3 : +- Project [c4#x AS c3#x] : +- Project [c4#x] : +- SubqueryAlias w4 -: +- CTERelationRef xxxx, true, [c4#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c4#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w2 : +- Project [c3#x AS c2#x] : +- Project [c3#x] : +- SubqueryAlias w3 -: +- CTERelationRef xxxx, true, [c3#x], false -:- CTERelationDef xxxx, false +: +- CTERelationRef xxxx, true, [c3#x], false, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias w1 : +- Project [c2#x AS c1#x] : +- Project [c2#x] : +- SubqueryAlias w2 -: +- CTERelationRef xxxx, true, [c2#x], false +: +- CTERelationRef xxxx, true, [c2#x], false, false +- Project [c1#x] +- SubqueryAlias w1 - +- CTERelationRef xxxx, true, [c1#x], false + +- CTERelationRef xxxx, true, [c1#x], false, false -- !query @@ -379,14 +379,14 @@ WITH same_name AS (SELECT 42) SELECT * FROM same_name, (SELECT 10) AS same_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias same_name : +- Project [42 AS 42#x] : +- OneRowRelation +- Project [42#x, 10#x] +- Join Inner :- SubqueryAlias same_name - : +- CTERelationRef xxxx, true, [42#x], false + : +- CTERelationRef xxxx, true, [42#x], false, false +- SubqueryAlias same_name +- Project [10 AS 10#x] +- OneRowRelation @@ -419,13 +419,13 @@ WITH q AS (SELECT 'foo' AS x) SELECT x, typeof(x) FROM q -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q : +- Project [foo AS x#x] : +- OneRowRelation +- Project [x#x, typeof(x#x) AS typeof(x)#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -479,13 +479,13 @@ SELECT * FROM Project [y#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias q : +- Project [1 AS x#x] : +- OneRowRelation +- Project [(x#x + 1) AS y#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -493,13 +493,13 @@ select (with q as (select 1 x) select * from q) -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [1 AS x#x] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- OneRowRelation @@ -508,13 +508,13 @@ select 1 in (with q as (select 1) select * from q) -- !query analysis Project [1 IN (list#x []) AS (1 IN (listquery()))#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [1 AS 1#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [1#x], false +: +- CTERelationRef xxxx, true, [1#x], false, false +- OneRowRelation @@ -549,11 +549,11 @@ from T1 z -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T1 : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T1 : +- Project [2 AS b#x] : +- OneRowRelation @@ -562,14 +562,14 @@ WithCTE :- Join Inner : :- SubqueryAlias x : : +- SubqueryAlias T1 - : : +- CTERelationRef xxxx, true, [a#x], false + : : +- CTERelationRef xxxx, true, [a#x], false, false : +- SubqueryAlias y : +- Project [b#x] : +- SubqueryAlias T1 - : +- CTERelationRef xxxx, true, [b#x], false + : +- CTERelationRef xxxx, true, [b#x], false, false +- SubqueryAlias z +- SubqueryAlias T1 - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -580,15 +580,15 @@ from (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias TTtt : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias tTTt_2 : +- Project [2 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias TtTt : +- Project [3 AS c#x] : +- OneRowRelation @@ -597,9 +597,9 @@ WithCTE +- Project [c#x, a#x] +- Join Inner :- SubqueryAlias ttTT - : +- CTERelationRef xxxx, true, [c#x], false + : +- CTERelationRef xxxx, true, [c#x], false, false +- SubqueryAlias tttT_2 - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -609,13 +609,13 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x] AS scalarsubquery(x)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [outer(x#x)] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -628,13 +628,13 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x && y#x] AS scalarsubquery(x, y)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias q : : +- Project [3 AS z#x] : : +- OneRowRelation : +- Project [((outer(x#x) + outer(y#x)) + z#x) AS ((outer(T.x) + outer(T.y)) + z)#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [z#x], false +: +- CTERelationRef xxxx, true, [z#x], false, false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -646,20 +646,20 @@ select * from (with q2 as (select * from q1) select * from q2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q2 : +- Project [x#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- Project [x#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [x#x] +- SubqueryAlias q2 - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -668,20 +668,20 @@ select * from (with q1 as (select x+1 from q1) select * from q1) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [(x#x + 1) AS (x + 1)#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x], false +: +- CTERelationRef xxxx, true, [x#x], false, false +- Project [(x + 1)#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [(x + 1)#x] +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [(x + 1)#x], false + +- CTERelationRef xxxx, true, [(x + 1)#x], false, false -- !query @@ -709,7 +709,7 @@ with cte1 as (Select id as j from t) select * from cte1 where j = (select max(j) from cte1 as cte2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [id#x AS j#x] : +- SubqueryAlias t @@ -723,9 +723,9 @@ WithCTE : +- Aggregate [max(j#x) AS max(j)#x] : +- SubqueryAlias cte2 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [j#x], false + : +- CTERelationRef xxxx, true, [j#x], false, false +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [j#x], false + +- CTERelationRef xxxx, true, [j#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out index 22dfeac5fd0b6..35713167bc38d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out @@ -411,14 +411,14 @@ CREATE TEMPORARY VIEW "myview"("c1") AS -- !query analysis CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v", false, false, LocalTempView, UNSUPPORTED, true +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query @@ -431,14 +431,14 @@ Project [a1#x AS a2#x] +- View (`myview`, [c1#x]) +- Project [cast(a#x as int) AS c1#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false + +- CTERelationRef xxxx, true, [a#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 20e6ca1e6a2ec..9384f17fa74c1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -1012,17 +1012,17 @@ WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias S : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias T - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query @@ -1031,17 +1031,17 @@ WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias S : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias T - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query @@ -1049,13 +1049,13 @@ WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias ABC : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Aggregate [max(c1#x) AS max(c1)#x] +- SubqueryAlias ABC - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index 5bf893605423c..c93109e428362 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -1358,14 +1358,14 @@ WITH cte1 AS ( SELECT * FROM cte2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [c1#x] : +- SubqueryAlias spark_catalog.default.t1 : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x]) : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte2 : +- Project [c1#x, c2#x] : +- LateralJoin lateral-subquery#x [c1#x], Inner @@ -1377,10 +1377,10 @@ WithCTE : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : : +- LocalRelation [col1#x, col2#x] : +- SubqueryAlias cte1 -: +- CTERelationRef xxxx, true, [c1#x], false +: +- CTERelationRef xxxx, true, [c1#x], false, false +- Project [c1#x, c2#x] +- SubqueryAlias cte2 - +- CTERelationRef xxxx, true, [c1#x, c2#x], false + +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out index 6b2c60f25bae3..7f1ca3dda902d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out @@ -36,7 +36,7 @@ WITH tmp AS ( SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias tmp : +- Intersect false : :- Project [id#xL] @@ -47,7 +47,7 @@ WithCTE +- Filter (id#xL > scalar-subquery#x []) : +- Aggregate [max(id#xL) AS max(id)#xL] : +- SubqueryAlias tmp - : +- CTERelationRef xxxx, true, [id#xL], false + : +- CTERelationRef xxxx, true, [id#xL], false, false +- Range (0, 3, step=1) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out index 9f2dd9bcb1783..441b3627bc928 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -89,7 +89,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -99,7 +99,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -111,7 +111,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -121,7 +121,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -134,7 +134,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false @@ -154,7 +154,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query @@ -167,7 +167,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false @@ -187,7 +187,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL], false + +- CTERelationRef xxxx, true, [x#xL], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out index f29b12d5af171..b92e70e91a4ba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out @@ -4,7 +4,7 @@ WITH q1(x,y) AS (SELECT 1,2) SELECT * FROM q1, q1 AS q2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias q1 : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -12,10 +12,10 @@ WithCTE +- Project [x#x, y#x, x#x, y#x] +- Join Inner :- SubqueryAlias q1 - : +- CTERelationRef xxxx, true, [x#x, y#x], false + : +- CTERelationRef xxxx, true, [x#x, y#x], false, false +- SubqueryAlias q2 +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [x#x, y#x], false + +- CTERelationRef xxxx, true, [x#x, y#x], false, false -- !query @@ -185,7 +185,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte : +- Project [42#x AS foo#x] : +- Project [42 AS 42#x] @@ -194,7 +194,7 @@ WithCTE +- SubqueryAlias q +- Project [foo#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [foo#x], false + +- CTERelationRef xxxx, true, [foo#x], false, false -- !query @@ -207,11 +207,11 @@ WITH outermost(x) AS ( SELECT * FROM outermost ORDER BY 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias innermost : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias outermost : +- Project [1#x AS x#x] : +- Distinct @@ -222,13 +222,13 @@ WithCTE : +- Union false, false : :- Project [2#x] : : +- SubqueryAlias innermost -: : +- CTERelationRef xxxx, true, [2#x], false +: : +- CTERelationRef xxxx, true, [2#x], false, false : +- Project [3 AS 3#x] : +- OneRowRelation +- Sort [x#x ASC NULLS FIRST], true +- Project [x#x] +- SubqueryAlias outermost - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -412,13 +412,13 @@ org.apache.spark.sql.catalyst.parser.ParseException with ordinality as (select 1 as x) select * from ordinality -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias ordinality : +- Project [1 AS x#x] : +- OneRowRelation +- Project [x#x] +- SubqueryAlias ordinality - +- CTERelationRef xxxx, true, [x#x], false + +- CTERelationRef xxxx, true, [x#x], false, false -- !query @@ -453,13 +453,13 @@ with test as (select 42) insert into test select * from test InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test], Append, `spark_catalog`.`default`.`test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test), [i] +- Project [cast(42#x as int) AS i#x] +- WithCTE - :- CTERelationDef xxxx, false + :- CTERelationDef xxxx, false, false : +- SubqueryAlias test : +- Project [42 AS 42#x] : +- OneRowRelation +- Project [42#x] +- SubqueryAlias test - +- CTERelationRef xxxx, true, [42#x], false + +- CTERelationRef xxxx, true, [42#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out index 02e7c39ae83fd..da4542fe9bb1c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out @@ -2044,13 +2044,13 @@ Project [1 AS 1#x] WITH v1 AS (SELECT var1 AS c1) SELECT c1 AS `1` FROM v1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias v1 : +- Project [variablereference(system.session.var1=1) AS c1#x] : +- OneRowRelation +- Project [c1#x AS 1#x] +- SubqueryAlias v1 - +- CTERelationRef xxxx, true, [c1#x], false + +- CTERelationRef xxxx, true, [c1#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out index 7c3678c66c117..f210bc49030fc 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out @@ -101,7 +101,7 @@ WHERE a.bonus_amt > 30 WHERE a.emp_name = b.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias bonus_cte : +- Project [emp_name#x, bonus_amt#x] : +- Filter exists#x [emp_name#x] @@ -133,7 +133,7 @@ WithCTE : +- Filter (outer(emp_name#x) = emp_name#x) : +- SubqueryAlias b : +- SubqueryAlias bonus_cte - : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false + : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false, false +- SubqueryAlias a +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) @@ -162,7 +162,7 @@ WHERE EXISTS (SELECT * WHERE bonus.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -172,7 +172,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -189,10 +189,10 @@ WithCTE : +- Join Inner, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -225,7 +225,7 @@ WHERE e.dept_id = d.dept_id WHERE e.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -235,7 +235,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -253,10 +253,10 @@ WithCTE : +- Join LeftOuter, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false, false +- Join Inner :- Join Inner : :- SubqueryAlias b @@ -268,7 +268,7 @@ WithCTE : : +- LocalRelation [emp_name#x, bonus_amt#x] : +- SubqueryAlias e : +- SubqueryAlias emp_cte - : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false + : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false, false +- SubqueryAlias d +- SubqueryAlias dept +- View (`DEPT`, [dept_id#x, dept_name#x, state#x]) @@ -299,7 +299,7 @@ WHERE EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) @@ -322,7 +322,7 @@ WithCTE : +- Filter (count(1)#xL > cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -352,7 +352,7 @@ WHERE NOT EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) @@ -375,7 +375,7 @@ WithCTE : +- Filter (count(1)#xL < cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false, false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x, bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out index 39748a324e527..85e28de736a0c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out @@ -307,7 +307,7 @@ FROM ( on cte1.t1b = cte2.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter named_struct(t1b, t1b#x, t1d, t1d#xL) IN (list#x [t1c#x]) @@ -330,7 +330,7 @@ WithCTE +- Project [t1a#x, t1b#x, t1a#x, t1b#x] +- Join Inner, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out index 0074991b4ea6a..03a4c3ffd4fcb 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out @@ -123,7 +123,7 @@ WHERE t1b IN (SELECT cte1.t1b WHERE cte1.t1b > 0) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1a) @@ -138,7 +138,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) > 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -177,7 +177,7 @@ GROUP BY t1a, t1b, t1c HAVING t1c IS NOT NULL -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- SubqueryAlias t1 @@ -197,21 +197,21 @@ WithCTE : : : :- Project [t1b#x] : : : : +- Filter (cast(t1b#x as int) > 0) : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : : +- Project [t1b#x] : : : +- Filter (cast(t1b#x as int) > 5) : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- Intersect false : : :- Project [t1b#x] : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- Project [t1b#x] : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : +- Project [t1b#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -250,7 +250,7 @@ WHERE t1c IN ON cte1.t1d > cte6.t1d) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] : +- SubqueryAlias t1 @@ -268,22 +268,22 @@ WithCTE : : : :- Join FullOuter, (t1c#x = t1c#x) : : : : :- Join Inner, (t1b#x > t1b#x) : : : : : :- SubqueryAlias cte1 - : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : : : +- SubqueryAlias cte2 : : : : : +- SubqueryAlias cte1 - : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : : +- SubqueryAlias cte3 : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : : +- SubqueryAlias cte4 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : : +- SubqueryAlias cte5 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false : +- SubqueryAlias cte6 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -316,7 +316,7 @@ FROM (SELECT * ON cte1.t1b = cte4.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1b#x IN (list#x []) AND (t1a#x = val1b)) @@ -354,16 +354,16 @@ WithCTE :- Join FullOuter, (t1a#x = t1a#x) : :- Join Inner, ((cast(t1b#x as int) > 5) AND (t1a#x = t1a#x)) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -393,7 +393,7 @@ WHERE t1b IN GROUP BY t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1h#x] : +- Filter t1a#x IN (list#x [t1b#x]) @@ -424,10 +424,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join Inner, (t1h#x >= t1h#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false, false -- !query @@ -453,7 +453,7 @@ FROM ( ) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x] : +- Filter (t1b#x IN (list#x [t1c#x]) AND (t1a#x = val1b)) @@ -485,16 +485,16 @@ WithCTE :- Join RightOuter, (t1b#x = t1b#x) : :- Join Inner, (t1a#x = t1a#x) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false, false -- !query @@ -515,7 +515,7 @@ FROM (SELECT cte1.t1a, GROUP BY s.t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -538,10 +538,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join RightOuter, (t1a#x = t1a#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -569,7 +569,7 @@ WHERE s.t1b IN ON t1.t1a = cte1.t1a) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -599,15 +599,15 @@ WithCTE : : +- SubqueryAlias t1 : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias s +- Project [t1b#x] +- Join LeftOuter, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false -- !query @@ -627,7 +627,7 @@ WHERE t1b NOT IN (SELECT cte1.t1b t1c > 10 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1d) @@ -642,7 +642,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) < 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -683,7 +683,7 @@ WHERE t1b NOT IN ORDER BY t1c DESC -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] : +- Filter NOT t1d#xL IN (list#x []) @@ -722,16 +722,16 @@ WithCTE : :- Join RightOuter, (t1b#x = t1b#x) : : :- Join Inner, (t1a#x = t1a#x) : : : :- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : : : +- SubqueryAlias cte2 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : : +- SubqueryAlias cte3 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false : +- SubqueryAlias cte4 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false, false +- SubqueryAlias t1 +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 72e230f9bb881..3b1ffa94c17f2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -617,13 +617,13 @@ SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1 -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [1 AS a#x] : : +- OneRowRelation : +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [a#x], false +: +- CTERelationRef xxxx, true, [a#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -636,7 +636,7 @@ SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- Filter (c1#x = outer(c1#x)) @@ -647,7 +647,7 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- LocalRelation [c1#x, c2#x] : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -664,7 +664,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t3 : : +- Project [(c1#x + 1) AS c1#x, (c2#x + 1) AS c2#x] : : +- SubqueryAlias t2 @@ -672,15 +672,15 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] : : +- SubqueryAlias t2 : : +- LocalRelation [c1#x, c2#x] -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t4 : : +- Project [c1#x, c2#x] : : +- Filter (outer(c1#x) = c1#x) : : +- SubqueryAlias t3 -: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t4 -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -697,7 +697,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false +: :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -713,10 +713,10 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- Union false, false : :- Project [c1#x, c2#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- Project [c2#x, c1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -732,7 +732,7 @@ SELECT * FROM t1 WHERE c1 > ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias v : +- Project [c1#x, c2#x] : +- SubqueryAlias t2 @@ -743,7 +743,7 @@ WithCTE +- Project [c1#x, c2#x] +- Filter (cast(c1#x as bigint) > scalar-subquery#x [c1#x]) : +- WithCTE - : :- CTERelationDef xxxx, false + : :- CTERelationDef xxxx, false, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -756,9 +756,9 @@ WithCTE : : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : : +- Filter (c1#x = outer(c1#x)) : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false + : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false : +- SubqueryAlias v - : +- CTERelationRef xxxx, true, [c1#x, c2#x], false + : +- CTERelationRef xxxx, true, [c1#x, c2#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -771,7 +771,7 @@ WITH t AS (SELECT 1 AS a) SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t : +- Project [1 AS a#x] : +- OneRowRelation @@ -779,7 +779,7 @@ WithCTE : +- Project [a#x] : +- Filter (a#x = outer(c1#x)) : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [a#x], false + : +- CTERelationRef xxxx, true, [a#x], false, false +- SubqueryAlias t1 +- View (`t1`, [c1#x, c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -1019,7 +1019,7 @@ SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') FROM (SELECT null as col) as foo -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias T : +- Project [1 AS a#x] : +- OneRowRelation @@ -1027,7 +1027,7 @@ WithCTE : +- Aggregate [sum(1) AS sum(1)#xL] : +- Filter ((a#x = cast(outer(col#x) as int)) OR (upper(cast(outer(col#x) as string)) = Y)) : +- SubqueryAlias T - : +- CTERelationRef xxxx, true, [a#x], false + : +- CTERelationRef xxxx, true, [a#x], false, false +- SubqueryAlias foo +- Project [null AS col#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out index f0d3b8b999a29..04e019fdbaa41 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out @@ -874,7 +874,7 @@ WITH temp AS ( SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias temp : +- ScriptTransformation cat, [b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) : +- Project [a#x] @@ -888,10 +888,10 @@ WithCTE +- Join Inner, (b#x = b#x) :- SubqueryAlias t1 : +- SubqueryAlias temp - : +- CTERelationRef xxxx, true, [b#x], false + : +- CTERelationRef xxxx, true, [b#x], false, false +- SubqueryAlias t2 +- SubqueryAlias temp - +- CTERelationRef xxxx, true, [b#x], false + +- CTERelationRef xxxx, true, [b#x], false, false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out index 5a74c4be107e3..89f988fe2b616 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out @@ -817,12 +817,12 @@ FROM t1 FULL OUTER JOIN t2 USING (key) WHERE t1.key NOT LIKE 'bb.%' -- !query analysis WithCTE -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t1 : +- Project [key#x] : +- SubqueryAlias t : +- LocalRelation [key#x] -:- CTERelationDef xxxx, false +:- CTERelationDef xxxx, false, false : +- SubqueryAlias t2 : +- Project [key#x] : +- SubqueryAlias t @@ -833,6 +833,6 @@ WithCTE +- Project [coalesce(key#x, key#x) AS key#x, key#x, key#x, key#x] +- Join FullOuter, (key#x = key#x) :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [key#x], false + : +- CTERelationRef xxxx, true, [key#x], false, false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [key#x], false + +- CTERelationRef xxxx, true, [key#x], false, false From 2b41131d7fa66ef5b23fbe247e057d631ee5e4f6 Mon Sep 17 00:00:00 2001 From: Neil Ramaswamy Date: Wed, 18 Dec 2024 15:45:36 +0900 Subject: [PATCH 202/438] [MINOR][SS] Minor update to watermark propagation comments ### What changes were proposed in this pull request? A few minor changes to clarify (and fix one typo) in the comments for watermark propagation in Structured Streaming. ### Why are the changes needed? I found some of the terminology around "simulation" confusing, and the current comment describes incorrect logic for output watermark calculation. ### 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 #49188 from neilramaswamy/nr/minor-wm-prop. Authored-by: Neil Ramaswamy Signed-off-by: Jungtaek Lim --- .../sql/execution/streaming/WatermarkPropagator.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala index f0950063b1613..aaf8cbd69ea20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala @@ -124,12 +124,14 @@ class UseSingleWatermarkPropagator extends WatermarkPropagator { /** * This implementation simulates propagation of watermark among operators. * - * The simulation algorithm traverses the physical plan tree via post-order (children first) to - * calculate (input watermark, output watermark) for all nodes. + * It is considered a "simulation" because watermarks are not being physically sent between + * operators, but rather propagated up the tree via post-order (children first) traversal of + * the query plan. This allows Structured Streaming to determine the new (input watermark, output + * watermark) for all nodes. * * For each node, below logic is applied: * - * - Input watermark for specific node is decided by `min(input watermarks from all children)`. + * - Input watermark for specific node is decided by `min(output watermarks from all children)`. * -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded. * -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS. * - Output watermark for specific node is decided as following: From 229118ca7a127753635543909efdb27601985d42 Mon Sep 17 00:00:00 2001 From: Eric Marnadi Date: Wed, 18 Dec 2024 16:06:45 +0900 Subject: [PATCH 203/438] [SPARK-50599][SQL] Create the DataEncoder trait that allows for Avro and UnsafeRow encoding ### What changes were proposed in this pull request? Currently, we use the internal byte representation to store state for stateful streaming operators in the StateStore. This PR introduces Avro serialization and deserialization capabilities in the RocksDBStateEncoder so that we can instead use Avro encoding to store state. This is currently enabled for the TransformWithState operator via SQLConf to support all functionality supported by TWS ### Why are the changes needed? UnsafeRow is an inherently unstable format that makes no guarantees of being backwards-compatible. Therefore, if the format changes between Spark releases, this could cause StateStore corruptions. Avro is more stable, and inherently enables schema evolution. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Amended and added to unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #48944 from ericm-db/avro-ss. Lead-authored-by: Eric Marnadi Co-authored-by: Eric Marnadi <132308037+ericm-db@users.noreply.github.com> Signed-off-by: Jungtaek Lim --- .../streaming/state/RocksDBStateEncoder.scala | 1711 +++++++++-------- .../state/RocksDBStateStoreProvider.scala | 169 +- .../streaming/state/StateStore.scala | 41 + 3 files changed, 1080 insertions(+), 841 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index f39022c1f53a6..b4f6197811939 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -27,7 +27,7 @@ import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWri import org.apache.avro.io.{DecoderFactory, EncoderFactory} import org.apache.spark.internal.Logging -import org.apache.spark.sql.avro.{AvroDeserializer, AvroSerializer, SchemaConverters} +import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, AvroSerializer, SchemaConverters} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BoundReference, JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter @@ -51,93 +51,138 @@ sealed trait RocksDBValueStateEncoder { def decodeValues(valueBytes: Array[Byte]): Iterator[UnsafeRow] } -abstract class RocksDBKeyStateEncoderBase( - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short] = None) extends RocksDBKeyStateEncoder { - def offsetForColFamilyPrefix: Int = - if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 +/** + * The DataEncoder can encode UnsafeRows into raw bytes in two ways: + * - Using the direct byte layout of the UnsafeRow + * - Converting the UnsafeRow into an Avro row, and encoding that + * In both of these cases, the raw bytes that are written into RockDB have + * headers, footers and other metadata, but they also have data that is provided + * by the callers. The metadata in each row does not need to be written as Avro or UnsafeRow, + * but the actual data provided by the caller does. + * The classes that use this trait require specialized partial encoding which makes them much + * easier to cache and use, which is why each DataEncoder deals with multiple schemas. + */ +trait DataEncoder { + /** + * Encodes a complete key row into bytes. Used as the primary key for state lookups. + * + * @param row An UnsafeRow containing all key columns as defined in the keySchema + * @return Serialized byte array representation of the key + */ + def encodeKey(row: UnsafeRow): Array[Byte] - val out = new ByteArrayOutputStream /** - * Get Byte Array for the virtual column family id that is used as prefix for - * key state rows. + * Encodes the non-prefix portion of a key row. Used with prefix scan and + * range scan state lookups where the key is split into prefix and remaining portions. + * + * For prefix scans: Encodes columns after the prefix columns + * For range scans: Encodes columns not included in the ordering columns + * + * @param row An UnsafeRow containing only the remaining key columns + * @return Serialized byte array of the remaining key portion + * @throws UnsupportedOperationException if called on an encoder that doesn't support split keys */ - override def getColumnFamilyIdBytes(): Array[Byte] = { - assert(useColumnFamilies, "Cannot return virtual Column Family Id Bytes" + - " because multiple Column is not supported for this encoder") - val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId.get) - encodedBytes - } + def encodeRemainingKey(row: UnsafeRow): Array[Byte] /** - * Encode and put column family Id as a prefix to a pre-allocated byte array. + * Encodes key columns used for range scanning, ensuring proper sort order in RocksDB. * - * @param numBytes - size of byte array to be created for storing key row (without - * column family prefix) - * @return Array[Byte] for an array byte to put encoded key bytes - * Int for a starting offset to put the encoded key bytes + * This method handles special encoding for numeric types to maintain correct sort order: + * - Adds sign byte markers for numeric types + * - Flips bits for negative floating point values + * - Preserves null ordering + * + * @param row An UnsafeRow containing the columns needed for range scan + * (specified by orderingOrdinals) + * @return Serialized bytes that will maintain correct sort order in RocksDB + * @throws UnsupportedOperationException if called on an encoder that doesn't support range scans */ - protected def encodeColumnFamilyPrefix(numBytes: Int): (Array[Byte], Int) = { - val encodedBytes = new Array[Byte](numBytes + offsetForColFamilyPrefix) - var offset = Platform.BYTE_ARRAY_OFFSET - if (useColumnFamilies) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId.get) - offset = Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix - } - (encodedBytes, offset) - } + def encodePrefixKeyForRangeScan(row: UnsafeRow): Array[Byte] /** - * Get starting offset for decoding an encoded key byte array. + * Encodes a value row into bytes. + * + * @param row An UnsafeRow containing the value columns as defined in the valueSchema + * @return Serialized byte array representation of the value */ - protected def decodeKeyStartOffset: Int = { - if (useColumnFamilies) { - Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES - } else Platform.BYTE_ARRAY_OFFSET - } + def encodeValue(row: UnsafeRow): Array[Byte] + + /** + * Decodes a complete key from its serialized byte form. + * + * For NoPrefixKeyStateEncoder: Decodes the entire key + * For PrefixKeyScanStateEncoder: Decodes only the prefix portion + * + * @param bytes Serialized byte array containing the encoded key + * @return UnsafeRow containing the decoded key columns + * @throws UnsupportedOperationException for unsupported encoder types + */ + def decodeKey(bytes: Array[Byte]): UnsafeRow + + /** + * Decodes the remaining portion of a split key from its serialized form. + * + * For PrefixKeyScanStateEncoder: Decodes columns after the prefix + * For RangeKeyScanStateEncoder: Decodes non-ordering columns + * + * @param bytes Serialized byte array containing the encoded remaining key portion + * @return UnsafeRow containing the decoded remaining key columns + * @throws UnsupportedOperationException if called on an encoder that doesn't support split keys + */ + def decodeRemainingKey(bytes: Array[Byte]): UnsafeRow + + /** + * Decodes range scan key bytes back into an UnsafeRow, preserving proper ordering. + * + * This method reverses the special encoding done by encodePrefixKeyForRangeScan: + * - Interprets sign byte markers + * - Reverses bit flipping for negative floating point values + * - Handles null values + * + * @param bytes Serialized byte array containing the encoded range scan key + * @return UnsafeRow containing the decoded range scan columns + * @throws UnsupportedOperationException if called on an encoder that doesn't support range scans + */ + def decodePrefixKeyForRangeScan(bytes: Array[Byte]): UnsafeRow + + /** + * Decodes a value from its serialized byte form. + * + * @param bytes Serialized byte array containing the encoded value + * @return UnsafeRow containing the decoded value columns + */ + def decodeValue(bytes: Array[Byte]): UnsafeRow } -object RocksDBStateEncoder extends Logging { - def getKeyEncoder( - keyStateEncoderSpec: KeyStateEncoderSpec, - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short] = None, - avroEnc: Option[AvroEncoder] = None): RocksDBKeyStateEncoder = { - // Return the key state encoder based on the requested type - keyStateEncoderSpec match { - case NoPrefixKeyStateEncoderSpec(keySchema) => - new NoPrefixKeyStateEncoder(keySchema, useColumnFamilies, virtualColFamilyId, avroEnc) +abstract class RocksDBDataEncoder( + keyStateEncoderSpec: KeyStateEncoderSpec, + valueSchema: StructType) extends DataEncoder { - case PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) => - new PrefixKeyScanStateEncoder(keySchema, numColsPrefixKey, - useColumnFamilies, virtualColFamilyId, avroEnc) + val keySchema = keyStateEncoderSpec.keySchema + val reusedKeyRow = new UnsafeRow(keyStateEncoderSpec.keySchema.length) + val reusedValueRow = new UnsafeRow(valueSchema.length) - case RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals) => - new RangeKeyScanStateEncoder(keySchema, orderingOrdinals, - useColumnFamilies, virtualColFamilyId, avroEnc) + // bit masks used for checking sign or flipping all bits for negative float/double values + val floatFlipBitMask = 0xFFFFFFFF + val floatSignBitMask = 0x80000000 - case _ => - throw new IllegalArgumentException(s"Unsupported key state encoder spec: " + - s"$keyStateEncoderSpec") - } - } + val doubleFlipBitMask = 0xFFFFFFFFFFFFFFFFL + val doubleSignBitMask = 0x8000000000000000L - def getValueEncoder( - valueSchema: StructType, - useMultipleValuesPerKey: Boolean, - avroEnc: Option[AvroEncoder] = None): RocksDBValueStateEncoder = { - if (useMultipleValuesPerKey) { - new MultiValuedStateEncoder(valueSchema, avroEnc) - } else { - new SingleValueStateEncoder(valueSchema, avroEnc) - } - } + // Byte markers used to identify whether the value is null, negative or positive + // To ensure sorted ordering, we use the lowest byte value for negative numbers followed by + // positive numbers and then null values. + val negativeValMarker: Byte = 0x00.toByte + val positiveValMarker: Byte = 0x01.toByte + val nullValMarker: Byte = 0x02.toByte - def getColumnFamilyIdBytes(virtualColFamilyId: Short): Array[Byte] = { - val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId) - encodedBytes + + def unsupportedOperationForKeyStateEncoder( + operation: String + ): UnsupportedOperationException = { + new UnsupportedOperationException( + s"Method $operation not supported for encoder spec type " + + s"${keyStateEncoderSpec.getClass.getSimpleName}") } /** @@ -156,26 +201,6 @@ object RocksDBStateEncoder extends Logging { encodedBytes } - /** - * This method takes an UnsafeRow, and serializes to a byte array using Avro encoding. - */ - def encodeUnsafeRowToAvro( - row: UnsafeRow, - avroSerializer: AvroSerializer, - valueAvroType: Schema, - out: ByteArrayOutputStream): Array[Byte] = { - // InternalRow -> Avro.GenericDataRecord - val avroData = - avroSerializer.serialize(row) - out.reset() - val encoder = EncoderFactory.get().directBinaryEncoder(out, null) - val writer = new GenericDatumWriter[Any]( - valueAvroType) // Defining Avro writer for this struct type - writer.write(avroData, encoder) // Avro.GenericDataRecord -> byte array - encoder.flush() - out.toByteArray - } - def decodeToUnsafeRow(bytes: Array[Byte], numFields: Int): UnsafeRow = { if (bytes != null) { val row = new UnsafeRow(numFields) @@ -185,26 +210,6 @@ object RocksDBStateEncoder extends Logging { } } - /** - * This method takes a byte array written using Avro encoding, and - * deserializes to an UnsafeRow using the Avro deserializer - */ - def decodeFromAvroToUnsafeRow( - valueBytes: Array[Byte], - avroDeserializer: AvroDeserializer, - valueAvroType: Schema, - valueProj: UnsafeProjection): UnsafeRow = { - val reader = new GenericDatumReader[Any](valueAvroType) - val decoder = DecoderFactory.get().binaryDecoder(valueBytes, 0, valueBytes.length, null) - // bytes -> Avro.GenericDataRecord - val genericData = reader.read(null, decoder) - // Avro.GenericDataRecord -> InternalRow - val internalRow = avroDeserializer.deserialize( - genericData).orNull.asInstanceOf[InternalRow] - // InternalRow -> UnsafeRow - valueProj.apply(internalRow) - } - def decodeToUnsafeRow(bytes: Array[Byte], reusedRow: UnsafeRow): UnsafeRow = { if (bytes != null) { // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. @@ -219,470 +224,403 @@ object RocksDBStateEncoder extends Logging { } } -/** - * RocksDB Key Encoder for UnsafeRow that supports prefix scan - * - * @param keySchema - schema of the key to be encoded - * @param numColsPrefixKey - number of columns to be used for prefix key - * @param useColumnFamilies - if column family is enabled for this encoder - * @param avroEnc - if Avro encoding is specified for this StateEncoder, this encoder will - * be defined - */ -class PrefixKeyScanStateEncoder( - keySchema: StructType, - numColsPrefixKey: Int, - useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None, - avroEnc: Option[AvroEncoder] = None) - extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) { - - import RocksDBStateEncoder._ +class UnsafeRowDataEncoder( + keyStateEncoderSpec: KeyStateEncoderSpec, + valueSchema: StructType) extends RocksDBDataEncoder(keyStateEncoderSpec, valueSchema) { - private val usingAvroEncoding = avroEnc.isDefined - private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { - keySchema.zipWithIndex.take(numColsPrefixKey) + override def encodeKey(row: UnsafeRow): Array[Byte] = { + encodeUnsafeRow(row) } - private val remainingKeyFieldsWithIdx: Seq[(StructField, Int)] = { - keySchema.zipWithIndex.drop(numColsPrefixKey) + override def encodeRemainingKey(row: UnsafeRow): Array[Byte] = { + encodeUnsafeRow(row) } - private val prefixKeyProjection: UnsafeProjection = { - val refs = prefixKeyFieldsWithIdx.map(x => BoundReference(x._2, x._1.dataType, x._1.nullable)) - UnsafeProjection.create(refs) - } + override def encodePrefixKeyForRangeScan(row: UnsafeRow): Array[Byte] = { + assert(keyStateEncoderSpec.isInstanceOf[RangeKeyScanStateEncoderSpec]) + val rsk = keyStateEncoderSpec.asInstanceOf[RangeKeyScanStateEncoderSpec] + val rangeScanKeyFieldsWithOrdinal = rsk.orderingOrdinals.map { ordinal => + val field = rsk.keySchema(ordinal) + (field, ordinal) + } + val writer = new UnsafeRowWriter(rsk.orderingOrdinals.length) + writer.resetRowWriter() + rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => + val field = fieldWithOrdinal._1 + val value = row.get(idx, field.dataType) + // Note that we cannot allocate a smaller buffer here even if the value is null + // because the effective byte array is considered variable size and needs to have + // the same size across all rows for the ordering to work as expected. + val bbuf = ByteBuffer.allocate(field.dataType.defaultSize + 1) + bbuf.order(ByteOrder.BIG_ENDIAN) + if (value == null) { + bbuf.put(nullValMarker) + writer.write(idx, bbuf.array()) + } else { + field.dataType match { + case BooleanType => + case ByteType => + val byteVal = value.asInstanceOf[Byte] + val signCol = if (byteVal < 0) { + negativeValMarker + } else { + positiveValMarker + } + bbuf.put(signCol) + bbuf.put(byteVal) + writer.write(idx, bbuf.array()) - private val remainingKeyProjection: UnsafeProjection = { - val refs = remainingKeyFieldsWithIdx.map(x => - BoundReference(x._2, x._1.dataType, x._1.nullable)) - UnsafeProjection.create(refs) - } + case ShortType => + val shortVal = value.asInstanceOf[Short] + val signCol = if (shortVal < 0) { + negativeValMarker + } else { + positiveValMarker + } + bbuf.put(signCol) + bbuf.putShort(shortVal) + writer.write(idx, bbuf.array()) - // Prefix Key schema and projection definitions used by the Avro Serializers - // and Deserializers - private val prefixKeySchema = StructType(keySchema.take(numColsPrefixKey)) - private lazy val prefixKeyAvroType = SchemaConverters.toAvroType(prefixKeySchema) - private val prefixKeyProj = UnsafeProjection.create(prefixKeySchema) + case IntegerType => + val intVal = value.asInstanceOf[Int] + val signCol = if (intVal < 0) { + negativeValMarker + } else { + positiveValMarker + } + bbuf.put(signCol) + bbuf.putInt(intVal) + writer.write(idx, bbuf.array()) - // Remaining Key schema and projection definitions used by the Avro Serializers - // and Deserializers - private val remainingKeySchema = StructType(keySchema.drop(numColsPrefixKey)) - private lazy val remainingKeyAvroType = SchemaConverters.toAvroType(remainingKeySchema) - private val remainingKeyProj = UnsafeProjection.create(remainingKeySchema) + case LongType => + val longVal = value.asInstanceOf[Long] + val signCol = if (longVal < 0) { + negativeValMarker + } else { + positiveValMarker + } + bbuf.put(signCol) + bbuf.putLong(longVal) + writer.write(idx, bbuf.array()) - // This is quite simple to do - just bind sequentially, as we don't change the order. - private val restoreKeyProjection: UnsafeProjection = UnsafeProjection.create(keySchema) + case FloatType => + val floatVal = value.asInstanceOf[Float] + val rawBits = floatToRawIntBits(floatVal) + // perform sign comparison using bit manipulation to ensure NaN values are handled + // correctly + if ((rawBits & floatSignBitMask) != 0) { + // for negative values, we need to flip all the bits to ensure correct ordering + val updatedVal = rawBits ^ floatFlipBitMask + bbuf.put(negativeValMarker) + // convert the bits back to float + bbuf.putFloat(intBitsToFloat(updatedVal)) + } else { + bbuf.put(positiveValMarker) + bbuf.putFloat(floatVal) + } + writer.write(idx, bbuf.array()) - // Reusable objects - private val joinedRowOnKey = new JoinedRow() + case DoubleType => + val doubleVal = value.asInstanceOf[Double] + val rawBits = doubleToRawLongBits(doubleVal) + // perform sign comparison using bit manipulation to ensure NaN values are handled + // correctly + if ((rawBits & doubleSignBitMask) != 0) { + // for negative values, we need to flip all the bits to ensure correct ordering + val updatedVal = rawBits ^ doubleFlipBitMask + bbuf.put(negativeValMarker) + // convert the bits back to double + bbuf.putDouble(longBitsToDouble(updatedVal)) + } else { + bbuf.put(positiveValMarker) + bbuf.putDouble(doubleVal) + } + writer.write(idx, bbuf.array()) + } + } + } + encodeUnsafeRow(writer.getRow()) + } - override def encodeKey(row: UnsafeRow): Array[Byte] = { - val (prefixKeyEncoded, remainingEncoded) = if (usingAvroEncoding) { - ( - encodeUnsafeRowToAvro( - extractPrefixKey(row), - avroEnc.get.keySerializer, - prefixKeyAvroType, - out - ), - encodeUnsafeRowToAvro( - remainingKeyProjection(row), - avroEnc.get.suffixKeySerializer.get, - remainingKeyAvroType, - out - ) - ) - } else { - (encodeUnsafeRow(extractPrefixKey(row)), encodeUnsafeRow(remainingKeyProjection(row))) + override def encodeValue(row: UnsafeRow): Array[Byte] = encodeUnsafeRow(row) + + override def decodeKey(bytes: Array[Byte]): UnsafeRow = { + keyStateEncoderSpec match { + case NoPrefixKeyStateEncoderSpec(_) => + decodeToUnsafeRow(bytes, reusedKeyRow) + case PrefixKeyScanStateEncoderSpec(_, numColsPrefixKey) => + decodeToUnsafeRow(bytes, numFields = numColsPrefixKey) + case _ => throw unsupportedOperationForKeyStateEncoder("decodeKey") } - val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( - prefixKeyEncoded.length + remainingEncoded.length + 4 - ) + } - Platform.putInt(encodedBytes, startingOffset, prefixKeyEncoded.length) - Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, startingOffset + 4, prefixKeyEncoded.length) - // NOTE: We don't put the length of remainingEncoded as we can calculate later - // on deserialization. - Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, startingOffset + 4 + prefixKeyEncoded.length, - remainingEncoded.length) - - encodedBytes + override def decodeRemainingKey(bytes: Array[Byte]): UnsafeRow = { + keyStateEncoderSpec match { + case PrefixKeyScanStateEncoderSpec(_, numColsPrefixKey) => + decodeToUnsafeRow(bytes, numFields = numColsPrefixKey) + case RangeKeyScanStateEncoderSpec(_, orderingOrdinals) => + decodeToUnsafeRow(bytes, keySchema.length - orderingOrdinals.length) + case _ => throw unsupportedOperationForKeyStateEncoder("decodeRemainingKey") + } } - override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { - val prefixKeyEncodedLen = Platform.getInt(keyBytes, decodeKeyStartOffset) - val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) - Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4, - prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) + override def decodePrefixKeyForRangeScan(bytes: Array[Byte]): UnsafeRow = { + assert(keyStateEncoderSpec.isInstanceOf[RangeKeyScanStateEncoderSpec]) + val rsk = keyStateEncoderSpec.asInstanceOf[RangeKeyScanStateEncoderSpec] + val writer = new UnsafeRowWriter(rsk.orderingOrdinals.length) + val rangeScanKeyFieldsWithOrdinal = rsk.orderingOrdinals.map { ordinal => + val field = rsk.keySchema(ordinal) + (field, ordinal) + } + writer.resetRowWriter() + val row = decodeToUnsafeRow(bytes, numFields = rsk.orderingOrdinals.length) + rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => + val field = fieldWithOrdinal._1 - // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes - val remainingKeyEncodedLen = keyBytes.length - 4 - prefixKeyEncodedLen - - offsetForColFamilyPrefix + val value = row.getBinary(idx) + val bbuf = ByteBuffer.wrap(value.asInstanceOf[Array[Byte]]) + bbuf.order(ByteOrder.BIG_ENDIAN) + val isNullOrSignCol = bbuf.get() + if (isNullOrSignCol == nullValMarker) { + // set the column to null and skip reading the next byte(s) + writer.setNullAt(idx) + } else { + field.dataType match { + case BooleanType => + case ByteType => + writer.write(idx, bbuf.get) - val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) - Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4 + prefixKeyEncodedLen, - remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) + case ShortType => + writer.write(idx, bbuf.getShort) - val (prefixKeyDecoded, remainingKeyDecoded) = if (usingAvroEncoding) { - ( - decodeFromAvroToUnsafeRow( - prefixKeyEncoded, - avroEnc.get.keyDeserializer, - prefixKeyAvroType, - prefixKeyProj - ), - decodeFromAvroToUnsafeRow( - remainingKeyEncoded, - avroEnc.get.suffixKeyDeserializer.get, - remainingKeyAvroType, - remainingKeyProj - ) - ) - } else { - (decodeToUnsafeRow(prefixKeyEncoded, numFields = numColsPrefixKey), - decodeToUnsafeRow(remainingKeyEncoded, numFields = keySchema.length - numColsPrefixKey)) - } + case IntegerType => + writer.write(idx, bbuf.getInt) - restoreKeyProjection(joinedRowOnKey.withLeft(prefixKeyDecoded).withRight(remainingKeyDecoded)) - } + case LongType => + writer.write(idx, bbuf.getLong) - private def extractPrefixKey(key: UnsafeRow): UnsafeRow = { - prefixKeyProjection(key) - } + case FloatType => + if (isNullOrSignCol == negativeValMarker) { + // if the number is negative, get the raw binary bits for the float + // and flip the bits back + val updatedVal = floatToRawIntBits(bbuf.getFloat) ^ floatFlipBitMask + writer.write(idx, intBitsToFloat(updatedVal)) + } else { + writer.write(idx, bbuf.getFloat) + } - override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { - val prefixKeyEncoded = if (usingAvroEncoding) { - encodeUnsafeRowToAvro(prefixKey, avroEnc.get.keySerializer, prefixKeyAvroType, out) - } else { - encodeUnsafeRow(prefixKey) + case DoubleType => + if (isNullOrSignCol == negativeValMarker) { + // if the number is negative, get the raw binary bits for the double + // and flip the bits back + val updatedVal = doubleToRawLongBits(bbuf.getDouble) ^ doubleFlipBitMask + writer.write(idx, longBitsToDouble(updatedVal)) + } else { + writer.write(idx, bbuf.getDouble) + } + } + } } - val (prefix, startingOffset) = encodeColumnFamilyPrefix( - prefixKeyEncoded.length + 4 - ) - - Platform.putInt(prefix, startingOffset, prefixKeyEncoded.length) - Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefix, - startingOffset + 4, prefixKeyEncoded.length) - prefix + writer.getRow() } - override def supportPrefixKeyScan: Boolean = true + override def decodeValue(bytes: Array[Byte]): UnsafeRow = decodeToUnsafeRow(bytes, reusedValueRow) } -/** - * RocksDB Key Encoder for UnsafeRow that supports range scan for fixed size fields - * - * To encode a row for range scan, we first project the orderingOrdinals from the oridinal - * UnsafeRow into another UnsafeRow; we then rewrite that new UnsafeRow's fields in BIG_ENDIAN - * to allow for scanning keys in sorted order using the byte-wise comparison method that - * RocksDB uses. - * - * Then, for the rest of the fields, we project those into another UnsafeRow. - * We then effectively join these two UnsafeRows together, and finally take those bytes - * to get the resulting row. - * - * We cannot support variable sized fields in the range scan because the UnsafeRow format - * stores variable sized fields as offset and length pointers to the actual values, - * thereby changing the required ordering. - * - * Note that we also support "null" values being passed for these fixed size fields. We prepend - * a single byte to indicate whether the column value is null or not. We cannot change the - * nullability on the UnsafeRow itself as the expected ordering would change if non-first - * columns are marked as null. If the first col is null, those entries will appear last in - * the iterator. If non-first columns are null, ordering based on the previous columns will - * still be honored. For rows with null column values, ordering for subsequent columns - * will also be maintained within those set of rows. We use the same byte to also encode whether - * the value is negative or not. For negative float/double values, we flip all the bits to ensure - * the right lexicographical ordering. For the rationale around this, please check the link - * here: https://en.wikipedia.org/wiki/IEEE_754#Design_rationale - * - * @param keySchema - schema of the key to be encoded - * @param orderingOrdinals - the ordinals for which the range scan is constructed - * @param useColumnFamilies - if column family is enabled for this encoder - * @param avroEnc - if Avro encoding is specified for this StateEncoder, this encoder will - * be defined - */ -class RangeKeyScanStateEncoder( - keySchema: StructType, - orderingOrdinals: Seq[Int], - useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None, - avroEnc: Option[AvroEncoder] = None) - extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { +class AvroStateEncoder( + keyStateEncoderSpec: KeyStateEncoderSpec, + valueSchema: StructType) extends RocksDBDataEncoder(keyStateEncoderSpec, valueSchema) + with Logging { - import RocksDBStateEncoder._ + private val avroEncoder = createAvroEnc(keyStateEncoderSpec, valueSchema) + // Avro schema used by the avro encoders + private lazy val keyAvroType: Schema = SchemaConverters.toAvroType(keySchema) + private lazy val keyProj = UnsafeProjection.create(keySchema) - private val rangeScanKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { - orderingOrdinals.map { ordinal => - val field = keySchema(ordinal) - (field, ordinal) - } - } + private lazy val valueAvroType: Schema = SchemaConverters.toAvroType(valueSchema) + private lazy val valueProj = UnsafeProjection.create(valueSchema) - private def isFixedSize(dataType: DataType): Boolean = dataType match { - case _: ByteType | _: BooleanType | _: ShortType | _: IntegerType | _: LongType | - _: FloatType | _: DoubleType => true - case _ => false + // Prefix Key schema and projection definitions used by the Avro Serializers + // and Deserializers + private lazy val prefixKeySchema = keyStateEncoderSpec match { + case PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) => + StructType(keySchema.take (numColsPrefixKey)) + case _ => throw unsupportedOperationForKeyStateEncoder("prefixKeySchema") } - - // verify that only fixed sized columns are used for ordering - rangeScanKeyFieldsWithOrdinal.foreach { case (field, ordinal) => - if (!isFixedSize(field.dataType)) { - // NullType is technically fixed size, but not supported for ordering - if (field.dataType == NullType) { - throw StateStoreErrors.nullTypeOrderingColsNotSupported(field.name, ordinal.toString) - } else { - throw StateStoreErrors.variableSizeOrderingColsNotSupported(field.name, ordinal.toString) + private lazy val prefixKeyAvroType = SchemaConverters.toAvroType(prefixKeySchema) + private lazy val prefixKeyProj = UnsafeProjection.create(prefixKeySchema) + + // Range Key schema nd projection definitions used by the Avro Serializers and + // Deserializers + private lazy val rangeScanKeyFieldsWithOrdinal = keyStateEncoderSpec match { + case RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals) => + orderingOrdinals.map { ordinal => + val field = keySchema(ordinal) + (field, ordinal) } - } + case _ => + throw unsupportedOperationForKeyStateEncoder("rangeScanKey") } - private val remainingKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { - 0.to(keySchema.length - 1).diff(orderingOrdinals).map { ordinal => - val field = keySchema(ordinal) - (field, ordinal) - } - } + private lazy val rangeScanAvroSchema = StateStoreColumnFamilySchemaUtils.convertForRangeScan( + StructType(rangeScanKeyFieldsWithOrdinal.map(_._1).toArray)) - private val rangeScanKeyProjection: UnsafeProjection = { - val refs = rangeScanKeyFieldsWithOrdinal.map(x => - BoundReference(x._2, x._1.dataType, x._1.nullable)) - UnsafeProjection.create(refs) - } + private lazy val rangeScanAvroType = SchemaConverters.toAvroType(rangeScanAvroSchema) - private val remainingKeyProjection: UnsafeProjection = { - val refs = remainingKeyFieldsWithOrdinal.map(x => - BoundReference(x._2, x._1.dataType, x._1.nullable)) - UnsafeProjection.create(refs) + private lazy val rangeScanAvroProjection = UnsafeProjection.create(rangeScanAvroSchema) + + // Existing remainder key schema definitions + // Remaining Key schema and projection definitions used by the Avro Serializers + // and Deserializers + private lazy val remainingKeySchema = keyStateEncoderSpec match { + case PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) => + StructType(keySchema.drop(numColsPrefixKey)) + case RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals) => + StructType(0.until(keySchema.length).diff(orderingOrdinals).map(keySchema(_))) + case _ => throw unsupportedOperationForKeyStateEncoder("remainingKeySchema") } - // The original schema that we might get could be: - // [foo, bar, baz, buzz] - // We might order by bar and buzz, leading to: - // [bar, buzz, foo, baz] - // We need to create a projection that sends, for example, the buzz at index 1 to index - // 3. Thus, for every record in the original schema, we compute where it would be in - // the joined row and created a projection based on that. - private val restoreKeyProjection: UnsafeProjection = { - val refs = keySchema.zipWithIndex.map { case (field, originalOrdinal) => - val ordinalInJoinedRow = if (orderingOrdinals.contains(originalOrdinal)) { - orderingOrdinals.indexOf(originalOrdinal) - } else { - orderingOrdinals.length + - remainingKeyFieldsWithOrdinal.indexWhere(_._2 == originalOrdinal) - } + private lazy val remainingKeyAvroType = SchemaConverters.toAvroType(remainingKeySchema) - BoundReference(ordinalInJoinedRow, field.dataType, field.nullable) - } - UnsafeProjection.create(refs) + private lazy val remainingKeyAvroProjection = UnsafeProjection.create(remainingKeySchema) + + private def getAvroSerializer(schema: StructType): AvroSerializer = { + val avroType = SchemaConverters.toAvroType(schema) + new AvroSerializer(schema, avroType, nullable = false) } - private val rangeScanAvroSchema = StateStoreColumnFamilySchemaUtils.convertForRangeScan( - StructType(rangeScanKeyFieldsWithOrdinal.map(_._1).toArray)) + private def getAvroDeserializer(schema: StructType): AvroDeserializer = { + val avroType = SchemaConverters.toAvroType(schema) + val avroOptions = AvroOptions(Map.empty) + new AvroDeserializer(avroType, schema, + avroOptions.datetimeRebaseModeInRead, avroOptions.useStableIdForUnionType, + avroOptions.stableIdPrefixForUnionType, avroOptions.recursiveFieldMaxDepth) + } - private lazy val rangeScanAvroType = SchemaConverters.toAvroType(rangeScanAvroSchema) + /** + * Creates an AvroEncoder that handles both key and value serialization/deserialization. + * This method sets up the complete encoding infrastructure needed for state store operations. + * + * The encoder handles different key encoding specifications: + * - NoPrefixKeyStateEncoderSpec: Simple key encoding without prefix + * - PrefixKeyScanStateEncoderSpec: Keys with prefix for efficient scanning + * - RangeKeyScanStateEncoderSpec: Keys with ordering requirements for range scans + * + * For prefix scan cases, it also creates separate encoders for the suffix portion of keys. + * + * @param keyStateEncoderSpec Specification for how to encode keys + * @param valueSchema Schema for the values to be encoded + * @return An AvroEncoder containing all necessary serializers and deserializers + */ + private def createAvroEnc( + keyStateEncoderSpec: KeyStateEncoderSpec, + valueSchema: StructType): AvroEncoder = { + val valueSerializer = getAvroSerializer(valueSchema) + val valueDeserializer = getAvroDeserializer(valueSchema) + + // Get key schema based on encoder spec type + val keySchema = keyStateEncoderSpec match { + case NoPrefixKeyStateEncoderSpec(schema) => + schema + case PrefixKeyScanStateEncoderSpec(schema, numColsPrefixKey) => + StructType(schema.take(numColsPrefixKey)) + case RangeKeyScanStateEncoderSpec(schema, orderingOrdinals) => + val remainingSchema = { + 0.until(schema.length).diff(orderingOrdinals).map { ordinal => + schema(ordinal) + } + } + StructType(remainingSchema) + } - private val rangeScanAvroProjection = UnsafeProjection.create(rangeScanAvroSchema) + // Handle suffix key schema for prefix scan case + val suffixKeySchema = keyStateEncoderSpec match { + case PrefixKeyScanStateEncoderSpec(schema, numColsPrefixKey) => + Some(StructType(schema.drop(numColsPrefixKey))) + case _ => + None + } - // Existing remainder key schema stuff - private val remainingKeySchema = StructType( - 0.to(keySchema.length - 1).diff(orderingOrdinals).map(keySchema(_)) - ) + val keySerializer = getAvroSerializer(keySchema) + val keyDeserializer = getAvroDeserializer(keySchema) + + // Create the AvroEncoder with all components + AvroEncoder( + keySerializer, + keyDeserializer, + valueSerializer, + valueDeserializer, + suffixKeySchema.map(getAvroSerializer), + suffixKeySchema.map(getAvroDeserializer) + ) + } - private lazy val remainingKeyAvroType = SchemaConverters.toAvroType(remainingKeySchema) + /** + * This method takes an UnsafeRow, and serializes to a byte array using Avro encoding. + */ + def encodeUnsafeRowToAvro( + row: UnsafeRow, + avroSerializer: AvroSerializer, + valueAvroType: Schema, + out: ByteArrayOutputStream): Array[Byte] = { + // InternalRow -> Avro.GenericDataRecord + val avroData = + avroSerializer.serialize(row) + out.reset() + val encoder = EncoderFactory.get().directBinaryEncoder(out, null) + val writer = new GenericDatumWriter[Any]( + valueAvroType) // Defining Avro writer for this struct type + writer.write(avroData, encoder) // Avro.GenericDataRecord -> byte array + encoder.flush() + out.toByteArray + } - private val remainingKeyAvroProjection = UnsafeProjection.create(remainingKeySchema) + /** + * This method takes a byte array written using Avro encoding, and + * deserializes to an UnsafeRow using the Avro deserializer + */ + def decodeFromAvroToUnsafeRow( + valueBytes: Array[Byte], + avroDeserializer: AvroDeserializer, + valueAvroType: Schema, + valueProj: UnsafeProjection): UnsafeRow = { + if (valueBytes != null) { + val reader = new GenericDatumReader[Any](valueAvroType) + val decoder = DecoderFactory.get().binaryDecoder( + valueBytes, 0, valueBytes.length, null) + // bytes -> Avro.GenericDataRecord + val genericData = reader.read(null, decoder) + // Avro.GenericDataRecord -> InternalRow + val internalRow = avroDeserializer.deserialize( + genericData).orNull.asInstanceOf[InternalRow] + // InternalRow -> UnsafeRow + valueProj.apply(internalRow) + } else { + null + } + } - // Reusable objects - private val joinedRowOnKey = new JoinedRow() + private val out = new ByteArrayOutputStream - private def extractPrefixKey(key: UnsafeRow): UnsafeRow = { - rangeScanKeyProjection(key) + override def encodeKey(row: UnsafeRow): Array[Byte] = { + keyStateEncoderSpec match { + case NoPrefixKeyStateEncoderSpec(_) => + encodeUnsafeRowToAvro(row, avroEncoder.keySerializer, keyAvroType, out) + case PrefixKeyScanStateEncoderSpec(_, _) => + encodeUnsafeRowToAvro(row, avroEncoder.keySerializer, prefixKeyAvroType, out) + case _ => throw unsupportedOperationForKeyStateEncoder("encodeKey") + } } - // bit masks used for checking sign or flipping all bits for negative float/double values - private val floatFlipBitMask = 0xFFFFFFFF - private val floatSignBitMask = 0x80000000 - - private val doubleFlipBitMask = 0xFFFFFFFFFFFFFFFFL - private val doubleSignBitMask = 0x8000000000000000L - - // Byte markers used to identify whether the value is null, negative or positive - // To ensure sorted ordering, we use the lowest byte value for negative numbers followed by - // positive numbers and then null values. - private val negativeValMarker: Byte = 0x00.toByte - private val positiveValMarker: Byte = 0x01.toByte - private val nullValMarker: Byte = 0x02.toByte - - // Rewrite the unsafe row by replacing fixed size fields with BIG_ENDIAN encoding - // using byte arrays. - // To handle "null" values, we prepend a byte to the byte array indicating whether the value - // is null or not. If the value is null, we write the null byte followed by zero bytes. - // If the value is not null, we write the null byte followed by the value. - // Note that setting null for the index on the unsafeRow is not feasible as it would change - // the sorting order on iteration. - // Also note that the same byte is used to indicate whether the value is negative or not. - private def encodePrefixKeyForRangeScan(row: UnsafeRow): UnsafeRow = { - val writer = new UnsafeRowWriter(orderingOrdinals.length) - writer.resetRowWriter() - rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => - val field = fieldWithOrdinal._1 - val value = row.get(idx, field.dataType) - // Note that we cannot allocate a smaller buffer here even if the value is null - // because the effective byte array is considered variable size and needs to have - // the same size across all rows for the ordering to work as expected. - val bbuf = ByteBuffer.allocate(field.dataType.defaultSize + 1) - bbuf.order(ByteOrder.BIG_ENDIAN) - if (value == null) { - bbuf.put(nullValMarker) - writer.write(idx, bbuf.array()) - } else { - field.dataType match { - case BooleanType => - case ByteType => - val byteVal = value.asInstanceOf[Byte] - val signCol = if (byteVal < 0) { - negativeValMarker - } else { - positiveValMarker - } - bbuf.put(signCol) - bbuf.put(byteVal) - writer.write(idx, bbuf.array()) - - case ShortType => - val shortVal = value.asInstanceOf[Short] - val signCol = if (shortVal < 0) { - negativeValMarker - } else { - positiveValMarker - } - bbuf.put(signCol) - bbuf.putShort(shortVal) - writer.write(idx, bbuf.array()) - - case IntegerType => - val intVal = value.asInstanceOf[Int] - val signCol = if (intVal < 0) { - negativeValMarker - } else { - positiveValMarker - } - bbuf.put(signCol) - bbuf.putInt(intVal) - writer.write(idx, bbuf.array()) - - case LongType => - val longVal = value.asInstanceOf[Long] - val signCol = if (longVal < 0) { - negativeValMarker - } else { - positiveValMarker - } - bbuf.put(signCol) - bbuf.putLong(longVal) - writer.write(idx, bbuf.array()) - - case FloatType => - val floatVal = value.asInstanceOf[Float] - val rawBits = floatToRawIntBits(floatVal) - // perform sign comparison using bit manipulation to ensure NaN values are handled - // correctly - if ((rawBits & floatSignBitMask) != 0) { - // for negative values, we need to flip all the bits to ensure correct ordering - val updatedVal = rawBits ^ floatFlipBitMask - bbuf.put(negativeValMarker) - // convert the bits back to float - bbuf.putFloat(intBitsToFloat(updatedVal)) - } else { - bbuf.put(positiveValMarker) - bbuf.putFloat(floatVal) - } - writer.write(idx, bbuf.array()) - - case DoubleType => - val doubleVal = value.asInstanceOf[Double] - val rawBits = doubleToRawLongBits(doubleVal) - // perform sign comparison using bit manipulation to ensure NaN values are handled - // correctly - if ((rawBits & doubleSignBitMask) != 0) { - // for negative values, we need to flip all the bits to ensure correct ordering - val updatedVal = rawBits ^ doubleFlipBitMask - bbuf.put(negativeValMarker) - // convert the bits back to double - bbuf.putDouble(longBitsToDouble(updatedVal)) - } else { - bbuf.put(positiveValMarker) - bbuf.putDouble(doubleVal) - } - writer.write(idx, bbuf.array()) - } - } - } - writer.getRow() - } - - // Rewrite the unsafe row by converting back from BIG_ENDIAN byte arrays to the - // original data types. - // For decode, we extract the byte array from the UnsafeRow, and then read the first byte - // to determine if the value is null or not. If the value is null, we set the ordinal on - // the UnsafeRow to null. If the value is not null, we read the rest of the bytes to get the - // actual value. - // For negative float/double values, we need to flip all the bits back to get the original value. - private def decodePrefixKeyForRangeScan(row: UnsafeRow): UnsafeRow = { - val writer = new UnsafeRowWriter(orderingOrdinals.length) - writer.resetRowWriter() - rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => - val field = fieldWithOrdinal._1 - - val value = row.getBinary(idx) - val bbuf = ByteBuffer.wrap(value.asInstanceOf[Array[Byte]]) - bbuf.order(ByteOrder.BIG_ENDIAN) - val isNullOrSignCol = bbuf.get() - if (isNullOrSignCol == nullValMarker) { - // set the column to null and skip reading the next byte(s) - writer.setNullAt(idx) - } else { - field.dataType match { - case BooleanType => - case ByteType => - writer.write(idx, bbuf.get) - - case ShortType => - writer.write(idx, bbuf.getShort) - - case IntegerType => - writer.write(idx, bbuf.getInt) - - case LongType => - writer.write(idx, bbuf.getLong) - - case FloatType => - if (isNullOrSignCol == negativeValMarker) { - // if the number is negative, get the raw binary bits for the float - // and flip the bits back - val updatedVal = floatToRawIntBits(bbuf.getFloat) ^ floatFlipBitMask - writer.write(idx, intBitsToFloat(updatedVal)) - } else { - writer.write(idx, bbuf.getFloat) - } - - case DoubleType => - if (isNullOrSignCol == negativeValMarker) { - // if the number is negative, get the raw binary bits for the double - // and flip the bits back - val updatedVal = doubleToRawLongBits(bbuf.getDouble) ^ doubleFlipBitMask - writer.write(idx, longBitsToDouble(updatedVal)) - } else { - writer.write(idx, bbuf.getDouble) - } - } - } - } - writer.getRow() - } + override def encodeRemainingKey(row: UnsafeRow): Array[Byte] = { + keyStateEncoderSpec match { + case PrefixKeyScanStateEncoderSpec(_, _) => + encodeUnsafeRowToAvro(row, avroEncoder.suffixKeySerializer.get, remainingKeyAvroType, out) + case RangeKeyScanStateEncoderSpec(_, _) => + encodeUnsafeRowToAvro(row, avroEncoder.keySerializer, remainingKeyAvroType, out) + case _ => throw unsupportedOperationForKeyStateEncoder("encodeRemainingKey") + } + } /** * Encodes an UnsafeRow into an Avro-compatible byte array format for range scan operations. @@ -704,10 +642,8 @@ class RangeKeyScanStateEncoder( * @throws UnsupportedOperationException if a field's data type is not supported for range * scan encoding */ - def encodePrefixKeyForRangeScan( - row: UnsafeRow, - avroType: Schema): Array[Byte] = { - val record = new GenericData.Record(avroType) + override def encodePrefixKeyForRangeScan(row: UnsafeRow): Array[Byte] = { + val record = new GenericData.Record(rangeScanAvroType) var fieldIdx = 0 rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => val field = fieldWithOrdinal._1 @@ -810,146 +746,486 @@ class RangeKeyScanStateEncoder( } record.put(fieldIdx + 1, ByteBuffer.wrap(valueBuffer.array())) - case _ => throw new UnsupportedOperationException( - s"Range scan encoding not supported for data type: ${field.dataType}") - } - } - fieldIdx += 2 - } + case _ => throw new UnsupportedOperationException( + s"Range scan encoding not supported for data type: ${field.dataType}") + } + } + fieldIdx += 2 + } + + out.reset() + val writer = new GenericDatumWriter[GenericRecord](rangeScanAvroType) + val encoder = EncoderFactory.get().binaryEncoder(out, null) + writer.write(record, encoder) + encoder.flush() + out.toByteArray + } + + override def encodeValue(row: UnsafeRow): Array[Byte] = + encodeUnsafeRowToAvro(row, avroEncoder.valueSerializer, valueAvroType, out) + + override def decodeKey(bytes: Array[Byte]): UnsafeRow = { + keyStateEncoderSpec match { + case NoPrefixKeyStateEncoderSpec(_) => + decodeFromAvroToUnsafeRow(bytes, avroEncoder.keyDeserializer, keyAvroType, keyProj) + case PrefixKeyScanStateEncoderSpec(_, _) => + decodeFromAvroToUnsafeRow( + bytes, avroEncoder.keyDeserializer, prefixKeyAvroType, prefixKeyProj) + case _ => throw unsupportedOperationForKeyStateEncoder("decodeKey") + } + } + + + override def decodeRemainingKey(bytes: Array[Byte]): UnsafeRow = { + keyStateEncoderSpec match { + case PrefixKeyScanStateEncoderSpec(_, _) => + decodeFromAvroToUnsafeRow(bytes, + avroEncoder.suffixKeyDeserializer.get, remainingKeyAvroType, remainingKeyAvroProjection) + case RangeKeyScanStateEncoderSpec(_, _) => + decodeFromAvroToUnsafeRow( + bytes, avroEncoder.keyDeserializer, remainingKeyAvroType, remainingKeyAvroProjection) + case _ => throw unsupportedOperationForKeyStateEncoder("decodeRemainingKey") + } + } + + /** + * Decodes an Avro-encoded byte array back into an UnsafeRow for range scan operations. + * + * This method reverses the encoding process performed by encodePrefixKeyForRangeScan: + * - Reads the marker byte to determine null status or sign + * - Reconstructs the original values from big-endian format + * - Handles special cases for floating point numbers by reversing bit manipulations + * + * The decoding process preserves the original data types and values, including: + * - Null values marked by nullValMarker + * - Sign information for numeric types + * - Proper restoration of negative floating point values + * + * @param bytes The Avro-encoded byte array to decode + * @param avroType The Avro schema defining the structure for decoding + * @return UnsafeRow containing the decoded data + * @throws UnsupportedOperationException if a field's data type is not supported for range + * scan decoding + */ + override def decodePrefixKeyForRangeScan(bytes: Array[Byte]): UnsafeRow = { + val reader = new GenericDatumReader[GenericRecord](rangeScanAvroType) + val decoder = DecoderFactory.get().binaryDecoder(bytes, 0, bytes.length, null) + val record = reader.read(null, decoder) + + val rowWriter = new UnsafeRowWriter(rangeScanKeyFieldsWithOrdinal.length) + rowWriter.resetRowWriter() + + var fieldIdx = 0 + rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => + val field = fieldWithOrdinal._1 + + val markerBytes = record.get(fieldIdx).asInstanceOf[ByteBuffer].array() + val markerBuf = ByteBuffer.wrap(markerBytes) + markerBuf.order(ByteOrder.BIG_ENDIAN) + val marker = markerBuf.get() + + if (marker == nullValMarker) { + rowWriter.setNullAt(idx) + } else { + field.dataType match { + case BooleanType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + rowWriter.write(idx, bytes(0) == 1) + + case ByteType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.get()) + + case ShortType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.getShort()) + + case IntegerType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.getInt()) + + case LongType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + rowWriter.write(idx, valueBuf.getLong()) + + case FloatType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + if (marker == negativeValMarker) { + val floatVal = valueBuf.getFloat + val updatedVal = floatToRawIntBits(floatVal) ^ floatFlipBitMask + rowWriter.write(idx, intBitsToFloat(updatedVal)) + } else { + rowWriter.write(idx, valueBuf.getFloat()) + } + + case DoubleType => + val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() + val valueBuf = ByteBuffer.wrap(bytes) + valueBuf.order(ByteOrder.BIG_ENDIAN) + if (marker == negativeValMarker) { + val doubleVal = valueBuf.getDouble + val updatedVal = doubleToRawLongBits(doubleVal) ^ doubleFlipBitMask + rowWriter.write(idx, longBitsToDouble(updatedVal)) + } else { + rowWriter.write(idx, valueBuf.getDouble()) + } + + case _ => throw new UnsupportedOperationException( + s"Range scan decoding not supported for data type: ${field.dataType}") + } + } + fieldIdx += 2 + } + + rowWriter.getRow() + } + + override def decodeValue(bytes: Array[Byte]): UnsafeRow = + decodeFromAvroToUnsafeRow( + bytes, avroEncoder.valueDeserializer, valueAvroType, valueProj) +} + +abstract class RocksDBKeyStateEncoderBase( + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short] = None) extends RocksDBKeyStateEncoder { + def offsetForColFamilyPrefix: Int = + if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + + val out = new ByteArrayOutputStream + + /** + * Get Byte Array for the virtual column family id that is used as prefix for + * key state rows. + */ + override def getColumnFamilyIdBytes(): Array[Byte] = { + assert(useColumnFamilies, "Cannot return virtual Column Family Id Bytes" + + " because multiple Column is not supported for this encoder") + val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId.get) + encodedBytes + } + + /** + * Encode and put column family Id as a prefix to a pre-allocated byte array. + * + * @param numBytes - size of byte array to be created for storing key row (without + * column family prefix) + * @return Array[Byte] for an array byte to put encoded key bytes + * Int for a starting offset to put the encoded key bytes + */ + protected def encodeColumnFamilyPrefix(numBytes: Int): (Array[Byte], Int) = { + val encodedBytes = new Array[Byte](numBytes + offsetForColFamilyPrefix) + var offset = Platform.BYTE_ARRAY_OFFSET + if (useColumnFamilies) { + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId.get) + offset = Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix + } + (encodedBytes, offset) + } + + /** + * Get starting offset for decoding an encoded key byte array. + */ + protected def decodeKeyStartOffset: Int = { + if (useColumnFamilies) { + Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES + } else Platform.BYTE_ARRAY_OFFSET + } +} + +/** + * Factory object for creating state encoders used by RocksDB state store. + * + * The encoders created by this object handle serialization and deserialization of state data, + * supporting both key and value encoding with various access patterns + * (e.g., prefix scan, range scan). + */ +object RocksDBStateEncoder extends Logging { + + /** + * Creates a key encoder based on the specified encoding strategy and configuration. + * + * @param dataEncoder The underlying encoder that handles the actual data encoding/decoding + * @param keyStateEncoderSpec Specification defining the key encoding strategy + * (no prefix, prefix scan, or range scan) + * @param useColumnFamilies Whether to use RocksDB column families for storage + * @param virtualColFamilyId Optional column family identifier when column families are enabled + * @return A configured RocksDBKeyStateEncoder instance + */ + def getKeyEncoder( + dataEncoder: RocksDBDataEncoder, + keyStateEncoderSpec: KeyStateEncoderSpec, + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short] = None): RocksDBKeyStateEncoder = { + keyStateEncoderSpec.toEncoder(dataEncoder, useColumnFamilies, virtualColFamilyId) + } + + /** + * Creates a value encoder that supports either single or multiple values per key. + * + * @param dataEncoder The underlying encoder that handles the actual data encoding/decoding + * @param valueSchema Schema defining the structure of values to be encoded + * @param useMultipleValuesPerKey If true, creates an encoder that can handle multiple values + * per key; if false, creates an encoder for single values + * @return A configured RocksDBValueStateEncoder instance + */ + def getValueEncoder( + dataEncoder: RocksDBDataEncoder, + valueSchema: StructType, + useMultipleValuesPerKey: Boolean): RocksDBValueStateEncoder = { + if (useMultipleValuesPerKey) { + new MultiValuedStateEncoder(dataEncoder, valueSchema) + } else { + new SingleValueStateEncoder(dataEncoder, valueSchema) + } + } + + /** + * Encodes a virtual column family ID into a byte array suitable for RocksDB. + * + * This method creates a fixed-size byte array prefixed with the virtual column family ID, + * which is used to partition data within RocksDB. + * + * @param virtualColFamilyId The column family identifier to encode + * @return A byte array containing the encoded column family ID + */ + def getColumnFamilyIdBytes(virtualColFamilyId: Short): Array[Byte] = { + val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId) + encodedBytes + } +} + +/** + * RocksDB Key Encoder for UnsafeRow that supports prefix scan + * + * @param dataEncoder - the encoder that handles actual encoding/decoding of data + * @param keySchema - schema of the key to be encoded + * @param numColsPrefixKey - number of columns to be used for prefix key + * @param useColumnFamilies - if column family is enabled for this encoder + */ +class PrefixKeyScanStateEncoder( + dataEncoder: RocksDBDataEncoder, + keySchema: StructType, + numColsPrefixKey: Int, + useColumnFamilies: Boolean = false, + virtualColFamilyId: Option[Short] = None) + extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { + + private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { + keySchema.zipWithIndex.take(numColsPrefixKey) + } + + private val remainingKeyFieldsWithIdx: Seq[(StructField, Int)] = { + keySchema.zipWithIndex.drop(numColsPrefixKey) + } + + private val prefixKeyProjection: UnsafeProjection = { + val refs = prefixKeyFieldsWithIdx.map(x => BoundReference(x._2, x._1.dataType, x._1.nullable)) + UnsafeProjection.create(refs) + } + + private val remainingKeyProjection: UnsafeProjection = { + val refs = remainingKeyFieldsWithIdx.map(x => + BoundReference(x._2, x._1.dataType, x._1.nullable)) + UnsafeProjection.create(refs) + } + + // This is quite simple to do - just bind sequentially, as we don't change the order. + private val restoreKeyProjection: UnsafeProjection = UnsafeProjection.create(keySchema) + + // Reusable objects + private val joinedRowOnKey = new JoinedRow() + + override def encodeKey(row: UnsafeRow): Array[Byte] = { + val prefixKeyEncoded = dataEncoder.encodeKey(extractPrefixKey(row)) + val remainingEncoded = dataEncoder.encodeRemainingKey(remainingKeyProjection(row)) + + val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( + prefixKeyEncoded.length + remainingEncoded.length + 4 + ) + + Platform.putInt(encodedBytes, startingOffset, prefixKeyEncoded.length) + Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, + encodedBytes, startingOffset + 4, prefixKeyEncoded.length) + // NOTE: We don't put the length of remainingEncoded as we can calculate later + // on deserialization. + Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, + encodedBytes, startingOffset + 4 + prefixKeyEncoded.length, + remainingEncoded.length) + + encodedBytes + } + + override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { + val prefixKeyEncodedLen = Platform.getInt(keyBytes, decodeKeyStartOffset) + val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) + Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4, + prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) + + // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes + val remainingKeyEncodedLen = keyBytes.length - 4 - prefixKeyEncodedLen - + offsetForColFamilyPrefix - out.reset() - val writer = new GenericDatumWriter[GenericRecord](rangeScanAvroType) - val encoder = EncoderFactory.get().binaryEncoder(out, null) - writer.write(record, encoder) - encoder.flush() - out.toByteArray - } + val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) + Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4 + prefixKeyEncodedLen, + remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) - /** - * Decodes an Avro-encoded byte array back into an UnsafeRow for range scan operations. - * - * This method reverses the encoding process performed by encodePrefixKeyForRangeScan: - * - Reads the marker byte to determine null status or sign - * - Reconstructs the original values from big-endian format - * - Handles special cases for floating point numbers by reversing bit manipulations - * - * The decoding process preserves the original data types and values, including: - * - Null values marked by nullValMarker - * - Sign information for numeric types - * - Proper restoration of negative floating point values - * - * @param bytes The Avro-encoded byte array to decode - * @param avroType The Avro schema defining the structure for decoding - * @return UnsafeRow containing the decoded data - * @throws UnsupportedOperationException if a field's data type is not supported for range - * scan decoding - */ - def decodePrefixKeyForRangeScan( - bytes: Array[Byte], - avroType: Schema): UnsafeRow = { + val prefixKeyDecoded = dataEncoder.decodeKey( + prefixKeyEncoded) + val remainingKeyDecoded = dataEncoder.decodeRemainingKey(remainingKeyEncoded) - val reader = new GenericDatumReader[GenericRecord](avroType) - val decoder = DecoderFactory.get().binaryDecoder(bytes, 0, bytes.length, null) - val record = reader.read(null, decoder) + restoreKeyProjection(joinedRowOnKey.withLeft(prefixKeyDecoded).withRight(remainingKeyDecoded)) + } - val rowWriter = new UnsafeRowWriter(rangeScanKeyFieldsWithOrdinal.length) - rowWriter.resetRowWriter() + private def extractPrefixKey(key: UnsafeRow): UnsafeRow = { + prefixKeyProjection(key) + } - var fieldIdx = 0 - rangeScanKeyFieldsWithOrdinal.zipWithIndex.foreach { case (fieldWithOrdinal, idx) => - val field = fieldWithOrdinal._1 + override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { + val prefixKeyEncoded = dataEncoder.encodeKey(prefixKey) + val (prefix, startingOffset) = encodeColumnFamilyPrefix( + prefixKeyEncoded.length + 4 + ) - val markerBytes = record.get(fieldIdx).asInstanceOf[ByteBuffer].array() - val markerBuf = ByteBuffer.wrap(markerBytes) - markerBuf.order(ByteOrder.BIG_ENDIAN) - val marker = markerBuf.get() + Platform.putInt(prefix, startingOffset, prefixKeyEncoded.length) + Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefix, + startingOffset + 4, prefixKeyEncoded.length) + prefix + } - if (marker == nullValMarker) { - rowWriter.setNullAt(idx) - } else { - field.dataType match { - case BooleanType => - val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() - rowWriter.write(idx, bytes(0) == 1) + override def supportPrefixKeyScan: Boolean = true +} - case ByteType => - val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() - val valueBuf = ByteBuffer.wrap(bytes) - valueBuf.order(ByteOrder.BIG_ENDIAN) - rowWriter.write(idx, valueBuf.get()) +/** + * RocksDB Key Encoder for UnsafeRow that supports range scan for fixed size fields + * + * To encode a row for range scan, we first project the orderingOrdinals from the oridinal + * UnsafeRow into another UnsafeRow; we then rewrite that new UnsafeRow's fields in BIG_ENDIAN + * to allow for scanning keys in sorted order using the byte-wise comparison method that + * RocksDB uses. + * + * Then, for the rest of the fields, we project those into another UnsafeRow. + * We then effectively join these two UnsafeRows together, and finally take those bytes + * to get the resulting row. + * + * We cannot support variable sized fields in the range scan because the UnsafeRow format + * stores variable sized fields as offset and length pointers to the actual values, + * thereby changing the required ordering. + * + * Note that we also support "null" values being passed for these fixed size fields. We prepend + * a single byte to indicate whether the column value is null or not. We cannot change the + * nullability on the UnsafeRow itself as the expected ordering would change if non-first + * columns are marked as null. If the first col is null, those entries will appear last in + * the iterator. If non-first columns are null, ordering based on the previous columns will + * still be honored. For rows with null column values, ordering for subsequent columns + * will also be maintained within those set of rows. We use the same byte to also encode whether + * the value is negative or not. For negative float/double values, we flip all the bits to ensure + * the right lexicographical ordering. For the rationale around this, please check the link + * here: https://en.wikipedia.org/wiki/IEEE_754#Design_rationale + * + * @param dataEncoder - the encoder that handles the actual encoding/decoding of data + * @param keySchema - schema of the key to be encoded + * @param orderingOrdinals - the ordinals for which the range scan is constructed + * @param useColumnFamilies - if column family is enabled for this encoder + */ +class RangeKeyScanStateEncoder( + dataEncoder: RocksDBDataEncoder, + keySchema: StructType, + orderingOrdinals: Seq[Int], + useColumnFamilies: Boolean = false, + virtualColFamilyId: Option[Short] = None) + extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { - case ShortType => - val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() - val valueBuf = ByteBuffer.wrap(bytes) - valueBuf.order(ByteOrder.BIG_ENDIAN) - rowWriter.write(idx, valueBuf.getShort()) + private val rangeScanKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { + orderingOrdinals.map { ordinal => + val field = keySchema(ordinal) + (field, ordinal) + } + } - case IntegerType => - val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() - val valueBuf = ByteBuffer.wrap(bytes) - valueBuf.order(ByteOrder.BIG_ENDIAN) - rowWriter.write(idx, valueBuf.getInt()) + private def isFixedSize(dataType: DataType): Boolean = dataType match { + case _: ByteType | _: BooleanType | _: ShortType | _: IntegerType | _: LongType | + _: FloatType | _: DoubleType => true + case _ => false + } - case LongType => - val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() - val valueBuf = ByteBuffer.wrap(bytes) - valueBuf.order(ByteOrder.BIG_ENDIAN) - rowWriter.write(idx, valueBuf.getLong()) + // verify that only fixed sized columns are used for ordering + rangeScanKeyFieldsWithOrdinal.foreach { case (field, ordinal) => + if (!isFixedSize(field.dataType)) { + // NullType is technically fixed size, but not supported for ordering + if (field.dataType == NullType) { + throw StateStoreErrors.nullTypeOrderingColsNotSupported(field.name, ordinal.toString) + } else { + throw StateStoreErrors.variableSizeOrderingColsNotSupported(field.name, ordinal.toString) + } + } + } - case FloatType => - val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() - val valueBuf = ByteBuffer.wrap(bytes) - valueBuf.order(ByteOrder.BIG_ENDIAN) - if (marker == negativeValMarker) { - val floatVal = valueBuf.getFloat - val updatedVal = floatToRawIntBits(floatVal) ^ floatFlipBitMask - rowWriter.write(idx, intBitsToFloat(updatedVal)) - } else { - rowWriter.write(idx, valueBuf.getFloat()) - } + private val remainingKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { + 0.to(keySchema.length - 1).diff(orderingOrdinals).map { ordinal => + val field = keySchema(ordinal) + (field, ordinal) + } + } - case DoubleType => - val bytes = record.get(fieldIdx + 1).asInstanceOf[ByteBuffer].array() - val valueBuf = ByteBuffer.wrap(bytes) - valueBuf.order(ByteOrder.BIG_ENDIAN) - if (marker == negativeValMarker) { - val doubleVal = valueBuf.getDouble - val updatedVal = doubleToRawLongBits(doubleVal) ^ doubleFlipBitMask - rowWriter.write(idx, longBitsToDouble(updatedVal)) - } else { - rowWriter.write(idx, valueBuf.getDouble()) - } + private val rangeScanKeyProjection: UnsafeProjection = { + val refs = rangeScanKeyFieldsWithOrdinal.map(x => + BoundReference(x._2, x._1.dataType, x._1.nullable)) + UnsafeProjection.create(refs) + } - case _ => throw new UnsupportedOperationException( - s"Range scan decoding not supported for data type: ${field.dataType}") - } + private val remainingKeyProjection: UnsafeProjection = { + val refs = remainingKeyFieldsWithOrdinal.map(x => + BoundReference(x._2, x._1.dataType, x._1.nullable)) + UnsafeProjection.create(refs) + } + + // The original schema that we might get could be: + // [foo, bar, baz, buzz] + // We might order by bar and buzz, leading to: + // [bar, buzz, foo, baz] + // We need to create a projection that sends, for example, the buzz at index 1 to index + // 3. Thus, for every record in the original schema, we compute where it would be in + // the joined row and created a projection based on that. + private val restoreKeyProjection: UnsafeProjection = { + val refs = keySchema.zipWithIndex.map { case (field, originalOrdinal) => + val ordinalInJoinedRow = if (orderingOrdinals.contains(originalOrdinal)) { + orderingOrdinals.indexOf(originalOrdinal) + } else { + orderingOrdinals.length + + remainingKeyFieldsWithOrdinal.indexWhere(_._2 == originalOrdinal) } - fieldIdx += 2 + + BoundReference(ordinalInJoinedRow, field.dataType, field.nullable) } + UnsafeProjection.create(refs) + } - rowWriter.getRow() + // Reusable objects + private val joinedRowOnKey = new JoinedRow() + + private def extractPrefixKey(key: UnsafeRow): UnsafeRow = { + rangeScanKeyProjection(key) } override def encodeKey(row: UnsafeRow): Array[Byte] = { // This prefix key has the columns specified by orderingOrdinals val prefixKey = extractPrefixKey(row) - val rangeScanKeyEncoded = if (avroEnc.isDefined) { - encodePrefixKeyForRangeScan(prefixKey, rangeScanAvroType) - } else { - encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) - } + val rangeScanKeyEncoded = dataEncoder.encodePrefixKeyForRangeScan(prefixKey) val result = if (orderingOrdinals.length < keySchema.length) { - val remainingEncoded = if (avroEnc.isDefined) { - encodeUnsafeRowToAvro( - remainingKeyProjection(row), - avroEnc.get.keySerializer, - remainingKeyAvroType, - out - ) - } else { - encodeUnsafeRow(remainingKeyProjection(row)) - } + val remainingEncoded = dataEncoder.encodeRemainingKey(remainingKeyProjection(row)) val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( rangeScanKeyEncoded.length + remainingEncoded.length + 4 ) @@ -986,12 +1262,8 @@ class RangeKeyScanStateEncoder( Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4, prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) - val prefixKeyDecoded = if (avroEnc.isDefined) { - decodePrefixKeyForRangeScan(prefixKeyEncoded, rangeScanAvroType) - } else { - decodePrefixKeyForRangeScan(decodeToUnsafeRow(prefixKeyEncoded, - numFields = orderingOrdinals.length)) - } + val prefixKeyDecoded = dataEncoder.decodePrefixKeyForRangeScan( + prefixKeyEncoded) if (orderingOrdinals.length < keySchema.length) { // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes @@ -1003,14 +1275,7 @@ class RangeKeyScanStateEncoder( remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) - val remainingKeyDecoded = if (avroEnc.isDefined) { - decodeFromAvroToUnsafeRow(remainingKeyEncoded, - avroEnc.get.keyDeserializer, - remainingKeyAvroType, remainingKeyAvroProjection) - } else { - decodeToUnsafeRow(remainingKeyEncoded, - numFields = keySchema.length - orderingOrdinals.length) - } + val remainingKeyDecoded = dataEncoder.decodeRemainingKey(remainingKeyEncoded) val joined = joinedRowOnKey.withLeft(prefixKeyDecoded).withRight(remainingKeyDecoded) val restored = restoreKeyProjection(joined) @@ -1023,11 +1288,7 @@ class RangeKeyScanStateEncoder( } override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { - val rangeScanKeyEncoded = if (avroEnc.isDefined) { - encodePrefixKeyForRangeScan(prefixKey, rangeScanAvroType) - } else { - encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) - } + val rangeScanKeyEncoded = dataEncoder.encodePrefixKeyForRangeScan(prefixKey) val (prefix, startingOffset) = encodeColumnFamilyPrefix(rangeScanKeyEncoded.length + 4) Platform.putInt(prefix, startingOffset, rangeScanKeyEncoded.length) @@ -1046,36 +1307,23 @@ class RangeKeyScanStateEncoder( * It uses the first byte of the generated byte array to store the version the describes how the * row is encoded in the rest of the byte array. Currently, the default version is 0, * - * If the avroEnc is specified, we are using Avro encoding for this column family's keys * VERSION 0: [ VERSION (1 byte) | ROW (N bytes) ] * The bytes of a UnsafeRow is written unmodified to starting from offset 1 * (offset 0 is the version byte of value 0). That is, if the unsafe row has N bytes, * then the generated array byte will be N+1 bytes. */ class NoPrefixKeyStateEncoder( + dataEncoder: RocksDBDataEncoder, keySchema: StructType, useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None, - avroEnc: Option[AvroEncoder] = None) + virtualColFamilyId: Option[Short] = None) extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { - import RocksDBStateEncoder._ - - // Reusable objects - private val usingAvroEncoding = avroEnc.isDefined - private val keyRow = new UnsafeRow(keySchema.size) - private lazy val keyAvroType = SchemaConverters.toAvroType(keySchema) - private val keyProj = UnsafeProjection.create(keySchema) - override def encodeKey(row: UnsafeRow): Array[Byte] = { if (!useColumnFamilies) { - encodeUnsafeRow(row) + dataEncoder.encodeKey(row) } else { - // If avroEnc is defined, we know that we need to use Avro to - // encode this UnsafeRow to Avro bytes - val bytesToEncode = if (usingAvroEncoding) { - encodeUnsafeRowToAvro(row, avroEnc.get.keySerializer, keyAvroType, out) - } else row.getBytes + val bytesToEncode = dataEncoder.encodeKey(row) val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( bytesToEncode.length + STATE_ENCODING_NUM_VERSION_BYTES @@ -1098,26 +1346,23 @@ class NoPrefixKeyStateEncoder( override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { if (useColumnFamilies) { if (keyBytes != null) { - // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. - if (usingAvroEncoding) { - val dataLength = keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - - VIRTUAL_COL_FAMILY_PREFIX_BYTES - val avroBytes = new Array[Byte](dataLength) - Platform.copyMemory( - keyBytes, decodeKeyStartOffset + STATE_ENCODING_NUM_VERSION_BYTES, - avroBytes, Platform.BYTE_ARRAY_OFFSET, dataLength) - decodeFromAvroToUnsafeRow(avroBytes, avroEnc.get.keyDeserializer, keyAvroType, keyProj) - } else { - keyRow.pointTo( - keyBytes, - decodeKeyStartOffset + STATE_ENCODING_NUM_VERSION_BYTES, - keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - VIRTUAL_COL_FAMILY_PREFIX_BYTES) - keyRow - } + // Create new byte array without prefix + val dataLength = keyBytes.length - + STATE_ENCODING_NUM_VERSION_BYTES - VIRTUAL_COL_FAMILY_PREFIX_BYTES + val dataBytes = new Array[Byte](dataLength) + Platform.copyMemory( + keyBytes, + decodeKeyStartOffset + STATE_ENCODING_NUM_VERSION_BYTES, + dataBytes, + Platform.BYTE_ARRAY_OFFSET, + dataLength) + dataEncoder.decodeKey(dataBytes) } else { null } - } else decodeToUnsafeRow(keyBytes, keyRow) + } else { + dataEncoder.decodeKey(keyBytes) + } } override def supportPrefixKeyScan: Boolean = false @@ -1139,28 +1384,14 @@ class NoPrefixKeyStateEncoder( * This encoder supports RocksDB StringAppendOperator merge operator. Values encoded can be * merged in RocksDB using merge operation, and all merged values can be read using decodeValues * operation. - * If the avroEnc is specified, we are using Avro encoding for this column family's values */ class MultiValuedStateEncoder( - valueSchema: StructType, - avroEnc: Option[AvroEncoder] = None) + dataEncoder: RocksDBDataEncoder, + valueSchema: StructType) extends RocksDBValueStateEncoder with Logging { - import RocksDBStateEncoder._ - - private val usingAvroEncoding = avroEnc.isDefined - // Reusable objects - private val out = new ByteArrayOutputStream - private val valueRow = new UnsafeRow(valueSchema.size) - private lazy val valueAvroType = SchemaConverters.toAvroType(valueSchema) - private val valueProj = UnsafeProjection.create(valueSchema) - override def encodeValue(row: UnsafeRow): Array[Byte] = { - val bytes = if (usingAvroEncoding) { - encodeUnsafeRowToAvro(row, avroEnc.get.valueSerializer, valueAvroType, out) - } else { - encodeUnsafeRow(row) - } + val bytes = dataEncoder.encodeValue(row) val numBytes = bytes.length val encodedBytes = new Array[Byte](java.lang.Integer.BYTES + bytes.length) @@ -1179,12 +1410,7 @@ class MultiValuedStateEncoder( val encodedValue = new Array[Byte](numBytes) Platform.copyMemory(valueBytes, java.lang.Integer.BYTES + Platform.BYTE_ARRAY_OFFSET, encodedValue, Platform.BYTE_ARRAY_OFFSET, numBytes) - if (usingAvroEncoding) { - decodeFromAvroToUnsafeRow( - encodedValue, avroEnc.get.valueDeserializer, valueAvroType, valueProj) - } else { - decodeToUnsafeRow(encodedValue, valueRow) - } + dataEncoder.decodeValue(encodedValue) } } @@ -1210,12 +1436,7 @@ class MultiValuedStateEncoder( pos += numBytes pos += 1 // eat the delimiter character - if (usingAvroEncoding) { - decodeFromAvroToUnsafeRow( - encodedValue, avroEnc.get.valueDeserializer, valueAvroType, valueProj) - } else { - decodeToUnsafeRow(encodedValue, valueRow) - } + dataEncoder.decodeValue(encodedValue) } } } @@ -1235,29 +1456,13 @@ class MultiValuedStateEncoder( * The bytes of a UnsafeRow is written unmodified to starting from offset 1 * (offset 0 is the version byte of value 0). That is, if the unsafe row has N bytes, * then the generated array byte will be N+1 bytes. - * If the avroEnc is specified, we are using Avro encoding for this column family's values */ class SingleValueStateEncoder( - valueSchema: StructType, - avroEnc: Option[AvroEncoder] = None) - extends RocksDBValueStateEncoder with Logging { - - import RocksDBStateEncoder._ - - private val usingAvroEncoding = avroEnc.isDefined - // Reusable objects - private val out = new ByteArrayOutputStream - private val valueRow = new UnsafeRow(valueSchema.size) - private lazy val valueAvroType = SchemaConverters.toAvroType(valueSchema) - private val valueProj = UnsafeProjection.create(valueSchema) + dataEncoder: RocksDBDataEncoder, + valueSchema: StructType) + extends RocksDBValueStateEncoder { - override def encodeValue(row: UnsafeRow): Array[Byte] = { - if (usingAvroEncoding) { - encodeUnsafeRowToAvro(row, avroEnc.get.valueSerializer, valueAvroType, out) - } else { - encodeUnsafeRow(row) - } - } + override def encodeValue(row: UnsafeRow): Array[Byte] = dataEncoder.encodeValue(row) /** * Decode byte array for a value to a UnsafeRow. @@ -1266,15 +1471,7 @@ class SingleValueStateEncoder( * the given byte array. */ override def decodeValue(valueBytes: Array[Byte]): UnsafeRow = { - if (valueBytes == null) { - return null - } - if (usingAvroEncoding) { - decodeFromAvroToUnsafeRow( - valueBytes, avroEnc.get.valueDeserializer, valueAvroType, valueProj) - } else { - decodeToUnsafeRow(valueBytes, valueRow) - } + dataEncoder.decodeValue(valueBytes) } override def supportsMultipleValuesPerKey: Boolean = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index c9c987fa1620d..fb0bf84d7aabc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -30,7 +30,6 @@ import org.apache.spark.{SparkConf, SparkEnv, SparkException} import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec -import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, AvroSerializer, SchemaConverters} import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StreamExecution} @@ -76,17 +75,28 @@ private[sql] class RocksDBStateStoreProvider isInternal: Boolean = false): Unit = { verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) val newColFamilyId = rocksDB.createColFamilyIfAbsent(colFamilyName) - // Create cache key using store ID to avoid collisions - val avroEncCacheKey = s"${getRunId(hadoopConf)}_${stateStoreId.operatorId}_" + - s"${stateStoreId.partitionId}_$colFamilyName" - - val avroEnc = getAvroEnc( - stateStoreEncoding, avroEncCacheKey, keyStateEncoderSpec, valueSchema) - - keyValueEncoderMap.putIfAbsent(colFamilyName, - (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, useColumnFamilies, - Some(newColFamilyId), avroEnc), RocksDBStateEncoder.getValueEncoder(valueSchema, - useMultipleValuesPerKey, avroEnc))) + val dataEncoderCacheKey = StateRowEncoderCacheKey( + queryRunId = getRunId(hadoopConf), + operatorId = stateStoreId.operatorId, + partitionId = stateStoreId.partitionId, + stateStoreName = stateStoreId.storeName, + colFamilyName = colFamilyName) + + val dataEncoder = getDataEncoder( + stateStoreEncoding, dataEncoderCacheKey, keyStateEncoderSpec, valueSchema) + + val keyEncoder = RocksDBStateEncoder.getKeyEncoder( + dataEncoder, + keyStateEncoderSpec, + useColumnFamilies, + Some(newColFamilyId) + ) + val valueEncoder = RocksDBStateEncoder.getValueEncoder( + dataEncoder, + valueSchema, + useMultipleValuesPerKey + ) + keyValueEncoderMap.putIfAbsent(colFamilyName, (keyEncoder, valueEncoder)) } override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { @@ -387,17 +397,28 @@ private[sql] class RocksDBStateStoreProvider defaultColFamilyId = Some(rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME)) } - val colFamilyName = StateStore.DEFAULT_COL_FAMILY_NAME - // Create cache key using store ID to avoid collisions - val avroEncCacheKey = s"${getRunId(hadoopConf)}_${stateStoreId.operatorId}_" + - s"${stateStoreId.partitionId}_$colFamilyName" - val avroEnc = getAvroEnc( - stateStoreEncoding, avroEncCacheKey, keyStateEncoderSpec, valueSchema) + val dataEncoderCacheKey = StateRowEncoderCacheKey( + queryRunId = getRunId(hadoopConf), + operatorId = stateStoreId.operatorId, + partitionId = stateStoreId.partitionId, + stateStoreName = stateStoreId.storeName, + colFamilyName = StateStore.DEFAULT_COL_FAMILY_NAME) - keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, - (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, - useColumnFamilies, defaultColFamilyId, avroEnc), - RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey, avroEnc))) + val dataEncoder = getDataEncoder( + stateStoreEncoding, dataEncoderCacheKey, keyStateEncoderSpec, valueSchema) + + val keyEncoder = RocksDBStateEncoder.getKeyEncoder( + dataEncoder, + keyStateEncoderSpec, + useColumnFamilies, + defaultColFamilyId + ) + val valueEncoder = RocksDBStateEncoder.getValueEncoder( + dataEncoder, + valueSchema, + useMultipleValuesPerKey + ) + keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, (keyEncoder, valueEncoder)) } override def stateStoreId: StateStoreId = stateStoreId_ @@ -605,6 +626,15 @@ private[sql] class RocksDBStateStoreProvider } } + +case class StateRowEncoderCacheKey( + queryRunId: String, + operatorId: Long, + partitionId: Int, + stateStoreName: String, + colFamilyName: String +) + object RocksDBStateStoreProvider { // Version as a single byte that specifies the encoding of the row data in RocksDB val STATE_ENCODING_NUM_VERSION_BYTES = 1 @@ -615,30 +645,48 @@ object RocksDBStateStoreProvider { private val AVRO_ENCODER_LIFETIME_HOURS = 1L // Add the cache at companion object level so it persists across provider instances - private val avroEncoderMap: NonFateSharingCache[String, AvroEncoder] = + private val dataEncoderCache: NonFateSharingCache[StateRowEncoderCacheKey, RocksDBDataEncoder] = NonFateSharingCache( maximumSize = MAX_AVRO_ENCODERS_IN_CACHE, expireAfterAccessTime = AVRO_ENCODER_LIFETIME_HOURS, expireAfterAccessTimeUnit = TimeUnit.HOURS ) - def getAvroEnc( + /** + * Creates and returns a data encoder for the state store based on the specified encoding type. + * This method handles caching of encoders to improve performance by reusing encoder instances + * when possible. + * + * The method supports two encoding types: + * - Avro: Uses Apache Avro for serialization with schema evolution support + * - UnsafeRow: Uses Spark's internal row format for optimal performance + * + * @param stateStoreEncoding The encoding type to use ("avro" or "unsaferow") + * @param encoderCacheKey A unique key for caching the encoder instance, typically combining + * query ID, operator ID, partition ID, and column family name + * @param keyStateEncoderSpec Specification for how to encode keys, including schema and any + * prefix/range scan requirements + * @param valueSchema The schema for the values to be encoded + * @return A RocksDBDataEncoder instance configured for the specified encoding type + */ + def getDataEncoder( stateStoreEncoding: String, - avroEncCacheKey: String, + encoderCacheKey: StateRowEncoderCacheKey, keyStateEncoderSpec: KeyStateEncoderSpec, - valueSchema: StructType): Option[AvroEncoder] = { - - stateStoreEncoding match { - case "avro" => Some( - RocksDBStateStoreProvider.avroEncoderMap.get( - avroEncCacheKey, - new java.util.concurrent.Callable[AvroEncoder] { - override def call(): AvroEncoder = createAvroEnc(keyStateEncoderSpec, valueSchema) + valueSchema: StructType): RocksDBDataEncoder = { + assert(Set("avro", "unsaferow").contains(stateStoreEncoding)) + RocksDBStateStoreProvider.dataEncoderCache.get( + encoderCacheKey, + new java.util.concurrent.Callable[RocksDBDataEncoder] { + override def call(): RocksDBDataEncoder = { + if (stateStoreEncoding == "avro") { + new AvroStateEncoder(keyStateEncoderSpec, valueSchema) + } else { + new UnsafeRowDataEncoder(keyStateEncoderSpec, valueSchema) } - ) - ) - case "unsaferow" => None - } + } + } + ) } private def getRunId(hadoopConf: Configuration): String = { @@ -651,53 +699,6 @@ object RocksDBStateStoreProvider { } } - private def getAvroSerializer(schema: StructType): AvroSerializer = { - val avroType = SchemaConverters.toAvroType(schema) - new AvroSerializer(schema, avroType, nullable = false) - } - - private def getAvroDeserializer(schema: StructType): AvroDeserializer = { - val avroType = SchemaConverters.toAvroType(schema) - val avroOptions = AvroOptions(Map.empty) - new AvroDeserializer(avroType, schema, - avroOptions.datetimeRebaseModeInRead, avroOptions.useStableIdForUnionType, - avroOptions.stableIdPrefixForUnionType, avroOptions.recursiveFieldMaxDepth) - } - - private def createAvroEnc( - keyStateEncoderSpec: KeyStateEncoderSpec, - valueSchema: StructType - ): AvroEncoder = { - val valueSerializer = getAvroSerializer(valueSchema) - val valueDeserializer = getAvroDeserializer(valueSchema) - val keySchema = keyStateEncoderSpec match { - case NoPrefixKeyStateEncoderSpec(schema) => - schema - case PrefixKeyScanStateEncoderSpec(schema, numColsPrefixKey) => - StructType(schema.take(numColsPrefixKey)) - case RangeKeyScanStateEncoderSpec(schema, orderingOrdinals) => - val remainingSchema = { - 0.until(schema.length).diff(orderingOrdinals).map { ordinal => - schema(ordinal) - } - } - StructType(remainingSchema) - } - val suffixKeySchema = keyStateEncoderSpec match { - case PrefixKeyScanStateEncoderSpec(schema, numColsPrefixKey) => - Some(StructType(schema.drop(numColsPrefixKey))) - case _ => None - } - AvroEncoder( - getAvroSerializer(keySchema), - getAvroDeserializer(keySchema), - valueSerializer, - valueDeserializer, - suffixKeySchema.map(getAvroSerializer), - suffixKeySchema.map(getAvroDeserializer) - ) - } - // Native operation latencies report as latency in microseconds // as SQLMetrics support millis. Convert the value to millis val CUSTOM_METRIC_GET_TIME = StateStoreCustomTimingMetric( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index e2b93c147891d..de10518035e2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -322,8 +322,22 @@ case class StateStoreCustomTimingMetric(name: String, desc: String) extends Stat } sealed trait KeyStateEncoderSpec { + def keySchema: StructType def jsonValue: JValue def json: String = compact(render(jsonValue)) + + /** + * Creates a RocksDBKeyStateEncoder for this specification. + * + * @param dataEncoder The encoder to handle the actual data encoding/decoding + * @param useColumnFamilies Whether to use RocksDB column families + * @param virtualColFamilyId Optional column family ID when column families are used + * @return A RocksDBKeyStateEncoder configured for this spec + */ + def toEncoder( + dataEncoder: RocksDBDataEncoder, + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder } object KeyStateEncoderSpec { @@ -347,6 +361,14 @@ case class NoPrefixKeyStateEncoderSpec(keySchema: StructType) extends KeyStateEn override def jsonValue: JValue = { ("keyStateEncoderType" -> JString("NoPrefixKeyStateEncoderSpec")) } + + override def toEncoder( + dataEncoder: RocksDBDataEncoder, + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder = { + new NoPrefixKeyStateEncoder( + dataEncoder, keySchema, useColumnFamilies, virtualColFamilyId) + } } case class PrefixKeyScanStateEncoderSpec( @@ -356,6 +378,15 @@ case class PrefixKeyScanStateEncoderSpec( throw StateStoreErrors.incorrectNumOrderingColsForPrefixScan(numColsPrefixKey.toString) } + override def toEncoder( + dataEncoder: RocksDBDataEncoder, + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder = { + new PrefixKeyScanStateEncoder( + dataEncoder, keySchema, numColsPrefixKey, useColumnFamilies, virtualColFamilyId) + } + + override def jsonValue: JValue = { ("keyStateEncoderType" -> JString("PrefixKeyScanStateEncoderSpec")) ~ ("numColsPrefixKey" -> JInt(numColsPrefixKey)) @@ -370,6 +401,14 @@ case class RangeKeyScanStateEncoderSpec( throw StateStoreErrors.incorrectNumOrderingColsForRangeScan(orderingOrdinals.length.toString) } + override def toEncoder( + dataEncoder: RocksDBDataEncoder, + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder = { + new RangeKeyScanStateEncoder( + dataEncoder, keySchema, orderingOrdinals, useColumnFamilies, virtualColFamilyId) + } + override def jsonValue: JValue = { ("keyStateEncoderType" -> JString("RangeKeyScanStateEncoderSpec")) ~ ("orderingOrdinals" -> orderingOrdinals.map(JInt(_))) @@ -758,6 +797,7 @@ object StateStore extends Logging { storeConf: StateStoreConf, hadoopConf: Configuration, useMultipleValuesPerKey: Boolean = false): ReadStateStore = { + hadoopConf.set(StreamExecution.RUN_ID_KEY, storeProviderId.queryRunId.toString) if (version < 0) { throw QueryExecutionErrors.unexpectedStateStoreVersion(version) } @@ -778,6 +818,7 @@ object StateStore extends Logging { storeConf: StateStoreConf, hadoopConf: Configuration, useMultipleValuesPerKey: Boolean = false): StateStore = { + hadoopConf.set(StreamExecution.RUN_ID_KEY, storeProviderId.queryRunId.toString) if (version < 0) { throw QueryExecutionErrors.unexpectedStateStoreVersion(version) } From 62d49b35ab24a4be7083fcf1da0c8710482415ac Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Wed, 18 Dec 2024 15:40:30 +0800 Subject: [PATCH 204/438] [SPARK-50504][SQL] Enable SQL pipe syntax by default ### What changes were proposed in this pull request? This PR enables SQL pipe syntax by default by updating the default value of the `spark.sql.operatorPipeSyntaxEnabled` configuration from `Utils.isTesting` to `true`. ### Why are the changes needed? This feature is ready for general usage. It is fully backwards-compatible, and will cause no breaking changes to existing SQL queries. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR also cleans up a unit test by removing the assignment of that configuration to true, since that is the default value now. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49079 from dtenedor/enable-config. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/execution/SparkSqlParserSuite.scala | 215 +++++++++--------- 2 files changed, 113 insertions(+), 104 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8bdb8b1addb00..e69fa996f6f29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -5132,7 +5132,7 @@ object SQLConf { "the sequence of steps that the query performs in a composable fashion.") .version("4.0.0") .booleanConf - .createWithDefault(Utils.isTesting) + .createWithDefault(true) val LEGACY_PERCENTILE_DISC_CALCULATION = buildConf("spark.sql.legacy.percentileDiscCalculation") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index e698c50e5631a..d4a0f987d4993 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -885,109 +885,118 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { // scalastyle:on test("Operator pipe SQL syntax") { - withSQLConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED.key -> "true") { - // Basic selection. - // Here we check that every parsed plan contains a projection and a source relation or - // inline table. - def check(query: String, patterns: Seq[TreePattern]): Unit = { - val plan: LogicalPlan = parser.parsePlan(query) - assert(patterns.exists(plan.containsPattern), s"Failed to parse $query, plan: $plan") - assert(plan.containsAnyPattern(UNRESOLVED_RELATION, LOCAL_RELATION)) - } - def checkPipeSelect(query: String): Unit = check(query, Seq(PROJECT)) - checkPipeSelect("TABLE t |> SELECT 1 AS X") - checkPipeSelect("TABLE t |> SELECT 1 AS X, 2 AS Y |> SELECT X + Y AS Z") - checkPipeSelect("VALUES (0), (1) tab(col) |> SELECT col * 2 AS result") - checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y") - checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y, X + 2 Z") - checkPipeSelect("TABLE t |> EXTEND 1 AS z, 2 AS Z |> SET z = 1, Z = 2") - // FROM operators. - def checkPipeSelectFrom(query: String): Unit = check(query, Seq(PROJECT)) - checkPipeSelectFrom("FROM t |> SELECT 1 AS X") - // Basic WHERE operators. - def checkPipeWhere(query: String): Unit = check(query, Seq(FILTER)) - checkPipeWhere("TABLE t |> WHERE X = 1") - checkPipeWhere("TABLE t |> SELECT X, LENGTH(Y) AS Z |> WHERE X + LENGTH(Y) < 4") - checkPipeWhere("TABLE t |> WHERE X = 1 AND Y = 2 |> WHERE X + Y = 3") - checkPipeWhere("VALUES (0), (1) tab(col) |> WHERE col < 1") - // PIVOT and UNPIVOT operations - def checkPivotUnpivot(query: String): Unit = check(query, Seq(PIVOT, UNPIVOT)) - checkPivotUnpivot( - """ - |SELECT * FROM VALUES - | ("dotNET", 2012, 10000), - | ("Java", 2012, 20000), - | ("dotNET", 2012, 5000), - | ("dotNET", 2013, 48000), - | ("Java", 2013, 30000) - | AS courseSales(course, year, earnings) - ||> PIVOT ( - | SUM(earnings) - | FOR course IN ('dotNET', 'Java') - |) - |""".stripMargin) - checkPivotUnpivot( - """ - |SELECT * FROM VALUES - | ("dotNET", 15000, 48000, 22500), - | ("Java", 20000, 30000, NULL) - | AS courseEarnings(course, `2012`, `2013`, `2014`) - ||> UNPIVOT ( - | earningsYear FOR year IN (`2012`, `2013`, `2014`) - |) - |""".stripMargin) - // Sampling operations - def checkSample(query: String): Unit = { - val plan: LogicalPlan = parser.parsePlan(query) - assert(plan.collectFirst(_.isInstanceOf[Sample]).nonEmpty) - assert(plan.containsAnyPattern(UNRESOLVED_RELATION, LOCAL_RELATION)) - } - checkSample("TABLE t |> TABLESAMPLE (50 PERCENT)") - checkSample("TABLE t |> TABLESAMPLE (5 ROWS)") - checkSample("TABLE t |> TABLESAMPLE (BUCKET 4 OUT OF 10)") - // Joins. - def checkPipeJoin(query: String): Unit = check(query, Seq(JOIN)) - Seq("", "INNER", "LEFT", "LEFT OUTER", "SEMI", "LEFT SEMI", "RIGHT", "RIGHT OUTER", "FULL", - "FULL OUTER", "ANTI", "LEFT ANTI", "CROSS").foreach { joinType => - checkPipeJoin(s"TABLE t |> $joinType JOIN other ON (t.x = other.x)") - } - // Set operations - def checkDistinct(query: String): Unit = check(query, Seq(DISTINCT_LIKE)) - def checkExcept(query: String): Unit = check(query, Seq(EXCEPT)) - def checkIntersect(query: String): Unit = check(query, Seq(INTERSECT)) - def checkUnion(query: String): Unit = check(query, Seq(UNION)) - checkDistinct("TABLE t |> UNION DISTINCT TABLE t") - checkExcept("TABLE t |> EXCEPT ALL TABLE t") - checkExcept("TABLE t |> EXCEPT DISTINCT TABLE t") - checkExcept("TABLE t |> MINUS ALL TABLE t") - checkExcept("TABLE t |> MINUS DISTINCT TABLE t") - checkIntersect("TABLE t |> INTERSECT ALL TABLE t") - checkUnion("TABLE t |> UNION ALL TABLE t") - // Sorting and distributing operators. - def checkSort(query: String): Unit = check(query, Seq(SORT)) - def checkRepartition(query: String): Unit = check(query, Seq(REPARTITION_OPERATION)) - def checkLimit(query: String): Unit = check(query, Seq(LIMIT)) - checkSort("TABLE t |> ORDER BY x") - checkSort("TABLE t |> SELECT x |> SORT BY x") - checkLimit("TABLE t |> LIMIT 1") - checkLimit("TABLE t |> LIMIT 2 OFFSET 1") - checkRepartition("TABLE t |> DISTRIBUTE BY x |> WHERE x = 1") - checkRepartition("TABLE t |> CLUSTER BY x |> TABLESAMPLE (100 PERCENT)") - checkRepartition("TABLE t |> SORT BY x DISTRIBUTE BY x") - // Aggregation - def checkAggregate(query: String): Unit = check(query, Seq(AGGREGATE)) - checkAggregate("SELECT a, b FROM t |> AGGREGATE SUM(a)") - checkAggregate("SELECT a, b FROM t |> AGGREGATE SUM(a) AS result GROUP BY b") - checkAggregate("SELECT a, b FROM t |> AGGREGATE GROUP BY b") - checkAggregate("SELECT a, b FROM t |> AGGREGATE COUNT(*) AS result GROUP BY b") - // Window - def checkWindow(query: String): Unit = check(query, Seq(WITH_WINDOW_DEFINITION)) - checkWindow( - """ - |TABLE windowTestData - ||> SELECT cate, SUM(val) OVER w - | WINDOW w AS (PARTITION BY cate ORDER BY val) - |""".stripMargin) + // Basic selection. + // Here we check that every parsed plan contains a projection and a source relation or + // inline table. + def check(query: String, patterns: Seq[TreePattern]): Unit = { + val plan: LogicalPlan = parser.parsePlan(query) + assert(patterns.exists(plan.containsPattern), s"Failed to parse $query, plan: $plan") + assert(plan.containsAnyPattern(UNRESOLVED_RELATION, LOCAL_RELATION)) + } + def checkPipeSelect(query: String): Unit = check(query, Seq(PROJECT)) + checkPipeSelect("TABLE t |> SELECT 1 AS X") + checkPipeSelect("TABLE t |> SELECT 1 AS X, 2 AS Y |> SELECT X + Y AS Z") + checkPipeSelect("VALUES (0), (1) tab(col) |> SELECT col * 2 AS result") + checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y") + checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y, X + 2 Z") + checkPipeSelect("TABLE t |> EXTEND 1 AS z, 2 AS Z |> SET z = 1, Z = 2") + // FROM operators. + def checkPipeSelectFrom(query: String): Unit = check(query, Seq(PROJECT)) + checkPipeSelectFrom("FROM t |> SELECT 1 AS X") + // Basic WHERE operators. + def checkPipeWhere(query: String): Unit = check(query, Seq(FILTER)) + checkPipeWhere("TABLE t |> WHERE X = 1") + checkPipeWhere("TABLE t |> SELECT X, LENGTH(Y) AS Z |> WHERE X + LENGTH(Y) < 4") + checkPipeWhere("TABLE t |> WHERE X = 1 AND Y = 2 |> WHERE X + Y = 3") + checkPipeWhere("VALUES (0), (1) tab(col) |> WHERE col < 1") + // PIVOT and UNPIVOT operations + def checkPivotUnpivot(query: String): Unit = check(query, Seq(PIVOT, UNPIVOT)) + checkPivotUnpivot( + """ + |SELECT * FROM VALUES + | ("dotNET", 2012, 10000), + | ("Java", 2012, 20000), + | ("dotNET", 2012, 5000), + | ("dotNET", 2013, 48000), + | ("Java", 2013, 30000) + | AS courseSales(course, year, earnings) + ||> PIVOT ( + | SUM(earnings) + | FOR course IN ('dotNET', 'Java') + |) + |""".stripMargin) + checkPivotUnpivot( + """ + |SELECT * FROM VALUES + | ("dotNET", 15000, 48000, 22500), + | ("Java", 20000, 30000, NULL) + | AS courseEarnings(course, `2012`, `2013`, `2014`) + ||> UNPIVOT ( + | earningsYear FOR year IN (`2012`, `2013`, `2014`) + |) + |""".stripMargin) + // Sampling operations + def checkSample(query: String): Unit = { + val plan: LogicalPlan = parser.parsePlan(query) + assert(plan.collectFirst(_.isInstanceOf[Sample]).nonEmpty) + assert(plan.containsAnyPattern(UNRESOLVED_RELATION, LOCAL_RELATION)) + } + checkSample("TABLE t |> TABLESAMPLE (50 PERCENT)") + checkSample("TABLE t |> TABLESAMPLE (5 ROWS)") + checkSample("TABLE t |> TABLESAMPLE (BUCKET 4 OUT OF 10)") + // Joins. + def checkPipeJoin(query: String): Unit = check(query, Seq(JOIN)) + Seq("", "INNER", "LEFT", "LEFT OUTER", "SEMI", "LEFT SEMI", "RIGHT", "RIGHT OUTER", "FULL", + "FULL OUTER", "ANTI", "LEFT ANTI", "CROSS").foreach { joinType => + checkPipeJoin(s"TABLE t |> $joinType JOIN other ON (t.x = other.x)") + } + // Set operations + def checkDistinct(query: String): Unit = check(query, Seq(DISTINCT_LIKE)) + def checkExcept(query: String): Unit = check(query, Seq(EXCEPT)) + def checkIntersect(query: String): Unit = check(query, Seq(INTERSECT)) + def checkUnion(query: String): Unit = check(query, Seq(UNION)) + checkDistinct("TABLE t |> UNION DISTINCT TABLE t") + checkExcept("TABLE t |> EXCEPT ALL TABLE t") + checkExcept("TABLE t |> EXCEPT DISTINCT TABLE t") + checkExcept("TABLE t |> MINUS ALL TABLE t") + checkExcept("TABLE t |> MINUS DISTINCT TABLE t") + checkIntersect("TABLE t |> INTERSECT ALL TABLE t") + checkUnion("TABLE t |> UNION ALL TABLE t") + // Sorting and distributing operators. + def checkSort(query: String): Unit = check(query, Seq(SORT)) + def checkRepartition(query: String): Unit = check(query, Seq(REPARTITION_OPERATION)) + def checkLimit(query: String): Unit = check(query, Seq(LIMIT)) + checkSort("TABLE t |> ORDER BY x") + checkSort("TABLE t |> SELECT x |> SORT BY x") + checkLimit("TABLE t |> LIMIT 1") + checkLimit("TABLE t |> LIMIT 2 OFFSET 1") + checkRepartition("TABLE t |> DISTRIBUTE BY x |> WHERE x = 1") + checkRepartition("TABLE t |> CLUSTER BY x |> TABLESAMPLE (100 PERCENT)") + checkRepartition("TABLE t |> SORT BY x DISTRIBUTE BY x") + // Aggregation + def checkAggregate(query: String): Unit = check(query, Seq(AGGREGATE)) + checkAggregate("SELECT a, b FROM t |> AGGREGATE SUM(a)") + checkAggregate("SELECT a, b FROM t |> AGGREGATE SUM(a) AS result GROUP BY b") + checkAggregate("SELECT a, b FROM t |> AGGREGATE GROUP BY b") + checkAggregate("SELECT a, b FROM t |> AGGREGATE COUNT(*) AS result GROUP BY b") + // Window + def checkWindow(query: String): Unit = check(query, Seq(WITH_WINDOW_DEFINITION)) + checkWindow( + """ + |TABLE windowTestData + ||> SELECT cate, SUM(val) OVER w + | WINDOW w AS (PARTITION BY cate ORDER BY val) + |""".stripMargin) + withSQLConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED.key -> "false") { + val sql = s"TABLE t |> SELECT 1 AS X" + checkError( + exception = parseException(sql), + condition = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "Operator pipe SQL syntax using |>"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = sql.length - 1)) } } } From cb84939e02ffdbe19830ff67216348470b0bfa6b Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 18 Dec 2024 16:41:05 +0900 Subject: [PATCH 205/438] [SPARK-50596][PYTHON] Upgrade Py4J from 0.10.9.7 to 0.10.9.8 ### What changes were proposed in this pull request? This PR aim to upgrade Py4J 0.10.9.8, with relevant changes. ### Why are the changes needed? Py4J 0.10.9.8 has several bug fixes especially https://github.com/py4j/py4j/pull/538 Release notes: https://www.py4j.org/changelog.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing test cases Closes #49214 from HyukjinKwon/SPARK-50596. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_python_connect.yml | 4 ++-- .github/workflows/build_python_connect35.yml | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- python/docs/Makefile | 2 +- python/docs/make2.bat | 2 +- python/docs/source/getting_started/install.rst | 2 +- python/lib/py4j-0.10.9.7-src.zip | Bin 42424 -> 0 bytes python/lib/py4j-0.10.9.8-src.zip | Bin 0 -> 42908 bytes python/packaging/classic/setup.py | 2 +- sbin/spark-config.sh | 2 +- 14 files changed, 13 insertions(+), 13 deletions(-) delete mode 100644 python/lib/py4j-0.10.9.7-src.zip create mode 100644 python/lib/py4j-0.10.9.8-src.zip diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index 471ad31279da4..b15e15abed1c3 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -82,7 +82,7 @@ jobs: sed -i 's/rootLogger.level = info/rootLogger.level = warn/g' conf/log4j2.properties # Start a Spark Connect server for local - PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ + PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ --jars "`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" @@ -101,7 +101,7 @@ jobs: mv pyspark.back python/pyspark # Start a Spark Connect server for local-cluster - PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ + PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --master "local-cluster[2, 4, 1024]" \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ --jars "`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml index b292553f99098..080932c6c09c0 100644 --- a/.github/workflows/build_python_connect35.yml +++ b/.github/workflows/build_python_connect35.yml @@ -85,7 +85,7 @@ jobs: sed -i 's/rootLogger.level = info/rootLogger.level = warn/g' conf/log4j2.properties # Start a Spark Connect server for local - PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ + PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ --jars "`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" diff --git a/bin/pyspark b/bin/pyspark index 2f08f78369159..d719875b3659e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -77,7 +77,7 @@ fi # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 232813b4ffdd6..97a4406977668 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.9.7-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.9.8-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 7805a3f37ae53..5bc007fa068a7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -426,7 +426,7 @@ net.sf.py4j py4j - 0.10.9.7 + 0.10.9.8 org.apache.spark diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 045ed0e4c01cb..9a944a44f655b 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.ArrayImplicits.SparkArrayOps import org.apache.spark.util.Utils private[spark] object PythonUtils extends Logging { - val PY4J_ZIP_NAME = "py4j-0.10.9.7-src.zip" + val PY4J_ZIP_NAME = "py4j-0.10.9.8-src.zip" /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ def sparkPythonPath: String = { diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index a1b9d66f66579..4045ee137f835 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -247,7 +247,7 @@ parquet-format-structures/1.15.0//parquet-format-structures-1.15.0.jar parquet-hadoop/1.15.0//parquet-hadoop-1.15.0.jar parquet-jackson/1.15.0//parquet-jackson-1.15.0.jar pickle/1.5//pickle-1.5.jar -py4j/0.10.9.7//py4j-0.10.9.7.jar +py4j/0.10.9.8//py4j-0.10.9.8.jar remotetea-oncrpc/1.1.2//remotetea-oncrpc-1.1.2.jar rocksdbjni/9.7.3//rocksdbjni-9.7.3.jar scala-collection-compat_2.13/2.7.0//scala-collection-compat_2.13-2.7.0.jar diff --git a/python/docs/Makefile b/python/docs/Makefile index 428b0d24b568e..f49adb0df80af 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -21,7 +21,7 @@ SPHINXBUILD ?= sphinx-build SOURCEDIR ?= source BUILDDIR ?= build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.9.7-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.9.8-src.zip) # Put it first so that "make" without argument is like "make help". help: diff --git a/python/docs/make2.bat b/python/docs/make2.bat index 41e33cd07d418..4127a045bf2fc 100644 --- a/python/docs/make2.bat +++ b/python/docs/make2.bat @@ -25,7 +25,7 @@ if "%SPHINXBUILD%" == "" ( set SOURCEDIR=source set BUILDDIR=build -set PYTHONPATH=..;..\lib\py4j-0.10.9.7-src.zip +set PYTHONPATH=..;..\lib\py4j-0.10.9.8-src.zip if "%1" == "" goto help diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index d0dc285b5257c..2b9f28135bb17 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -177,7 +177,7 @@ PySpark requires the following dependencies. ========================== ========================= ============================= Package Supported version Note ========================== ========================= ============================= -`py4j` >=0.10.9.7 Required to interact with JVM +`py4j` >=0.10.9.8 Required to interact with JVM ========================== ========================= ============================= Additional libraries that enhance functionality but are not included in the installation packages: diff --git a/python/lib/py4j-0.10.9.7-src.zip b/python/lib/py4j-0.10.9.7-src.zip deleted file mode 100644 index 6abba4efa0f4267bce6598cbb848d132b61cb489..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 42424 zcmafZQw9U$*zdUb~*T=M*k;2dAQMR_Iqi14eBIjtG@q~z5yx4!PJCLe|;kG zAloE+3&&RWV))?8K@+ddX@G*9_JepTJji$y&I$V+G^h$iGEJ+ybzn~z*$-P?w@LIx znuwNcJ(OY8Mlq4M#Y`5?LlXwYtJf=h5>Kq5CPgBp_)F$A2xIgWp>fhylJHL5a% zbphD_{d75;UXT;#uGJ73+92@Nj$teq>{L>;=5D$0Q*Z6QXZ6wbKKBsPF= z=vB)gs6wL8lAO}TTDk=(EQmS^LFkv&b&+Zla|JFE{L_3TU!3u3_HQ)O^}9EpH>~v~ zBvza~E-v=9>8mS&j>ERn%U!l__4jo*Pv1?ahB!B3{x^}dCws|L2!t?(&TQp{D_1m= zo;>g`9k)F^WU0r>6psr|uTro1+ePR$zzoWR%A(Q{Yi%syb;Okzf1$iqmumz1ciHPX zCjuNnd&zk7pU>k7by1Y@X-+LkBuYT9vz87jtO+%T&#gCjb1`*YxxtNy^pNSHtb0Y=24g_S7N+ z04cgo7e8i){T?>rquwduJu8t1@s4z6;u+Y15U8Ho(}#0Ri~$h|9CjW|kcTIaJ;t;_ z?=h7E;;}4G5i1UX3lf0VrvTcB)wb`0zU`jGT(mP(c-i6H_mRrvR6=sZV-9(9baSY% zGWt;0CCF_qLR7u3lj$;#l9uQhQmeEp93X3nkur9Z2@sU=oc$VKf}xY42>7%`Hwm{q zW%RQ*>hS{`^_fHn)WRtrbJci9gOg1_-TCo@hDXJ*EEa9e1miy!>egB2O zCu)8F^&D3^m%YifO(@p?rE#=d7WnHSMl@KTUe&iT=d9Owg&Y=C-ddENua?@}@+qt! z!p(IE!*iWFX<^F?PUC^1nqs+gk&w;0*5#$Qo^w3qn_s~GbX+-A{y7=*dbG9zr6MEI zw!N#U80Aa8EpztI;|8zK=hXR-X4?-^TYjy&6E|%0;c@X4*`fs2s zw9fWB(xwM4k5C=0M_WTy^)d3;nNSYT>y3soSO_(F(+DUd7U_7dA4xbASo~N2#0s8K zOb~EW9>x5mDRH&xNH-9fO~ihEv%3hKJ~y2yt+}H5l*;&PMt4t#(RDjLDYK`x@R!aX z5^Y@fv2Yeq^!nV$jx)N*Wt?j7S`NKJx4F9lx&a%rVJ2;}Rhp5F!toEwD>5_(SgFZl zo;W*zUfQ}inv{d=VG+Xl!Z;OZD=lSnxBiL>t$%$hK6 z%wbAtI6-8xoIE#I$Ebmzn?AUY6zVQg8gyfW0A}85sO%VivqvAKp8=Q+>a;rNdQMm= zTn~J$NtLv$J9cS~O@*639gZe)^Rb-$&;E8wF}?V~v^B|3`_468 z&CBJ8iR<5$41GWOxapjH^?Xn#xdH;no4c zzp(%RQ^&uszdRs`4C@~z_?P_;_Pd%mI$7A+{s-*W)BEpvdU}LBgufW*7#Zj|>DYfk z|8I7IQ-JgQ3;fRxPyhh@zuCpe&c@!r`9EyXtSV)@$$+?hfeK9?X2QkFp zRza1O`S&pyJ05EE7;LJb5D^+n00yp;_+4KP2M}e3nJ~~Ums1?D?E!k)*gl)>A>mqm zhN=+&w6nT*}d0XYGB`y8(6cZzcyC}lb zbnR1YW@c-bQXgq2`LWCNkT^bEL{Pah7Ch4Zg1EX?_&eN!KCmhOHp5uw0R;O_7(ZT4 zbbb{Tt_gPhdV*kQ0FcL}Os4*fO6r^CF-aY(LT5>9edD9_@DRuXl4h?THRAnPS9Ox} zeYYsXFlcVYXTPX4QrA6|iQTzfqgXD-`bq^{hdonK*IfBcq$)yg`KHiQeCc8jhgR#n@R zc6n3oU1ozV%VW)kxGIbk|3X%ymdh=P@0KEG&0lJW66GlQF|SV|q|b05u>TY(iFp?) z>P6i01!wjsHp1@!L-UC|Uy#cv_AG7nf}XPe!DZz&f8$uZ*Zvqyf8M@%^S3Z98~dX^ z$I6>mFD|&v_#1|OQESf0jyRonVJ-hSJ))Q8RzYm+7%bx^^8gE8;R$Sznc@ zx_?%SubBAf6lF0>8{Wtc>@UU#{&sS)_sZd?j=cCrUjwWa2{IhY=B_@09fDqQ9};UtCIw6e0^({zj3B}S83K*9PPATK zr&`1}J&oX!a~#r~Of*%7TT}=jxg&gZLe2_a)NCP8u3K9F>#$zF7xMf6Bx6eNg+YrT z0Kftw0D$!Ga51&`Cl|BuG;#b75p$)s9l0rr=)0=Ns7+vrp;bgSN6c}mGbuo5LDbEg ze4r&oV&zn+Q{NeJ81#9Q{S`;He!tzm8Y)=u%Znwq_iY;X9Xw^1mPidOH52x`t6Ssy zh;)2-Sma(5t(w0Mtx;ZooD5svevl01lf-fij2WCILsVjLwMdgn=7)Vted!1>jYb?M zy@*d@9gBRN9y@&)I4Ou-dN3B|b$|GUK7rJzx;klQFT+ABzt0n*w*gLaBwDU~sUl># z!rG?Np&3}0T~f=OtQ6Uvj04PcfX1&n8o7E{WllVxxSFDlx|!MgL-Y9M&ZN=njc!_H zTfY5{wY^d%fi!(J$t?@#FjuSlI67UP&F&4pNdzP ze&pp&=e&A>XgW$xHV6oFj-jNBN3;J#%4S3nCyL?p{f9?}J3%X&2&{iwtV#=T3!3CW zR#aM(DQebJJ%92N2V}B`8L#M!%MNtO#~Qhd!)?&5WP1?!0t4ZWF?e4;t;2pkFm!wUU`VRReM7D zVH*YL=dd_B=BldDjmfB)Rmt z-e?Lb7DUHm0y9ne8x2e!cDGb(|$gDT$cF zSxhq=ON7)#P1q(RGGE()d5sXk^yg*Gt)u+M>9p`{7kTuLJQ&osk&qb0s?J2#0_>y!CZT`r$~^im)R@P(sz39kHy1u$mp#LHocC_eKAA(`k7!;;0eE_~ zzHaR~uFd%E>c37IhJ#A$`igTsCQKG{VVVHljk4vwXa?U1V~0?U5QiQ^XM9TAPHYxj z1Je5OIUzju#}*ks{|6o5z)Ln^kpX~ZP5=P$-{@fPXyGavCR_kEPnbT5^s=t}^W`FkeOi3LW8F);+>eE4 z`=9#7CRDoUaH1vZ>Y-|CfSyRIb!8-~YK>Q1MMy>vnNSvWy&qpf{T+cuv@LQLatYHy zJQ?WZ<$kA0a);Z8g$DYqpx7XHoJc0ZAyc%qfGoD|i5H9{CDkzF*kC|8Y4K^?I_sv< zpB0q6X)Pj%J<4ApmhzFX)%eNp$INp_MAb@K@)i>BoJ*sj8+V-$jEd>xw+NEY$?R_L z-x^6#>?h*1XtFbM5&rl1&s741W*mc5r1`L;KDT5rrn!P6c@_n&M$SSSjRv+YN`|kn z0_x_afA!IZWDKckaQhgAi@u>^oV85EsT}zb7TFYuq;4gTeWrt_6?O_R4w-{07Y8Mf zMqI&^ziz+U2yd`fhOM8iS_E}5)I~f)3BRUyZ>fiHDx=eF{V9@a2>ygY8nfoW6b%^) zq04iDSWb3MHkWMfHIT~AFANq4S;G|EF*2~HxiWh$ea4I5QsRhJFUP!Q0{jKWsmEmN zr3z`)>WE@wRLJ7uln)FC%V%I;zO zke1iQ?n!~2gOLcjX=L0>et=Uh(C>q&O@uhKh6_s7{o{ONmf5p?jo82gX226aR&y;b zG=VEbob00VLnn z9Bog5`<4BuyWe*ZU7z8v7U7w|t9!49Z;v+i+(3T<#)qwZIS@osA+=ny_KkmsU=)sX zv}McMew}?*!=bq1>W}}}+_I&;EwXKA^@fOvcBU1eP25L(d}(_p!=%q z-+ub-cjM^#op02QhhXdNjqn|DeV8ztMc`UxPq#DkrNQ9qH92a4O*7&PwN<93Fxs9I zrFZ`KBF2d^UHc=j{!cTgVPa>bm0qs~(cAv1seQGc&H;7_35e3XgFfr`5&=SGy2mJ% zbZMRa6L4Dgo6}btn5$^xIp;M8d!W{rIQPu!bw@#lYu0A}Eni?NS}-2PUr&ZgbhwPR zyT7&MsqZqt>g4^=zE1m2+h%xwr;U}PcLUQ-zMUCzl^r(##jSq97auPXKdczf)sF@G z%-RA~hp&rgK}J7I4mIFTh{=B*epS8May*(bdvoPwowzMB+OuT#E8ZUO&I$VxGu!i< zw9&(`62(n|VFn)rr4i5hX+xr`lc7`QSN`BY=({WRAjLoFbF{Z(_5H1O53l6(?~ca? zR$@T6bqRt{&c+G zzOS{zA<@D`Qq>_S-~i=*oF4nB zW_$&L_}UHqwxMR`#_hpnEv`FIOq2WSmoc&qG0F=B>ab9}OwCse(Z40TY+m0MG1@87 z9K3`SH}KPr)t74p8b+II=CcJe2x-%YdvF!b*B(o%HbPH93NmwIEfelZ>%8+FF^9F` zc$VD8Rm`x%(KiIT*LUCnsFAmfJ3RaKUX)SL&a~}MlPl2;9hPuD_20~XvgK)mSa7yR zXk@&)bC?&4uuj2^grkQE3q{5F3xnr(I#7>kTAouG-c29}uN^NOY0}E_Pi+uEhrfo9 zY1ra4ZWX3}yKD|6*GEGNPih zzd(x7Muxs;YFUU}{XsM^{nn>aUG32+B?O&{h$);Ps}ve5Pi6TOkH9ldjH}$-uaT5? zWoiP8&QtMn5ESkUJk1kofhDIB^}!QCGkhHy-~N2I#lJPz8O|3oRdLWUv9$`>zU&ZBzPIYMq%G z6WsJdp8i>dQnd6}K7vaE?v%z@fmcX8tR#fuuk7P_@&M@FE2owGlk5f^X(4b{Rxd(G zp}gCfeBsfU!FvEJ*LxiB#5JvknZuM9(5DY{FKz+8F`%%|7 z{3)q${nFoQAcaxGN`C#gJP)X5IRbf$Az=CH0AJAyXe?iUU(P^H-eGtlU2wstWYjM0 zs-DPULj)<&SLbfNM{9q9{v%-Wn=}-qJEp5PZ}do*mAo2A$!ZoE!I}yYTFdShRaEeY zZqy5v6qM(1@=nZ4yX31>#!6v&71H+92pax4@;Kq3t46ZTg%3yuLjr0^SC1AW5AVLw zpp@bjkejzpJ z5)OM)WY$|vl6xH%D&ZQ%KP09w^2ZGGvG+uXFH~DXZO|Z!WT9z z=AGExAG(aFX#VD@;Cm>1cpY1AA8>V0k10}L3F$Z78*3-1Giy)u1y(f&c%}musAnZ?k4De9%=Nr zLzCdEg|Z-UO&0EZP5zW6#oqHn%JWYxNaEtt4OXhVjMpA1W6qYwT}d81#8OCUIp_#5 z5jg?M`B^ER>I_T>7w-XKDQL&|jA*Er4K%!eEoxQg7^M0GlShb)qxcW7MnvKx^FR#Z zoRgDYX!btfH!;$Rt+7T`_(%ubR53;C3$WPklb@8KeWCs}*H!5^5MULBjH*$i{YgB1SCbbJyUzHZ=7L zi7G=6jjPaK5E6l8I)o*aBhGdgxF?Mex&502d{y!ew2vriRkeOXJ%*8|$kMv-bS4zw zE0lr=$Y%!^N!}UYiDWM!cV!Eq0WP7Nrpn6_;}XmaEhu%fec!!hO}2Ihp=cqtg6s8hurRGm?%YryDGt0*Kdtk_F88qg_{!iKmVyaGcfNc#nW zMMZXsZY`-jDak#J69hFf1Arrd=@rJb#i>Gm336&R;<9llVNeI*dSa{L?u$eOU27-m z*gyv;HG!fQAT74H5VRWK#`)Q4~zy8k_SJmRpM-NSz0!#az1j7S<`))L#N4B^^=>Ff5q zgMIXYBzn<3=f-i-E&P71@#EvuxPiuXn|Ei#Y1)aFNpMYV4k|)V?q{ybtEUAL24esq z_r_mDT|v&Xw!+=KZ*TTgO1yM5*QwUM;Sqg+4d0CCQ51L_w}oGxC}u zpZ&r4lkW+@*#05;`uv=Xy{%ENb+^0!QpxGf-bDu|K(W>;_gX@px4Er0N6Hr2ziUr^kIxA9Z^96DgI1{8xYW_x#); zRb0}rEmzz%Re&;?9vbQPDC2BsGc&FBrWmR}A>j%@(AEJ9SJB2@J7iyXYW;^~)sSNO z@`=6bl)i(o{MFV5$424?%Y8(7yK0%H7P{OFvLkmQt76pXT-J(8=?aXuyE>^D8TjQG@% zD5)PcgUdRK6Y&%)n?p;H1Bf)9ChP{oQ6CqTDrPcnst1f5{2ehhGVnXq>d_jLL%j<5 z1my8L);-ZC2$8j+74hCHufBf+ec{@no6RkLn7K2S27ApY3G>L9VSla30V$#;O>Glu zBv`FKX?cO`ib{Bs9C(4^f^?(WfhJ{63LdfEjgrH7h+Z@8 znHV{0@j0OZE~{Ko_}qZGM%d=OAK9wFfNh$~;VFqQctf9f{obaak*2arzhLEfbGLs} zOvfQAUCFf!98rLbT4W85bJE6hRquYXd5h86d6r!p3Ho%Y?{c1%ds@7NHHr; z%6k}(Rpt-56&tLoY7DMgWPI!1Plf*^dX33ZuMHLxelk*YAs-{qpn5~z2a}Q|hnQH~ zDUg@X&h)j@g}Vi+$4^a|>b7jqOsfI=&&E-9& z0^|FIAO#V`!wOHvMA8+Wt&M=t+gN`D(6RRSEB}@yeT5D3g0e(8bxU&R9=DxA@71zn zXz(#`k;%(A?HJe#58u$XutfxC9ZPG-(wul!3!X-2PTt30Lc~cA_+NkfxI-mo`zFwy zca{#JCLN=g!!V^-s)tnw@a3N6TH$oRbUtp`y)^!54`0I(NH8%^G1O= zTJD&u>3qj!0g2faHCsVr-OybH+O zsC_-^d7@v#9MrpH1lzMnH_N(m7jrsZmC zmoGFeqj&gvt>Y&z^lR7$FdTm$X(W_=0_`w_B%R-DjR}3HNO1Jmg4t>ul;Y+z%UXn{hfdJR6rPKQk&~4Ks#IHd;5T& z2VybDj4uP%6{33YEgaWq3qVKQ&3F8j=*&1$*-1KTsIjojinTA1QYv6zX2p?hTFBx* zC<7yi2d?}5t^GUI??DF7P&9)SimoZjz{$U3BBdHDDvYPK#|hX>B?O>IsVY%V)5gh% zOT?_-GE^f}gV`Wemu*0riTsHG!0NBN`1PegL0<(SqN~{k4Fd$IO}nmq7HmgXLX4Rq z7^gdtT`S=j29|t!49B3vsfJ9XhGL>!Fmu{D7`T(}B3LfEg9rOA+%7{~HW7J$nu+9f z{OowC5&x7`z}?;avTgNIY_QLiBsp@khIV)nP5yJ91D3J7yZ>;r_v=*#r0#X8kDRud zq54Y?)OXEoHWo;#XZxy>9gN<1;j*6@(ZH~cc2Z~F_uUy}N56N7poB+fp-X#yT4_sH zuCQRO@!;%XF_$xhbh4fdU`l|YE2G&JaihiBWl+Tkp+SnBB7(JDO zxHQ$wEG4Z(bm%S7#G`(LNx8^y`kRNg_T8O4xqKh1(WJ zW><|uSOd!-vq2I9C^8Zc->$S$dw_~i$!5qu>T<$`fwz~JmsOVaRMalQ*FFs8hC~uh z9^%`&J=$ONUW-<5US=bVlPTv>iKlT?#@moCnl!=;-*ty&;N*f)cgMLYbxTzm6Vf>n z?}(?9(^Q5P$99fOG*CTWY}PKiQ&gx*(i#-HX-5_k_bPcL4#-Oo-4`GwM?@Yg9z!wt(#Ln~G5SYSy2{@& zN%56gGXwBrsmO4WA@e+|4DXiZfA8-8c-68eP^YP|BJTp$gC~|8Nn(z*kD00!EO|1# zQKwa>D-;z-7fdt&AeFztC@U$X6jC&JP#rZf)+-c{T2CEj+s2p>>8DEAbWolaC7&Ks ztI}o-R!*!GoLU_dQR~m-f%%dx6sU{LdeQD6j(b>_p?8piJy~fhiB+5NUK%J=cu`Y- zr5bJavo*-EJsW{=6_hZ2_km`Dg&{so%ys5N2syYwOQ*?Nk?|>8t}rjU-|+@DQwY#H z2j>0S(<6Im$BqcDGFeof4K<{)0=ooxBQgiLG93NXD^j!Go=z`lo1XP5FPk1|&MZki zOt7{JRz;NfEh9FZ%;JlD)C?aUpA5SlR!~(C`3U1Z_2g8V>L?SuhTHf;ZUjx>NNGFX z9BZ>8S=LV_mVv6^EF?<$s~*p1PpL+1s#xZ%!pLDosiZQQ`(I`$)c(3NFzRkdofi{Pwhb36iKvnL_JZx!z5MyI>pr}j9Nb*YJiUE=we3~F)nzXn8#9v1K0iIa{(6AUm4%NV z-P_H^A@O@+)tzh=buEy4(hwp{G;u=B3H;@neU7=d}2Zctano8YW`5-G>UzuG*BY=Q+jtfKz zR5yh$&~CCn&=06zw#Y4WPuvuXE8oA3+;9L1OY-si zMkZ-%)J$6G*r8rfSdnB<+NN8ad0A_+ssGCqT(0O>Og2+q41$cW_2;@>?$h7ea;-5P zr#%Dw#t`XiX7c-X)l=!4*>rIz{V8eFddmdEY~LO0BUD#y;52?&!tQL#XNak)vdmpK zOL4H}XSpn8gcrvF{-8mqVDlra=}JQtQGyi)cf8XpJg(>mY{b?WN_$X9_6A{fd96rw zBhM&pLRs~`x)a*@pmyTwOp%BxafYo2MvNLuZBmO&n&19T-bD(&w zK_Mg|>db&M13IFfR~YL!k3~1XpuhZH#8IWZ6&xAIUr4THDoQhz_bAOO8r81P1z7%( z877XpLSmt_gN2oM`_mj;1ib?0_Mmyho-Qs9UT@{44kW}9GE$SBSn3j~zQVL03cT*I=fpQlt5MSlZ()5>sMB!)e#2whXy0J(%X+2V8PAJGV z4dk@UOIQo=rT|s>6Jgs5&@=)Fckvq0J8ac}rha}PEREA-|Gqe4{vn3>%)!LW{zHs% zG7xQyPf!|&7O>`n)m4nr`KRIem&43_ucHz!)LHPHy8YImbkX{jVDXW*zrr#d%6mEi z&#N(?E@u7FLAF67W@8pBn@TdMWtvg0h>ryQAOnqN`+t1bGH4g9aBhpM_&wXGK?aJ& ziKt8VobVy2%0~&pV3q_W#eRn-dE#CCLpc^ zArYh2XGifQjd_CgJH^pBt`i1&IxH@6k02S|``TvekXrUE6B6^IN@h+%@f^XCh3bG- zC6lm0qZNS zF*5j!FjgNK$}Gjy;57!EG5drI_42vJJ8MTYjJdTaCLS6gPTo|6Fr=IZQ|@0Vqos< z^C43jtwhSMpTWQT&G*3jd0vDIY$stuA{GdN3*y2&alDV#iM%Ghf|}4;ib4#^XpzkS zgYW)9WTslD_iG)QVb~yaPd*Ey{UDb&8=bSdnR)G};H4o#a-HV)mZMT}m?pGk-YP~5 zRi8!jL2X)cK#ci@hJfp~>)9IqUEii#RRhBa>;cgu@WnGpGSPr`LCLsK2*k)Xx;e1%dJoR$+J{~s=2l;A1ggz%b2t(jAF13WG#B& zmhvYzO<2b_qQhvE9d@qjkAHAA5ga{2y*~GqAy@$G9Ns}0?Mgp<`j_lM8AgNR&>)>e zlc+PzXk`U3+)kx(iE;G?kFWo-C&(@ps}vtF2tw?)aFWINDZVR_;_`6?ZkWc-aZdz%`KV;<*3`oz6HkcZQHQ2^QhFcfNeN7Ne)tBcN z?8+*`a3aJ&o5%l9x$zFcmU87bsyIZN+Gp6-E{c)Wi7S!ww@4u`8*n~^b1mvGY3VQtn80lHdrb3o8jwDGXltNOMK@8RC=`g*wV*HgyW}1qm-OWvzz^zk;NFF|#~ecH(d;c2J>23?}uwfN{P}3Nup~ zlV+mdhdg1UN59PC#_$^(t~rHB3XBTf60)3?jMkDybT{m$*)ImiHkn!`{OkQq`ABB9 zvWA9@Jh_GI75MIy=5=NU(?NA2`g@+KFtP?o%qN2`a2SMbm+P3<{ziQv(EzjbX;j+_ z!-MBjs1s$10{=5=**Zk8Y6Ds{E@C^G1yA;zk}t^vcm54LiFzg6d}9-peZwhG3Jo$B z;4prMocVpb0+gl@C+cPj)jSczg$`B{346~4*E9|{hN3hd!?d&@kX-gzqFDJp6z818 z{LHxLFE${ne~Vexku~di{q+{Ut8XY!0~Jj+cSxFHxfK=33mQti zsAaSpYTLB_*j*_9tk_U`6A0EnMMtNu&Tc8 z(}D1f6B}zevqHA&K+S5UWnts+obJMaqzX}ZFnop?=*OHXe$5L~>^&)WF)uDa#Id}zLSVMca6<_Q7+1|8>r9dE z9^I&rbh%|+mW1R7a^2d!AW|n&wW81!j6z1*+;{8^ydHt*u5MOzuIyqKB6Cfm*JU5s zvAzHr@*G;FfyJRYW1r>6f>#~--C+`t%Wn3U#^>Q*-YzGYl=6E7{>%Dc2m?Qm&yn+b z5$Jj78mPWAR%|T>3j9xF2MN>-u zU@hMG1quXq3-BQX8b;RT{#QBG^#UlkD{xXZHop0zz_Is~X-r=KwI)}ctSGD?c6C4$ zuKjk@{Nf4~r&GC*i~o~Q&)WKDgr!lPISyXy?;4S#9p^X7SbgWpWWPjZYPGJ^`ApO#a!n zBLW_H`}4pm9{@L2$k9b;`%7(BfnCqy9;e>3DHjW|juZ|hNB<5Rgm`v$p5Ko|tH)t2 zvp5|?Z7y>MV%<&Olip4-b6ti(cw>
    N5hgk$$cB+*WpKmH*?7r~u z2V`CuH2$$lDUsP$PJ+d1it$8d)tLst8zLcRUrV>1f}GH!N(^n4DkjSA$0*;ag5)=U z1WG8G-p(Oy?uWY0KGo<}(q4p%X0zWzRJFJ&_b`KXm)|pLV>HNczvaYDWSij*cR0SRkw!b&h{TDgyMsdsIyDgZDU2 z5idoFX=NHdBn8fXebjxN!CbBb`T_sMm%mPb;miuT^K{EY%!tD2boQsnkWKTd0@Li@(5ME6wN6 zrb-mxTSHg|#-Wp8{@$~JO>shQiw(yUJ;gK_z2y+4(?E6wEn-Bq<4*K`q3~Rx;Q3*{ z$DKs@=gXJ?xBv(LFruYSd55}6{lc%u0g)WNU_IY6my#F}TSNXBb`)NdQbp>uG!NKS**&aCt&%?3LNr6;7ED#DtSr?^KBU@1Vrn^=;uz+k!7Hd;Fom zZp=_87}jO`gK=U5fwWx9P&UaC#D!jY@P4-9C%TE%XBM>{VF zUY9Ewlliu^zV=QKMZFETDgy1ZzDF8DR{*g!{_5ghUxhBCLaRbeIcEWoW=5vgB!v%= zr`F!j)2|bC;cqP50cst_qirw55}R(%4}$a<$=vICXjWygMVI`R`x4IXv5yeh65Jmb zA%xZ7Hit_ittvbV@Z2tx;~N%_TMmBD^Dz6!q_YHC9MC5KqN{}*0LuA!6J&*9r z*_!Nt!^VaXxqn#;p7H*mgU}<>a3VFToD`iPYwX7haT!5$VEx^?+o-)G&R~Z}XzHRY z%!<0r@`FcoW;s39)Tvd!_usd>cat8uIfruFJUmml3FOyQKupP|l#T_@fl_avRe3e( zcIPjTDkyR>CdWi;O;KEo(%=0NtNx_$oYi#6u*xv^Qok*?q02sq-`+vlG?XW_DscX!)Q0n+*1qPsetDo z&G+jM`BDA4NywTm6#NS4Sz}&=aFhEqTkvabbe<(V)i2hT1c)uf3caxe@pZUMdnrvD zgiY7w6&a_P#=`g&87_#($O=UljUy<$&T+br&aA{*{zt4#t6}5#Hwoa+a0{Sc)xLXL ziN$5*`nj2yx&D;a9Z#)g5v-D+kwINn1v$T^SL564ed-!ML zyxnMxl_*Rm#JF(g6PFk34pgPBy8G-d;b?$crP3JqGY-l$-^Bnuo)S%pYr!hmV;Jf@ zxX$dd?R~Tf!xk+O5QEttHU`W*fL9MQsG)=O(3y09JvZo2X3&upASqtxH4@{!lQ>+7 z?`4ZlkA6`v65al#kq-^8Bk8$)-`vbRJ%62E{!5Fe10>?nGUT%>X}A`S4=nG$<)d-? zPRzx_>C4(fye29(@PVsKbdIdH%I;hVMxJM&r7(Q9v|d1oaC_aBd8pB~vP4mc*uiEU zA{NJiLsNAmIULNxG1SB*h29^tlo$Gq7}m*Si3*F*-tIU|9Ww+#k9~j2<}T_@%A}l( z7B(H&YH~=o4DnRoFfVa(az2xz6njPB_SSl8C>x7hhh=Lvd*`0S#=mshdYn5Rf}khA z&*0!6@kJybDuyiam09eCpFS1hzU-^=?2^BOxH0Y3`5=;T`Peu>a-DmkTnQ+*mm8lGH98}d(I?JnMo8w@2B`Dn~ zK_2f6+L#AkVOASka-=go_YRkOVLA_XI!!X-xQlWIYV;cTIxm@f@ug{Q(R{76rHi@_ z{rLEpl{OFEG^?)4X!;C$SwRNA+Xb7|iV^B%?6Pg=>e07~8yroTxcNTt$LLmC+}#`R zx^r{bUBIUAx8>kLT>R4uvt9KU`kh@{#H`9iOY9NPJ!Kl;(b&4-45au#vc^#A-nyWO|Odb7FGz(hp#p} zvqBU<=Dv>&exI0~D*0egES54Su6!=N64%K=hjo<+YyNF3=i7AB&&*<_Oj0b7LvgzT z=Md=~pw;WHf$BrG@Awwl2=}|F*YW7wv9+V<5pVQJ~Jnu zQnQ3MK{KIYGhY=Tk|&-52Fw6WIQ;b9Y1c#jaI69_c{{feZWTfHAfVrU^U~`R^%Enc z0#fi}@jW)X&(f}OfZSVDuYna|svTE>swII3*=p#E)dO{U=$I-eJpWaWM++6Hy!-2i z=lvMdP>uHt^fHJCabg-OImg}1Dy{yPzLuWx^|F0GTn}y+_#?!{qr_iuTymKUT*G=p z84!f6RFL9aZSI}Zw^{$2Oz!-=jJS-TRDFd#%N1C80p@v6O*#=A<^Xb0RS*lb_I;YR zL^0{ewQbmCD&Z88YpJ~64tGv&p}0S%Gq4vDViJD&&kvP>81#(}WgG};9HuV4^3m^MiYRUC|U>u}RA=-=Y$nM6*YSm@u zce&Pts?^GqZ2X;)c z6nAj#jbwSeK2~PPW0c=k=QpC0i|5@H*J3|5$!5tvK5g)!rF?-;`XWO4dSe4##4w%w zYtYi>ZT6x;FA-4e+#AXJRx)T~QZ0e#P6f0i@)%Cn&^+iJD=OtVK)`wVxoGB5e!b@y z6Qu$c!Ew_itD(0*`|jp^YBcZa?&gk%^^UN-c|LMWqHOpr{==qPDr4Y{&jqld6MjCY zpH*xvCunK0qgM8Bzi8jL8-(2}GUB2W>LL~rqQUwnh%x(cPWd?TTVXx1a>n$60N34I zKNBQ=?bI2Vu#j6r)EInUTW*mLi?e}UNEC}Uy^y#R`-6XlK5JP}jnxQHc5yDwaKteB zq+o9a?J?kjSvq^XD{cRjpmiH>fLX8NB=jmzdq&aeG&KCGozd&zBKYIwXnuANkl&qJ zPB!14jK|+`zfrnIQ zpi)H9;6_ha>owo3UBC#_Bz^+V2uQ?kFsS@OXHnMsWzs5L4Q8q4>O)Nq1{1b^TGCZ9 zDugWD5ic;mZLkA{Z z_(>Sp#2=$Bdc>+PSGHaf9;^`yn6(!L{{;Hi$2ez<8hR_+bvFI@?eok`OU2Z(N+fik z(}A_y@FwB!LAb-r9e=~)0$|gIQ>tj{{TG>YMtD4hxC*yO3!WM^8g6beW!<1iFRPOT zdE#KNiIq47?#MU>#G>^VxRh97jpu(Z_tyJse#Xf`j@Bos4{j@0^vS+A1V82jWk*g= zr&|$pIrUslh~dM|Hn*Na8Gxn@2H>rxW~k*$zIHX{tgp}h+Ny*V*)t}YecY{$U*+{% zPlX#yxZkmgAk;tN1|rd=;H2Le6Vi*UQ-l{o9o@p8FbQMv{=`s`i<|P?Q$Oa3_O~T` zIzIu@=!W|~RfBm<=5G#OKCsu5!9%BcL0n!dqW2TYL-4MYk{!_B@mFgjb!CJ1GA;yl zAh!k+9;0Daj8~v#R7;Owsn|i((ANQEN}FD>{=5o80|y(8hdhVBonb!lI2Np$+=2^S zWJG})@Qmg~v+_)Y@aG58L2PQ|yRqSE}L-^%Ee2 z4$0Dcfoxi70nWOf#93sTBL`#68O57^+DPPot*mLCzydEMB^oQ_i zy2rPq3}P1BXqI?`RhU%X+F!ZOF|2y%Po|52n5nBR4}_%=kw>cM*8%gnF5lHLsS#8WUx)cS65K&WFmO2;@le zM)&Bi#fIIG!`VYL2A(v-19gP}DA8bqOgxop_$n^~R(yUB3O=kxiF?z8JCb%RdodNh zc1R(K$$GsOJXx+P;DGj;pg*x1^}&MPVb+we8}ktBoP$%yATo?9{&ulEy;F;vX&cwt zoiXVsFL+e(pt#ET$QukpHBNJik7S&GfpUXliSLph)=hlC=<-Y$6>IzpQ=IT@#Psr_ z+ya{reWG4Am(K{_gB^`!{A~7H+bE1%kND(M5dE(!_|Z5)Z{C$rWSE1Y0g%HZ@-d}M z4Nu-4cA!`3K5ZpPf#+xv>nR2u+z(<9ykLcrBIgBBbAhO|np3Z=0Ieg&PKR%Yo{j z*jbytHm!6%Y-l7%Uxp^B?*oT^L}hT^1Pa*tYithBkXCE=c69jTi=g?0%l-dH*f}kW zqAbdC*tTukn8UVh+qP}nwr$(CZR@W6c%JS*bVo;2WoH$QXYLQ*3m6F>kwQd-Y8DXy zHnKHeS^-2e04aOTmWc+i42)d>7h`tLvgD0=z*nX?7&C2>aL!Q!KAq?&VQ}fbN$%LY z?p04m%@WX`g(~A^1gyop=qI*vqD5-GW;*rcI|_S9u2Q=zjL!j}RLV>WDb6S$e&GUM z5H#A97rUXIFzqBKQ2`VV5e`^#O=8iTp29AatD|b;LR&8Z+`AO{y%Vqn1pAo;ss77;snbSR>symN5T z{zg@O0Oh+llVM*0ny^y35=~r9)m_0^VKL+b!deq5B6@Zd_=FNix1M&uY=X*`k3Zd{*Wq!Q|#6f6;udHm%8#ko}xRiZJ{{z zfi!k>;S*$a?gh3om0@endZR)`PZHX%aa$}rvIZ?I7ST~VmRZz4Jc*K~k|L#SN{ob@M z%^6;rwK<73`2`cICCCw=Qj*xOJ~|~=h`Y6QqTSg$!{>j)_}ZdcV$G)m^+HI~2Vlf0 zEMP>C*^|lFnZSLDdp<#e7fgPX^=osFQ#G5y2!xSXgzXbb=mNw-U9203?!#ZSn5|wz ziH3U%bas$}$j)A#a*b$Of?YD~I|fk(7?+?g_jyJq9PM8?BjgH}Y~^+ZsN%FeIvhco zPG9mULI)IL453cVudtEqn#2)U&(MPGlRL&1%RytIIKkSo%tsh}kI&;ml6$Uf_)M3w zTpfMnhg=$c9qUTz7>8K5Qy{yk{Rx#e3uzNh2ItoGtRa=J0@Ss$7zQDN&p>T~b$eS? zI6ZYem@~jh(+=4R4pxd}TZY<9DZ0rmQ<%Mtj7!mR8;VuPcT(U60?{J`_-S2-WDvFh zyzIM$J+6qLCJLzE#D|dqY7P52|H6u*Q8LX;E~kkf8ay<-2V8Mp=gV2nH6~oo@F5Z? zexm&YI736;l3*Lr5J8l7KR#Wpiw^3)lB#2b<+p`hPu#=e&7+@kC`oM8Hw^ReF4EQz z7a%j0o<^k(3lXRqTW)vC6&SnG8=WV{7N1_nk11%23;9_6&895G=hN{{_dMq#xr3`~ z6)PKXuwlMKqxu^WzV_d8mZh^w4XxtDfEk?J*B{g_hoYf859i+w$?qU0nMNRC0UaC?9!uv=GEHU-ZtWM)ccIMmQ_Hrpj3%PoDYam z8vmE^)%3-p74US^+{T3iW_D=x3p>O8jPrWv}$Q;V*& zxb^MlSwtbj{$T-{2}5_KZ3Ur;+ET1m-`<9MT-DDywQ`-i>CsCW3{DpkR|}y{dzw)yD{&1 zn+>%*QN14bzc-z^rJrMcU;7|jy(SV3cue>G=?e~@U%Dg`tf0Yz9QLhM$Q~>gQ_g#i zpzV7GmZUjny&4rqyHWwR7}Cy7G*Ke~Xm*f^j-B0RFT}tZ26#Xie+DjfB7dKQRQ$T8loD?5hz(^2nsVuZ_F9flEz!eB-}nlzHvBJLTlo(v~}KZIK?%c zT@X|KqJv~{!J{CWHfGxaO;2looy_)HAhznPKS_ zePm|7H(1w%j<2bh?XlsWG9S3tXAv=oV(Y?Ox{gMu9G_c5ZUY816fYAjuzeI>z#f0SzM`SE~tzU4#CrrN8ptePx0(fB5@Oh&Ee4b#ub_Xil>``zD9X^@Ci9Z<^k zown3T*etw$GFd<=qgv6GY^8|r49>G|#|FjRC!9*J*ph#boB!SR&g(vC;hwkDNf@hP6Od~3C{)~&y^wt z91O!f55WfXR}^n!h02)JXBEZ#Fcj7akZo)Q$}!45IYF_Q%Bv}gVK4=Teaq-}XM|bU zWFdx#9S(NyneMiGcB%?`^; zRdXCdZ^5+u69A+wv=-Ffy(i(@N*j0X^t6 zwQi;vG;t3{87W`SDXo--=Kga?t^%If-FMNvKbt@|D$0>g-7z(+Kutu(f~`l+wo0}G z%#4z8j6hCc^z!FiJ6|<1EGFW#_-~rUnUT}i!ZATO4ZyVW?gg~C)hX8CjwaVqy~Q<< zGB8e<*=(tlLJ6#+`G0zx50m`-gnN!Y-Gv-_5Bqd`15^&xP){(Jb|MYmWN#JgTy<^3 z6oV-|tkC{?#LHRGi>;zU=%rI6XW3=_Lgb|)lTJ+~*N))9pu*404w2vo*^zKIe5r-g zKd5KL14E0&4A5`+qoXFwo@DRL;uGK4Cd|W0l03_I29(SDM(q@(z=B)Jte*BYaW!!i zI&-FF=E1NunvvkTYA%qAOACW8ms$U(t9+@qC%O5mEZRV@8kSY!Bn4% zcb6i+)(vk=Q?hM6zAW;2F2j&+gxn&#%CD%74)Cx`e4Bb4p@!c zby74oTJuQ;8&MQseBiukwx@gg0Om(A(0>sKS>hB_fiZ@f-dPlqrraWY8*LCw+)cBn zbn?ng6tmNnnJ*>dRO$ysAH|u90G9IK)s^Aix^=2ire{lXb(bqIpN9Ic$4k(}zS5Ib za)3VXOCz3KKhI5bx&2DW&43+)04dqId#_0g5p4!lL-her)qL(C0VLCM)T@N;mnc^!B7B_q>~H5kR0pXLE$CNk~I-yA-sD2*kUdokek9o)afmqjyP3AJu${KD<^oQ zMQ_%k^_(S~GJ~R1xI?-MH3uE@oH1g=h(v=djLJ~^4)IZFg+TKI)9vi>PC&&ZPC(80 zacm%Rgv|X_mC&-nrGj?Qr0cGoh#&3_)||TeI&naGu1R*rgeYs0*w*>!vH{{0GqTTy zsw>)$FEBdt+7SM%()kSEHa*&y7o{9jgrvr-=23Uj6&i`9`TX@VW1SLZ*PQw#-8pLn zU-_H;E+J8?rV}5)n~;-JgyOLF^%2m_)ojgS*0HypcbPXmzK^5db9Zs@88%_R7mi<0 z%^2RwvvTmSu{=&nhS|AidV(GZ2XnUOMoNC09V}}*qDLbD_0t&G?0}m`nwL#XQzB%S zD{w^VS}p8-HKb>7fg6}gz>5SV*5iCz0rl7;JB|2S1<^BJtd0IW#O=66$18GI?jJBtTRYbwQWvu{gYrP1j-kjeYbHzrJm>!Bw zl^wW|uo4;UVjr8JP-oa2n7GVCe$&$^e~exn`39NSEwKYthjkI44l>~r>;Y(DOJT4z z;hIi^^jY-MX`$1F7N=K5hH^nv3c9AU0lob}htDD7G-@X_3*t1XyDBuzvuje!d0SKn zi`SNvtfI03^O1FFsh@qqpU&w)*W42WQ5_POb{ z)WMKaF$9B)`D9yAAhgGrAaCnGovEED=L24vm)Rb&?5}*OTqe0oGA){yc$RMPFtx9$|ZM45_V3>CEFE)_{@zMsXy zT^X9>h3`?+#&AMDEzMsu4Xq0ePJsKkMN?ZVxUO3 zJNf*v4G(HIU$ zbI{+Z(?+iFjI=U{lr|H_oATp{o^PG6Gh5z&B5GMZi(2P7T68W)diw_O2*9aakzOAynisCW`P@H;eN$^RyYQkf$(BE4CEUHOYn7h zx80C%BWmZLOALmgb~k9(a@|p(+=PaZxo^ypu_(lu?+RfcpV14Aq-HEj2W&CfPc_i7#pua(rD_#oZ+{f za7$41*jLAYgx86W@!DBkr^fTbP&Vw2pGxIXIdUbBY##kZ9>D96l#xm{?LfXFS&43e z?_v4QWj~+zaxG$AW|PAKTKETAb9$0!Tb&K^!(qevexHxO8ZKWJEm*|uG-tVrHTs4` zX>~L4G$g`EQGgC?n&m5E01THa^{Jg5I%5z3C^SeGmBATN;;r)Jpg5EIvIEXwC_T)vBOUbg9`5u)eqAv z8DW0icY#)heL?&Y&!CqRv-@ElxbLmqu}oqeOQ6avWp^nshcey!o%F0qHrxQaOii1b z=K$R#$F1t8T3Wp8W5)|m4q3q<#q1}>W=UFwD%xnEjiin!CG>!#YxQ?EEH?RE;(6sl zV0jE47?}5(OJOCubNS(I6?aJ+Re20EZ6%+No)A9O$=&dIas8vypR{}!D0I<@_lx2k zzN5rKDWt^pHpG)0De=0A$94D&SUzL=<%EL{t;?wQ;wDl>;6aGpv?oE|_WL9(y>Nux z2gD^{+NIcg3C{2AIO0=)q3B1zc>Dm|@3?S!_Mfes4heM?d~g-n#(ym2!gG^Jf#DJs zX@T=B%t;^PI@SGEsJa|3GXI7%=`EAtQP+CYg_aOh7Xo|>jQq>YA%-n*)A5Ff^sRV8TU`Qq-iwjUGJF&?nx<`;z`{?{GfY7nZENqIrGqR&8*=EHtmmW1#lgOQ z=o@(27W;(h!hZbAW|U+;4i%(YIos5e;RcB?2~)9G8|e{&4auFOyh{H*4SOo(gU*BS z3E)1wO-dXK{1=dSU?;2(4Q>VfA-&h$@bvX0hk#fn$^p;CHpd}e5E(FaK$Hgyf*-3Ycv*2ts!kM@5rXMLmZE41e8k1zz?3=zOlx`+4&k}p;ZLNPFp8?-Si{svlW%KD2 za6MowhB7&VF_|2k4;WuQF)$V_p#C#sVBfW+agm$ziB#|n@BR+@tfcBcOA6my@K7(6GeT?37__QRk zl@Eh=CnBRP6}cK2ZeCjn!bmSG(sePI5JWrlVOvb$;o|taH15*b_62X-=+WzRg5Pa{ zZ!#L}JYyT9hH0cu5RqKXoR_rw^ZwkrN7$?#`|V}feZhgIm<3;BihJfG&q%rNCc?Df zZRp&73VM<;bLn!bq|xb=js!HeCL8)7noR_qLII z9s?=U8j4)yr=5*fab^Zu5*Vl4!aa!@xWjfN5$9io4?=T~=S(GTpksYzUcvN{nwDym za8KOuU~LDaZFP};e)O@`O}}QXQe>UaJLRI+jY2Ntx1%x*F)hY_c2<2@0*=@Xg?xkY z5iwh1<28Ku8O=BP=6j=P%6w90-N$OMS^kw3*4V6> zSTkhRW7PJez4reas3nX@<`a)455#_BeA|04J??)?#$$d8v9w{UEe%OzD@sQJ{7t}@ z99>!SXJ8$F$ddij_-ENZh+t$eoepOF{1Qy1xBl3v91X$!4UR(rh9o($V2tzv>ifbt zpx;5y_gViQNyb;-^{JwYdEUSiOTK{Lg`;)^Wfj~u5$_~+BO=` zaocXBSKztTKWhW*z@8dL&PJ3Tmx#+2$n9{5ybdX#euV7&*{#RQc#3Qo=t& zIGDkE11QHyn@-pE@IDS)82okb!r`PmUjYU9EWg!AZy@ zx{JTxaT07LW5|F*?2dyew6a@6`I^a~Q+!mqk+O_!OzsOV{}Qew>YL zSF?Yc^zT~pB2+3=m{@XIg!i_zq?&%6vMa7w)~r@}a&e5@A;DJIk>*5A&5{RIV|&1l zs6`+2nm^m_OPYw*_&CtVysY%|3m84yXvhsziEoY?pq~W?YRi({|6!%}Whdq~TJrfw zMA<-07~gh^{E+Ie9YmkJ3n*gwQx5hm%(aGoYN^3);!rukgLSKs*z%cRf>dr?({_1Q zYRk@I#3eND?kiwyB?(h03Q9CRp=a}^xMYW_BI=Z?R+{F+E8^3Q{CLN(?iq0b%>SGE zOa_uUkfBOz#Izjmt$fM~x(kT?m5KfFu9*Eg!<8eSp-xzmd|J!Z#Tuu#Froa;^-{pH zq@{Wo49*B}gfiVaz}Fo4*6lQL>an#8eIWti|6+(Vby_a}V4ERJtEb|Wy)g_QzrWuo zlGgY%C~{4-c)eor!AlqVI5jBb9s#^Hh2UdJUv?>u8lzJ~n54?qEYl&m$mY%`7;cIe z;)g)m__o~y+jl<)@en-t7d!}Go$BuLJ;SIkX}u^lUcXogewY1Yx(V5$Q$U;1f>0Xw zWA|T8P7;?!mdVrON|=6HmT{U|?|=zt3BKsMidF(ph28w$+57W>>3m*#%3;qM?wl$7 zNS)-;P0fY!C8Hz#n4DK528hhqgM=yvA4biW^R%exIkvr35TIq|=RBmnYy1E~TfQY7 z6Up)#P^&L9I6dC1A}`y(0mvcxEzUb2_x}}^O7um>Gc`_zOE>mf!!utJo(ip41FYzY z1nf_`9Bo%|k)nW;g^zuMmme^>ZM8B%Xcv0}x^!nNduN-@H0#i^>b;#c3xuU*a+o7q zgU%FuN*RH{3SMQN+B2HG_H@;uwTfhe+;phKZmh>eb5l3b?5n}SI*&|*8oFR1x6v$l z?9e6p{#)_1Z07)>MQmjhLiaduTloZ)7`vyn1&CRtnL`j(X1cjS)8R6z+k0l?+kt0Y z=}@c>4Flh^Ia5w9c}%e1V{2KH+ZA3w_Cq4<+9?g&XRNHdc43a@2=~?MT=-$^sd^e? zD@YB!$h5heij#BOO;1`LwJ?p&%|L#6j$sY@rJJr$_2JnsH|1z1;ePYo&bH%FTwAxQ zU4i*Dz^twI8G=${8s#S)V!cL&nTt=lWaxSLO%-l*J`teSKL$=S+**wi?W5%e597yB zhvN@BkSvi0(ECjUy$y#q#^);gY|XGj6sf-pej<>25arzFe0XEbip>xslZ`#>14f&M zwwMl~X>?P&ZcDkEM49zDp)_;faGb<6@S;p>DrhU&;FYG;RutwjNYhoW>VhIHPIvfx z6?bYYffD*D4$nk$!Mb02aJwyPsf-^#?Oel)Cq?B^*GcrM=W^y-M5C zr8Ex)Y<*9}f(cE;4ptR}yNKs?W?)L}U{7bZwc2v<$D4e;dXp3b;XR%gA5KJUiuOr0 z#DfGhPcuw6C0UmRD7r3(617j z^bU8L&m8OL_nTs1rJI}k{qrUK>EOK)_Go*bbBf2Yz`VhKVwY0TK1B$?R62?nr1Eqh zgjWbuv0<;!!Whx`3D#@o8{LzU)~#!vC>6(xWD>sCl*Uoi4%+Jn+hIJ8HHg!bU5mlT z(nWCK-z_Wlr5#6MY8Y*)eZg45R;~Yhc^KZ{zQLHD&vy{bq0*Q9vdIaUci7dbptbYQ zcMxuXI#Pr}H|$#XE}X~`GEuCF2xw@NJC`;VA0@RPxHFzmR z_!pxzR9+u$f0bDuOj_YdBIx7CgZ9~IM^)-26pNEULs-UwO?+xnT7Zs9?>aX(#qU#- z|5+dLu`hGnOSR?bNg-C?Df)scvsMwnpHhd_)+&`49a8(+y`}KoI9cGFS)f>(RoR({ z<(;0IX~{m2Oh2BaTf;ue)NbiXU|5MR>xJv`jbfl^nVxelTy8)A0wdhC%%?YBK6dSc zf9&?6%~}lgpTBG(y?bk)dgcnSc&muEj5`C8`aaX)c7_z^;Te3LW(>tiq@?h$JHJ*< zo!(fj0}En?oaM#IlsYtG2JSj|B-6vko~uw`nh%(O;H(e&d2r;n1X6PGXNs61djGbw*0e^H{(MEw{K=FVm3n z8MAVv&CnJMOPwIng;YBt{n?X6+uB}bMD`^mAF)R&TTeAtphLa-*<+qbRtv@E*jX9D z5iVXHyj<`5r~aR}`#Vh+>8s|3jD+@T@;|Ion|*b%JQ#*9Z7(cNv{LNa!{j$;jOj+O zwpjDAgO>pmR!Ob5p&MAP6~@-Mj~4XPXO=&qwbcpQaD{!45I&uz3d|9_QZxZ060T|; zd0%1hC%jyKb67ey4*iM_J0YvbX;HTRWh@l|J)a2~0)ZQfF*m2sHo4RtrfX_{upIW4*_9&8$Ahgc3?Pd?z(%* zVlU2`><*`0(p=b{#H2?`Y-fxx2sOJAZYNQ=xts6V@_H%xq#(;z;Iw&5^BooyPVBi=h%{5jH(Sm@^qWVjybcz0C- z=BR`0^&SOwlG7WWJRa_bZ8&0Dg6YP6!%d}0YL~(I1ym+_jZ<6NK97xqGH5;7tj%QG zI84F$DR*zKEE%M{|6nGoPfNe*e*6j(-V^Pxttvfwqb+==e0FTM9`zv`n-N&#?>PXK zN^%o-X5DwHPucJ7UAzn1;-nRQ(RQ*v zjAJ@x<57hdiG|2-k#_J2oePxpvC08@%cUPYd#^`G@X&0_I5h12!A0f}4p4esT;ivoc-t@iCCuDoP}Ip7mYp>kQ*-o+lFuv@Y`qbociCc^YAcKZ76h|YFdc;IE)>9 zrOw!mMQYFukH1z=Z_&b>NI$0fMcV^*guw~wdLIi^f^I%g02yP_ar7y-zkDAihV9*% z>#f;13;okTu8uas{2LRj41-D8RN$*vd$g<|T7+NUz=lGJ48Y~j&w?C8#; z98wnS#?aoqk6&o55>A2$a#e1y-V;F!suR|@=3Q_+ge0IMY^$Y}NUP%u=2`_x&|&L7 zb}uaAd6|6>J8*QK!1wIkv^n6EBnuY=LLu)WH?98yQaF}(+Fi8Bb5Hf${ATo{aGLwk zK!nfgQc#BLhh~RpQ=;Fem(3~8&LnrZ`Z=RHDcW!a zB1&(ilVYSmc3(n%ybTN*&=mb73o=g@7S+s;95}cA>akqzEdhEQ`z zI{YFDRrh>T*3Go3w2X5oeny){dcv%zSQKF9YqwzOxGcp$6fkubB*(=%?e~FcnDeq$ z&$OvJGn(Sxb&v3RCZOEF>l=`(6SM#ZEoYR z1d!&G6}O-dHO5>EU!{rc(y@h-O;E0<(32tQ4M0suzPa9h@#p(iI|5XjPmWDanTe$L z;O)xkO9}X|snV>v%EJ26SY)sI=xzrb2g$7Pieo($BRdD;)1(!WiPt4#bZ~cl0dk{H zQ(2)7EZ@y1ZZV^z)eLt9AqPkZvG}u-B8H)#L@npuIM}XEDtIdYA!2Y3K!rO@kq(oU`Ky?nH0neARnONBkiUt{lcZ+-mt95FB~N9DrPx9vf4>$qY>-^9rQ@N zPcQYjMbUTjp6=7QyPNWa(sxhhMGew=#HyNAY55%E=lgKvyGtK(s9S~=dpnKud6)S4 z5|Kz}M1<}8x9zF~6f|PuZx3TPx$?dVQ{7E<;2wWs)>=t8;OWCkE#sGB2f&r5MEW?b zE)k0lQ<=~OI64FO<7nM`oyqBZ=;BBYfT_a;+PYFII!{%T({|@m{LebwV&J5Je_ZsT z0qEF$2@+i5pH+I}pcUl5U&5P%;VafPZ9$m`y^4u{E^dYc4?32_qlJ-{T`%l-Q(36x zgoV0-4f!PrD%LAN7zndbUy=;V^>g+S5T-mnWx(Xc7`Qw&6RYUjdp2G*8Fb(9x@sxV zP!J={+H<38s+7q0$TE4_d3pWJjVwOecdlGMoo}z~1R_r(l^Z@kqkjL$U0Ac{V3PE< zATS|Su-Hyrw95=togs;J8AmdcLp|p%|CC@)yh^o?&%%TP@Po*+Z_@MY%=G#XRvdeE z*ZpxHwSdWOK9=5X6ERXcRE9>a$5}@Fd;3{o2GK{}%Ya82%w_%8F$iaJj7N+owZ7XR zcCGV!f4zqZX>oWDbX8=+<|-@KJ6MRfF+Z|$EO+P)xBx0)=)h}lFh}lzm}Tw&_+vSd11VnwVH~jFCMzx{9 zgN_^-gP#G3k4PHZA*IVO4AyH0rO2;FE>HP4yn6fj91_SCRtROQRmlvsK zK#fRQm7SoCK_jK)Y|-%5EmK49_PqWUne};z#c3Q{sI5G8!5h(^;-fBN7&vu25DB3y~_~xc1BpdEkPV*IDZ&43Sa0~)yeb2 zZ<$%x9DLjyAciR8i{Z&8r?JK`|7@r;`;WgZI09gMzp2hNVJ4Pu-&`QsxXXY4 z7}dvbSf2LTpzRT9=DmGfaBboaiGpwu&E(JV1YHDLiCc4ar5$BnJ_0dEVFC%pyFo&P z3rhu`1vLZXnH||T5YyCV1^u((t}7tq-=CY@5|0%C)kf^O&o={GgHITG(iP@X*;SLu z&h;0i@oW=Hp~QBI)g;lF+>t7HEDp~0u9?#6bsGePnlD^ zkAsVPPo*B6k&O%Y-+W1sLW0?TC^#f2f0I(PuI02MWjpAITf?M<7@QXAoD$9)N#6VmIziIU^j9LtM6Aa2DuGiVI5ir;se%*w#c z%;2ND!%yx%l#lN}mbXyyAElP)tRbHwr#lMO(TBXO=Yzo?#{OgTpvM8p0kRsq9iJP2 z80Y^z9#AJZTp1AR_c7)QRrO5kH33$tF!UkePM`^Ggh5*SAy5TvBmbt+V}6rI5|sJh z$tX-Zz;Lkli0IH=sM%#wu0N`-blG&Ol?)x2&F`}yZqwdYKHfBT;&_|=@>Vq)To)aV zY-ejv#6Uu7W^qDDoo3V_jvzA_2LKTP)D(ulZ+azA_?JP~-abJd>S&nhUQI0^)O8Bu za$7o-E?Hg@ULpsc_n07zonVm&aabf(W-DS0_2*ronAU4$4@wK(g?3^_4y>R&Fe6+6 z^>f1aD5e%G{gWH?8Un2cO%vK8Nzsgo$qODc(lS&87>1Hp1YB_Ay?O=9bWKhZdLG|3 zNy`HSA<8rg>81VTc-^T8_zAEIPj&K3=KL zmD{|Dpj5`nj_Nb;)ReIG;ET*fBr`>lj?T^{$lLP+dZcfbU&+Yt+_Oz-Zjhlnm!&d_ z%f#&Gk#mCt@G^ts#SVV#BX1IY80m#B!lhuXui#EExDOuGOEE9rpEfCg4I9b3qf8-c z(Jk*P5ik(Z2%nGAPZj3Boj`i@lT6;Jw1f55XTk#TQ9lWd$8B%x-EXm#lU8QJ3ayL1 z&Dzu~!B2C?m>+ZVE0zdQsw2*A_8i&CY~Z?wusKUWMJ4e3c>ENWx}STW>ZUbWzhNdN} zT_=H|;YA-6eO7MYyCI%A(d&Vy8OrUi-CY<*we&b9twdDE)PVYx`E9+@u(&`OzTuNT zAV0ONWmzy<=Aik=b<_FT5KlEkDBi8;Xi}v+vHg&Ue}ER|04UvIT|imBRf(rs%-Zo*BfCgHd^e(7VYSxG#z-*~PoOsP6x95H zByrQA^u*h7?IcQRC}^8;^m?y7@7zu8aj*dWM;KV|AY2wILQ*-_UNQp z9g3cnwDPW$4{GhY;cj(nZ(9MN8n0uq`;+3paEYjOOAS_t0Xzuff484~|C5n+0*8CS ziV{ubI2l>)_KQ$RWK=n))U2mGb;lj{-2rdC0ciD>%}V%bkg0Hbz~^o=8y6j(i|Lv3~gO#RoR2H3gh_h8#SuzI;tu-JlsARjv^7 z(8xEy!2x*Om**h=d#GhLldk2Gbjj|s*OVWC5uwN8AyV>^&c;h5c|H7|LXLT1zxhfh zTL~sCt?m5hJ|YyHoGc0sf?PHAfK(s{c<}elv(62V-v^m>z-%f@AhSCPdG0l7is$zH zFKYmZ!6j;{F_#TR$63$4_W2XJ_QQC#4I{_xagm`;LoXq@x0LXQC;?uA+U3NP7sg3t*naAQPC zO_9~^W|etP7nL9W=wjfbYU$`8;bUXkJ_~fxB38IV07Jjd)Xt0oyGzIhwk}XLB{!Oo zbY%?TRHAnN+|gQrbJ*+85bCw?0aXHr)T7!g)2Q63nQL%+25E@4T^&e7e6Q+FeLiSu zDQT%DO^LAZ)&p(^QeW94hvQ`Y>v&e@U8Zp4}y<0V+pH3y2b_N^Cx| z;#nuBJw_4lQkFjqb0xOuF+sAQEhvaxWxhcmcKh|hoOGv-4HH~bw z3jCsU5M{h{V4fe8X1|F#fIVWl6%p1S1_vT$u90Qv6)GLZ0%U?D%Elul3CVTdghUbi z?jMaKI4pxR3u?nQ4m1W~*C{mqgBnqz`yRyHi3~M zRCi9ZwyvnjDY*Xh%k+e~M7WiNesx6F)IXSHdJxTEIgqe-MqoZ1KI+wHKJRTe-nIkj zrdN9X;nw|u_>dO5NPk^9ZJQ+-dnAWT2memDx_c|lCUze@xwO+!=t5Ar|+Bp@$--7YWS$N_L&*4NpfNI!-CQtk&G=>aUaTf#ifWm&8U z&j@T@3fLVM7G%FC=?HSyY7 z_|AsCe4t(QI|(2eq-Y38kJt+zhBxc>R`B6AF8!s=|)p97y zg~93*+nr7J-T#=U-#oLGF4PLL-1aLt=H@#-(MC)|mRji!VTkEnkVVy}hy-<2^<;`s z;UTt-!xQ7fwZ;@04pxB$^wVT)fnl7K5|1<(HI*XUF~bLFwnVHXir`4Z43V2^XL7kp+4+G6l#BWaHdn7jl8~FG&v(y8fz*X*(FLD{$gR%`O9;R zKn?6l)1;T~n%foKR@%hZ_hDI?74}XGCzahMi*{q*uH11TbZU+(o0^T(YDfuKBot== z;Fn)$eJXW2bkBvFEGfLbDcVhtLw)RP1M_`!6Qwvt&0%?=B*w-hUfwLrr#YdXaEf z_s8$j?th%e#XJvKjh^FPZrt1t>n-Yf$*s=G2 zfdALa40xQfwgd|R;2;J7fbo9>$p5Wa)eP$I~3xm%Q4BydHi=nRpZa;{cQ^)AdFR=alV%P0^}_&nlp*fb5mqOeS29y+i; z9m7YOP=Ce1pJLHHC6Y(UnT6v9St<3zqyMD$ zgbDpO$a7>c`(OUTd_Yk!>)2`<$m~n{Z(ThcB(qi@O&=$!c5ci7AA*O{pJNW)@o%F^ z0&DU&Q1rHJeJ$4yLF&fm}JK16sM;4fM#_YCu0lVq$)`<~{AnpK%(WwX2A z)jkZ56WAM&*T*V*4gqg~wEdPEY}#iwNuu6AT!A*je_uS18jpUUHKRS6WSuBH zaRh0Ca$P_O$<%Yn0sL;&CbTMm%WJ(Mg)A2G@WevktxBA3rsGV7WB zlxYAE948Hr*-<^DN?5V-o`tJ)=GF zfyN@Ozz}_9gu*m#HvhS;v|KwfnKnXUj*2in(G~dGX!2*nB@`}@*mv+$u9H_X+02~R zxFGxi<_6UVYl0q=l!gRgCK!h>bI?zk5cO8W0)vkEG~(B!4L(z;@AlJ2pX)}@3BGyS^Mp8sWvsnAM@JcA;kGR?c5+`?PeV>rm^1LNp z1FsBg^LzH19#(3*wo(m>q7^Ji6xG6g~u3)p5O+mwi>X6vp=wEW4FcYFfR!d|h zGKQH-3vX)qmjGZ#u(>-ce}}Np8oDzDB6fxhvY$Bh7234DLQw$hM<5HF*20=WF-T~d z_9&p8x-P$)raCV_m#_G`IgeYywzrY79o$~+k6xiTi63-z0%6i5u!@53%E+BMJ%LZX zl5;wDwLd2;QoM!_TOxlj8wmO78O=cC&g$}DJ-2Yl`^ndQn81OHk8>XeoM`(s5Y4+G zu|6q?ow$&OaD7j|p7{L|{6^hCSM0e}8?DMp-y9U}DJV#tH_T0sI(L zZzhXjDDyj23&N;<4Pw@p<#QDTHl}hq>=^z2!X`ukd4;-2yd?2Z9<3=g0D+Vyb6eBk zmm)H+B-_mjf<_DUCUy?>?NXa=5!gm8)?8X6uPwoo8~--KSrwb*VCR+#UY2^y>dQTlehupYr~0R-@4xCSO?eOa|q)_7uhVoy*v?n! zduY64bu*3xk{&40>GXspeVUX32G{X4MM_X$0b%(r5RO>2=&k~jIe_DVEtR3`DS^_{akv^%{Gb0HLkjejiy;wXI&{Qe5CR)G-P&cs) zNKkj3m2_iZsey9=MJDG;JEq8jlJn`bq=#Jnlz8Xkjxn+lNu{*1H2$ejpT7qqBL`pm z=QkG@2OD<-8wVHuydO&oo26gq1cUcGc2*ZMx!A?W?E8ed@T1qi8O~OgJe=h@MeZZc zmt6b{lJNTGHG>}qBjfUq!n#q@7=tfUy(kRAgwxc&bpc8*2xCf2TasLDP!PsAA)eeI zS3~d?w9Bre__;Xvuy~{25Ga#33{ondoxPp50pU+~?L1bMxc`NGgJ20x0G2973!18I zuQJTkLMI{hbojZRZ zLKG_r=K;?|DzP?Pj)nIrYO9p8q+FFlDE=HHuX&*gz9}F=G*)e0Ito@)KRb|$;GjT! zaF3Hw&1}GuA?1Mgb+Q;GK@bB=V=BWY@w>-yc)X*yR=n7XKvFA%_fhFE3=C)e=uo;V zaxoi+E|FZYR$WN1AkkAYaOv}d9il9KU>zc2K|dm?W-t`?@V6KGtZLChit|a<64|Ti zNgI^B-=G_v!H%D|L>KwWwp@G~57p6oZqfIA!eSOH5_@u6w0jp7pILt{eouXQru}|X z9WT0D0vOcvQ9>kwL@c$crg< z2?-`NC;f&P~(RLuj{T> zTCW;sQLFqFdEe>O;nmaVBYsa*hFx}21L0?D`!$3U73N_T) z`Q<+&Wx{^3)SaY`OKzEt(MCanHPF0Ym69?d3NVP#PrZS!H0$YggsvX_KfC zfxUe|4$BkUQ(L(%yNCzXVi<}j#`*9Bz9@goQf(IBJGKj5=F%~%Scqty!>m`zX zb`xrsD13GJ>v(MEE(1;RepLTsq2D~z^16OWzhART(pH$iebBW(*Rg6Iy0$s7w&8Gf zvoV^V(Fv_l4t|0A`%mI7P#N}~2mk=Ry7&AqmmCutD>FM6XEP_r|JyTO80@^s8^8L8 zx%whB7s69IOsCuMjma=+Y)0|>VNH?EryZwxTqT9rkyPpiS{`!!gXhO*PfeQ$D8h5h;+L`ZXzp5(mjo}47%lfG=WU4h$R^W2b(HVUWnqe?@Aw-zQ0 zAQ}EncThNc`t$cn>%G+V-yf+Bf1EWGjKU&-7s~CnNrg)1J496usm76_=&KRu;@Bhd z{0yd5oxooFd73*#?gJ=2FsMuxV0cnp!U2<>!@Pe0>UxO=e0^j_Rfv9`zy!U$oYu2(6jLs-uIMjkBZDxwvD1^)-O&R704C9Go{k8X?b0}Y@bDXr>%)1tR8+C9zP-d~@%>c5=Y+z&Qj8l}xjo0uxB&}2r&N!9YB(Q{3& zSZs>}n9lIcS196@OljzQFQ5t~f?2vR8v|sh!FO%ac$cHO18pN1zBx0l@OOPx7L+v_ z-c{M$3Yg(8tZ6OwaW2zN$Q!YhYw&DvIRRUhhn#;>@xZ?v&_OHgFJ8&(gGfsb_VyUs zDl;2mms^C;oXI*Mp|m_P)zQ^cjY&Axyb;98Z*cR*H7qOH;Y%4+)Ub&h@%!Ofpvr3TE&3;YDsG zVr>=~7R57zVhjSAI;yQ;LTs_#>$^gj6k1)bI= zbSKvl2fmdGDB_w=m!eC+A|KWl*D{0^GFD46i#8xyXnZ*yIx39aS7j}OCIm$LKkpgM zNiK~VR|BDH1)zqRA3H>TjN-75YxjpO3X2B@$-kH51#3qi`h)hqo%2&ERQdpFfHoFx zSF!i06`eQioEAOGyD`1LBy38rQD_~ezS1EG8X9Dib3iI#6g_ucI=N02irk5hbb4wm)4*P}O!{Rj+tPVzdk_M9^;mhzA zSMQ)tN z-D9N^dz}pK0JRfL(7SIW{Y=OLCK@9dNS3nGVMMYN&_+Izh{h8Kq(FMvXnF@&N2afw zWpVAYi(q=q(Z|BBgWfDuwO`WN*2P(_o=n52)i+P%=7Hr}uxNO#nzX%J-*cSXYCs}R zHbMOy;S<#a?&_}BSgHCm5v2%bTqe{@1KN!t)4NPPV%E$|@)c{>AANN2#3J#Nzy7G^ zzl0KFqf&5X#iNB4+_zLX%Q35|cUR$ld%Bt-B1bCLzHyqWcM{010HeM-JJ)ptf7wOj zs750})zoevu01SERgr0ix=a!KM-x9uc=oL8B=ZINntjU$*DJ!m$Np&g9z$nxUHZM^ z5UOz%&w;9&eAtL`UC6*ujTkqRZmwqDUMgkPFOC682xMBQ!;~BwIY@?(WQ^9)x&%*E zqtz&6fKnd+gIeyEgqw;AJLCW>X)%q*jJC8nD+({u6p0XaN8*`n*TydK(>FD2z$38FcK!)=F6GOuzld9NzKjL#93!3i~YHPB9dc55{8j>-%%gc z$5HKPJsAzo-mbV@7k}!<-!oz?v01KPCiQLlHq=+l>!OOos{CxlDvb5^OQ+uzI@rVn z0PjyXgj_Zl7jv`oTWl!VnZ#i%B4z>?wS~T2vDbr_uG4LTfw|35Cq~M;dc+yxFavP# zWK2O?5K7AOA_$h`AikTEX-FpX0h^g-Z58p4x@J|D%f+_jE!=(O%9F;l+<-j}ckJ_!BnZ7&sHhEtKt=gQf{uSO)ealU&0 zLV7>IFYpyL&xLXZG+LV!$M8NfqKnd20XO92tdz%bu2PQu&zCojlze5V`ca9saDCF2 zP~*w>9GTD08WPR+1W!-Zy)24HdjUTJ+w-VekaKYgyiK~`ja5FyCH(S3rXf{-$lR47 z;)J4zCn&ov+YJ{HNE;C_-4$v2ksLIQ)N7L-l=tTZO^bGK7FSg)Wt?P;g!&rQ_sK=bg6Y-pHq_teR9mk zlHWz`w9ITa^Z<~vGR)C8%#N(&^;mf1XNsHqsy|9k_-zac74)H@d{9*xL_Ds77?hOR zB?X=AC5313=n!T)rE2!ZIgHV7_5fpE7B=fzAKWaek~GvP8`C?&1x!}JA3aFlG%d+i zy(vM+_=VTNPF!H*AjNOdL5k%CLrk$FD|uBo3}hP-n!eL&FF?-E;r!rtNbZIgBFVL4 zev3(08C!!2cmlm6z?pif`|SUws8s2yKr)2#b8BsG5lMxR_p z#g;PHZXzkeloFJDUHo~%3aF*m)~(XmqsEdN4`=g;#Zffm!iiyc&()2k?kT@6eNLlmqRSN9k@{~jH=M2u#0FM$HvrU*yq#@+ z+W}E{beM5-xCm6+N_q;fp6+GxmP~mGKe?AutfI}dcFYpq7i3irza`AROS@cL)5tCOf@y~f0f+{XfUwe6W3MIY11(Xg4CM*Fh3<{f@)EWye_Hqn(NyS>Nw zaDU_HpXvE&X$OWvEOaexF&KVC)ka+frjY-H@_U=A1I$3ip1^j)#QpX2xV&a>^MHKARZK|G zBv|8zs5SO7TuN;yT63X*gYbifFv1U{js-r>nI8ntcCpAF7s6)_ZBiIuzqYsiBn&cm zq^QjXcqLKmojp2RG^#wx;`wHDcq}as0SSji3{K05XfYF*G-dBB?)GG(88$H@j9Po- zO2|457#L6vOyU7RKIV3}oclTom~4r_CRy_ETb%T@^p*1r0xe}l2rkHa~fd(-xt za`L5sM^~f43pozpfI!c4J3FaDPZtG$iK z=kenHW1?VT?~e*rwF6%DxDr2l;|SZ@Iz(~#d^lR|yYY%@st+1g7>49lKPs!*XdtKz zbny2B!-;9&7&>=gB8=ZKMpRNl|UFz-3u>?vB!8z);Kbt!O zo}MgTWlL`x)oI3%{L)&kEnbEFW#j#8UOuUj_oJ@%8|_L)f5O_$-AW>AW8TV>DHmUE z35x7AG$-_CCV;GWgS;HFsQW53OUv!2R}^yNkOyVj*@a%9#nG9wZ*{kd491gN2a|Nk zYlsQZa?4Yyv+=q3`Z7j`3cnfCR?sKq+)b({MY=rHjMltuQz?--KojJLFEcDCIct>h zpht12HvrMOi5wdHNE$bVESL-EB;b5lgOrxY&gOidnEG_IvXIm^-ne017cbdkgTeJq zauE|AEx((#TLq)&zEUp+%Z)^!cLOs1?To}VY@C)S8!wVyyrX;h-X*;$HB@kj>reeg zIQ3vJcKdNVT{}t`?kvft#=~Qs3cviuhjsEnmh19iQ0EwCh0p44L2_4F%E-oY@D6%C z5kEp*E3CAb_A(M-Lrzm!3T6ez=_QtX(DI)*Pt{dl>t}hh2t>P$OKPKa3XIb#FXgeb zPk|UJ20~4@)vXTtbQMt&G#J~wh)q%P&+9V{DYue8WRQ&zptQdkv7BRH=fGP`mv6XF z8!+}KT;?Xm0w5NtW50k0m#9Jo%uVlNAOmt2K%A!E@rMX5A(4%NmMW!%Jg9Xqvi2fe z?e}*(L4uNvJo(L~rPQ*x(dSl~qJ&Y1yj3l-z#+}(is0Emz|`soiv21a=v_>sfUmr; zLp@iQ5zu_-#SPRyN0$`m^=?4i-c(}gC)mV(Za>AnYz5!Xh6m9frz|3r<>=g?c6qfq z?6-o_&bHFbS8&*gYEu>0?<8!Qc^$`aCJxuymuHVv`)ZCY>?xHc)`sUf@`vz;OJ*IO z&~O|JXTRBe(6rOL2_+RP${RS+`teZ)#4YGl#&@7=Uhm4s>0SyB$UT&4MxJDC!j*Ob z4(d={fETu;WUzJI2VWjRgpoHjS=Ii_D9-p3qKFwe>$Sk9Lw*rjVeAGP!Pim?P-z;E z>D;SNn#S94K8><3vngB<1M4)yxPX|Bg zA|l}mhik6M1V_(vrl2alN4>!mqR$!iG13f+`E{uh$2DG0W@TUYY3)5 z^U+PmjUIYnrd1_Yc~i{kUWLRJCR`EsT@VEOt)ttu0_Ql@7xKn@Bu%LyC%arX0lwkD zoF6}Q+@5Ih?;oS!x)85!N=DV3TbYO8GoX8$+;J_f+Px!55h>IcF*at!GbhJ}OvLJb$>l(Y5hft+qIv z9!Uq%vv_?Hp9kYD(kfrafAhSa!T#gas9iN0Gu8_lx^-3w$wD{r1E@Kd>-M9xGW4PJ zaXIM4leicoep<)`Su*MywBF6*S5$ z37M-9X>#&yK4%$Mvjx1R&!{y*;*T4Lg=o&Uh74*l zZjljqAW!}o+S)EN?ADFZdrQtKmPCBUP+abeWMcRvEBz@-@W4I`oS z;X#*qO4;Kl%mmC6_oW6&7%En+7-oL6FG@)XwJ+?>{^G!t0D8Kxz5tJxzVI^Ij$o+Y zTIL_@G;!yg_Aw4Kd8+W&BbW3|oh(kQ@0SSLf=ROPY}%#^6_D(zq4T)Ae%~+KZIM5@ z(LyWo|9mMya|=7>t(J7OakWQhdxk?oR1gx*BG(tT%5{9Y;f*mgPD%CZlZ`vKD&FJE z?Bd5fjfwUXRZG!w>(!w!+4xvO069fG9eSLEh&3%DzSOaa3=qt;lUlaA`{JWUi>%5b zYOZT{c2-B5E&`qXojCBM zz&q%6=~6Ba@4}eBQeWiT4G~&@@Jj7&m=J1_ge>0nWajY*wi%E(VmEA9{M~>u+wddY z7uFVR;E7ze(e9fgyESjZ@V(LD&2%zu4*fB)*6xnXa8jy0HhgV^ak6fABi^t#kFi@T z2MbleX=}0&|$eE!K`z`3t4 zd?EM3d_z@VXqaG)HHT(}E0w}_lvQZ@hlA5>C!7Ndp(w*~&=dCPMv5zL&Kj*xcHkXt z&h{l~w@=ZTfHpi{-g4WE~rnh)5VyWLEM?CqcZojqWIht^Fr)xpX&~K9b2Udb~Tj9k}v2 zRmeos_m*t;WSrS#T!6@pQ?h((md0Wa$UKN!^$bg{)crG@ewGOiZCW>hqkpDQ5e0W_{7>gC~XHPuA+fntxRl{pK z$-5S=Vx|AX>$-DA2^)(7voyJF5YbNM0@*%3J}Abt7*PjZVp$L!j`#~Lx#;o#3Gyj4i!9N)jRm?(uVi<T4`7DFibQ{WYRCnsUTs)^4xp+dsc+LbV)>;}z$6w(GcObwFBD&yZ`^DT4Go z9J4%Tf1)9E$78(6N^t|&pG5leInBiL%;DF`&-jn{jkMvA$q`%>rx_4b(nr~m^? z4)+%DkIf?wL*t)~zfSP4^@ji2FagKn|7baQa9gu!sKW!`b_5=bUENUtFmG;P0Dw2R zuQ&gN`bP^1u#tcf?{8|&q>frIuNdHIZ|Be(*1r53j`wB${0MP#y zY1V&2P3&zQKra6VQ|%!?XbKJhn8XACu>QszetpY72h!Z?wewcczr?x!TmSz{s0f9N zsMr9&v;Y8r`8VvrYmMkXu?|l5F7~grq5np(v_^1qO0OgOsviC~su<;8R6|26J1ZAM z!+(QnF23}!?lS-&?+x=RO8zUC{*?f;|4Fq5K|qEUAQv+?kjKA4{(DXAe$6*|BZg$&PK?wr$(C?b&B)&U|OiOx0Jbx@!G+>gn~T zuevJw2oW3;=*a0ss&IxBxqM77Kbub5k3AD@QszcNJwQ z00@buzA5RyaCUt`9ZDF_AwU+xQ+7u&=YkGIjjv02v2p zV0u~wwg%*~cxATl7w-Xjh2{jSxpRZGL$5A(W?UXqdYeDBn(T^rY?lKcYV`&q_Wn>aSI7eo7B_L_LDj{Owmv>(Ki;Xy_t zaE{pTph1-=lBrrW))`k9!1db^EJiZBC>dV#@mL{eG_TsEnb?FuigK&_<(ZH5Fve0xo zHfYtXH1i2)Xfc9?>j4$E+ud*8t}pS{4Oi@%Q0ryfHng563Kfsv%==NzKm)d$<6MH{ z{SrBU)2T5z5ktV~n_~HA>#6cAQvOziu+9Vfzn?CJ(+hIq+_e}WgX}l{R2X|;Hlo30 zowadg8LLBgQ2n(hNu7_@D1kF-HI5138+_H$4=R`FwIHW-wvui}3JaqCiy-vN^14v9 zk+~cf3I1uWf-lx+CF?gD>Dt|!_Z!yQA`&akE*BU3>eSVhK>I;!$>k2)xBB~XV)1DFi|oLr0eK{FMuuad$5GmyYW$95_u9G>m?q#z2 zP?LY|>uYLifF$|q>#P3uK(?D-R5Oz*#emBo;0eA?=lA0yiPzTQyTc|AzCz8 zk6zWcA^WVyXqg-qRNhLIp09@5%;G65Kf={z5W{1QI&pr}6HeoSqKaavV}X#(s>b=H zr;c+h`I}$B?sQB!MgBPn^LnJF9Hl%x!KSUVun6T#zBOaU=W(6a`}1s}33yDAk0M&s zY31n;9}hwf@@rN+cMg`+Sy>R|zWQ&VE3}TbJJQAnF85F!tw$RJR`pTxnCVaskL&gN zQdkHzd6NhzLl)^ct{+J_6j=OM|Aca$5lj$p6CTC9#7S|rsz_H5nGM7~J=41goL*O* zNv+w!y5x$uYeqK@`;j$UT`AM2*6^2(9};a`x6yDGQS`c;$o4b3$R(UAuNn^B0@vBQ ze7b&X(;+5p(-oTG^@6bviz_lTdswN7Bc51WfNtuVIGU8b>_H*I`TQ6aXbUZ6QfafiZ0%IXx9C-2+~|Ykl5m2^BsqC*uJ#dqL03I+ZzZ5u(8X*Si|M`@qTenCq3H_w30BpPK)E| zy^4wsCK2Yt-b8pYqKu1UrJBlMYr*C|f&4%AL;qtx;D53|acGnd>)-o-#Q$c$i?M^F zxvkBAWk1>f&3;|o|8-7Rmyn0>7Xuw5105$F$1mvr$ngKE3!DO+#|iMi3{U_7{C}#8 zp{=!@zSDnA@E-#6-^8U!Rmx_A0deaB6~veXq8fdynBiPRzEB61LNnLXKVFnkDLhev zDvoX$=FtEz{#~;=0ksqZ{q1-vbl_J^4PgQ&1_HO4^xd^LVu=2&f+{QX?;|pHJk+RB z*c3q_A~co&3|vR?yWVULAj))8VW1r@$5>#See~4PJvN&I!qvJE`KG`QWUbdqpC4oh z4OKd+JR>j(MV4ZPRMOrGO)5had686k8|BJHExod2V?&0!zl5i0+Nape%vR1N-qMcp zqnBwRv3$6QpmL=wc%*yzv9&GmcewezV3Yo>1~E?k2zDJXe!QIM{3yjm9`(Q3l}u*6kbWJsJ12U@Fw3o&jeeP#h4Cq zl^ZIaKvtoa$t{ZSlptr$U22FDWh?nHuT3DNO>-cy{}d{Tc@-$?M%?lRXLKtzz;6RX z^NBoPkjp4`FK+gLp0fJjvhtd}aV*?xe~hF(Z{584o12u5`l!#c^5)iw3vMy~hGAdO znsu}#PUD?l%{xkqD543~yJK2BY<}utdhC|2SwqZ>xJ*&jQ)Q~^o6+JcBK|o=S;*9e zH?##i!T7-6N-FYNKKRs;7hmtKhqVOZ9EiB&%Sp@c69-1x92AN_YmXnqjwhX0N7rg4 zXVaN^YI?}n(Ic=$&@JjkVlB_0fJsL{T&a&1M3^T-ps~`4(yi@Ki}`HAbazhl+cSV;`o4^7?tB`D#nB!DuLV(bmsEapgUrUO_(y>CP zt|Q_g=<_D)E0%2SeyeRIR51USCreJx+Z60Oc=8S{ks4S^2JCldm&W%Y>DbVa$h|09 z6@M*SgS_4t8MdC?02#_BiNz=wGdN4SsKmfZp(d5g5BsM2;vr%xjW|qNA)mw=7Wo)G zcG?hdVi3FZKn%?5-p~tuJgH$-RpRt+y17&7+q);|9++x+#?{`L;XOwhEbe z(!6;s+b;-e`QyrR?2+t9@ytK`o1w?Rj%7TT%)>9$lb$U6y3*xv7;5cFjtn0Q{6;Mrwi zo>nxKuCfE3%$3*5hI8z~e4oANqPzWX5#idwz1_Q(Gh-M2hZX|R#fnuF{8UoBs?R2 z4@LGswbHGFjG9+2dHx2!L1Zdn1^??+mibY2{7`QLm=T5b1LJTl5Qq~pd zh!f;NNoiD=nq)j+*Mvqu8o#pJDlh=15s*yqbQj{rkkb<$bG0V50`H57UHh2WRgqXI z((2T5g9tR&!+u#BxCu(!b_k4ANRo`$!W7mdvl&9ktFyVhji4=4;(Is}>@d`n;^Zb&&r!of4kuB#-)$2ZQ=H6cVFY(HYO2haItEduHQnj)01r zv${@BI~Zw8CE8> z*LXHMaLueKPeosrV4VJ&%4e?R=EB?avU>=R^WOE@J7e(s5zX^308h8Z*R?I%r3t@H z{nshOP*6#2Z&8l>xbZ>`Oe3JHLAJ~nP5&EV^Z=>>;=q0Ij8AFHkym$i^830(~1OO2KlMZ$cwobN&wpRZ?aA@MTc3c;q-+7`wUPTC<7OqQJ z+(viZruusnlnLhpFlA4Y2^WCP6{b%ly{s+!e7VSGpAzqPU-Q&1^JC%J`uBXX36(6^ zA8U!axT~7zqbHDRT^Wk1THzH{5|R-_#+UxR-is@y{*FK++7dYnxrAvZo(OdGbi30e zxx?+nLIeF)P^_0bN+1*AkSW}pM;2T2zzar_lB%C}sMn{QF#j}anQ_(V%M41|uo4l( z9^o$+Oa6%8Z208&W9GRdqG};6ehZ0n%AwKt6MG#WjEd>#w*ZpI$?Rt5-x5hty53Zvs~-6@i42>!T!Dzj$)Bn=q~q4RUTSaw!+ z7MEWE@w zRLJ6U#&JU*@nFFHNX!Z-d_D3F5y*r54=S~vZ(yS+h>w-7)FC&A%5Gu&kQUcPZi#^% z1Ca=SQpva%{Q#$2px^s{HxOdc>Mtl&_m1+6SZ2=lG-3kxnE_AySWPv!&;%|Jv9j;l za2Mvr;30Dfx^Kd-2-48mIwSUf5Vm0hGWu@%%zpnmIb2(d^zD8PxApYf@5aI9 zJI}BS55dOE3*kHB`XGKLlfb3Yj&6JUOM}7Jb7Djvn`YP-YO_>LVWchlukQKDMYJPh zn)XLvolg^}K|)8QrEZS~(c9jsiCvYh&OUZA35e31y&miLA^}20n)?WrbV;q<6L4zQ zo8wn2n2Tt`Ip;M8d!W{rIQR7Hb$fogOXf!3Eni>?S}-2Pi3dXkI$V0|-AN63%DW7( zI(c7|uj8KMmMPxJl#z1OPGIWsw-ZB-vco!{sKqb%;^QUahZWM>uBSzDm;;C10F z$na;;z8c&SG0ErQSLK@x$D=8;7guiPvFie(9ZN=^;_cDStgtUJvmL*2D?JP=QS1a5 zX7GMcD)FqJHYB<_89HTN#SadIo|{rPQrwdsM_U_K?@5(gcm=0_R~$C55(BygPlPl= z11Zpm3daCYsbroq8#ZxsMVznq8m8s};w243VFwX(4?H4K@8#0gr^EdQhrDMgyb2*| zOa8m~UlP5<6^D!E5l7t6VCiC+N~q|r`x;vu5-nULRULwS4p8pLsnMS*##bPSubt3u zYiee0+-_XfqS}4MRJpG{8AGcO!`wiic5}tclsv@{y<4)&rnM~*!|h_tflEkneLw9O zJ-HU3A+*^hJ{vIokXAjo2N&Ty?a{<4L-b^%AX7)yQsM5@jyvCBGgxbmXUQ#G#dKR7 zJp-V7J$oL28hP`W{j*=sMJWaCbnD&}xf0#rK{4l3-_6V?Tdp>UIcH0RM*6E8hgp#b z>m=N8IC_|{(BEi(Veq^Td+Je5i*qW2yK&^;)uZ`CO&4QF~ z=Z(Rnx+sW;H@(;MQZK+4HTn(Bp;y9$*%lLh1+G6+jNvd0_fG%cI?eUpFOXuik)iJy zTIM2GK8X4z-+EN4E8RLJgrJiV(FNmV6+)wBDJ-Aj5qL%ku@xJ8)soUKOpRbsxhkIa zg2KIlr@2DSu;f&t-gqKt2Csu-Tc6K1__t;{LwRB*D)u@CcHpiLYH*aWT*T_(Lu|?c zZJUq4LYH<9QGExHyCISHOyjgw1qdf0+ZNQ8N5>-Z^Uw(tm}%tr8#Ivl`7$t*JQ#k7 zL>%z9{Epc`-p6Yls!BOdhUqkFiC*V%LOFjM65EIe*c}I;FSCa|hXvl)=|iv?ZrTrV zQ98Y5xEz|NSvHCd+~<#CszKQuC!4YAuj_uLLD_7Oj$U3tsYGmo)16oIr|`1ES9G>o z-*ZiyMWx4)woI*-be8o7B=YcsrPW(JV?e~ev>$JGov6UrG^TB))S8+y!A;HQ>YbG< zMM;0)Mcr;9d>@hpOOmKEuKsP zDU29Y@ax6qxEpy7`tj};EOY9QN~|A1sLAfT3Xac?$s_v#%9N-kOkxw*Sk zn$yv``<8;6jqXn{#hxseMytPlk7pE^4bJ&?6v)*cw z-0QGV30EunkeI;8A2H0u+*6i%EoCe+$VSzgMY`;qZJW5B-B+~Ql*=@}de%3=F$g@xdcEs0As8s!j)=|Os*x-tO&k-Wca=^+<9 zudMdxvHPtL@VYD#M8~Udu0bQ-e;d~8(wLs=R5XDAi*tkJW{mFhNTbUgnh0MdlnH@r zJb&L~>{FT;bI%Vc&p$aYiHlD+P@(QTR&%I~Ia3mQCAt3)Lm{PQuOq-jPE?OM&BvrWo$sAzPOZx4@X5H zrlAE(cNloAJD~Wbp9X-gQk1ohuV&B$3CsISHoC(TF>E29vkKq2uBlr3;mfDG@Gp+z%t`tN-J~OaD z^3DKHBzpe0T9|IOHfM7b#Jz zmaqkdmx2+BIz{Y3)d_{V8jKFLl0x#rlD%ZT9-SgFY>?aDGca_Vv`-LNRAjsG)`HrD zlH9{6UQi<=05}4eUSU*QoGRp(Ag5LXE*pmu26Ygw2eumSo=8N{wRVDzHFSVdBPeP< z(n4D^L9;?Dt2yvZEa!eIIpNv2&?NT<#C1)u*KIRRbVGHAoj&dFy!<-Td;g#eny|sU z4~^Z$?l_tTT8=h@=sw%`v^o5H9VQzOK*P*oPlTq8DAW zt{fL#!td7_Ki=LA>u6lJxp#(~CLL%Q1lQDNpd$3-erBq?x>_J%F!}&8ZukTm*cZ<2 z9w7G-X&n&a9JwAZ-jDmYH4KCelqwG4m7)q-=yS6glFY~r6y#bpL(fU_nI9aVJP!cI z)(^?o=jTMsZMAxho9+FVN_J1y4mvmiij`KG=OXf)^=;k5d1ihy;`I@}R#_M?pWcYd z-~9kcXYd<6_1{6z|PwbVa^zDRY zuQt|59)>Va`>YI|Zo|r3l}j`=&}F8O?K$I_Cf%|2^c8J*e&*?C1Y^m z9plY+O5fE6i0W+Bv=YFa8~p3Q!^u}t43+$3PVQ>m{0CQOzvdv?3)YiL8S3CVb>WBd%37oF_Ump++pP4?}(|9f#0cC4p*7%>r}|cA&=Iu?uj-) zh^!1OiFaRl_5ADU3sw(Yt#9$e%$%?^*sDiKn1@FV`f7~#Nf9+^Y8p`^!D{?T%ko{8 zRl*zP!1HYfbzmwZ&IgF&(8i^uwvsC5r5n`tH7T=`@rZSAl+1DXZy`G!Zzmo$W{dUZBm^NPDzBp>wCrPb~gkKHI-HR1S`gxy8IiX+YeCbimzqh zhyrBPBCBzn64#$Ad-jsdnhnp+Gi_T*(5Fg#mvSxTLwMMaH;!pSidb=y-@|w;Gk(Y| z*-3@ z+{{tkf2zYI7an0_bus)<6(K2ZVJTJjM~e7|`xsT8E2@u5G&BlmF7GiF7~d}hDTp8* zmU+_06R+@WtObPLM*AXwjv{@iaQJb3X>+BaXYlPyFp-50seg8bQ0?S=xmfbqu2q z!jxjD9#$a0m%5i~gwy=e_|%DqjEu1wgq1<>4?&UK@s=(`LyQzO34DWkrNikffVR?vA(j)anZKc~@@;v?Km%GA^@Uuc>~?(lV6 z#*Sa;SF!hDI8GjEB$T}aZ83x-o!)DV2z@6>aP(G#*{bc8XU8p}VuE^(C|lg0W7!M< zJw0`xTF`Z5!>YI`cBkko{gxKMFcQvv9VfdgAdJ%~O||8q9nJDRy}-}?F_@!9mw{^v ze|zrD9ad@cL5JPUw*8gpOgU26N!qKav9L^wv@em8%VA(<#F4F=$>KgJ10#t0ulxM1 z{5#a|K?crHG=mfhuPI8w$-ko`r5ei1ji$862-r*{1fWQ%Do{^T$H<3@#jM`aRU=e` z*&tPytU;QH{D}a->aRQabtOPSU-==TD_Q#W{RF6uJ1%_YY=>7uj2R&qr`wU8%i$RM z7JRx4N1((h227*|VxpZev)b7hxD#$7Sk8Y2_V=8*od-9qBXWJ3h~#zrYeM-yW z?rwhBw0JAl+hs_S9J*RT+rNk=`J88irSI(QJ>2a6dX)jGeO>G&r)^@W`qBmUU3Hy_ z0g~$8x~gCYqc@tr>|;jMH)y4u(3$gncLLeg>lq{{=Fyq&)SjDC+WaF|kiXinfA+AD z!x=(4QAY+aA;9<}quCj8qs7{(U&$j?T>YT?e|pZOdLXMZ{{}#||KfkUO1iq{Hs(&c zy8qQx`hN^*lvE*0S?&}e#igfZrR9z!smEtj!eZp8X(?$Xqhi!XCuv9NsT9PesitQr zX(ggUZ;8eq_3DkwM26Dd+_kmu?&QhkOK7Fl5hM~}{+rpQw7Jd4uldgg4*~+9{-@bx zq3@!vYpU;L?5gknUq6lY|DGRRYHPV}h{pSz)STwMWCEc;C*zsAgr1xV#F>eO+Z04* zRgOSd0m~q>K@tKe(i0BeuC!9RfeKN{rpZ5Qv%`gfx0aTcRF-sA)GoqTKMdpsMdFVi z;##}h+g@~E3s-JlW+IG|DCba#r*KrpT9GaqHNp(u{|w2%$pxeCjB!)`DN$*NPvc0q zBc4i1RT)wo-99STKy`nyUcKl_R-r0MD;H^qjZ!au5Fy!@UWUQX)1bK9x?omdub&Sb z*RH%5R&mx=F2|UyJ#m22jw~SVQF2e{mzN;A&qqp%h&)m}f@1Qek89s$^pB!+k-ue< z;w!ac2H?j~k>Mmk=6X~b+%3uf-r4!_tYMF*PE}z=-T|xyk1slqL?3A%F;yv8@T7a8 zPN`0nD=LuA8*2bSDt?1eR!~SOBx~@XI%s07l`9~%oZ8Q{jxr(APZqQ3pgb)|K0T&X zrq1XuA6qIowm2l9)}6@%^Cg)pP#2o^pxr?nb+ax(ZzBbJu+ml#t2W`i)KjSNqNe;x zG2G~5tCwSYHU#0yFJ}7g1A{iOZL!)9T8krz$7%7RGz(&aN=gQ6_i|xAuix4;sgj(ssBx z(q=`nsGCeE1y#YBPmuIiJ(|m!RE^kBvB+M|RaB_67)N^svMk;0lV6lN#UO)zvY}AM zn@|*K(+q?^ZgZz4#$Kn}>t zfg6KU4)+QDkt75;bRa~fZ7tz2o5OpAHBLRH^a?WdHmc==G>%U-;RU-EOuRnB>4uOA zJKmLDc32>I^w`<#_Tuc0sFwTogzfrs`SWGlb#ON^u(6PFdi(ln;pGh%^D&eCW7>-F zLJ)FWmH#N5Qx{uD4iH{%E z)5XDque-<5o`iFBdi(g8F;H+*aB;U}x!bUX*Y35|@p(7oI`{PQD`jqO={II5~~nzh+TnzZ)BnDD4Y zcK{D4EJAY>OQOlxlCn*7l(fSpaH00vqP+@3Ii2Uf@KCbywfW@uBZoW#Fl7EJ5WgWdSP{Wtw?o4 zkH6Z4vg*CH$Fy@nZNya>A`!82`pQ84v!g|ShF}H*R({pUPTO(}`)~U-k#XY9g5tFV zg^+}(GXu`_>4>^tVXR`^7hL^ z8$0|F5(}LfD5$vGo8sUi=n*ip1I;D&aCWx$d@C!lCn1)Qk(%fr4{9q{0ab?()_@SL zqOh>b?^Y$mw4ij7Qb_E)o1_2^l)GSo_$q6XroRj%3O{=%ZpYsF6N7Y<+AT%qh=N>Q zPfp9ch&2yy0#KDd7Pct|O(lSE6R#G%!&VJw?BfT*(l|}>?~NtqA7q%z7)Z$IJHRL- z1JTC#1f_v!25Z`1S-~iodm5U1ImpQKJS^ryodM6T-D?R-6Rm3w79Vas5teCJ-qi_s zUWxv6HtmZFvI!bC9W`IxP?A9{)%@#%_(;$P(%)dZ=i|GYPCIXjb6Zr&@6k#P(qAM_ zL|we=hz~(kHd1hBnju6hXZxo$S_nu$$gm$W%#^=kZiIi=bv9JOSZ{mHU*K}&z}80B zTBa@_HG4ljFl84gF}X+8J|=e|$(TnB_4DET*CTs0*)2Pi8R4tK?4~4bB)5wY`Q_W& zTFEz`)<7u0Ah~{xF{KYa4AupOCQ_ad$!CkWFEstrEn08d{@~ZeUYJ5ux3Na|7c@qB* zzS{?pscNn6uQg4oXE~n$Q-xD;UjG zz2-^#HK|Df(Prx!0xnxFXRG*ky&JBT^$f$X`$UhBs~DYvkTdVYc3p?i51oks=X{%% zF5OG9gAinX9`OaI6E!(KsPrq*+OhqYTkp8zXQ_BqGi@6_R(imu5ou)@#Xv2{YE=I% z};ive{dBM96dsv9`}_2SODuR-hL_Vavyx!m+XEiM!n+T0G&jm zs1wadML96sc7<}WQPnz+um6$<$PN{&6dy1MLd>^tqWRbb{Fh5wf>8ktsIjx)YX25lh6{V z*BPEGc1x4xnepJGVI0A@rY6y|`DgMfC9kFEv+JwA5cRKdjh_$m*N@3IS)3=a$v5`W z$C}!Eh^GulgZ5Bua;x^iNFkKJ7v`8xNXohJw+$)C&)RI=cRHT= zyw~+;RT%ja>rGeeiALGv*hYm0n-|EvjS!7hm**Gk z$|^%}BE&!&NB^YUXq#YDxnc`d93oZiGi-AQ#nAHD1xbMue^u*J+E@U6NW?rf^ZWBB1%1tpgj{xh5;DJBBVR5)u57`x{zz7Wl``* z;E+W(;Bt#X9py5MIvM$bgcq2y#@(1-LDH^>SspGcVWaX zGr{jep0L5aPiA3#=#356j6x(CMulz>SS48oT4b@XdrgPxFRfN9zks?CML z{_`o+u`)%z{~5JxEuv?YJ}nv-v8~L!2YYt$m*l=1|2m#Tof2-Ik+I62!6Yb!2AMN( z7(YYy+@5VdN@Iv4brXeZt_b3MJ1dEVoyWXODu*jWVXC)5YHAQj4*LvIjC?POQ+7gL zM(pz!n~r!0Xct9>#an)qCVjMdIE$rbi>8;j!Ypl>)93eS?{|M2iyE9i?dam{R0O}N z4Y(|wRf_r69ak+F`8hlSt&qLH`3&pus@0s{S~K6(Hx#J8iYA*IB+ZcAvWnyd4JF>+ zCA3j6PQ-b2=@AYE)O8$-63?5$K2NXb!a1Cu@B}va7?s^_30vE_LW@S31`U6(%HFNh z{_yoW0$LvXd%?ncOT`4y)PcA^jk-W4*V3y2KeK80aSM>nvbm5;} zx)CAiGK<{aaKbcWEvJ07r%+(2=m%U_1 zdID(3vuG9i<_Bhsy%rzyp0((A2Z=z=J6T^Ep9d$rosKZcW%mgDmvzAq`hFmv!{>D( z&~wn$P`ziYR$TD4yKy0IP|LZzVqO-9||m-5wxy4E;>2?bIwp;V_lZ&}E1Cqju{^60XPVNX#Qxn#CIMVDH@@JgoL9bdIdgwaa;C430<`>rr6SmU~h?Tj{EFbZuSTyuQ5gR9T^TWTk zA)}Hd5j>P>T+G;eE2i^2=4!j91w?gA~Cw!OPQg)(U|*%skklL#?>zd_xIi=Y@wq zAmd8E!N)SWSY}H(5f-a4+5?$YX9@&wkc6ClHO*=ga$J`xA+$xRh$yQMqinkplHcqR zD86`VE1R^b59&JWRHI8tdjT$r&2AS_)%>c=-4xb=ukxEgTr-0zSPur)eu>g8>)C;K zF*x@~4IL$U${n`qyoTLl{Iq{bVEKz&@jkvlP zmNk?s&gL%*q1!*Chiuq0eIXT0{>Oa<7J`q2K0htq>+A!p7nNb zfU2RqB^+O{Q0z=t8Na}O+VGAe=|AtP90&|K+6%2r-UVx@r3Nq1^hE z>%)UfW6HlJ(P`92A_X0eRWr6E&@;5V&=Q!m8LUSg`So-07x-(XxxASa ziF|x32#dg2bTZ7}yVkJDj>xSs;dr8_nEE5P9HMj@$S$CTjHtHU30^N09?KLwKkWCo z6A1rS#st6xIQWMUEp*D-)m7@|e?9h#Wa|d&`kpx#M~m1P@JF+w@EVsWQl~?dm?=iG zZ#god#%wG4$fK^M?!x~d$sxhz3aPPIWMP*(W|bvB*M`hiuLjAttYW8+4Bh7qj2Tm`#(elFgV_#TvMS5^vM9fj4OlzPRl6hXT7X zMIC2YlkE$}i3tSKaw$dGAV&}vdga0U*^C?Sa(eAWDS}0v=?Q-9KtrUL8!j2?xFC34 zs$fjw+tm8nJw_DuGT1B+w9EV+t`A)X#8msMi+g?*IFAUe2sLJ(1wfh_nplw(JVc&a zc|A|Pj@O32v2X{dwHuAJz7R`nxIRA!(qklXujQgymckZZ@>}dlIJw00p3J4KE8l2tuhi+zHV~m zwE+0WfJdE*=vOC-rIxE#KavB06u4$mif18Fg6k=%Y=4CtS3iV-`NdbpLg|>l9()4T zUccnIJ37*#1(5%SaqTn>$rCi0@t)xSdGyKX zX#duAQmY#CbqxV%G4D%@0~>)4RFg88hwAp0hy+|XrRYaq6u*=SOq%_1Dyw# z=^eJ+k5*yW!bJjNFuQ|>fawSDsv!n7bdYX3P8DF}x%yfPLuZR?`IHE^*R2@`8lB6F6a|RwY*ry+ zu^c!wm4}i;!8{y;ja*XbebI}#q2GvM9XuAOun28!4nx$@g8=lH_opoG!k)wo%84jp zlm5*{`!tIX5A}7kVn;`(GdW7JR|IY^t*83Z(a1GewkFee?g?!COUKQ}*`q-Sdh+{p z4*p?ZMDoER$YNiag&z2+Qz7ol-b#;7`8$XklOCN9A_?b@^?f9l*(b{7fHFHNa-2Uw z>P4>5H->3FctYXBi~Jrs0{xNThn50qAELH8ijy$CwP6DTs#;2CxfN`)9PBOxCELZw zV;w>3bHK~YYNLw|bSCFs;ZiS5=fRGriH00^f1Q9DJo~@Si)UYaX_}ffUn^|r{$7WE ze0siJQjKq)MeDC{XbSW+D?2dKb zx!Ui{W7GH9aPS~5`1HVRRh~eP-e%L&8Ai0I@;^7t}tQEy=`WHn@sqbnlG11iY0I;Zk6L4 zAiV>$dfYTny{Yyb-a;GTei!yQ9DaT0Ja>1sMUjj>_Ei~(MIthCrA!hEE_w;ZPctYP`U&?`?cKJHdLvY!$ zF1TkX%5eK}Q@pRVhqC68rss(R!}T69-rNZwvg~rbrM6hx1I9u7G^D-we&||Qs8U^W zdY5a7w@Rr<&ce@r*|o$>{bRP^w{KCW3*<#y!%?;;NHVK~@0@`HjYqeHmis%q@R-iV z5AzPLqk$}!*W1z*`IPe8`us+8V(EOV{95eCCdo4Cr>h4pw1hA4Nl!>9&vvxGgBYfR ze-&EVtko_C@&*mX#-)*ZU?rJqHpL2pb|$>}FTc)Y8}+05sislxi zy;88Zf_4~ip)4KUUKO%F#b{kd>tI$ZIPpEoQyzcmbn5FzYNr2mbK&@SdYQf51LSum zSGg~ih6a8VFJUflB*n6=l>gGPFCX)gKsrr+$;n2`h5Ujrt&xomp@)~1d&$OYba|1m zclms5^^2mA?eh`18k5R>A{*iRarG&D^}gX1Zjf`s6X*q6^ z(^oN~s5hn8tMQzx)802kSC5;-G60aX8j+}e(3+KY%bssBHKCj7tUc*6QmXT3M^#Tu zc>E?6Da7HSSXZKtK6iV_&Y1-8RE2U%r6_|K>Ado7?`u9b@69 zK{o>z1&ei*0yZlC<5bhm%}9Um<1Yt1?ZU6lCj{e-$k#) zOo<2!=D)uL6*z4dl^-QTrp##KiV`}SdMAv8il+@1D`vmfcr)c9FwD@5g4DfkXc>D` za0yOCOB~pCf;~&ZAVoCR>Rs-=EvcHCSr7Xh8+4Qk+ysBb?+iGd(K+mto2l}Dg9>EQ zg-Go?4gx#(FlJ*wF8gq2?8Ilm9yEeoeu(&|)4x8(x?oh(YuUV6_Y!w3t*k5+(#@%o z5Ch)^*Ttcjhkr)m46}EE4UD^k^&8EppsV%YWGouua^(}M+@LRbXxFH_x2^s_u(Dd55y^o`q@Tn>b&`KakH=n*~9h#XxY(-;Dk9@!b=ueiM zdc0px*HIl-_j+PSOnJJVc!}fzhwahz367s7cj@5U1 zi;iQVSi(8xhGEp=L>%W2`{vf5df5r^v%=B#Z_QyA*u}8fcbizA$YaB;M%Y3`kS5rd zbO_;lLpe`qNvelBQF6;?=P__CsoT(5Dd_IR&3AI2!r)4&f0P@d~-U!HIB=b7nZ)gt~gFFic>- zV+jdeJmzp#O$N@ZnzeR}iOuF|_h_ZbReb|Z8-1Z|(>z+;siizl4ZGIDrwLh)ng}$0lB>YcE*a>8 zu}QUgkz;IC6onWoy$(DB?d9mAgRqPmgV0H6uXPop1LxY_ZfN$T;KSa7zZk}AcSehw zu087U;Ok5g^f9gK-%uqv+CeXcOBpfk@%urnepjO65&1w5l<9M2%vdtKd*Z7{BL@2Z zj`YJ1>}kzG-H{;O#x#K%u_S}{fCG|dLH8Sdth8}w*@CG0f}9q6I{KRfS_5&X>QMmX zs{G{{$&^|7`7T3kzpjD>IBZvZ=g}WRhrT1KFJdqgL#cNRNh1SKM=l56!I$cxS>Q}v zzt--E#)|lWj}ixlr^0};#xz*rG^2cv&wSvQ?U#x>9Dlpa?*_$~t;eKX=~aNy5!;x< zpg7nfq!lJO_;PXP3iHXs)j-kHdZnd%U;mCy?OAMBUS8aU5$02LCyxYu`WT6T8IHjPTaaWygTDnNNuGWCq0ReG3Y>m3X3gwWV;s6| z5xy>Dd5;v5TRIJc^nGmV63=Z}6>1^0X!>QgN^GR21+JC zwZ)@wUdyg%wnh>&hsRH+vvGbL>kw*PF5RSPK@~|?xtlZzfT~3WzVG`}_Nr<}r*t(! z>kaQui~p{T^S2qrAmL}T-0>C00y2&Lg3)w8 zgVg^)*g3T4qI6p{<{R6#o$MGpwr$(?j&0kvZQHhO($~f{jPA`8};nDPyMb2BnX66_f(M-~(E@Vc%9k$^7 zLzmhqxKg+2iLEohgQKbXF)YhnM?Jpu0{f_t!=$cL4F+orrQEnDI0wu-mcL-Lgok(q z7qw|G2_fE43tZ}xxdQy63s*S^;U8S&s*=O1dqdM3#OMf>{`vYocDTJ8#G=U6L`S?E z&wZkmn9v^2j&yuPsNCSaiajSHm9-l1HK%Mp)XGfJ(DktTIbK2~^^k)=G8r3dF{8o!IVo z@a=o{;Vp&$+nX84vyz+EylQ}p@ML-+E z&L0=!14bOpJsCnwDB_Pl@?gW8$c_%Q>!ODB3~pXr)UF(*hMSb^QVr=5hB#TP?xI(} z-NskDvVXpLPJCUt-J0ISFdfm}!4%_#K+LLn#L^?2-70DB;l*BoCT90eRr5=Lqp9K| z_s+@W51FX3N#?<3z%H7n1QUxma_tc-vgE)|)&y1a*^Hq6kV;+8nCkDI0$vK=I%97(zywU%;xlT)EEa}yDts_i-yqnPii@COWBmk{K) zWet{P&=UBf{}%qa@{!v3U)@GNKnA9z?Dy;oCyJKYI3u~7A#Ql^z~~-y)oD#IcPht( zWIfY|T%hWS&L3!oj=W{jI$|+`I4$nzWTiIhU*9EcEfXBSEkPYwFPkTyUeci?nMwZ; z{NtNQdqZ5HbPf}fS`{MfKh@ZBhf|(^F`ESBbA_1V18W2_d9A6z?<+r9R0RxtI^Jm= zXM81B@U1PQrTq>zjaTT{zsKQgz^&(5+AkW|F(>+rP&EIXus(OzkGjC!g@C$t7r2he zG5{F^@R~ylhOd0@>$uQC(O}T)T?1M)%o_0Gat!CUeDWBc9X&lggWg9yFDTvIe>IAV zmAOO(AbqohfG4tYw<*?Pvg}H`m-qi0uH%xK&@fQT+!6l(#=hA1W1boBYu;ea*A%l$ zN3Qicpl_k9#Vss=UXV6leIMiaZpU2Jwz!^ob~7hLTJJ?L?eXyQg#Gq*sN4dZ^XSKQ z6Ku1m7JW%k%j@^kuzb4x{X8r)rtS*IGGZf*rC5!=y$yZkya*{D_m=V%qvEmLSYS|y zd|Tk$1<-4}X_KLdb3@Z9B6+LfxqOa376JP)c)Fg8Rvu(ny(ga+XE`G0N^{8otssp1 z*oZGpALmeVF`1eVDtemNqNo)w zkmWAte&`k8IH{*6EVLOut=ZpSjJ+(7ap!7AnvO=bM@xI?=(PBfjUzY;g*f$ezupgn zg^F8>YSCkx@zqeSgG}oJwQa2rE~qRh;oo4PPG)p>$)gj34*Q$WEYc>4sd11hp*6ly z(o*L)l zs`1s`9%NVwR12R#mdNruX)Ilq8DYE~TR&M>qWIjH0qXdA1a-)NQgq*TlqVw~hZ<}0 z(h*Ms>D23$6I2h(#}*hR!^Z>B!Twt=<1F`-)I6k=P^I(GbYAhvcW5`$>bDnwKyYU# z*(DCG+LwUm8+nd9k40R?{G3;wQ>Az-T_GlO3h!CFZG-CW6Gp99WXZqD&HZM3>lOdb z!HO6kiDZ6jn*~x<(Le6M?@wg>SC*!-%X{g5YAT8AE ze}DM?a@LI2*Sq(GJU{|HoNh&h)hOy){)@tVI_tD!DM`C?!!3BM*A?@L*}-j0YO=s>2_cF({?Y))|d zwshNzbrx5+(xAAZX6B2N7&-38ow4||OJ1Jw3afNKDz{ojr+ zN{`Alp8B>S>cJE;?2x`Xq>CBw^G%Gu7$wtWVX{m5#1ZH7%sMqxo!SF?f(kynF-|*nm4DUnM5_x4;3K@!ujwF6DnL}m=`ro z>ycj3*1<#jf#ky6B;M}hibCR%m`Kci?yXuustQ92apa0L;N+=yt_sO2Q3)1a{hp!A4x8;Sgzj>W=%#O+!z`}*^%>ii;^hCP6#d2(YnB$E2BQ)Fqlb3 zF?OH~&EB;02Ltpc@*s_36C|g9bu)%a^VE{*JX);n2WOV*v7`5}gg8|J-nCE=n^=G`5eVBk*66|$j1LdX3;nnb46yxyDINR12 zDnkF$U;hW;bkvZL;9+dfj*eY(|38V zP@>)Gf=U-8Z)`GOi8X{oMn309|BYngRpi~=sl_}i+tzwJoClE)9M|~i}#P#@YexoDzlI9rJS?C@Q#s}L~Bn!d@29T zp@5&fSimD>DHpCtM}3n5bw9bLIBIKFBT*&J&@8#--z8Kp9wQE8Pxg?hbE&q$;D@<_ zUT~Y@De6rmW$Qo}KN!|vb|-QgSUOcfTiU~GBo*as50DxG(xRcQJDcQA*4;VbE~I&d zruFCgmYCR(Qb;j5omhI*wbnky0=O;H8@${)OXiuDBum6c@%mH{5WB=W6J9i2&rvRS z5>*V0zL2O1tCfodEL~i0FJo$DotyqMkX6XFtOcmQhH%IVNM3(0yH25R0I$(te)+|BazXCek-LNxBw^ z^ObBw>0Uc9G+urN@KWiJW+tc3m3-QCYgbRRJr@K`jV$NEl@LB_#yy$8fp(Iwndc^H zfpBhaO<6JS=Ra?{1V^r%#N9!9-_SFJ=zoC>v=m@Ag{H>cblhfKcl&;ge9hj*hEKH& zrh>HA*1h&GKU^K@-v*zu#bcJ9_T|(Ch%sK^Y^*Ki#>9`gwBG-x42a8eBq^bjPN4DM zG1m6)*1ZM_D||*eXmt^91LsZ1&nL{38$g>_%|YU=Xw!KSPOH;t#{LOx&Fe$U$?xg5 z2E!@Qlep7hzP}R<(+EI2+G0`;bP-Wvx?LVSn26qUj!F>Y|5hGnow9Ljgrt84GpTgh6s5(cYL^mnx(3@st(s#->4?@pr1A;5|`_ zImgw#0|hs7ewp`Pa4(NH_hz?}OXywMk{vU)@g*N7-~5{HPvg-lSMPkh0HGbsT9mtfv1?66ybQ`gQuAI?_R#J zchq~W@3WH@-E$S@1PZelp*}_^V;N|_fF{h-D%}@$8-=zbp zqMEQO<)fOvu_=C}`W{AO9+KD7tv(wGff5I3&Y?N=g!bd*I@en~@=g7*G1<2CML5Nh zD8%8KEz`!>f&MsroBFz#K_4%?PMqxF@67Itqt*borr4+YE_#E1yd=UA-b6py9+ZNb z{%j_lB~h2O?`2C}#|ttA`7)Tp^!Q;#^{Bk*uIYYc&-L?goWeh+J2IICy5_2=->&|4 z;UnoDF?24lFmKOgPHAD1qQJHUk{f1!)WA2)SbMaXD}_cUAbF4fsoB4?fbmrg9+Jw% z`}2MGF+P*&PuVjv8W<9g>s?A0P&d+OUW8(pf+)aVi0db)wS_aueBq)jXwA2NdSkQ#K=nlB6k^SnuF6?l&X}(d~rZA3@!8`L~wr-6kS^++?(Fo$XGm)_*l?i z?X1C?OGcZ!2BOX%f&qG}?3H9LD_KaDdDN_Wspza9)D6Q^rE**{zq@D~b@mJneb@MW z&gBrAJ+-FvV+yE4SW&l)+TVL=uO_b&XsI0K)IL+Ox^dt|29}o+xc-1$$@wfD&nPsxI3(#qeR&nFNIo zyv(fBvN3(_88B)1kJ-0|v-HLCm6IVB0L^a%r%JGTr-K^ejt0 zu7Qjvr!1|CLG^#-Hw?4wY_9cl6O1lOZLpE+c9Lq_rstz-4;m=ro>KdPyx}O@LwxLt zEkMzTd`iLqd|@KV0yc zhKD91G}W~xB6TFdAZmkT-vvk=i?EpRu(9q)$`vmG3Gu|EGI4@^LVSQ~Se4{48A-R z^r0@2L|EkG|Cp_@d(4d+aM_~ruUJz(N3#eX~ail^(C9M8*YrWCON_0ACR4u;vT-K^Hvq=D~kXMLxY+ z>L$VuOujT(qn-xMcKg1wh?@{=b_XUmB3a~~OD~MN+F}FX4C5uBClVXuq>nJH!Td?q z(w8w5sIOwSCF`>o;MmXn(}uy1j%)@Te=QE0jVQQ5t(8Yk0UPTXcJ(2u=6Ba(p z&{0Km z&8wZzo2_oQ>%*(Gu$%VjdzgPWlGgE#F%(q`n|irF>|3F&m-W32v2MJhFcFXa%>yLc zvl0emMQX+*)u@$|yxg3dB9)u-<(}ssjGN=%;;Ve(oqZLBMout#-X8j~blULwtEZQv zc0zTq%+*dEC;+i~aKp&wS|4r0bqNS@`uH5mQ{dMh<6_MxxVvFZM+|Y8(=Fe-CrE6O=W$2(9CX3tdu_i4>n2Jd~bGBBWTv$bChIG{E}UdXYJ$>L#1_FT0i{gu}Y zBMx8{rEZ(0wBb`|)=*AYIiO&F(`~9MIBE!I35)|UD{~k8Bc6IAqArEpT zi$u|rkxB-I+ZD=Qpkgn(cRUh#$I=`?I4=-y>qO@I8Y6^{aUY8>CH=uL>-%)^K*tde zjQ{1XG-ELru1Q(W0dHrBmV{PtHL>LO$-?(|`qufmx_dVT^8kVsaDP5`TD=BOy~Jii zK4)KL6_Pi<8qVT3UFDWvNoajm`Z)d_uN_1lK);{IFPg?0;)1(dYR!Dz@c?JU-B-~rIEX#?$s@? zAK}!J?5X?fuM7wcVgE9p{!^dWxJo#i_kW%gYrGHosdM2|_6$eOoiv1WB;om95z1~s zn8w#NR9eI>%rQglJo|yl9$BMiLZm*3Zf0oonOTMGU$)$i6GQOxw_{}P*$%3;^QU^9 z+`YPtSr_Lid;LC)ZA~CdkpJnV6ch~UG>PsnT7BkT`>1rp5_KfnY}z73A8Nh7!u@IN zaD{JuxQ@~ui;0z;mDRn@#cQo3lK}rxb0X`^-MIhhF-96J>6{DK@YBXSQ6M`EjN9{aUI2 zI7+%sRCqikpl>oJjBmL_zDW7&4`ac;3)-^zQw{Xa&$fhoXsN+%ASQD`g6_~La^%** zgDhJkH|X%H&{MFE7?A?0k_OiolQPF6OJ-#E_6@7vM(Og_5NSt*s3Pf=s8*QfL6;+h z9@zGbWZW|00h#BS_y`4**i$4+tpnJMcbDF$`(OLSe#wpb^{7363l7(dJjfhXEq*o{ zr9eN)DrtjvofsyBs#VBz(-`}8X8NqTzl1Z>b|+h`V^*i1>vDqj%m_y3ti_`<>>8~d zp{)`Q0VW~g0EAq=v7}-Gc1?q#h|9^ zJN1w|tn(cP7p?-fwT3`(irbj26}DKz{p0=fKfu^+Q|qqP0Fc)+-99|IQ`Fs%alLA1 ztK8bo$Mhmj#OyXRry&R#q@_-M2)Kb&7MfkqliKrx3e6Q%8|0=#X^y7>SM@E!WQ*?> zNAo;^n!M3jV9+B&&EkawJ$>}d$>$|;2PjQqL!%Iehk@IQXIMwG1D!1(>}ri1qO!85 zrdBQI@DbhKbDOS?+fU_F$$o5neDBt5C54npu?e5eC0%}Bl)rLc5=mDt9K2w$3O**W znMgx|&zlR;H?ikB*%X^W8rUUPE!_dX`g3WeDHZUEFtFzbmB`g$kL%jSfO3CwBFLljGwHP>NOShu|r;h2ENETWvP zhN1|ML28a#5_ z>3P;F5a01ld?*7cGRkMg5FZNEeAPq|Te^ubBsLmp*0VAh5wA)k*J-$4N?gIOH~{@3zK zl(IRAT_99!L9_=^2`&j;t9YLuD#ZjdjbGWZgNt7rN~ltxgkdlh^&>CVtMd=IQSUujh(Ynke+yxkCZ9BmawZ^q@i7%gB?#= z^D;W9OCH>`c-29{G3p7A7QJmkApBMzesZ}6lStu8-ld8wbz+*v5 z%b0WovDS>iLDT`%@fX{1e9j6=>Pbk4$=lpjZ~)jNJLaVw-(hM9bGdE7NYX~V|7>{} zcI3chR6pQdO!5y0As7pRtmk?)Sz;MGkMQbuNuKOKP?1vK><*=%(@0oT^ocY0hwKGZ2{81%23wNDk4IQ()(c22IF1AVc%Rok z9ciylxqxDI608qRpSOrhNem6pQR&&@=B503d=xn8B|GwEiG8lJ962t)4m`$~S7p&E z1o~B?TH9Qq9;HWaTeY_mc^xAUnPu%S+-zQaByN7KrEXNT_b0!@mo%$yqhq3JZnbFN zGbmS+>A-X8NGg~=$H=DmH+m;0GScsqSTgJ7Jjix4AFtJM-QjS2J1_A7G5dmH)#wQ{ z-XdNB4NVdNPERs3uRVdpFV|(sR**zPjf8agZQb1Ci`O|aFJjDBUYyEcL?iNa)yF5F z6h8agg2UW(#Ri4#c+=Tef_EKXdHIvlZm5l2cvN=$S>skCSGnwolt=r1EleD)e?k`q zzogrzS=}qhxEUuYEK-nyOW*D!0oQ5Q*I+EW zb?*!dOSZ^P=ypjAsy2R^nhq37v4hf8#)*C)%MMMxdv9Dn zxBkaLdq!B0GAxv&s)#YdqhD(OG+H;J|NT6>0(0&fENtArPxa$pR~{X zy>E$+5+A?Y+l76LpYP*k??{^LZr%Cy&BlYM9H+os*m?wTyZtnPzv1+3Q&I;&sil#? z-ho2hZ!5IR+FHvtcH`pZWU}(U#kcFDQt4tvR+E@XH{ft3jb!m(P86vhEPV|>dSHY| z@{v+go=aUT>G%OL&6*j8#P=j*J?T3LY!IX#G*-fyR(anF&DV4%XpB|d;%w{3))Q)sPP2lSm_<`*MtGN>J9W7( zO@o0vApjq(He@|U(Uy{ACWkQL(}v0h45BOoK4t!WW2p?!@qzxMA>w7}D%f4`&>5i! zPS2B@V5>I}e`T*MO0)BNEKQWqJp|$1Xd*#aJoKT1s+N3%U}AWZ2`AH%BS)Zeopfj- z3zB$|Vz&C@Q7rq0l%e>-q>zhrWK+Y}uCY%VsWZdZM81W~9F?bX^Xkf)PR1($n5a4_ zd93~RD~Nx~u*)>BFzK0E{7U)k*laoOM>h5#QqS9S1j(1wA?wV(@6ed8-9NZ^leQyB zMacOptrK+hHvsTYz;Y0d$OiNMnnQHY5xhd9k^La6@Na#Fqx7H)SAQEc=ev4;Pt!jg zFU?pr`tcSRAAba%u4$+)ND_Kna2{a+fb(Jb#C>wtneHrw)s#y}0bv6^)*p$~vmdB@ z*kX`H0nj@SlZe$v69&`!I(zQn`#?}0O3x6?(y{7)2jx?ircOOxpWQjj{J=p-$Bj&( zaQaND&r{L_*n|u+4tpfMM6*sv{{;q?a^}0{`w40xs#H2&E+iQGUtftCgh;G~Ii6;8 zJ%#ush1hlOT-!hoKwH<1;E!^?6G2HT;D)VgnpKZ5LWyVW&c8wm%ASRyymvRYwh!!U zioCJ9kZch-=7Gm5YrV~v^}YDS1f)q!A=P1Cd-S~7nUHq?y6l;_N`1nC&+#urdDcf6 znETwBRwhZlsrf@PZwobvoZx%X^kX5n=ynBbYy`bPg%Tf*hd>RG+Hz%kKNo7N4AcJ! zT~_5jet%4dbZbJo$S;|Mgrn<-+y0BlB#Oz*+9F_F+tHszIwa5AjiSF99zD}p#V7wG z%u%_d{7pjyWc& zColZ~U)V)i(;Wg+t@^K8wsWCSS(I(rap58ETdg3v<}ASKu^LS$YJJq{rkwDonA?#P z6PiZMvV(gri7^QkPu@DE*WJ`9-oEeE{=a}M9Ix!Ej1NkQ*@o#e#J&K>%AII*u|am? zN-LO=)g&5P&jb@Fw9V9b@zcldMgU|dW)b$3SLYkPKzd`!!}j`twjtvD30~Ny>~8%9 zZiTUkkB!~^>~!PmFn;| zJqLKHIJJ6mEjjXTUx@gGtLurU0Kw1Xy+4h)tk}qe>2PRQ&i2&qguoaVuwLcmCHAMr z0!QV0R~y6_SVnngEZd0~`56e`2CcA6oDMmYL!0X}h#NzS$}&x0*-jowvl$hQW|%7& zs{hzP(=SJ{LI}n&%vAQ3z3s~Qokx=|5@uH)4CK9J$zbUdoVF-bVwXTv=Ht7k2~lic zqCUD3bZx}B?t9}Q zMxKeBh<-YcNR`tD&F{NGGrj3kJYih7iLJYXnXh^T{&K%wT4)yXgZw=E4voUSxaka? zrtsPB*|^|&c~Z*yS~iwo)e9>gUb|utLm|QF+V~&;tWguu2uCwzImJOV2b0irrGOp~ zED$JkYuitdO(nl1WgsfJBfv-9%A+wOsVlBWcHL?1RP#c@JfVjJQ$$zlRABW5I2mqe zh~x+P28oE%?w``(v*HY$@0!T82<&}YuIo*E?)crcRp=@8Q5PL~ar86_r3vL&eH?v! z0jpyRPY%tS=TGNbo4fvK(-@Veug^HYAlXaH_IxZtzBVMrq{ZY`hb5YU}4cn3%F!7L`pXYy?^gaL7$&7NZ>8@&!F8ZE&04P zRfcDRvG!-icCVB#eW4eD?M>{t>}_T#Lok9YIw0>JLg5z_NCgB?c?0dzOox^vXg&Ho zh-=}fv2lw`6{5rc5=LN^?M7B!VB<3s^EiHpPnawm3ntwO{4s+*eC!=;E!i~M^O5~9 zGlPH3_5)3Z(>bQx6-dj$y1F9Tx=wWDVw#}*ycHfugN@g>*QO7N0Vjz+`iq z{?AT_@!W~@?y7&Vs_~bhk5jtVDwo|11<%;vuNPk-tIcO+H-M!xPM!D94mW!ur2Xl| zwx!0#UOAdb7%RF-#aRf0OpRB{d|&n4P3fcoUjw+<&BT2yu{k&i){^;LWG%>~3vo6q zB*hU!DRM1_5^Qk406dG?iHXw7%l@)YR}Fdtft6{|@!`KFtqpe=K7~wnLL{fx6CxtzNVU zAgLqilUBidyeinPenK)K0KCFi1^mv!IV)R21*a+tXwJ_(#!%Dysb<{UA7Mq81XvN3 z+FAuwa6d*G(D&EMCg%Q3@W{APEth`=T?fdxarQxCV61S8;pQ60QBMQ<8W{yrk=v&> zmfh`zlRawMtEp;~L{6y31)8$A;@~-83G|&*!s~4LT!uU^TiUCuq~@zEFr5&lV5aJ8 ze$xKvwtqPz`TnpjR*4?DrZ}gx1FD;qv^cHH-Bf!j#&r zokbHQN*ZYAYDQNqdXXf>B0o9CVj@R$+y7dcxt2FZ)ax+8?)&K4zj75xg{#0KOxoL9)oB-Aud80GlDso-3JOz-T$%V@@e}$d%9cSpT`Os@$Ddf_iop% z(&i4WihMh=R(D%Ocyk3p@`trv8AhC68vFE7WfiRDuEjOTZ(5xCw@BOfw@%>k`B52vQTV?>bixZUud5;K@{&Q*B*EE;rx5Q2V?%Oj7Jh#;BhVbRF6l zP9pxU3(~+4WbO23W>_V< zZ50sc@8?u6hk(!XRS_fRTNDFx;>Sal> zRv6Z^7jZ{?V3ik1a?Fa0bzuwAttd-6rA|iC`}TY>@!#U2m#FTL=bJb6f9Jvm#{Rw7 z0;46!zburWR0OOqiU`6||63V_2?qctNB3VH`g1kA463f< z>I%0_=UU0o!P&fi3-UItE!E>q6BnkpnJ+KR^U!*^h*Kxw_5>UhM4DIbxl%dXsnXZ-8LLv>9aCXR0n2+E8qf-_@2eIQl!ugaCo@_+b~B{mkz8% zyWi|`Roky>xMc2xQy;XI=0doLbuZ<(grm7!!&zn)ur0W?>a;4+$ui@C_$LYb=C^i# zz*^(NK0?w8R<@L%fM;h#Wk)|{ z&cHP^$@{z7=8-neuNg2Qxj~hp-}5hzW&1Hw&X>D1Oty$v9#c2h>4EKcX&c>~1lA#C zM#^g1ycC*H$RN;z-AKTE8P6g?qQM-?5jV``ug*(^*dz~w8>CbHNznfsbx zEl!hc6~s_XU*QU`0R;sE&b9rKj*0<6No{Q3IOVIbkTQQQw3xui;XD>j3&yCt@X^rw z`r_V*=yZ8Nehmw?AX&rgDHP9nCZX4KKEMSU7v4N6mLElMmttP!4 z*4m0UmE^IfRtB4BPA=eWQ;!2zZ3pKD;o@9sDO3=B52M{$ z)VcfDs@GiJTLMoDY|T7&B-B*g1j2ard>Buo+@Vw-?DrEGV4nkI$bWkjr8T$z1=d zl`1O#!4~0!I`07~U4grzV_WIDaCXadI9Y6)I4yOtX&D%~IFkUskm+G>r)yQ-79v=S zQ1iebjM$xOvy<}fBom#StZ%QHa?}p5zY)Kc10D zM~xfUg_E6>6T!96kjY@;kQPONT0$plp7wgI{MXs$3KpgLV}u3~rh`cB;XG1Fb23Ka zV6Cc=$9rLv6$29267+HSJ1M|$w(5iRjuK9fdT5uZl+#}UuHg*vErW#u+BeiU*?hNO zUN(iZx74)9vJ`z%l2SpH#OtDEV_P2FU#{=Cij01`FuG>8Y#?p`4g(m%)&&fsEvy$J zC$J{@LWfN^(Qa}F6`21{WGkZHmdy+!k@^nGQHGXA0Fowp28M-fH*u3%DGeKaD}l-I zb3j7Gk8i7+4ObRA-5=eE7lcm`GsoL;MB1Chj%G`hz71*<;Ny!n8pxh3mqA8iP5fp( zi=80!>GKZiy~K+eOyq$gfGAmKOa~f%S|CcA?t0A5&gWJ9rfyX5pnFkiKaBop+Wg!O zM_|89CKkfMfF`YNIaJ%CQpk7}Km@VReT~rOD=o!5D(kE@5lJmM5l-{^9hYA)s+?V7 z)?Jpe?GE?mh`&}3v6s55JaNPRKBcb~_}Vky%f z4#nG<4faoR0J(Sd)hDf~dJD&bkIOyiZ@1wHOd)TrJF*NS!6R^NB#yw#AvEOit)0t? zclk1Xde7o(^BZtjz?n>>n6jeVd8?X8cr<4011k!wZ0Ghc*RZ|2N8~m_?SJU4Vau7n zy`d2nK5`Qm;gw(QtLN!{B8LKswAKz_;YugVy(L%s#8K>b7l7_xuBDyy+?0P-&^h2# zyqar2P2t2heAbn?8sahi5mEfahH#D%P6}1SOekQ{`{#XX)~|@Q^0pf#ljKe=jGn@E zmq;qhg*kw=fuCOjOL>1#RE66M+m0cpWF0&QHj(_z7PJZF+XX0I4VEPwtSixb3^UDq zZK0A7JK!Jc=dC$yiF4!C`3dqxdzd#!?kUklwTxZihHclm46IJ?vmo0b+*Oqt2EuP; zqB;c(H$26Y_3hDQw;7}<`z+Ds;fj+KvShH;bw;b}Wx82xL~iK)(caba z`21b)`}&l9CisL#%ut&krhcudomm+UcR>q6ZJ=s$PLu)Jaw+18M9tipqqPFpkk_v_ z%%IOkm;eT`=juFwIWFUGLO&Llfpi2yrX$Pd*W2wx|v zd*YZi&OOjrU%=d3l@yD0t^au3-?$v(db(Oi#X!XR#4oH!I}!v2BUTV-sal51J%A-NiBx*`eANKZ?&xaUXIR0 zWne%O8{9!NmJ{pkqCu1Jp~0{nk|QX>7r)1U?YMGF5N%;O-Y>*-EzQ`8&|A-3%7d~0*z-})jk1Lz&L9l=_d8-`79$*OYu_4$4@<>)OaYix6 zF+toFVpPHcnjHRyYRY%DQqt(L=CEDTni+U#s{?rtBP>V5Ig zR=CkH%ka34;+dM?`a~JCOqpw^JBA~sbwd|bn?}T|tEwkaj))AhuOA#6AFMVc({i#2 z&SQj3L=zi|n^@eCk*F-2Z`P}S1x6QzoUBPs3TL$cHFgb$WK+j__k{J;<=(R3|2aF_ zs~q&hJVC|rcZX!2F zHFLsBS365#B1peWITRXiL@lh+qC5)#^r`mze%%f(+QI8Zz+K+Gohcjtavha*mZw5Z z2PTAmPF@;MbC{~w0;|ep^eLKA)UF|#w&9tsF}8X#Lr?ZFS0!O{v#UkBelDpU3QxTLgq9(!?M4_3hlz!1o@i2YDos(?+!? z2{rJToK|{=(3LbQIlru%qf4->4eV=m^x5J!-ZPsyOi&mOjL@(nDfL7>`j#$oO@Phj zqaAhcLwo&njpp3mT)Y1H^oA}p{=DMIIc7nMe2@5^VfZ^hBB7LKdY%Q|@0`xSf&@mJ z!LUGUHc1*t6-gN57=N$+f=MvRs(V5#kD5IL&keR*;)&1j!Qcr95g6b(G?*FDhGE&K z%%5>=F%4w#B@qPE^Z##h)obxwCuKTzsX?^}3TlmBv~ zN|1kYj+RzQoalVS||RnMstW^AA&?OZVRs52V3o=x<4Hiy~bk4oetD9;aFp6h=1nTr|0H z{zNQ61r&WTg{b71>T+r6d4PbC(Uuu&dx-!Z#@@tZ>-*6vIIn>~#&`k|KaD4U!TeE2 z&zp_)kwJQ1!=alHI&+}WMnr|6%oB@%B*%-#B24GS z^aZK><^AatkdFm{_HpEZiQGfTqxF#@T@VI1k;8NNd?hoVI8NB7#Mcn!g#QK2f^+NZ ze9g0gaa$)fIDoOl6qZA&5$PlLsmG*QWn!I%|7dJrVqv7W1>VzIq~;r9EDuwf#?ItD zwU(4=MoXB0dOG%PUbSx&-#OnSi*8lBq!W}s{Geb}Hfm;a?Ha98V(`Y4Z#P!0*e z&=xlm$Sh)<6s9HrYG@W|9~E2*2mg{(c|hal46f}F@miiW$7$e~VsCuSTrt2&ZPiq$ z!BDn9{1Zj9Fy~rr>XRZ3NWkc8x{v(F0z{SrS9X&!55d(d)^yxwYPk;=hx+4;IU1LU zstGhvqIk^LXCWdj0Ct!zP^KcQSoMhj0g(s7o$MvizZ;xVV@CrC*S49C)Dx!S^pzlu z&}rkUVRp;3AxKL#1m78C_Pilv7+)0}(-ZYW!4qmie86UjqD0O(U18x(BmW!#>IgA= zYvu0{8d6PvszA)am`?s3tG-N^x?3O$g!2Gmf!k70Js<`ROWPI+vR&KhcimX)<>&Gl zS3B!*L)7{@Ji3k7qy63^94qmSp-w15mIzUv|5XvOU8^Vfp;vrH@2>XigiVHD|87g{ z4`Bl(KQ*lxh|*D27NqAECV4mUk_QOvKYu^-VZ@EHUj@^=9Te-8g6dB25jP?g4{6IR z`R4isuzQT_G9rP}Bh~NH4f*@)gEh#)i-D3rM>Ui)+Vm4dqj@u148fS+s#*|5?rD&) zJujW9AhI);(c?tx_Z2iE3Cb(fM&Kukhwx}kssRZmH=5g;20a&&dnMX#loK{sU^H@Y zs&AFpbp3^E&|=G>GxFLLI==RAC7MyOSqgG)KIdhflfg>M+t5rkA2r3xx0Hwj8NtaV zyc1<;(+lku>JmhRJc6%hcGj0wD`ATxktgv4cNAud%|XvIZ+l6XzY9KvVa|Y(4&k2Q zM+HYCHwrKV%(EgRUbwpkQn9F?Gv}TsZI_6}!kghzV^wBgKi{vAjh+OILabZn;m}yh5_0Tg^JF}}QUsbGcBa}8hVbVx zv*_@bZDkC@(8+<*0r{p!${;fgVdTsD5eFV1b!Y?Pt1KWRW~D?aX$tbF%D z;0cf5tTc*&MP=X9&fti`ji_Ga1Ww=mco!yfNmUU^KW_VZ=f)Gz$ z$kTolOZo-ZA>wQTVkDxWS6Ixkb0!(pkH53q>;l4XFI)Jm%LrjFB>SM_odH&=#dQA{QJuQ!NFPthg0fptKX?~a$)=fqu+%&P34VPa362;Ciai;x1uYxpq%1so*& z<=p=G7r{&gEx{G-675)DR*Hm5ZJ#ZmXTX{-N&M~ImCzu4*%2b7IsvijSvaK1cMH^* zXKPEMmM+wn;KqA?*`!V&9NpA`7G>46VxEI5X+vL;8UwJZ32b)Nr6M;ump^YL?vMGT zE#f*F6#;k5{k`I04ETBA(miE)%$B6w(KSC;PXut|L?Aq%Xb6P2Co0gezD55kBPp#= zV&+Y_M#*bgtA=V0j1-Jh-Vuui7u3xTBE>!__&RpPLal5%VnLO1!r=_wK!P1iMc0%{ zwTDOXG6RKm7T<;yR~bZLsYf534n{?FHH_>@en%u^mnAS zqtD8^-}S8dGFlY++`eG{A>t=m-IatR70^=W)1C@eOuw)~aI=Jr)CCMiRF4)D$CL=e zunBy3F|9^i`S&9Q(BZa8w3>ntV?VB5X;W6TMe1-pPO-SawM@J(N%y zWox8Ib6%OQOb1HyiywR4|Dw(ox!p@v)XN z&rcF^DSQ@TRYUwNbe|pWsrBiY_F7Lx6{HFWJlAQik=)RSBAcOxPF&YQqhjmz`kABm z>eOMxJ*^*`=~A^LSG>yzsA0INj7=|%PnqbiEk0j21!T_)c~#zIJVf93WzaN6kL$kL zDBOH9u^!trnq5lidR(n|9h`oXG1&F>XTcGigf2>Zr3=F!-)Sda%^^UMz)uUT3UzHP-e!}PrkkUj^nIwbb18yE z?}v`X67+FX%M;6{8R$HF-i>md0cb0tOvC=NOs2c*^2sGdLy?2-IyrI9lnlBv_;!PbL)#;wYQf|l4>OSI*dZ=iAPqE7v{Q50C_SkG?ztC zJj&imJ*+fZBv4W7d7Dm-##J0aJi%~S>$c3;>S1AZqMV}blKT9a74F5v2V|Tj`rB2@ z&w^c-!UN&mHRH6s9wyejaC-~J$uxa&Gq!DSirw3*wi1^Z`Ep|5DyvTRyVpeJq9Fsc zA0k7jwrbgyRh4x%!=zVnVa2kWkHW}Em9(S=4{WH z$JdqC^E&@7_q2N4lY4GEQGf^k`Kp9?oksjMpay8PD&$yU_mRPcK_(3k-i{yqt zA1%p_>))%gqp%y*+(hX_eG|O z8W39L+3Ca6&M*z9Ms}@cL`s&{dLnA$Gx`UL8}FPctVb73`t0+0l4Jp!rA{#+ca7Ys z3YL330SCnedAXHoni#mwyPbbrA2tjnSkcNP<}$WOlk0)FRSbFJ`m7LFX0gebepDkP89^X~&NqqZK@VfbLk!T7b| zvO=^aj=-V7olc3_6+%MC(B)C4gJxfDzs3%F3uIqCgxMwW-<^)0PaiyHAjv*W=}Hv& zuXBC&4#k#-Fa z01yKL{>QS&*v8V-4(M#^XkyQV(@}MBD*XL{!&e*M zAIiL2rOCYSMzWQ!;#iSf&Dty26b;&FI1Lvd|b@ zd7m#gP>A|6Cv*liMUy9LQ?>c&=j+CeMBvdg?%-bjgmCp#@JFij#SryE2(gD3Sqo8mYTWM?N<@@%4%z@Nb1ZvgYqX+pf$A3Sv|5Q zFYbc8(m;z&KM(5GUax##iJ0OxO)5L4Iq)3nD7SB?T^QDZYAQ%1k=<;~r>=US6f&!s zo<#5iveS6?K~&^O0{6%#A^zQ_Fd^{F(U1zkk!LwIh*~h^=wnMkCwx`_zmRHjRJZzR zMRx4K?A?CvdgA!q_R*2Ir{gTYyR)nBsP1H94x=9*uiRE<&fD+l*S1q1cP=02G(Dp3 z?Cj2vWJ;K@dE#mLn4&|^Qm#axnfSMiCauaZvIcr!F%FBCO?LJY;BlJz=A1051@h`H z_l~S*3RSg;)9dhf5fwY#^ku1T$YPr|zK`n-RkS^ve>~vPwHMt^r0OAmN**Spy+BK; z)2EOiAH(Ziv0PD0UbHdHqiEJ9BOU?{kZz~VajT+wkbXZr#9^7zKd^s``S}};Uj~{@ zN{=Jo%%7jZ*W1ej_;A7IG5T~BW~36JSrlmX|o08$Z%<+m#vLET73~- z{AxX*Yp>`f54(^Q_^Jcpc@fB2XJOLeBfKl9tb|~N#r-7*#W0yG$bz_heC6=|=^~DSH^yB@mKtaAIJH6frb#)wSh|pv_h_E$afmQI1R9S=yYpoOt4(p zyXGK5qE0K~Te&i(d-*tC0-QulYc4Igz_f~`v0Ld5UV`W1A!0WZkhdflR3{Ddn99{$ zP|+ndip742ePxpiAk=!Q1qoJ4+vuAuhsdTlX)b190uk>dVET!6%{~Q%%w>E=DsP15 zmrdORskzk^TqG~VyDaHbh%!bG!)ZF*LxqPJ@wOL+O$^(-HHO7=lezNv z(ZygJDSs%D!8wt%jB;-W2S-Vb58-e8FCABf&mz@AiIw`@N$DbWj zr2hlr&WVHTIIrwtiS}y`etgdv=TNC}95+*C$(RvYl`iH_zwiaoMbWn~(G%LdCCMPR z9s?wyG6FAK3k`0!o=SD961Gk8je|2|83ze4$wsEGDj}nU?zV}GHjds2JVlbo5h&F3 zBs+rgFTbKx+3i2)lB$xKk@y^~>eAG}Y%Ea2UcFUWJRHp~WrgEeekC}MuJ9Iu2`ZQ9DMl?(%-mkZ08XtSb_yEY~!zyAurc;kiOE=66)rg(pbcEeYWC)|^pK1BN>o>0zIjc48z)J^*0~lsk>LTY(g6hKWGbtabDYaoL z!0CMFE5a6_j;}M_m<9^SI5~irQHwYrmvaLHS6{MbsEMja63OF8|8l;dk|wY$8i!Ey zZ2hCen^Fm1%j+x2*<+@-9aScfEQTZ<=%eZSXQjNE@G0JORyP*(u;|r!=L_TWD|~#U zPY*j>5m*Mq2o5-5WX!!VcR;Lv*-9UYO6&PP`HU{APcO;}wssgb6Bv~UF-9T{V?=mB zr;Qsi*VjUYIM*aRzMH_Jxxa7}#Jh+ag4FLhZzY%VZcFB%O1+K_wz?Piiq&TD2bibe9Kv zCJgny0Pi63)|AE(DUx;nm{_ycs-w_8`Y;ukMjf{vU@i2rfQzbyW-vrvS4Vl~I@H@6 zj(~c6z;PhC^5>}*Thv3QuLmSvLK|V_EF329%0S>%Ms>zK#VUN9xnqwypjB(rMa=NZqH3e5)Zc50L@>SdoO85a%X-b6#OR}-sc{I< zrqUQCAocx+!no&9JE>Ve3xS&iL^ZDxibl|y&_MvYQ%45} z?%wItNP0`HF!+z-NW*!(`oi>#Pre}6IgwR4yg-vGHobli0&)gH#%nH}579uzPt6FE zdi9J6Fi^sE1wa1dPvO^tbf{e%_&ZL%0N@&`b6?K^49 zJow$(Em6#*mBc+gUss|0{04JZxQV_T)qtD4;0n9Kua@C8*G#z*jI4rpaV?umtlE-vWoov(Or}U!!eT z)4wOs8K4Z&Lupp^56#m9@bABi7MmSeb0HBoWAF#JBs!TobdwcZ=O?6V*-TI^C zid;>c_XV0!bCi_;lr!CxuThaEJ1DcF9&Fm{Voa8H(h~HXDkVXhmg2w>Fo4uQ$H%qaM)2bt3Fdnua&OVi3uEum4cSA&%!fyY)~Wu!Fm$DFM1KQT zidURFTbBTztmD%=dXItGYBv@-Y!M!>n#k(y@I@l&I#P^Dceh|6MxPNg3F@9%3Q&@P zb#1=W@8R$4bsA1**M7Xuk`9%tEIJ~_#z`=gKd_6wKi_4VV_NdM^8i&3BQyc4))c( zVs9F9CA&P|cnGY*0bM$~0A*(1q9DfMcCMd7A(*XhJjUq?Wcvci+w^Xevt|;mB0B)$#0uHXw6Z>z} zvFX#-piZ$(a&S3&A*^Ib=gT)=l;yPSLzZDPT)H*f@Y9of{ZBAG%YJu-M>E3|s#!){ zWi-4ZcJ)b{NXdHk&LRsNyE?yVHrFASP*$v6Q`)FrYH}BLf_c7G?HcPvXlqVz;}*Hf zkH9Qb8*Nw>jy$L8pl{?O&j(NhJS~5Eu zcN}-vXwitzc`7|cOv|Z`N){8J5|r2A|FA09%neemc@|5Dx3^Ams{=@!VyQVD)-xP% zesQ%fu}#ir=kA5DD^C9od>h5*e;c@{)NOUPrDfoxro^vxCuv?^Xdqokg1K9;tMDo=-Qc70QOuH9eK zvGdxbeY0v2p3AD7_~Pe;()1X?!wcf<$#3(_1ts_UI?ayunT#n)V!j;s>mxhqa>HU2_dO)g{G96}ovBJ*mE1`V-PI>%#{kHSDVL zf<{G)Uq3f~Z8s^-32wAK-1x$-P=QSd#Rl_`8zG}>dS6zgs}fYN(eAqg1(MLwdoB=f zsd1%!vJJz4`%AxOk-4Ratag*NoIw6lakH+d^wNtBp@t8a$CH#roWz@%3g#nCW|-=# zs*n+V%mUH3F!_#K2kh8zR;y#RjaO~N2G8Y(WN$9QDDSfzhu=zJVqC>>5XWqu-Wkqt z+6!0`rXg1^qh-T+-EGg1c;C?N7uUh#Di25;j9jkcLijgjnqKg+<2_49`MRg@fJn2t zCl4;6ZAyJq+O+bD~vh()LI;#Go%fsAUPk*A^q%E`fra+yEZmJ{8#k zYVmhr-tz@&JvVaV(kY!C_Y_a`7YdjX*xj}CVk&j)fUDfo$+FjnB6^8KH%fy&7g@1i zKRXRg$GKqkyrKqQ!2*Oa7FweWQ7|U)I>Yq_M-oN*qOZ5UIqiP>z_hydo5XV; zxbqd80!+Vv=xLmwRGYQ6@WSJJ`|1$8kTUhBGrDeiiBC|AP_{lIt9;63e3`~0jpx)` z#+`L0D&|{x6f6&6+OtM%6ftu#LD0*3xDNm=*U`d}r;2sOsnD2imq6gMr3yF-H73n8 zPZnxMc#Tvx!-t@3dZcCr3vcMYB_IMN^TN)l0JjS5ZxOzpNj~8S$5^MSJdEtdTsVmb|b?%UdAxurg-GYZKs{d-9c$soKM(I5;oNQC2GM9Dr=Tqm>Wg!$aO3I*Jm6sKZLiTCt}=(XYh0r>4F+{a2DP8+j^H z#@0njFi&M#rmt5(sS!nv3i6p$F3nF&EKpOWwB-RtWaA3v`gV^OihI>F~qe+;YYhD1}GGqTxmZ~OcgXq*uAvJcz9lXm#hdES9K>O zQ05pNqouWbX_?~DLGpeXNOZe2M3bC$tW6`Oi^NxiH;^)-l+}ntpIoG)-i|3;pJQ1> zHagVqHl=L#T9#tU4b`ZJ2#adgU?Pd>=X7z3{q(o_<@w0SAoY2}X;Js~P{Y?7qZ{j) zviBiA!rHU)i&^o-`xmyH%OUCs8ZSjI6^oxk^Q0#`y`vFriaoAXd-dmP84SH6yKqR# z{;)SwZpsvVp!lPE^RsWVwk;wyoLPk5jmy1U9RBDFacQiLj^16}C?Z7fH8*>_=1mR! zvXS^66)Pg<=RhJGclU6G`pD|y;i6t~&rIRl5MtiyslEBbHLepR@R(!1oAo-><^?N< zy`1acJ8Rx@tRTR58XJ*4%WTD?k%wUM7;ND-Xh~LOEqfAtBIcJJZn5;-p7*3IyS`jw z9s`QPYrZ|+xqB=w-OmO(!aqNzHvO{ZY&V6!kM$Svb@vl#=8B2mfO&QZCq~9ac4$_h)BR zs#WR*4?mJw@;Hbw{xOWROn(qk2Z{(9D9uBsK8T`-)U}*@Lk&u>%7e-&z;E(TN+A1wSKEC|qn)1=r^p#!S9s@}TDH1|34N+EDGs0E=lulIFraMm{IEqxv3zUF8HDE{!Br|C3t;`%{;+ss#ay#!ja5bwT-Eh9(4$D9^d1Fk1tn?WKhu_p*(-S zz6j3Glf9dG$i)}F<1>yLumT_r?Av{1QdnR}@LX(#gts=JzomM)$zll!x~E>GaWOne z3v^9QZ=MU6|0bQTXj+1scyygwlsxiUZ7*m1@EXnuDc)^?mLP^J#P1s8CW14a4h!`1 zk#1nsnfkE2dKIia8EqeXkGyB+to$V3`$Xp3W}V0Id$!wuQ5V|O*Q!U@pGIK^31X~) z6GRvfl%fE;^@EYo=epd$+^NTe6-f~6~!Om7J*O}+Q zPwWs3pRM|`o1V(Rr9xLtqbX&iS!G0Xx+k=G%OVKCf{pKq$u z3A4Do+TFH-#^z&NykM)I@yo_jGv_p>o%XVR!>fn+_Mk!@Z%w}|41+s1mzZenBn8gb zOBg{@v){KFURY?pglA0%OZ-A@Y{G(S2P;H;I(1&|7XeV^zZ6jF+Tua=FiYnV0pESr zEQQ{cdbxZTpU?AAdT;ob=1mDZ5iN6kf%v$jszgF93k!lJk3dM#Z^gmpm>}GXH%0{+ zFmNIWOaK4?3*gI**PMDz53Pp+0N~I70F)0wt_puYZpKvmw@6cp0>=zcKRN2>H+CqU}|p;{Tb8tR)H0!`fvJ^lmo-<7)mN~ZAo wXS{!r|8L#y|89)`p5pyix=h4B>Hn7S{eO|6J|@t|Bk*xzd`t%Cf`4577xAUoH~;_u literal 0 HcmV?d00001 diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index d799af1216345..09f194278cdcc 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -343,7 +343,7 @@ def run(self): license="http://www.apache.org/licenses/LICENSE-2.0", # Don't forget to update python/docs/source/getting_started/install.rst # if you're updating the versions or dependencies. - install_requires=["py4j==0.10.9.7"], + install_requires=["py4j==0.10.9.8"], extras_require={ "ml": ["numpy>=%s" % _minimum_numpy_version], "mllib": ["numpy>=%s" % _minimum_numpy_version], diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 0bea4a45040ed..825ea0c63c6f6 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" - export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.7-src.zip:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.8-src.zip:${PYTHONPATH}" export PYSPARK_PYTHONPATH_SET=1 fi From 9c9bbf6a6d607942009b428fcd8ea4b87c17f35d Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 18 Dec 2024 15:42:05 +0800 Subject: [PATCH 206/438] [SPARK-50604][SQL][TESTS] Extract MariaDBDatabaseOnDocker and upgrade `Mariadb` docker image version ### What changes were proposed in this pull request? The pr aims to: - extract MariaDBDatabaseOnDocker - upgrade Mariadb docker image version from `10.6.19` to `10.11.10`. ### Why are the changes needed? - Align testing logic with other databases, eg: `DB2DatabaseOnDocker`, `MsSQLServerDatabaseOnDocker`, `MySQLDatabaseOnDocker`, `OracleDatabaseOnDocker` and `PostgresDatabaseOnDocker`. - Tests dependencies upgrading. ### 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? Closes #49059 from panbingkun/MariaDBDatabaseOnDocker. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../sql/jdbc/MariaDBDatabaseOnDocker.scala | 33 +++++++++++++++++++ .../sql/jdbc/MariaDBKrbIntegrationSuite.scala | 11 +------ 2 files changed, 34 insertions(+), 10 deletions(-) create mode 100644 connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBDatabaseOnDocker.scala diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBDatabaseOnDocker.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBDatabaseOnDocker.scala new file mode 100644 index 0000000000000..61930268eb2ab --- /dev/null +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBDatabaseOnDocker.scala @@ -0,0 +1,33 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging + +abstract class MariaDBDatabaseOnDocker extends DatabaseOnDocker with Logging { + override val imageName: String = + sys.env.getOrElse("MARIADB_DOCKER_IMAGE_NAME", "mariadb:10.11.10") + override val env: Map[String, String] = Map( + "MYSQL_ROOT_PASSWORD" -> "rootpass" + ) + override val usesIpc = false + override val jdbcPort = 3306 + + override def getEntryPoint: Option[String] = + Some("/docker-entrypoint/mariadb-docker-entrypoint.sh") +} diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala index 32c552eb8c7eb..962c70510b5bd 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala @@ -37,20 +37,11 @@ class MariaDBKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val userName = s"mariadb/$dockerIp" override protected val keytabFileName = "mariadb.keytab" - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("MARIADB_DOCKER_IMAGE_NAME", "mariadb:10.6.19") - override val env = Map( - "MYSQL_ROOT_PASSWORD" -> "rootpass" - ) - override val usesIpc = false - override val jdbcPort = 3306 + override val db = new MariaDBDatabaseOnDocker() { override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:mysql://$ip:$port/mysql?user=$principal" - override def getEntryPoint: Option[String] = - Some("/docker-entrypoint/mariadb-docker-entrypoint.sh") - override def beforeContainerStart( hostConfigBuilder: HostConfig, containerConfigBuilder: ContainerConfig): Unit = { From 76672cbf366ca4f11ae40da35229d670b205c780 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Wed, 18 Dec 2024 16:05:01 +0800 Subject: [PATCH 207/438] [SPARK-50472][SQL] Introduce initial implementation of the single-pass Analyzer ### What changes were proposed in this pull request? This is the initial PR for the single-pass Analyzer project. Coverage: - Scans of csv/json/parquet/orc/hive tables and files - `VALUES` - `WHERE` - `LIMIT` - Trivial star expansion - Literals - Aliases - Binary arithmetic - Binary logic Main components: `HybridAnalyzer`: The analysis is now routed between the fixed-point and single-pass by the `HybridAnalyzer`. There are 3 modes: - Fixed-point. In that case `HybridAnalyzer` just calls `executeAndTrack`. This is the default one. - Dual. This one is activated by the `spark.sql.analyzer.singlePassResolver.dualRunWithLegacy` flag and involves a filtering logic based on the `ResolverGuard`. First, the query is always analyzed by the fixed-point `Analyzer`, and then, if it's supported by the single-pass, it's analyzed by the `Resolver`. - Single-pass. In that case `HybridAnalyzer` calls the single-pass `Resolver` without any filtering or routing. This is useful for debugging. `ResolverGuard`: This component allows `HybridAnalyzer` to figure out whether a SQL query/DataFrame program is supported by the single-pass `Resolver`. Here we do a simple pass over the operator/expression tree with a short-circuited boolean logic. `Resolver`: This is the main entry point for the single-pass analysis. All the operators are resolved here. If the new operator is introduced into the Catalyst, it should be handled in the `resolve(...)` pattern match. The operator is either resolved here, or the resolution is delegated to the extensions. `ExpressionResolver`: Similar to the `Resolver`, this is the main entry point for the expression analysis. The resolution either happens here, or is delegated to specific resolvers (e.g. `BinaryArithmeticResolver`, `TimeZoneAwareExpressionResolver`). The `Resolver` calls the `ExpressionResolver` to resolve expressions nested in operators. `MetadataResolver`: The `MetadataResolver` is used by the `Resolver` to look up relation metadata and store it in a map by `RelationId`. There are two modes: - `AnalyzerBridgeState` from the fixed-point `Analyzer` is available. In that case we just reuse the existing relation metadata from the fixed-point run. This is done to avoid latency/consistency issues. - Otherwise, we collect the metadata using `RelationResolution`. `ResolverExtension`: This is a trait that allows extending the `Resolver`. Its `resolveOperator` must deterministically match the unresolved plan node - a conflict between the configured extensions is disallowed. `NameScope`: This component controls the visibility of names (tables, columns, aliases), star expansion for tables and structs and operator output handling. A new `NameScope` is created when a new scope is introduced (`Project`, `Aggregate`). We add operator output to the `NameScope` to later look up unresolved attributes and aliases. The `NameScope`s are stacked together to further support correlated names. `DataSourceResolver`/`DataSourceWithHiveResolver`: Those extensions allow us to resolve relation nodes (e.g. `UnresolvedCatalogRelation`, `LogicalRelation`, `HiveTableRelation`) defined outside of `catalyst` package to avoid depending on those packages (that would require a cyclic import). `ResolutionValidator`/`ExpressionResolutionValidator`: These components work together on the resolved tree and assert invariants that should be true no matter which SQL query/DataFrame program was passed to the single-pass analyzer. For example, we validate that expressions used in project lists actually exist in the lower operator outputs (using `AttributeScopeStack`). Every `resolve*` method in `Resolver`/`ExpressionResolver` must have its `validate*` counterpart in the `ResolutionValidator`/`ExpressionResolutionValidator`. `TracksResolvedNodes`: This is another validation mechanism which asserts that the analysis is actually single-pass - we store a reference to any resolved node in a set, and prior to the resolution check that a reference to the passed node hasn't been previously analyzed. Thanks mihailoale-db and mihailotim-db for working on that! ### Why are the changes needed? We want to build a new single-pass Analyzer to replace the fixed-point one. SPIP Jira: https://issues.apache.org/jira/browse/SPARK-49834 SPIP: https://docs.google.com/document/d/1dWxvrJV-0joGdLtWbvJ0uNyTocDMJ90rPRNWa4T56Og ### Does this PR introduce _any_ user-facing change? No, the single-pass Analyzer is disabled by default (under the `spark.sql.analyzer.singlePassResolver.enabled` and `spark.sql.analyzer.singlePassResolver.dualRunWithLegacy` flags). ### How was this patch tested? - New test suites - Ran some suites with `spark.sql.analyzer.singlePassResolver.dualRunWithLegacy` enabled ### Was this patch authored or co-authored using generative AI tooling? Copilot. Closes #49029 from vladimirg-db/vladimirg-db/single-pass-analyzer/initial-state. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 45 ++ .../sql/catalyst/analysis/Analyzer.scala | 92 ++- .../analysis/resolver/AliasResolver.scala | 142 ++++ .../resolver/AnalyzerBridgeState.scala | 40 ++ .../resolver/AttributeScopeStack.scala | 89 +++ .../resolver/BinaryArithmeticResolver.scala | 200 ++++++ .../resolver/BridgedRelationsProvider.scala | 50 ++ .../ConditionalExpressionResolver.scala | 65 ++ .../resolver/CreateNamedStructResolver.scala | 61 ++ .../DelegatesResolutionToExtensions.scala | 70 ++ ...ExplicitlyUnsupportedResolverFeature.scala | 57 ++ .../ExpressionResolutionValidator.scala | 367 ++++++++++ .../resolver/ExpressionResolver.scala | 340 +++++++++ .../analysis/resolver/FunctionResolver.scala | 91 +++ .../analysis/resolver/HybridAnalyzer.scala | 218 ++++++ .../analysis/resolver/IdentifierMap.scala | 37 + .../resolver/KeyTransformingMap.scala | 46 ++ .../resolver/LimitExpressionResolver.scala | 114 +++ .../analysis/resolver/MetadataResolver.scala | 133 ++++ .../analysis/resolver/NameScope.scala | 393 +++++++++++ .../analysis/resolver/NameTarget.scala | 81 +++ .../analysis/resolver/PlanLogger.scala | 58 ++ .../analysis/resolver/PredicateResolver.scala | 112 +++ .../resolver/ProducesUnresolvedSubtree.scala | 52 ++ .../analysis/resolver/RelationId.scala | 31 + .../resolver/RelationMetadataProvider.scala | 86 +++ .../resolver/ResolutionValidator.scala | 153 ++++ .../catalyst/analysis/resolver/Resolver.scala | 364 ++++++++++ .../analysis/resolver/ResolverExtension.scala | 43 ++ .../analysis/resolver/ResolverGuard.scala | 283 ++++++++ .../resolver/ResolvesExpressionChildren.scala | 33 + .../resolver/ResolvesOperatorChildren.scala | 36 + .../analysis/resolver/TimeAddResolver.scala | 72 ++ .../TimezoneAwareExpressionResolver.scala | 73 ++ .../resolver/TracksResolvedNodes.scala | 51 ++ .../analysis/resolver/TreeNodeResolver.scala | 30 + .../resolver/TypeCoercionResolver.scala | 50 ++ .../resolver/UnaryMinusResolver.scala | 60 ++ .../sql/catalyst/analysis/unresolved.scala | 19 +- .../catalog/TempVariableManager.scala | 4 + .../sql/errors/QueryCompilationErrors.scala | 77 ++ .../apache/spark/sql/internal/SQLConf.scala | 70 ++ .../LimitExpressionResolverSuite.scala | 94 +++ .../resolver/ResolutionValidatorSuite.scala | 265 +++++++ ...TimezoneAwareExpressionResolverSuite.scala | 58 ++ .../resolver/TypeCoercionResolverSuite.scala | 68 ++ .../datasources/DataSourceResolver.scala | 67 ++ .../execution/datasources/FileResolver.scala | 55 ++ .../sql/execution/datasources/rules.scala | 13 +- .../internal/BaseSessionStateBuilder.scala | 10 + ...citlyUnsupportedResolverFeatureSuite.scala | 91 +++ .../resolver/HybridAnalyzerSuite.scala | 404 +++++++++++ .../resolver/MetadataResolverSuite.scala | 277 ++++++++ .../analysis/resolver/NameScopeSuite.scala | 659 ++++++++++++++++++ .../resolver/ResolverGuardSuite.scala | 207 ++++++ .../sql/analysis/resolver/ResolverSuite.scala | 157 +++++ .../resolver/TracksResolvedNodesSuite.scala | 133 ++++ .../datasources/DataSourceResolverSuite.scala | 119 ++++ .../datasources/FileResolverSuite.scala | 97 +++ .../sql/hive/DataSourceWithHiveResolver.scala | 57 ++ .../sql/hive/HiveSessionStateBuilder.scala | 9 + .../DataSourceWithHiveResolverSuite.scala | 107 +++ .../hive/PartitionedTablePerfStatsSuite.scala | 3 + 63 files changed, 7425 insertions(+), 13 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AnalyzerBridgeState.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AttributeScopeStack.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BridgedRelationsProvider.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ConditionalExpressionResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CreateNamedStructResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/DelegatesResolutionToExtensions.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExplicitlyUnsupportedResolverFeature.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolutionValidator.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FunctionResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/IdentifierMap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameTarget.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PredicateResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationId.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationMetadataProvider.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverExtension.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesExpressionChildren.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesOperatorChildren.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimeAddResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TracksResolvedNodes.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TreeNodeResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolver.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnaryMinusResolver.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolverSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidatorSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolverSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolverSuite.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileResolver.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ExplicitlyUnsupportedResolverFeatureSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/HybridAnalyzerSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/MetadataResolverSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/NameScopeSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverGuardSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceResolverSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileResolverSuite.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolver.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolverSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index fca6a0d451de1..55265fa6dfc9c 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -70,6 +70,12 @@ ], "sqlState" : "42000" }, + "AMBIGUOUS_RESOLVER_EXTENSION" : { + "message" : [ + "The single-pass analyzer cannot process this query or command because the extension choice for is ambiguous: ." + ], + "sqlState" : "XX000" + }, "ARITHMETIC_OVERFLOW" : { "message" : [ ". If necessary set to \"false\" to bypass this error." @@ -1659,6 +1665,39 @@ ], "sqlState" : "22000" }, + "HYBRID_ANALYZER_EXCEPTION" : { + "message" : [ + "An failure occurred when attempting to resolve a query or command with both the legacy fixed-point analyzer as well as the single-pass resolver." + ], + "subClass" : { + "FIXED_POINT_FAILED_SINGLE_PASS_SUCCEEDED" : { + "message" : [ + "Fixed-point resolution failed, but single-pass resolution succeeded.", + "Single-pass analyzer output:", + "" + ] + }, + "LOGICAL_PLAN_COMPARISON_MISMATCH" : { + "message" : [ + "Outputs of fixed-point and single-pass analyzers do not match.", + "Fixed-point analyzer output:", + "", + "Single-pass analyzer output:", + "" + ] + }, + "OUTPUT_SCHEMA_COMPARISON_MISMATCH" : { + "message" : [ + "Output schemas of fixed-point and single-pass analyzers do not match.", + "Fixed-point analyzer output schema:", + "", + "Single-pass analyzer output schema:", + "" + ] + } + }, + "sqlState" : "XX000" + }, "IDENTIFIER_TOO_MANY_NAME_PARTS" : { "message" : [ " is not a valid identifier as it has more than 2 name parts." @@ -5659,6 +5698,12 @@ }, "sqlState" : "0A000" }, + "UNSUPPORTED_SINGLE_PASS_ANALYZER_FEATURE" : { + "message" : [ + "The single-pass analyzer cannot process this query or command because it does not yet support ." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_STREAMING_OPERATOR_WITHOUT_WATERMARK" : { "message" : [ " output mode not supported for on streaming DataFrames/DataSets without watermark." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1c31c45d27296..7c97fccade018 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -28,6 +28,13 @@ import scala.util.{Failure, Random, Success, Try} import org.apache.spark.{SparkException, SparkThrowable, SparkUnsupportedOperationException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ +import org.apache.spark.sql.catalyst.analysis.resolver.{ + AnalyzerBridgeState, + HybridAnalyzer, + Resolver => OperatorResolver, + ResolverExtension, + ResolverGuard +} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ @@ -146,7 +153,26 @@ case class AnalysisContext( // lookup a temporary function. And export to the view metadata. referredTempFunctionNames: mutable.Set[String] = mutable.Set.empty, referredTempVariableNames: Seq[Seq[String]] = Seq.empty, - outerPlan: Option[LogicalPlan] = None) + outerPlan: Option[LogicalPlan] = None, + + /** + * This is a bridge state between this fixed-point [[Analyzer]] and a single-pass [[Resolver]]. + * It's managed ([[setSinglePassResolverBridgeState]] method) by the [[HybridAnalyzer]] - the + * goal is to preserve it correctly between the fixed-point and single-pass runs. + * [[AnalysisContext.reset]] simply propagates it to prevent it from being reset in + * [[Analyzer.execute]]. Normally it's always [[None]], unless + * [[ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER]] is set to [[true]]. + * + * See [[AnalyzerBridgeState]] and [[HybridAnalyzer]] for more info. + */ + private var singlePassResolverBridgeState: Option[AnalyzerBridgeState] = None) { + + def setSinglePassResolverBridgeState(bridgeState: Option[AnalyzerBridgeState]): Unit = + singlePassResolverBridgeState = bridgeState + + def getSinglePassResolverBridgeState: Option[AnalyzerBridgeState] = + singlePassResolverBridgeState +} object AnalysisContext { private val value = new ThreadLocal[AnalysisContext]() { @@ -154,7 +180,16 @@ object AnalysisContext { } def get: AnalysisContext = value.get() - def reset(): Unit = value.remove() + + def reset(): Unit = { + // We need to preserve the single-pass resolver bridge state here, since it's managed by the + // [[HybridAnalyzer]] (set or reset to `None`) to avoid it being reset in [[execute]]. + // It acts as a bridge between the single-pass and fixed-point analyzers in the absence of any + // other explicit state. + val prevSinglePassResolverBridgeState = value.get.getSinglePassResolverBridgeState + value.remove() + value.get.setSinglePassResolverBridgeState(prevSinglePassResolverBridgeState) + } private def set(context: AnalysisContext): Unit = value.set(context) @@ -219,9 +254,15 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { if (plan.analyzed) return plan AnalysisHelper.markInAnalyzer { - val analyzed = executeAndTrack(plan, tracker) - checkAnalysis(analyzed) - analyzed + new HybridAnalyzer( + this, + new ResolverGuard(catalogManager), + new OperatorResolver( + catalogManager, + singlePassResolverExtensions, + singlePassMetadataResolverExtensions + ) + ).apply(plan, tracker) } } @@ -245,6 +286,20 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor errorOnExceed = true, maxIterationsSetting = SQLConf.ANALYZER_MAX_ITERATIONS.key) + /** + * Extensions for the single-pass analyzer. + * + * See [[ResolverExtension]] for more info. + */ + val singlePassResolverExtensions: Seq[ResolverExtension] = Nil + + /** + * Extensions used for early resolution of the single-pass analyzer. + * + * See [[ResolverExtension]] for more info. + */ + val singlePassMetadataResolverExtensions: Seq[ResolverExtension] = Nil + /** * Override to provide additional rules for the "Resolution" batch. */ @@ -1018,7 +1073,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case i @ InsertIntoStatement(table, _, _, _, _, _, _) => val relation = table match { case u: UnresolvedRelation if !u.isStreaming => - relationResolution.resolveRelation(u).getOrElse(u) + resolveRelation(u).getOrElse(u) case other => other } @@ -1035,7 +1090,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case write: V2WriteCommand => write.table match { case u: UnresolvedRelation if !u.isStreaming => - relationResolution.resolveRelation(u).map(unwrapRelationPlan).map { + resolveRelation(u).map(unwrapRelationPlan).map { case v: View => throw QueryCompilationErrors.writeIntoViewNotAllowedError( v.desc.identifier, write) case r: DataSourceV2Relation => write.withNewTable(r) @@ -1050,12 +1105,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } case u: UnresolvedRelation => - relationResolution.resolveRelation(u).map(resolveViews).getOrElse(u) + resolveRelation(u).map(resolveViews).getOrElse(u) case r @ RelationTimeTravel(u: UnresolvedRelation, timestamp, version) if timestamp.forall(ts => ts.resolved && !SubqueryExpression.hasSubquery(ts)) => val timeTravelSpec = TimeTravelSpec.create(timestamp, version, conf.sessionLocalTimeZone) - relationResolution.resolveRelation(u, timeTravelSpec).getOrElse(r) + resolveRelation(u, timeTravelSpec).getOrElse(r) case u @ UnresolvedTable(identifier, cmd, suggestAlternative) => lookupTableOrView(identifier).map { @@ -1119,6 +1174,25 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } } + + def resolveRelation( + unresolvedRelation: UnresolvedRelation, + timeTravelSpec: Option[TimeTravelSpec] = None): Option[LogicalPlan] = { + relationResolution + .resolveRelation( + unresolvedRelation, + timeTravelSpec + ) + .map { relation => + // We put the synchronously resolved relation into the [[AnalyzerBridgeState]] for + // it to be later reused by the single-pass [[Resolver]] to avoid resolving the relation + // metadata twice. + AnalysisContext.get.getSinglePassResolverBridgeState.map { bridgeState => + bridgeState.relationsWithResolvedMetadata.put(unresolvedRelation, relation) + } + relation + } + } } /** Handle INSERT INTO for DSv2 */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala new file mode 100644 index 0000000000000..7b652437dbd8b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala @@ -0,0 +1,142 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{AliasResolution, UnresolvedAlias} +import org.apache.spark.sql.catalyst.expressions.{ + Alias, + Cast, + CreateNamedStruct, + Expression, + NamedExpression +} + +/** + * Resolver class that resolves unresolved aliases and handles user-specified aliases. + */ +class AliasResolver(expressionResolver: ExpressionResolver, scopes: NameScopeStack) + extends TreeNodeResolver[UnresolvedAlias, Expression] + with ResolvesExpressionChildren { + + /** + * Resolves [[UnresolvedAlias]] by handling two specific cases: + * - Alias(CreateNamedStruct(...)) - instead of calling [[CreateNamedStructResolver]] which will + * clean up its inner aliases, we manually resolve [[CreateNamedStruct]]'s children, because we + * need to preserve inner aliases until after the alias name is computed. This is a hack because + * fixed-point analyzer computes [[Alias]] name before removing inner aliases. + * - Alias(...) - recursively call [[ExpressionResolver]] to resolve the child expression. + * + * After the children are resolved, call [[AliasResolution]] to compute the alias name. Finally, + * clean up inner aliases from [[CreateNamedStruct]]. + */ + override def resolve(unresolvedAlias: UnresolvedAlias): NamedExpression = { + val aliasWithResolvedChildren = withResolvedChildren( + unresolvedAlias, { + case createNamedStruct: CreateNamedStruct => + withResolvedChildren(createNamedStruct, expressionResolver.resolve) + case other => expressionResolver.resolve(other) + } + ) + + val resolvedAlias = + AliasResolution.resolve(aliasWithResolvedChildren).asInstanceOf[NamedExpression] + + scopes.top.addAlias(resolvedAlias.name) + AliasResolver.cleanupAliases(resolvedAlias) + } + + /** + * Handle already resolved [[Alias]] nodes, i.e. user-specified aliases. We disallow stacking + * of [[Alias]] nodes by collapsing them so that only the top node remains. + * + * For an example query like: + * + * {{{ SELECT 1 AS a }}} + * + * parsed plan will be: + * + * Project [Alias(1, a)] + * +- OneRowRelation + * + */ + def handleResolvedAlias(alias: Alias): Alias = { + val aliasWithResolvedChildren = withResolvedChildren(alias, expressionResolver.resolve) + scopes.top.addAlias(aliasWithResolvedChildren.name) + AliasResolver.collapseAlias(aliasWithResolvedChildren) + } +} + +object AliasResolver { + + /** + * For a query like: + * + * {{{ SELECT STRUCT(1 AS a, 2 AS b) AS st }}} + * + * After resolving [[CreateNamedStruct]] the plan will be: + * CreateNamedStruct(Seq("a", Alias(1, "a"), "b", Alias(2, "b"))) + * + * For a query like: + * + * {{{ df.select($"col1".cast("int").cast("double")) }}} + * + * After resolving top-most [[Alias]] the plan will be: + * Alias(Cast(Alias(Cast(col1, int), col1)), double), col1) + * + * Both examples contain inner aliases that are not expected in the analyzed logical plan, + * therefore need to be removed. However, in both examples inner aliases are necessary in order + * for the outer alias to compute its name. To achieve this, we delay removal of inner aliases + * until after the outer alias name is computed. + * + * For cases where there are no dependencies on inner alias, inner alias should be removed by the + * resolver that produces it. + */ + private def cleanupAliases(namedExpression: NamedExpression): NamedExpression = + namedExpression + .withNewChildren(namedExpression.children.map { + case cast @ Cast(alias: Alias, _, _, _) => + cast.copy(child = alias.child) + case createNamedStruct: CreateNamedStruct => + CreateNamedStructResolver.cleanupAliases(createNamedStruct) + case other => other + }) + .asInstanceOf[NamedExpression] + + /** + * If an [[Alias]] node appears on top of another [[Alias]], remove the bottom one. Here we don't + * handle a case where a node of different type appears between two [[Alias]] nodes: in this + * case, removal of inner alias (if it is unnecessary) should be handled by respective node's + * resolver, in order to preserve the bottom-up contract. + */ + private def collapseAlias(alias: Alias): Alias = + alias.child match { + case innerAlias: Alias => + val metadata = if (alias.metadata.isEmpty) { + None + } else { + Some(alias.metadata) + } + alias.copy(child = innerAlias.child)( + exprId = alias.exprId, + qualifier = alias.qualifier, + explicitMetadata = metadata, + nonInheritableMetadataKeys = alias.nonInheritableMetadataKeys + ) + case _ => alias + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AnalyzerBridgeState.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AnalyzerBridgeState.scala new file mode 100644 index 0000000000000..d3e93c82dfa21 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AnalyzerBridgeState.scala @@ -0,0 +1,40 @@ +/* + * 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.catalyst.analysis.resolver + +import java.util.HashMap + +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * The [[AnalyzerBridgeState]] is a state passed from legacy [[Analyzer]] to the single-pass + * [[Resolver]]. + * + * @param relationsWithResolvedMetadata A map from [[UnresolvedRelation]] to the relations with + * resolved metadata. It allows us to reuse the relation metadata and avoid duplicate + * catalog/table lookups in dual-run mode (when + * [[ANALYZER_SINGLE_PASS_RESOLVER_RELATION_BRIDGING_ENABLED]] is true). + */ +case class AnalyzerBridgeState( + relationsWithResolvedMetadata: AnalyzerBridgeState.RelationsWithResolvedMetadata = + new AnalyzerBridgeState.RelationsWithResolvedMetadata) + +object AnalyzerBridgeState { + type RelationsWithResolvedMetadata = HashMap[UnresolvedRelation, LogicalPlan] +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AttributeScopeStack.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AttributeScopeStack.scala new file mode 100644 index 0000000000000..6f9d6defd2edb --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AttributeScopeStack.scala @@ -0,0 +1,89 @@ +/* + * 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.catalyst.analysis.resolver + +import java.util.ArrayDeque + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet} + +/** + * The [[AttributeScopeStack]] is used to validate that the attribute which was encountered by the + * [[ExpressionResolutionValidator]] is in the current operator's visibility scope. We use + * [[AttributeSet]] as scope implementation here to check the equality of attributes based on their + * expression IDs. + * + * E.g. for the following SQL query: + * {{{ + * SELECT a, a, a + col2 FROM (SELECT col1 as a, col2 FROM VALUES (1, 2)); + * }}} + * + * Having the following logical plan: + * {{{ + * Project [a#2, a#2, (a#2 + col2#1) AS (a + col2)#3] + * +- SubqueryAlias __auto_generated_subquery_name + * +- Project [col1#0 AS a#2, col2#1] + * +- LocalRelation [col1#0, col2#1] + * }}} + * + * The [[LocalRelation]] outputs attributes with IDs #0 and #1, which can be referenced by the lower + * [[Project]]. This [[Project]] produces a new attribute ID #2 for an alias and retains the old + * ID #1 for col2. The upper [[Project]] references `a` twice using the same ID #2 and produces a + * new ID #3 for an alias of `a + col2`. + */ +class AttributeScopeStack { + private val stack = new ArrayDeque[AttributeSet] + push() + + /** + * Get the relevant attribute scope in the context of the current operator. + */ + def top: AttributeSet = { + stack.peek() + } + + /** + * Overwrite current relevant scope with a sequence of attributes which is an output of some + * operator. `attributes` can have duplicate IDs if the output of the operator contains multiple + * occurrences of the same attribute. + */ + def overwriteTop(attributes: Seq[Attribute]): Unit = { + stack.pop() + stack.push(AttributeSet(attributes)) + } + + /** + * Execute `body` in the context of a fresh attribute scope. Used by [[Project]] and [[Aggregate]] + * validation code since those operators introduce a new scope with fresh expression IDs. + */ + def withNewScope[R](body: => R): Unit = { + push() + try { + body + } finally { + pop() + } + } + + private def push(): Unit = { + stack.push(AttributeSet(Seq.empty)) + } + + private def pop(): Unit = { + stack.pop() + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala new file mode 100644 index 0000000000000..7d9c6752094d7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala @@ -0,0 +1,200 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{ + AnsiStringPromotionTypeCoercion, + AnsiTypeCoercion, + BinaryArithmeticWithDatetimeResolver, + DecimalPrecisionTypeCoercion, + DivisionTypeCoercion, + IntegralDivisionTypeCoercion, + StringPromotionTypeCoercion, + TypeCoercion +} +import org.apache.spark.sql.catalyst.expressions.{ + Add, + BinaryArithmetic, + DateAdd, + Divide, + Expression, + Multiply, + Subtract, + SubtractDates +} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DateType, StringType} + +/** + * [[BinaryArithmeticResolver]] is invoked by [[ExpressionResolver]] in order to resolve + * [[BinaryArithmetic]] nodes. During resolution, calling [[BinaryArithmeticWithDatetimeResolver]] + * and applying type coercion can result in [[BinaryArithmetic]] producing some other type of node + * or a subtree of nodes. In such cases a downwards traversal is necessary, but not going deeper + * than the original expression's children, since all nodes below that point are guaranteed to be + * already resolved. + * + * For example, given a query: + * + * SELECT '4 11:11' - INTERVAL '4 22:12' DAY TO MINUTE + * + * [[BinaryArithmeticResolver]] is called for the following expression: + * + * Subtract( + * Literal('4 11:11', StringType), + * Literal(Interval('4 22:12' DAY TO MINUTE), DayTimeIntervalType(0,2)) + * ) + * + * After calling [[BinaryArithmeticWithDatetimeResolver]] and applying type coercion, + * the expression is transformed into: + * + * Cast( + * DatetimeSub( + * TimeAdd( + * Literal('4 11:11', StringType), + * UnaryMinus( + * Literal(Interval('4 22:12' DAY TO MINUTE), DayTimeIntervalType(0,2)) + * ) + * ) + * ) + * ) + * + * A single [[Subtract]] node is replaced with a subtree of nodes. In order to resolve this subtree + * we need to invoke [[ExpressionResolver]] recursively on the top-most node's children. The + * top-most node itself is not resolved recursively in order to avoid recursive calls to + * [[BinaryArithmeticResolver]] and other sub-resolvers. To prevent a case where we resolve the + * same node twice, we need to mark nodes that will act as a limit for the downwards traversal by + * applying a [[ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY]] tag to them. These children + * along with all the nodes below them are guaranteed to be resolved at this point. When + * [[ExpressionResolver]] reaches one of the tagged nodes, it returns identity rather than + * resolving it. Finally, after resolving the subtree, we need to resolve the top-most node itself, + * which in this case means applying a timezone, if necessary. + */ +class BinaryArithmeticResolver( + expressionResolver: ExpressionResolver, + timezoneAwareExpressionResolver: TimezoneAwareExpressionResolver) + extends TreeNodeResolver[BinaryArithmetic, Expression] + with ProducesUnresolvedSubtree { + + private val shouldTrackResolvedNodes = + conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_TRACK_RESOLVED_NODES_ENABLED) + + private val typeCoercionRules: Seq[Expression => Expression] = + if (conf.ansiEnabled) { + BinaryArithmeticResolver.ANSI_TYPE_COERCION_RULES + } else { + BinaryArithmeticResolver.TYPE_COERCION_RULES + } + private val typeCoercionResolver: TypeCoercionResolver = + new TypeCoercionResolver(timezoneAwareExpressionResolver, typeCoercionRules) + + override def resolve(unresolvedBinaryArithmetic: BinaryArithmetic): Expression = { + val binaryArithmeticWithResolvedChildren: BinaryArithmetic = + withResolvedChildren(unresolvedBinaryArithmetic, expressionResolver.resolve) + val binaryArithmeticWithResolvedSubtree: Expression = + withResolvedSubtree(binaryArithmeticWithResolvedChildren, expressionResolver.resolve) { + transformBinaryArithmeticNode(binaryArithmeticWithResolvedChildren) + } + val binaryArithmeticWithResolvedTimezone = timezoneAwareExpressionResolver.withResolvedTimezone( + binaryArithmeticWithResolvedSubtree, + conf.sessionLocalTimeZone + ) + reallocateKnownNodesForTracking(binaryArithmeticWithResolvedTimezone) + } + + /** + * Transform [[BinaryArithmetic]] node by calling [[BinaryArithmeticWithDatetimeResolver]] and + * applying type coercion. Initial node can be replaced with some other type of node or a subtree + * of nodes. + */ + private def transformBinaryArithmeticNode(binaryArithmetic: BinaryArithmetic): Expression = { + val binaryArithmeticWithDateTypeReplaced: Expression = + replaceDateType(binaryArithmetic) + val binaryArithmeticWithTypeCoercion: Expression = + typeCoercionResolver.resolve(binaryArithmeticWithDateTypeReplaced) + // In case that original expression's children types are DateType and StringType, fixed-point + // fails to resolve the expression with a single application of + // [[BinaryArithmeticWithDatetimeResolver]]. Therefore, single-pass resolver needs to invoke + // [[BinaryArithmeticWithDatetimeResolver.resolve]], type coerce and only after that fix the + // date/string case. Instead of invoking [[BinaryArithmeticWithDatetimeResolver]] again, we + // handle the case directly. + ( + binaryArithmetic.left.dataType, + binaryArithmetic.right.dataType + ) match { + case (_: DateType, _: StringType) => + binaryArithmeticWithTypeCoercion match { + case add: Add => DateAdd(add.left, add.right) + case subtract: Subtract => SubtractDates(subtract.left, subtract.right) + case other => other + } + case _ => binaryArithmeticWithTypeCoercion + } + } + + /** + * When DateType like operand is given to [[BinaryArithmetic]], apply + * [[BinaryArithmeticWithDatetimeResolver]] in order to replace the [[BinaryArithmetic]] with + * the appropriate equivalent for DateTime types. + */ + private def replaceDateType(expression: Expression) = expression match { + case arithmetic @ (_: Add | _: Subtract | _: Multiply | _: Divide) => + BinaryArithmeticWithDatetimeResolver.resolve(arithmetic) + case other => other + } + + /** + * Since [[TracksResolvedNodes]] requires all the expressions in the tree to be unique objects, + * we reallocate the known nodes in [[ANALYZER_SINGLE_PASS_TRACK_RESOLVED_NODES_ENABLED]] mode, + * otherwise we preserve the old object to avoid unnecessary memory allocations. + */ + private def reallocateKnownNodesForTracking(expression: Expression): Expression = { + if (shouldTrackResolvedNodes) { + expression match { + case add: Add => add.copy() + case subtract: Subtract => subtract.copy() + case multiply: Multiply => multiply.copy() + case divide: Divide => divide.copy() + case _ => expression + } + } else { + expression + } + } +} + +object BinaryArithmeticResolver { + // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]]. + private val TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + StringPromotionTypeCoercion.apply, + DecimalPrecisionTypeCoercion.apply, + DivisionTypeCoercion.apply, + IntegralDivisionTypeCoercion.apply, + TypeCoercion.ImplicitTypeCoercion.apply, + TypeCoercion.DateTimeOperationsTypeCoercion.apply + ) + + // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]]. + private val ANSI_TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + AnsiStringPromotionTypeCoercion.apply, + DecimalPrecisionTypeCoercion.apply, + DivisionTypeCoercion.apply, + IntegralDivisionTypeCoercion.apply, + AnsiTypeCoercion.ImplicitTypeCoercion.apply, + AnsiTypeCoercion.AnsiDateTimeOperationsTypeCoercion.apply + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BridgedRelationsProvider.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BridgedRelationsProvider.scala new file mode 100644 index 0000000000000..bc7a9df064c33 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BridgedRelationsProvider.scala @@ -0,0 +1,50 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.RelationResolution +import org.apache.spark.sql.connector.catalog.CatalogManager + +/** + * The [[BridgedRelationMetadataProvider]] is a [[RelationMetadataProvider]] that just reuses + * resolved metadata from the [[AnalyzerBridgeState]]. This is used in the single-pass [[Resolver]] + * to avoid duplicate catalog/table lookups in dual-run mode, so metadata is simply reused from the + * fixed-point [[Analyzer]] run. We strictly rely on the [[AnalyzerBridgeState]] to avoid any + * blocking calls here. + */ +class BridgedRelationMetadataProvider( + override val catalogManager: CatalogManager, + override val relationResolution: RelationResolution, + analyzerBridgeState: AnalyzerBridgeState +) extends RelationMetadataProvider { + override val relationsWithResolvedMetadata = getRelationsFromBridgeState(analyzerBridgeState) + + private def getRelationsFromBridgeState( + analyzerBridgeState: AnalyzerBridgeState): RelationsWithResolvedMetadata = { + val result = new RelationsWithResolvedMetadata + analyzerBridgeState.relationsWithResolvedMetadata.forEach( + (unresolvedRelation, relationWithResolvedMetadata) => { + result.put( + relationIdFromUnresolvedRelation(unresolvedRelation), + relationWithResolvedMetadata + ) + } + ) + result + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ConditionalExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ConditionalExpressionResolver.scala new file mode 100644 index 0000000000000..75ba1b7a01a5c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ConditionalExpressionResolver.scala @@ -0,0 +1,65 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.{AnsiTypeCoercion, TypeCoercion} +import org.apache.spark.sql.catalyst.expressions.{ConditionalExpression, Expression} + +/** + * Resolver for [[If]], [[CaseWhen]] and [[Coalesce]] expressions. + */ +class ConditionalExpressionResolver( + expressionResolver: ExpressionResolver, + timezoneAwareExpressionResolver: TimezoneAwareExpressionResolver) + extends TreeNodeResolver[ConditionalExpression, Expression] + with ResolvesExpressionChildren + with SQLConfHelper { + + private val typeCoercionRules: Seq[Expression => Expression] = + if (conf.ansiEnabled) { + ConditionalExpressionResolver.ANSI_TYPE_COERCION_RULES + } else { + ConditionalExpressionResolver.TYPE_COERCION_RULES + } + private val typeCoercionResolver: TypeCoercionResolver = + new TypeCoercionResolver(timezoneAwareExpressionResolver, typeCoercionRules) + + override def resolve(unresolvedConditionalExpression: ConditionalExpression): Expression = { + val conditionalExpressionWithResolvedChildren = + withResolvedChildren(unresolvedConditionalExpression, expressionResolver.resolve) + + typeCoercionResolver.resolve(conditionalExpressionWithResolvedChildren) + } +} + +object ConditionalExpressionResolver { + // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]]. + private val TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + TypeCoercion.CaseWhenTypeCoercion.apply, + TypeCoercion.FunctionArgumentTypeCoercion.apply, + TypeCoercion.IfTypeCoercion.apply + ) + + // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]]. + private val ANSI_TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + AnsiTypeCoercion.CaseWhenTypeCoercion.apply, + AnsiTypeCoercion.FunctionArgumentTypeCoercion.apply, + AnsiTypeCoercion.IfTypeCoercion.apply + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CreateNamedStructResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CreateNamedStructResolver.scala new file mode 100644 index 0000000000000..12c3c71b5e8be --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CreateNamedStructResolver.scala @@ -0,0 +1,61 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.expressions.{Alias, CreateNamedStruct, Expression} + +/** + * Resolves [[CreateNamedStruct]] nodes by recursively resolving children. If [[CreateNamedStruct]] + * is not directly under an [[Alias]], removes aliases from struct fields. Otherwise, let + * [[AliasResolver]] handle the removal. + */ +class CreateNamedStructResolver(expressionResolver: ExpressionResolver) + extends TreeNodeResolver[CreateNamedStruct, Expression] + with ResolvesExpressionChildren { + + override def resolve(createNamedStruct: CreateNamedStruct): Expression = { + val createNamedStructWithResolvedChildren = + withResolvedChildren(createNamedStruct, expressionResolver.resolve) + CreateNamedStructResolver.cleanupAliases(createNamedStructWithResolvedChildren) + } +} + +object CreateNamedStructResolver { + + /** + * For a query like: + * + * {{{ SELECT STRUCT(1 AS a, 2 AS b) }}} + * + * [[CreateNamedStruct]] will be: CreateNamedStruct(Seq("a", Alias(1, "a"), "b", Alias(2, "b"))) + * + * Because inner aliases are not expected in the analyzed logical plan, we need to remove them + * here. However, we only do so if [[CreateNamedStruct]] is not directly under an [[Alias]], in + * which case the removal will be handled by [[AliasResolver]]. This is because in single-pass, + * [[Alias]] is resolved after [[CreateNamedStruct]] and in order to compute the correct output + * name, it needs to know complete structure of the child. + */ + def cleanupAliases(createNamedStruct: CreateNamedStruct): CreateNamedStruct = { + createNamedStruct + .withNewChildren(createNamedStruct.children.map { + case a: Alias if a.metadata.isEmpty => a.child + case other => other + }) + .asInstanceOf[CreateNamedStruct] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/DelegatesResolutionToExtensions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/DelegatesResolutionToExtensions.scala new file mode 100644 index 0000000000000..7d57e4683df40 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/DelegatesResolutionToExtensions.scala @@ -0,0 +1,70 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * The [[DelegatesResolutionToExtensions]] is a trait which provides a method to delegate the + * resolution of unresolved operators to a list of [[ResolverExtension]]s. + */ +trait DelegatesResolutionToExtensions { + + protected val extensions: Seq[ResolverExtension] + + /** + * Find the suitable extension for `unresolvedOperator` resolution and resolve it with that + * extension. Usually extensions return resolved relation nodes, so we generically update the name + * scope without matching for specific relations, for simplicity. + * + * We match the extension once to reduce the number of + * [[ResolverExtension.resolveOperator.isDefinedAt]] calls, because those can be expensive. + * + * @returns `Some(resolutionResult)` if the extension was found and `unresolvedOperator` was + * resolved, `None` otherwise. + * + * @throws `AMBIGUOUS_RESOLVER_EXTENSION` if there were several matched extensions for this + * operator. + */ + def tryDelegateResolutionToExtension(unresolvedOperator: LogicalPlan): Option[LogicalPlan] = { + var resolutionResult: Option[LogicalPlan] = None + var matchedExtension: Option[ResolverExtension] = None + extensions.foreach { extension => + matchedExtension match { + case None => + resolutionResult = extension.resolveOperator.lift(unresolvedOperator) + + if (resolutionResult.isDefined) { + matchedExtension = Some(extension) + } + case Some(matchedExtension) => + if (extension.resolveOperator.isDefinedAt(unresolvedOperator)) { + throw QueryCompilationErrors + .ambiguousResolverExtension( + unresolvedOperator, + Seq(matchedExtension, extension).map(_.getClass.getSimpleName) + ) + .withPosition(unresolvedOperator.origin) + } + } + } + + resolutionResult + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExplicitlyUnsupportedResolverFeature.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExplicitlyUnsupportedResolverFeature.scala new file mode 100644 index 0000000000000..e6279c9740395 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExplicitlyUnsupportedResolverFeature.scala @@ -0,0 +1,57 @@ +/* + * 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.catalyst.analysis.resolver + +/** + * This is an addon to [[ResolverGuard]] functionality for features that cannot be determined by + * only looking at the unresolved plan. [[Resolver]] will throw this control-flow exception + * when it encounters some explicitly unsupported feature. Later behavior depends on the value of + * [[HybridAnalyzer.checkSupportedSinglePassFeatures]] flag: + * - If it is true: It will later be caught by [[HybridAnalyzer]] to abort single-pass + * analysis without comparing single-pass and fixed-point results. The motivation for this + * feature is the same as for the [[ResolverGuard]] - we want to have an explicit allowlist of + * unimplemented features that we are aware of, and `UNSUPPORTED_SINGLE_PASS_ANALYZER_FEATURE` + * will signal us the rest of the gaps. + * - If it is false: It will be thrown by the [[HybridAnalyzer]] in order to get better sense + * of coverage. + * + * For example, [[UnresolvedRelation]] can be intermediately resolved by [[ResolveRelations]] as + * [[UnresolvedCatalogRelation]] or a [[View]] (among all others). Say that for now the views + * are not implemented, and we are aware of that, so [[ExplicitlyUnsupportedResolverFeature]] will + * be thrown in the middle of the single-pass analysis to abort it. + */ +class ExplicitlyUnsupportedResolverFeature(reason: String) + extends Exception( + s"The single-pass analyzer cannot process this query or command because it does not yet " + + s"support $reason." + ) { + override def getStackTrace(): Array[StackTraceElement] = new Array[StackTraceElement](0) + override def fillInStackTrace(): Throwable = this +} + +/** + * This object contains all the metadata on explicitly unsupported resolver features. + */ +object ExplicitlyUnsupportedResolverFeature { + val OPERATORS = Set( + "org.apache.spark.sql.catalyst.plans.logical.View", + "org.apache.spark.sql.catalyst.streaming.StreamingRelationV2", + "org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation", + "org.apache.spark.sql.execution.streaming.StreamingRelation" + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolutionValidator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolutionValidator.scala new file mode 100644 index 0000000000000..8c80992e2fa2c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolutionValidator.scala @@ -0,0 +1,367 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.expressions.{ + Alias, + ArrayDistinct, + ArrayInsert, + ArrayJoin, + ArrayMax, + ArrayMin, + ArraysZip, + AttributeReference, + BinaryExpression, + ConditionalExpression, + CreateArray, + CreateMap, + CreateNamedStruct, + Expression, + ExtractANSIIntervalDays, + GetArrayStructFields, + GetMapValue, + GetStructField, + Literal, + MapConcat, + MapContainsKey, + MapEntries, + MapFromEntries, + MapKeys, + MapValues, + NamedExpression, + Predicate, + RuntimeReplaceable, + StringRPad, + StringToMap, + TimeZoneAwareExpression, + UnaryMinus +} +import org.apache.spark.sql.types.BooleanType + +/** + * The [[ExpressionResolutionValidator]] performs the validation work on the expression tree for the + * [[ResolutionValidator]]. These two components work together recursively validating the + * logical plan. You can find more info in the [[ResolutionValidator]] scaladoc. + */ +class ExpressionResolutionValidator(resolutionValidator: ResolutionValidator) { + + /** + * Validate resolved expression tree. The principle is the same as + * [[ResolutionValidator.validate]]. + */ + def validate(expression: Expression): Unit = { + expression match { + case attributeReference: AttributeReference => + validateAttributeReference(attributeReference) + case alias: Alias => + validateAlias(alias) + case getMapValue: GetMapValue => + validateGetMapValue(getMapValue) + case binaryExpression: BinaryExpression => + validateBinaryExpression(binaryExpression) + case extractANSIIntervalDay: ExtractANSIIntervalDays => + validateExtractANSIIntervalDays(extractANSIIntervalDay) + case literal: Literal => + validateLiteral(literal) + case predicate: Predicate => + validatePredicate(predicate) + case stringRPad: StringRPad => + validateStringRPad(stringRPad) + case unaryMinus: UnaryMinus => + validateUnaryMinus(unaryMinus) + case getStructField: GetStructField => + validateGetStructField(getStructField) + case createNamedStruct: CreateNamedStruct => + validateCreateNamedStruct(createNamedStruct) + case getArrayStructFields: GetArrayStructFields => + validateGetArrayStructFields(getArrayStructFields) + case createMap: CreateMap => + validateCreateMap(createMap) + case stringToMap: StringToMap => + validateStringToMap(stringToMap) + case mapContainsKey: MapContainsKey => + validateMapContainsKey(mapContainsKey) + case mapConcat: MapConcat => + validateMapConcat(mapConcat) + case mapKeys: MapKeys => + validateMapKeys(mapKeys) + case mapValues: MapValues => + validateMapValues(mapValues) + case mapEntries: MapEntries => + validateMapEntries(mapEntries) + case mapFromEntries: MapFromEntries => + validateMapFromEntries(mapFromEntries) + case createArray: CreateArray => + validateCreateArray(createArray) + case arrayDistinct: ArrayDistinct => + validateArrayDistinct(arrayDistinct) + case arrayInsert: ArrayInsert => + validateArrayInsert(arrayInsert) + case arrayJoin: ArrayJoin => + validateArrayJoin(arrayJoin) + case arrayMax: ArrayMax => + validateArrayMax(arrayMax) + case arrayMin: ArrayMin => + validateArrayMin(arrayMin) + case arraysZip: ArraysZip => + validateArraysZip(arraysZip) + case conditionalExpression: ConditionalExpression => + validateConditionalExpression(conditionalExpression) + case runtimeReplaceable: RuntimeReplaceable => + validateRuntimeReplaceable(runtimeReplaceable) + case timezoneExpression: TimeZoneAwareExpression => + validateTimezoneExpression(timezoneExpression) + } + } + + def validateProjectList(projectList: Seq[NamedExpression]): Unit = { + projectList.foreach { + case attributeReference: AttributeReference => + validateAttributeReference(attributeReference) + case alias: Alias => + validateAlias(alias) + } + } + + private def validatePredicate(predicate: Predicate) = { + predicate.children.foreach(validate) + assert( + predicate.dataType == BooleanType, + s"Output type of a predicate must be a boolean, but got: ${predicate.dataType.typeName}" + ) + assert( + predicate.checkInputDataTypes().isSuccess, + "Input types of a predicate must be valid, but got: " + + predicate.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateStringRPad(stringRPad: StringRPad) = { + validate(stringRPad.first) + validate(stringRPad.second) + validate(stringRPad.third) + assert( + stringRPad.checkInputDataTypes().isSuccess, + "Input types of rpad must be valid, but got: " + + stringRPad.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateAttributeReference(attributeReference: AttributeReference): Unit = { + assert( + resolutionValidator.attributeScopeStack.top.contains(attributeReference), + s"Attribute $attributeReference is missing from attribute scope: " + + s"${resolutionValidator.attributeScopeStack.top}" + ) + } + + private def validateAlias(alias: Alias): Unit = { + validate(alias.child) + } + + private def validateBinaryExpression(binaryExpression: BinaryExpression): Unit = { + validate(binaryExpression.left) + validate(binaryExpression.right) + assert( + binaryExpression.checkInputDataTypes().isSuccess, + "Input types of a binary expression must be valid, but got: " + + binaryExpression.children.map(_.dataType.typeName).mkString(", ") + ) + + binaryExpression match { + case timezoneExpression: TimeZoneAwareExpression => + assert(timezoneExpression.timeZoneId.nonEmpty, "Timezone expression must have a timezone") + case _ => + } + } + + private def validateConditionalExpression(conditionalExpression: ConditionalExpression): Unit = + conditionalExpression.children.foreach(validate) + + private def validateExtractANSIIntervalDays( + extractANSIIntervalDays: ExtractANSIIntervalDays): Unit = { + validate(extractANSIIntervalDays.child) + } + + private def validateLiteral(literal: Literal): Unit = {} + + private def validateUnaryMinus(unaryMinus: UnaryMinus): Unit = { + validate(unaryMinus.child) + assert( + unaryMinus.checkInputDataTypes().isSuccess, + "Input types of a unary minus must be valid, but got: " + + unaryMinus.child.dataType.typeName.mkString(", ") + ) + } + + private def validateGetStructField(getStructField: GetStructField): Unit = { + validate(getStructField.child) + } + + private def validateCreateNamedStruct(createNamedStruct: CreateNamedStruct): Unit = { + createNamedStruct.children.foreach(validate) + assert( + createNamedStruct.checkInputDataTypes().isSuccess, + "Input types of CreateNamedStruct must be valid, but got: " + + createNamedStruct.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateGetArrayStructFields(getArrayStructFields: GetArrayStructFields): Unit = { + validate(getArrayStructFields.child) + } + + private def validateGetMapValue(getMapValue: GetMapValue): Unit = { + validate(getMapValue.child) + validate(getMapValue.key) + assert( + getMapValue.checkInputDataTypes().isSuccess, + "Input types of GetMapValue must be valid, but got: " + + getMapValue.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateCreateMap(createMap: CreateMap): Unit = { + createMap.children.foreach(validate) + assert( + createMap.checkInputDataTypes().isSuccess, + "Input types of CreateMap must be valid, but got: " + + createMap.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateStringToMap(stringToMap: StringToMap): Unit = { + validate(stringToMap.text) + validate(stringToMap.pairDelim) + validate(stringToMap.keyValueDelim) + } + + private def validateMapContainsKey(mapContainsKey: MapContainsKey): Unit = { + validate(mapContainsKey.left) + validate(mapContainsKey.right) + assert( + mapContainsKey.checkInputDataTypes().isSuccess, + "Input types of MapContainsKey must be valid, but got: " + + mapContainsKey.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateMapConcat(mapConcat: MapConcat): Unit = { + mapConcat.children.foreach(validate) + assert( + mapConcat.checkInputDataTypes().isSuccess, + "Input types of MapConcat must be valid, but got: " + + mapConcat.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateMapKeys(mapKeys: MapKeys): Unit = { + validate(mapKeys.child) + } + + private def validateMapValues(mapValues: MapValues): Unit = { + validate(mapValues.child) + } + + private def validateMapEntries(mapEntries: MapEntries): Unit = { + validate(mapEntries.child) + } + + private def validateMapFromEntries(mapFromEntries: MapFromEntries): Unit = { + mapFromEntries.children.foreach(validate) + assert( + mapFromEntries.checkInputDataTypes().isSuccess, + "Input types of MapFromEntries must be valid, but got: " + + mapFromEntries.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateCreateArray(createArray: CreateArray): Unit = { + createArray.children.foreach(validate) + assert( + createArray.checkInputDataTypes().isSuccess, + "Input types of CreateArray must be valid, but got: " + + createArray.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateArrayDistinct(arrayDistinct: ArrayDistinct): Unit = { + validate(arrayDistinct.child) + assert( + arrayDistinct.checkInputDataTypes().isSuccess, + "Input types of ArrayDistinct must be valid, but got: " + + arrayDistinct.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateArrayInsert(arrayInsert: ArrayInsert): Unit = { + validate(arrayInsert.srcArrayExpr) + validate(arrayInsert.posExpr) + validate(arrayInsert.itemExpr) + assert( + arrayInsert.checkInputDataTypes().isSuccess, + "Input types of ArrayInsert must be valid, but got: " + + arrayInsert.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateArrayJoin(arrayJoin: ArrayJoin): Unit = { + validate(arrayJoin.array) + validate(arrayJoin.delimiter) + if (arrayJoin.nullReplacement.isDefined) { + validate(arrayJoin.nullReplacement.get) + } + } + + private def validateArrayMax(arrayMax: ArrayMax): Unit = { + validate(arrayMax.child) + assert( + arrayMax.checkInputDataTypes().isSuccess, + "Input types of ArrayMax must be valid, but got: " + + arrayMax.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateArrayMin(arrayMin: ArrayMin): Unit = { + validate(arrayMin.child) + assert( + arrayMin.checkInputDataTypes().isSuccess, + "Input types of ArrayMin must be valid, but got: " + + arrayMin.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateArraysZip(arraysZip: ArraysZip): Unit = { + arraysZip.children.foreach(validate) + arraysZip.names.foreach(validate) + assert( + arraysZip.checkInputDataTypes().isSuccess, + "Input types of ArraysZip must be valid, but got: " + + arraysZip.children.map(_.dataType.typeName).mkString(", ") + ) + } + + private def validateRuntimeReplaceable(runtimeReplaceable: RuntimeReplaceable): Unit = { + runtimeReplaceable.children.foreach(validate) + } + + private def validateTimezoneExpression(timezoneExpression: TimeZoneAwareExpression): Unit = { + timezoneExpression.children.foreach(validate) + assert(timezoneExpression.timeZoneId.nonEmpty, "Timezone expression must have a timezone") + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala new file mode 100644 index 0000000000000..0a9e2b9c5a872 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala @@ -0,0 +1,340 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{ + withPosition, + FunctionResolution, + UnresolvedAlias, + UnresolvedAttribute, + UnresolvedFunction, + UnresolvedStar +} +import org.apache.spark.sql.catalyst.expressions.{ + Alias, + AttributeReference, + BinaryArithmetic, + ConditionalExpression, + CreateNamedStruct, + Expression, + ExtractANSIIntervalDays, + InheritAnalysisRules, + Literal, + NamedExpression, + Predicate, + RuntimeReplaceable, + TimeAdd, + TimeZoneAwareExpression, + UnaryMinus +} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.MetadataBuilder + +/** + * The [[ExpressionResolver]] is used by the [[Resolver]] during the analysis to resolve + * expressions. + * + * The functions here generally traverse unresolved [[Expression]] nodes recursively, + * constructing and returning the resolved [[Expression]] nodes bottom-up. + * This is the primary entry point for implementing expression analysis, + * wherein the [[resolve]] method accepts a fully unresolved [[Expression]] and returns + * a fully resolved [[Expression]] in response with all data types and attribute + * reference ID assigned for valid requests. This resolver also takes responsibility + * to detect any errors in the initial SQL query or DataFrame and return appropriate + * error messages including precise parse locations wherever possible. + * + * @param resolver [[Resolver]] is passed from the parent to resolve other + * operators which are nested in expressions. + * @param scopes [[NameScopeStack]] to resolve the expression tree in the correct scope. + * @param functionResolution [[FunctionResolution]] to resolve function expressions. + */ +class ExpressionResolver( + resolver: Resolver, + scopes: NameScopeStack, + functionResolution: FunctionResolution) + extends TreeNodeResolver[Expression, Expression] + with ProducesUnresolvedSubtree + with ResolvesExpressionChildren + with TracksResolvedNodes[Expression] { + private val shouldTrackResolvedNodes = + conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_TRACK_RESOLVED_NODES_ENABLED) + private val aliasResolver = new AliasResolver(this, scopes) + private val createNamedStructResolver = new CreateNamedStructResolver(this) + private val timezoneAwareExpressionResolver = new TimezoneAwareExpressionResolver(this) + private val conditionalExpressionResolver = + new ConditionalExpressionResolver(this, timezoneAwareExpressionResolver) + private val predicateResolver = + new PredicateResolver(this, timezoneAwareExpressionResolver) + private val binaryArithmeticResolver = { + new BinaryArithmeticResolver( + this, + timezoneAwareExpressionResolver + ) + } + private val functionResolver = new FunctionResolver( + this, + timezoneAwareExpressionResolver, + functionResolution + ) + private val timeAddResolver = new TimeAddResolver(this, timezoneAwareExpressionResolver) + private val unaryMinusResolver = new UnaryMinusResolver(this, timezoneAwareExpressionResolver) + + /** + * This method is an expression analysis entry point. The method first checks if the expression + * has already been resolved (necessary because of partially-unresolved subtrees, see + * [[ProducesUnresolvedSubtree]]). If not already resolved, method takes an unresolved + * [[Expression]] and chooses the right `resolve*` method using pattern matching on the + * `unresolvedExpression` type. This pattern matching enumerates all the expression node types + * that are supported by the single-pass analysis. + * When developers introduce a new [[Expression]] type to the Catalyst, they should implement + * a corresponding `resolve*` method in the [[ExpressionResolver]] and add it to this pattern + * match list. + * + * [[resolve]] will be called recursively during the expression tree traversal eventually + * producing a fully resolved expression subtree or a descriptive error message. + * + * [[resolve]] can recursively call `resolver` to resolve nested operators (e.g. scalar + * subqueries): + * + * {{{ SELECT * FROM VALUES (1), (2) WHERE col1 IN (SELECT 1); }}} + * + * In this case `IN` is an expression and `SELECT 1` is a nested operator tree for which + * the [[ExpressionResolver]] would invoke the [[Resolver]]. + */ + override def resolve(unresolvedExpression: Expression): Expression = + if (unresolvedExpression + .getTagValue(ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY) + .nonEmpty) { + unresolvedExpression + } else { + throwIfNodeWasResolvedEarlier(unresolvedExpression) + + val resolvedExpr = unresolvedExpression match { + case unresolvedBinaryArithmetic: BinaryArithmetic => + binaryArithmeticResolver.resolve(unresolvedBinaryArithmetic) + case unresolvedExtractANSIIntervalDays: ExtractANSIIntervalDays => + resolveExtractANSIIntervalDays(unresolvedExtractANSIIntervalDays) + case unresolvedNamedExpression: NamedExpression => + resolveNamedExpression(unresolvedNamedExpression) + case unresolvedFunction: UnresolvedFunction => + functionResolver.resolve(unresolvedFunction) + case unresolvedLiteral: Literal => + resolveLiteral(unresolvedLiteral) + case unresolvedPredicate: Predicate => + predicateResolver.resolve(unresolvedPredicate) + case unresolvedTimeAdd: TimeAdd => + timeAddResolver.resolve(unresolvedTimeAdd) + case unresolvedUnaryMinus: UnaryMinus => + unaryMinusResolver.resolve(unresolvedUnaryMinus) + case createNamedStruct: CreateNamedStruct => + createNamedStructResolver.resolve(createNamedStruct) + case unresolvedConditionalExpression: ConditionalExpression => + conditionalExpressionResolver.resolve(unresolvedConditionalExpression) + case unresolvedRuntimeReplaceable: RuntimeReplaceable => + resolveRuntimeReplaceable(unresolvedRuntimeReplaceable) + case unresolvedTimezoneExpression: TimeZoneAwareExpression => + timezoneAwareExpressionResolver.resolve(unresolvedTimezoneExpression) + case _ => + withPosition(unresolvedExpression) { + throwUnsupportedSinglePassAnalyzerFeature(unresolvedExpression) + } + } + + markNodeAsResolved(resolvedExpr) + + resolvedExpr + } + + private def resolveNamedExpression( + unresolvedNamedExpression: Expression, + isTopOfProjectList: Boolean = false): Expression = + unresolvedNamedExpression match { + case alias: Alias => + aliasResolver.handleResolvedAlias(alias) + case unresolvedAlias: UnresolvedAlias => + aliasResolver.resolve(unresolvedAlias) + case unresolvedAttribute: UnresolvedAttribute => + resolveAttribute(unresolvedAttribute, isTopOfProjectList) + case unresolvedStar: UnresolvedStar => + withPosition(unresolvedStar) { + throwInvalidStarUsageError(unresolvedStar) + } + case attributeReference: AttributeReference => + handleResolvedAttributeReference(attributeReference) + case _ => + withPosition(unresolvedNamedExpression) { + throwUnsupportedSinglePassAnalyzerFeature(unresolvedNamedExpression) + } + } + + /** + * The [[Project]] list can contain different unresolved expressions before the resolution, which + * will be resolved using generic [[resolve]]. However, [[UnresolvedStar]] is a special case, + * because it is expanded into a sequence of [[NamedExpression]]s. Because of that this method + * returns a sequence and doesn't conform to generic [[resolve]] interface - it's called directly + * from the [[Resolver]] during [[Project]] resolution. + * + * The output sequence can be larger than the input sequence due to [[UnresolvedStar]] expansion. + */ + def resolveProjectList(unresolvedProjectList: Seq[NamedExpression]): Seq[NamedExpression] = { + unresolvedProjectList.flatMap { + case unresolvedStar: UnresolvedStar => + resolveStar(unresolvedStar) + case other => + Seq(resolveNamedExpression(other, isTopOfProjectList = true).asInstanceOf[NamedExpression]) + } + } + + /** + * [[UnresolvedAttribute]] resolution relies on [[NameScope]] to lookup the attribute by its + * multipart name. The resolution can result in three different outcomes which are handled in the + * [[NameTarget.pickCandidate]]: + * + * - No results from the [[NameScope]] mean that the attribute lookup failed as in: + * {{{ SELECT col1 FROM (SELECT 1 as col2); }}} + * + * - Several results from the [[NameScope]] mean that the reference is ambiguous as in: + * {{{ SELECT col1 FROM (SELECT 1 as col1), (SELECT 2 as col1); }}} + * + * - Single result from the [[NameScope]] means that the attribute was found as in: + * {{{ SELECT col1 FROM VALUES (1); }}} + * + * If the attribute is at the top of the project list (which is indicated by + * [[isTopOfProjectList]]), we preserve the [[Alias]] or remove it otherwise. + */ + private def resolveAttribute( + unresolvedAttribute: UnresolvedAttribute, + isTopOfProjectList: Boolean): Expression = + withPosition(unresolvedAttribute) { + if (scopes.top.isExistingAlias(unresolvedAttribute.nameParts.head)) { + // Temporarily disable referencing aliases until we support LCA resolution. + throw new ExplicitlyUnsupportedResolverFeature("unsupported expression: LateralColumnAlias") + } + + val nameTarget: NameTarget = scopes.top.matchMultipartName(unresolvedAttribute.nameParts) + + val candidate = nameTarget.pickCandidate(unresolvedAttribute) + if (isTopOfProjectList && nameTarget.aliasName.isDefined) { + Alias(candidate, nameTarget.aliasName.get)() + } else { + candidate + } + } + + /** + * [[AttributeReference]] is already resolved if it's passed to us from DataFrame `col(...)` + * function, for example. + */ + private def handleResolvedAttributeReference(attributeReference: AttributeReference) = + tryStripAmbiguousSelfJoinMetadata(attributeReference) + + /** + * [[ExtractANSIIntervalDays]] resolution doesn't require any specific resolution logic apart + * from resolving its children. + */ + private def resolveExtractANSIIntervalDays( + unresolvedExtractANSIIntervalDays: ExtractANSIIntervalDays) = + withResolvedChildren(unresolvedExtractANSIIntervalDays, resolve) + + /** + * [[UnresolvedStar]] resolution relies on the [[NameScope]]'s ability to get the attributes by a + * multipart name ([[UnresolvedStar]]'s `target` field): + * + * - Star target is defined: + * + * {{{ + * SELECT t.* FROM VALUES (1) AS t; + * -> + * Project [col1#19] + * }}} + * + * + * - Star target is not defined: + * + * {{{ + * SELECT * FROM (SELECT 1 as col1), (SELECT 2 as col2); + * -> + * Project [col1#19, col2#20] + * }}} + */ + def resolveStar(unresolvedStar: UnresolvedStar): Seq[NamedExpression] = + withPosition(unresolvedStar) { + scopes.top.expandStar(unresolvedStar) + } + + /** + * [[Literal]] resolution doesn't require any specific resolution logic at this point. + * + * Since [[TracksResolvedNodes]] requires all the expressions in the tree to be unique objects, + * we reallocate the literal in [[ANALYZER_SINGLE_PASS_TRACK_RESOLVED_NODES_ENABLED]] mode, + * otherwise we preserve the old object to avoid unnecessary memory allocations. + */ + private def resolveLiteral(literal: Literal): Expression = { + if (shouldTrackResolvedNodes) { + literal.copy() + } else { + literal + } + } + + /** + * When [[RuntimeReplaceable]] is mixed in with [[InheritAnalysisRules]], child expression will + * be runtime replacement. In that case we need to resolve the children of the expression. + * otherwise, no resolution is necessary because replacement is already resolved. + */ + private def resolveRuntimeReplaceable(unresolvedRuntimeReplaceable: RuntimeReplaceable) = + unresolvedRuntimeReplaceable match { + case inheritAnalysisRules: InheritAnalysisRules => + withResolvedChildren(inheritAnalysisRules, resolve) + case other => other + } + + /** + * [[DetectAmbiguousSelfJoin]] rule in the fixed-point Analyzer detects ambiguous references in + * self-joins based on special metadata added by [[Dataset]] code (see SPARK-27547). Just strip + * this for now since we don't support joins yet. + */ + private def tryStripAmbiguousSelfJoinMetadata(attributeReference: AttributeReference) = { + val metadata = attributeReference.metadata + if (ExpressionResolver.AMBIGUOUS_SELF_JOIN_METADATA.exists(metadata.contains(_))) { + val metadataBuilder = new MetadataBuilder().withMetadata(metadata) + for (metadataKey <- ExpressionResolver.AMBIGUOUS_SELF_JOIN_METADATA) { + metadataBuilder.remove(metadataKey) + } + attributeReference.withMetadata(metadataBuilder.build()) + } else { + attributeReference + } + } + + private def throwUnsupportedSinglePassAnalyzerFeature(unresolvedExpression: Expression): Nothing = + throw QueryCompilationErrors.unsupportedSinglePassAnalyzerFeature( + s"${unresolvedExpression.getClass} expression resolution" + ) + + private def throwInvalidStarUsageError(unresolvedStar: UnresolvedStar): Nothing = + // TODO(vladimirg-db): Use parent operator name instead of "query" + throw QueryCompilationErrors.invalidStarUsageError("query", Seq(unresolvedStar)) +} + +object ExpressionResolver { + private val AMBIGUOUS_SELF_JOIN_METADATA = Seq("__dataset_id", "__col_position") + val SINGLE_PASS_SUBTREE_BOUNDARY = TreeNodeTag[Unit]("single_pass_subtree_boundary") +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FunctionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FunctionResolver.scala new file mode 100644 index 0000000000000..b7311b83e872e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FunctionResolver.scala @@ -0,0 +1,91 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{ + AnsiTypeCoercion, + CollationTypeCoercion, + FunctionResolution, + TypeCoercion, + UnresolvedFunction, + UnresolvedStar +} +import org.apache.spark.sql.catalyst.expressions.Expression + +/** + * A resolver for [[UnresolvedFunction]]s that resolves functions to concrete [[Expression]]s. + * It resolves the children of the function first by calling [[ExpressionResolver.resolve]] on them + * if they are not [[UnresolvedStar]]s. If the children are [[UnresolvedStar]]s, it resolves them + * using [[ExpressionResolver.resolveStar]]. Examples are following: + * + * - Function doesn't contain any [[UnresolvedStar]]: + * {{{ SELECT ARRAY(col1) FROM VALUES (1); }}} + * it is resolved only using [[ExpressionResolver.resolve]]. + * - Function contains [[UnresolvedStar]]: + * {{{ SELECT ARRAY(*) FROM VALUES (1); }}} + * it is resolved using [[ExpressionResolver.resolveStar]]. + * + * It applies appropriate [[TypeCoercion]] (or [[AnsiTypeCoercion]]) rules after resolving the + * function using the [[FunctionResolution]] code. + */ +class FunctionResolver( + expressionResolver: ExpressionResolver, + timezoneAwareExpressionResolver: TimezoneAwareExpressionResolver, + functionResolution: FunctionResolution) + extends TreeNodeResolver[UnresolvedFunction, Expression] + with ProducesUnresolvedSubtree { + + private val typeCoercionRules: Seq[Expression => Expression] = + if (conf.ansiEnabled) { + FunctionResolver.ANSI_TYPE_COERCION_RULES + } else { + FunctionResolver.TYPE_COERCION_RULES + } + private val typeCoercionResolver: TypeCoercionResolver = + new TypeCoercionResolver(timezoneAwareExpressionResolver, typeCoercionRules) + + override def resolve(unresolvedFunction: UnresolvedFunction): Expression = { + val functionWithResolvedChildren = + unresolvedFunction.copy(arguments = unresolvedFunction.arguments.flatMap { + case s: UnresolvedStar => expressionResolver.resolveStar(s) + case other => Seq(expressionResolver.resolve(other)) + }) + val resolvedFunction = functionResolution.resolveFunction(functionWithResolvedChildren) + typeCoercionResolver.resolve(resolvedFunction) + } +} + +object FunctionResolver { + // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]]. + private val TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + CollationTypeCoercion.apply, + TypeCoercion.InTypeCoercion.apply, + TypeCoercion.FunctionArgumentTypeCoercion.apply, + TypeCoercion.IfTypeCoercion.apply, + TypeCoercion.ImplicitTypeCoercion.apply + ) + + // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]]. + private val ANSI_TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + CollationTypeCoercion.apply, + AnsiTypeCoercion.InTypeCoercion.apply, + AnsiTypeCoercion.FunctionArgumentTypeCoercion.apply, + AnsiTypeCoercion.IfTypeCoercion.apply, + AnsiTypeCoercion.ImplicitTypeCoercion.apply + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala new file mode 100644 index 0000000000000..8a1a8602368e0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala @@ -0,0 +1,218 @@ +/* + * 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.catalyst.analysis.resolver + +import scala.util.control.NonFatal + +import org.apache.spark.sql.catalyst.{QueryPlanningTracker, SQLConfHelper} +import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, Analyzer} +import org.apache.spark.sql.catalyst.plans.NormalizePlan +import org.apache.spark.sql.catalyst.plans.logical.{AnalysisHelper, LogicalPlan} +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf + +/** + * The HybridAnalyzer routes the unresolved logical plan between the legacy Analyzer and + * a single-pass Analyzer when the query that we are processing is being run from unit tests + * depending on the testing flags set and the structure of this unresolved logical plan: + * - If the "spark.sql.analyzer.singlePassResolver.soloRunEnabled" is "true", the + * [[HybridAnalyzer]] will unconditionally run the single-pass Analyzer, which would + * usually result in some unexpected behavior and failures. This flag is used only for + * development. + * - If the "spark.sql.analyzer.singlePassResolver.dualRunEnabled" is "true", the + * [[HybridAnalyzer]] will invoke the legacy analyzer and optionally _also_ the fixed-point + * one depending on the structure of the unresolved plan. This decision is based on which + * features are supported by the single-pass Analyzer, and the checking is implemented in + * the [[ResolverGuard]]. After that we validate the results using the following + * logic: + * - If the fixed-point Analyzer fails and the single-pass one succeeds, we throw an + * appropriate exception (please check the + * [[QueryCompilationErrors.fixedPointFailedSinglePassSucceeded]] method) + * - If both the fixed-point and the single-pass Analyzers failed, we throw the exception + * from the fixed-point Analyzer. + * - If the single-pass Analyzer failed, we throw an exception from its failure. + * - If both the fixed-point and the single-pass Analyzers succeeded, we compare the logical + * plans and output schemas, and return the resolved plan from the fixed-point Analyzer. + * - Otherwise we run the legacy analyzer. + * */ +class HybridAnalyzer(legacyAnalyzer: Analyzer, resolverGuard: ResolverGuard, resolver: Resolver) + extends SQLConfHelper { + private var singlePassResolutionDuration: Option[Long] = None + private var fixedPointResolutionDuration: Option[Long] = None + + def apply(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { + val dualRun = + conf.getConf(SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER) && + resolverGuard.apply(plan) + + withTrackedAnalyzerBridgeState(dualRun) { + if (dualRun) { + resolveInDualRun(plan, tracker) + } else if (conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_ENABLED)) { + resolveInSinglePass(plan) + } else { + resolveInFixedPoint(plan, tracker) + } + } + } + + def getSinglePassResolutionDuration: Option[Long] = singlePassResolutionDuration + + def getFixedPointResolutionDuration: Option[Long] = fixedPointResolutionDuration + + /** + * Call `body` in the context of tracked [[AnalyzerBridgeState]]. Set the new bridge state + * depending on whether we are in dual-run mode or not: + * - If [[dualRun]] and [[ANALYZER_SINGLE_PASS_RESOLVER_RELATION_BRIDGING_ENABLED]] are true, + * create and set a new [[AnalyzerBridgeState]]. + * - Otherwise, reset [[AnalyzerBridgeState]]. + * + * Finally, set the bridge state back to the previous one after the `body` is executed to avoid + * disrupting the possible upper-level [[Analyzer]] invocation in case it's recursive + * [[Analyzer]] call. + * */ + private def withTrackedAnalyzerBridgeState(dualRun: Boolean)( + body: => LogicalPlan): LogicalPlan = { + val bridgeRelations = dualRun && conf.getConf( + SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_RELATION_BRIDGING_ENABLED + ) + + val prevSinglePassResolverBridgeState = AnalysisContext.get.getSinglePassResolverBridgeState + + AnalysisContext.get.setSinglePassResolverBridgeState(if (bridgeRelations) { + Some(new AnalyzerBridgeState) + } else { + None + }) + + try { + body + } finally { + AnalysisContext.get.setSinglePassResolverBridgeState(prevSinglePassResolverBridgeState) + } + } + + /** + * This method is used to run both the legacy Analyzer and single-pass Analyzer, + * and then compare the results or check the errors. For more context please check the + * [[HybridAnalyzer]] scaladoc. + * */ + private def resolveInDualRun(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { + var fixedPointException: Option[Throwable] = None + val fixedPointResult = try { + val (resolutionDuration, result) = recordDuration { + Some(resolveInFixedPoint(plan, tracker)) + } + fixedPointResolutionDuration = Some(resolutionDuration) + result + } catch { + case NonFatal(e) => + fixedPointException = Some(e) + None + } + + var singlePassException: Option[Throwable] = None + val singlePassResult = try { + val (resolutionDuration, result) = recordDuration { + Some(resolveInSinglePass(plan)) + } + singlePassResolutionDuration = Some(resolutionDuration) + result + } catch { + case NonFatal(e) => + singlePassException = Some(e) + None + } + + fixedPointException match { + case Some(fixedPointEx) => + singlePassException match { + case Some(_) => + throw fixedPointEx + case None => + throw QueryCompilationErrors.fixedPointFailedSinglePassSucceeded( + singlePassResult.get, + fixedPointEx + ) + } + case None => + singlePassException match { + case Some(singlePassEx: ExplicitlyUnsupportedResolverFeature) => + fixedPointResult.get + case Some(singlePassEx) => + throw singlePassEx + case None => + validateLogicalPlans(fixedPointResult.get, singlePassResult.get) + fixedPointResult.get + } + } + } + + /** + * This method is used to run the single-pass Analyzer which will return the resolved plan + * or throw an exception if the resolution fails. Both cases are handled in the caller method. + * */ + private def resolveInSinglePass(plan: LogicalPlan): LogicalPlan = { + val resolvedPlan = resolver.lookupMetadataAndResolve( + plan, + analyzerBridgeState = AnalysisContext.get.getSinglePassResolverBridgeState + ) + if (conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_VALIDATION_ENABLED)) { + val validator = new ResolutionValidator + validator.validatePlan(resolvedPlan) + } + resolvedPlan + } + + /** + * This method is used to run the legacy Analyzer which will return the resolved plan + * or throw an exception if the resolution fails. Both cases are handled in the caller method. + * */ + private def resolveInFixedPoint(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { + val resolvedPlan = legacyAnalyzer.executeAndTrack(plan, tracker) + QueryPlanningTracker.withTracker(tracker) { + legacyAnalyzer.checkAnalysis(resolvedPlan) + } + resolvedPlan + } + + private def validateLogicalPlans(fixedPointResult: LogicalPlan, singlePassResult: LogicalPlan) = { + if (fixedPointResult.schema != singlePassResult.schema) { + throw QueryCompilationErrors.hybridAnalyzerOutputSchemaComparisonMismatch( + fixedPointResult.schema, + singlePassResult.schema + ) + } + if (normalizePlan(fixedPointResult) != normalizePlan(singlePassResult)) { + throw QueryCompilationErrors.hybridAnalyzerLogicalPlanComparisonMismatch( + fixedPointResult, + singlePassResult + ) + } + } + + private def normalizePlan(plan: LogicalPlan) = AnalysisHelper.allowInvokingTransformsInAnalyzer { + NormalizePlan(plan) + } + + private def recordDuration[T](thunk: => T): (Long, T) = { + val start = System.nanoTime() + val res = thunk + (System.nanoTime() - start, res) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/IdentifierMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/IdentifierMap.scala new file mode 100644 index 0000000000000..899eb7d71e813 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/IdentifierMap.scala @@ -0,0 +1,37 @@ +/* + * 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.catalyst.analysis.resolver + +import java.util.Locale + +/** + * The [[IdentifierMap]] is an implementation of a [[KeyTransformingMap]] that uses SQL/DataFrame + * identifiers as keys. The implementation is case-insensitive for keys. + */ +private class IdentifierMap[V] extends KeyTransformingMap[String, V] { + override def mapKey(key: String): String = key.toLowerCase(Locale.ROOT) +} + +/** + * The [[OptionalIdentifierMap]] is an implementation of a [[KeyTransformingMap]] that uses optional + * SQL/DataFrame identifiers as keys. The implementation is case-insensitive for non-empty keys. + */ +private class OptionalIdentifierMap[V] extends KeyTransformingMap[Option[String], V] { + override def mapKey(key: Option[String]): Option[String] = + key.map(_.toLowerCase(Locale.ROOT)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala new file mode 100644 index 0000000000000..ff6e118fcc3c9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala @@ -0,0 +1,46 @@ +/* + * 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.catalyst.analysis.resolver + +import scala.collection.mutable + +/** + * The [[KeyTransformingMap]] is a partial implementation of [[mutable.Map]] that transforms input + * keys with a custom [[mapKey]] method. + */ +private abstract class KeyTransformingMap[K, V] { + private val impl = new mutable.HashMap[K, V] + + def get(key: K): Option[V] = impl.get(mapKey(key)) + + def contains(key: K): Boolean = impl.contains(mapKey(key)) + + def iterator: Iterator[(K, V)] = impl.iterator + + def +=(kv: (K, V)): this.type = { + impl += (mapKey(kv._1) -> kv._2) + this + } + + def -=(key: K): this.type = { + impl -= mapKey(key) + this + } + + def mapKey(key: K): K +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolver.scala new file mode 100644 index 0000000000000..a25616ba50b6a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolver.scala @@ -0,0 +1,114 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.errors.QueryErrorsBase +import org.apache.spark.sql.types.IntegerType + +/** + * The [[LimitExpressionResolver]] is a resolver that resolves a [[LocalLimit]] or [[GlobalLimit]] + * expression and performs all the necessary validation. + */ +class LimitExpressionResolver(expressionResolver: TreeNodeResolver[Expression, Expression]) + extends TreeNodeResolver[Expression, Expression] + with QueryErrorsBase { + + /** + * Resolve a limit expression of [[GlobalLimit]] or [[LocalLimit]] and perform validation. + */ + override def resolve(unresolvedLimitExpression: Expression): Expression = { + val resolvedLimitExpression = expressionResolver.resolve(unresolvedLimitExpression) + validateLimitExpression(resolvedLimitExpression, expressionName = "limit") + resolvedLimitExpression + } + + /** + * Validate a resolved limit expression of [[GlobalLimit]] or [[LocalLimit]]: + * - The expression has to be foldable + * - The result data type has to be [[IntegerType]] + * - The evaluated expression has to be non-null + * - The evaluated expression has to be positive + * + * The `foldable` check is implemented in some expressions + * as a recursive expression tree traversal. + * It is not an ideal approach for the single-pass [[ExpressionResolver]], + * but __is__ practical, since: + * - We have to call `eval` here anyway, and it's recursive + * - In practice `LIMIT` expression trees are very small + */ + private def validateLimitExpression(expression: Expression, expressionName: String): Unit = { + if (!expression.foldable) { + throwInvalidLimitLikeExpressionIsUnfoldable(expressionName, expression) + } + if (expression.dataType != IntegerType) { + throwInvalidLimitLikeExpressionDataType(expressionName, expression) + } + expression.eval() match { + case null => + throwInvalidLimitLikeExpressionIsNull(expressionName, expression) + case value: Int if value < 0 => + throwInvalidLimitLikeExpressionIsNegative(expressionName, expression, value) + case _ => + } + } + + private def throwInvalidLimitLikeExpressionIsUnfoldable( + name: String, + expression: Expression): Nothing = + throw new AnalysisException( + errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + messageParameters = Map( + "name" -> name, + "expr" -> toSQLExpr(expression) + ), + origin = expression.origin + ) + + private def throwInvalidLimitLikeExpressionDataType( + name: String, + expression: Expression): Nothing = + throw new AnalysisException( + errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", + messageParameters = Map( + "name" -> name, + "expr" -> toSQLExpr(expression), + "dataType" -> toSQLType(expression.dataType) + ), + origin = expression.origin + ) + + private def throwInvalidLimitLikeExpressionIsNull(name: String, expression: Expression): Nothing = + throw new AnalysisException( + errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL", + messageParameters = Map("name" -> name, "expr" -> toSQLExpr(expression)), + origin = expression.origin + ) + + private def throwInvalidLimitLikeExpressionIsNegative( + name: String, + expression: Expression, + value: Int): Nothing = + throw new AnalysisException( + errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", + messageParameters = + Map("name" -> name, "expr" -> toSQLExpr(expression), "v" -> toSQLValue(value, IntegerType)), + origin = expression.origin + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala new file mode 100644 index 0000000000000..e1334fc56575e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala @@ -0,0 +1,133 @@ +/* + * 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.catalyst.analysis.resolver + +import java.util.ArrayDeque + +import org.apache.spark.sql.catalyst.analysis.{withPosition, RelationResolution, UnresolvedRelation} +import org.apache.spark.sql.catalyst.expressions.{Expression, PlanExpression} +import org.apache.spark.sql.catalyst.plans.logical.{AnalysisHelper, LogicalPlan} +import org.apache.spark.sql.connector.catalog.CatalogManager + +/** + * The [[MetadataResolver]] performs relation metadata resolution based on the unresolved plan + * at the start of the analysis phase. Usually it does RPC calls to some table catalog and to table + * metadata itself. + * + * [[RelationsWithResolvedMetadata]] is a map from relation ID to the relations with resolved + * metadata. It's produced by [[resolve]] and is used later in [[Resolver]] to replace + * [[UnresolvedRelation]]s. + * + * This object is one-shot per SQL query or DataFrame program resolution. + */ +class MetadataResolver( + override val catalogManager: CatalogManager, + override val relationResolution: RelationResolution, + override val extensions: Seq[ResolverExtension] = Seq.empty) + extends RelationMetadataProvider + with DelegatesResolutionToExtensions { + override val relationsWithResolvedMetadata = new RelationsWithResolvedMetadata + + /** + * Resolves the relation metadata for `unresolvedPlan`. Usually this involves several blocking + * calls for the [[UnresolvedRelation]]s present in that tree. During the `unresolvedPlan` + * traversal we fill [[relationsWithResolvedMetadata]] with resolved metadata by relation id. + * This map will be used to resolve the plan in single-pass by the [[Resolver]] using + * [[getRelationWithResolvedMetadata]]. If the generic metadata resolution using + * [[RelationResolution]] wasn't successful, we resort to using [[extensions]]. + * Otherwise, we fail with an exception. + */ + def resolve(unresolvedPlan: LogicalPlan): Unit = { + traverseLogicalPlanTree(unresolvedPlan) { unresolvedOperator => + unresolvedOperator match { + case unresolvedRelation: UnresolvedRelation => + val relationId = relationIdFromUnresolvedRelation(unresolvedRelation) + + if (!relationsWithResolvedMetadata.containsKey(relationId)) { + val relationWithResolvedMetadata = resolveRelation(unresolvedRelation).orElse { + // In case the generic metadata resolution returned `None`, we try to check if any + // of the [[extensions]] matches this `unresolvedRelation`, and resolve it using + // that extension. + tryDelegateResolutionToExtension(unresolvedRelation) + } + + relationWithResolvedMetadata match { + case Some(relationWithResolvedMetadata) => + relationsWithResolvedMetadata.put( + relationId, + relationWithResolvedMetadata + ) + case None => + withPosition(unresolvedRelation) { + unresolvedRelation.tableNotFound(unresolvedRelation.multipartIdentifier) + } + } + } + case _ => + } + } + } + + /** + * Resolves the metadata for the given unresolved relation and returns a relation with the + * resolved metadata. This method is blocking. + */ + private def resolveRelation(unresolvedRelation: UnresolvedRelation): Option[LogicalPlan] = + AnalysisHelper.allowInvokingTransformsInAnalyzer { + relationResolution.resolveRelation( + u = unresolvedRelation + ) + } + + /** + * Traverse the logical plan tree from `root` in a pre-order DFS manner and apply `visitor` to + * each node. + */ + private def traverseLogicalPlanTree(root: LogicalPlan)(visitor: LogicalPlan => Unit) = { + val stack = new ArrayDeque[Either[LogicalPlan, Expression]] + stack.push(Left(root)) + + while (!stack.isEmpty) { + stack.pop() match { + case Left(logicalPlan) => + visitor(logicalPlan) + + for (child <- logicalPlan.children) { + stack.push(Left(child)) + } + for (expression <- logicalPlan.expressions) { + stack.push(Right(expression)) + } + case Right(expression) => + for (child <- expression.children) { + stack.push(Right(child)) + } + + expression match { + case planExpression: PlanExpression[_] => + planExpression.plan match { + case plan: LogicalPlan => + stack.push(Left(plan)) + case _ => + } + case _ => + } + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala new file mode 100644 index 0000000000000..8abf4e04b8836 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala @@ -0,0 +1,393 @@ +/* + * 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.catalyst.analysis.resolver + +import java.util.{ArrayDeque, ArrayList, HashSet} + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.{Resolver => NameComparator, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions.{ + Alias, + Attribute, + AttributeSeq, + Expression, + NamedExpression +} +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * The [[NameScope]] is used during the analysis to control the visibility of names: plan names + * and output attributes. New [[NameScope]] can be created both in the [[Resolver]] and in + * the [[ExpressionResolver]] using the [[NameScopeStack]] api. The name resolution for identifiers + * is case-insensitive. + * + * In this example: + * + * {{{ + * WITH table_1_cte AS ( + * SELECT + * col1, + * col2, + * col2 + * FROM + * table_1 + * ) + * SELECT + * table_1_cte.col1, + * table_2.col1 + * FROM + * table_1_cte + * INNER JOIN + * table_2 + * ON + * table_1_cte.col2 = table_2.col3 + * ; + * }}} + * + * there are two named subplans in the scope: table_1_cte -> [col1, col2, col2] and + * table_2 -> [col1, col3]. + * + * State breakout: + * - `planOutputs`: list of named plan outputs. Order matters here (e.g. to correctly expand `*`). + * Can contain duplicate names, since it's possible to select same column twice, or to select + * columns with the same name from different relations. [[OptionalIdentifierMap]] is used here, + * since some plans don't have an explicit name, so output attributes from those plans will reside + * under the `None` key. + * In our example it will be {{{ [(table_1_cte, [col1, col2, col2]), (table_2, [col1, col3])] }}} + * + * - `planNameToOffset`: mapping from plan output names to their offsets in the `planOutputs` array. + * It's used to lookup attributes by plan output names (multipart names are not supported yet). + * In our example it will be {{{ [table_1_cte -> 0, table_2 -> 1] }}} + */ +class NameScope extends SQLConfHelper { + private val planOutputs = new ArrayList[PlanOutput]() + private val planNameToOffset = new OptionalIdentifierMap[Int] + private val nameComparator: NameComparator = conf.resolver + private val existingAliases = new HashSet[String] + + /** + * Register the named plan output in this [[NameScope]]. The named plan is usually a + * [[NamedRelation]]. `attributes` sequence can contain duplicate names both for this named plan + * and for the scope in general, despite the fact that their further resolution _may_ throw an + * error in case of ambiguous reference. After calling this method, the code can lookup the + * attributes using `get*` methods of this [[NameScope]]. + * + * Duplicate plan names are merged into the same [[PlanOutput]]. For example, this query: + * + * {{{ SELECT t.* FROM (SELECT * FROM VALUES (1)) as t, (SELECT * FROM VALUES (2)) as t; }}} + * + * will have the following output schema: + * + * {{{ [col1, col1] }}} + * + * Same logic applies for the unnamed plan outputs. This query: + * + * {{{ SELECT * FROM (SELECT * FROM VALUES (1)), (SELECT * FROM VALUES (2)); }}} + * + * will have the same output schema: + * + * {{{ [col1, col1] }}} + * + * @param name The name of this named plan. + * @param attributes The output of this named plan. Can contain duplicate names. + */ + def update(name: String, attributes: Seq[Attribute]): Unit = { + update(attributes, Some(name)) + } + + /** + * Register the unnamed plan output in this [[NameScope]]. Some examples of the unnamed plan are + * [[Project]] and [[Aggregate]]. + * + * See the [[update]] method for more details. + * + * @param attributes The output of the unnamed plan. Can contain duplicate names. + */ + def +=(attributes: Seq[Attribute]): Unit = { + update(attributes) + } + + /** + * Get all the attributes from all the plans registered in this [[NameScope]]. The output can + * contain duplicate names. This is used for star (`*`) resolution. + */ + def getAllAttributes: Seq[Attribute] = { + val attributes = new mutable.ArrayBuffer[Attribute] + + planOutputs.forEach(planOutput => { + attributes.appendAll(planOutput.attributes) + }) + + attributes.toSeq + } + + /** + * Expand the [[UnresolvedStar]] using `planOutputs`. The expected use case for this method is + * star expansion inside [[Project]]. Since [[Project]] has only one child, we assert that the + * size of `planOutputs` is 1, otherwise the query is malformed. + * + * Some examples of queries with a star: + * + * - Star without a target: + * {{{ SELECT * FROM VALUES (1, 2, 3) AS t(a, b, c); }}} + * - Star with a multipart name target: + * {{{ SELECT catalog1.database1.table1.* FROM catalog1.database1.table1; }}} + * - Star with a struct target: + * {{{ SELECT d.* FROM VALUES (named_struct('a', 1, 'b', 2)) AS t(d); }}} + * - Star as an argument to a function: + * {{{ SELECT concat_ws('', *) AS result FROM VALUES (1, 2, 3) AS t(a, b, c); }}} + * + * It is resolved by correctly resolving the star qualifier. + * Please check [[UnresolvedStarBase.expandStar]] for more details. + * + * @param unresolvedStar [[UnresolvedStar]] to expand. + * @return The output of a plan expanded from the star. + */ + def expandStar(unresolvedStar: UnresolvedStar): Seq[NamedExpression] = { + if (planOutputs.size != 1) { + throw QueryCompilationErrors.invalidStarUsageError("query", Seq(unresolvedStar)) + } + + planOutputs.get(0).expandStar(unresolvedStar) + } + + /** + * Get all matched attributes by a multipart name. It returns [[Attribute]]s when we resolve a + * simple column or an alias name from a lower operator. However this function can also return + * [[Alias]]es in case we access a struct field or a map value using some key. + * + * Example that contains those major use-cases: + * + * {{{ + * SELECT col1, a, col2.field, col3.struct.field, col4.key + * FROM (SELECT *, col5 AS a FROM t); + * }}} + * + * has a Project list that looks like this: + * + * {{{ + * AttributeReference(col1), + * AttributeReference(a), + * Alias(col2.field, field), + * Alias(col3.struct.field, field), + * Alias(col4[CAST(key AS INT)], key) + * }}} + * + * Also, see [[AttributeSeq.resolve]] for more details. + * + * Since there can be several identical attribute names for several named plans, this function + * can return multiple values: + * - 0 values: No matched attributes + * - 1 value: Unique attribute matched + * - 1+ values: Ambiguity, several attributes matched + * + * One example of a query with an attribute that has a multipart name: + * + * {{{ SELECT catalog1.database1.table1.col1 FROM catalog1.database1.table1; }}} + * + * @param multipartName Multipart attribute name. Can be of several forms: + * - `catalog.database.table.column` + * - `database.table.column` + * - `table.column` + * - `column` + * @return All the attributes matched by the `multipartName`, encapsulated in a [[NameTarget]]. + */ + def matchMultipartName(multipartName: Seq[String]): NameTarget = { + val candidates = new mutable.ArrayBuffer[Expression] + val allAttributes = new mutable.ArrayBuffer[Attribute] + var aliasName: Option[String] = None + + planOutputs.forEach(planOutput => { + allAttributes.appendAll(planOutput.attributes) + val nameTarget = planOutput.matchMultipartName(multipartName) + if (nameTarget.aliasName.isDefined) { + aliasName = nameTarget.aliasName + } + candidates.appendAll(nameTarget.candidates) + }) + + NameTarget(candidates.toSeq, aliasName, allAttributes.toSeq) + } + + /** + * Add an alias, by name, to the list of existing aliases. + */ + def addAlias(aliasName: String): Unit = existingAliases.add(aliasName.toLowerCase()) + + /** + * Returns whether an alias exists in the current scope. + */ + def isExistingAlias(aliasName: String): Boolean = + existingAliases.contains(aliasName.toLowerCase()) + + private def update(attributes: Seq[Attribute], name: Option[String] = None): Unit = { + planNameToOffset.get(name) match { + case Some(index) => + val prevPlanOutput = planOutputs.get(index) + planOutputs.set( + index, + new PlanOutput(prevPlanOutput.attributes ++ attributes, name, nameComparator) + ) + case None => + val index = planOutputs.size + planOutputs.add(new PlanOutput(attributes, name, nameComparator)) + planNameToOffset += (name -> index) + } + } +} + +/** + * The [[NameScopeStack]] is a stack of [[NameScope]]s managed by the [[Resolver]] and the + * [[ExpressionResolver]]. Usually a top scope is used for name resolution, but in case of + * correlated subqueries we can lookup names in the parent scopes. Low-level scope creation is + * managed internally, and only high-level api like [[withNewScope]] is available to the resolvers. + * Freshly-created [[NameScopeStack]] contains an empty root [[NameScope]]. + */ +class NameScopeStack extends SQLConfHelper { + private val stack = new ArrayDeque[NameScope] + push() + + /** + * Get the top scope, which is a default choice for name resolution. + */ + def top: NameScope = { + stack.peek() + } + + /** + * Completely overwrite the top scope state with a named plan output. + * + * See [[NameScope.update]] for more details. + */ + def overwriteTop(name: String, attributes: Seq[Attribute]): Unit = { + val newScope = new NameScope + newScope.update(name, attributes) + + stack.pop() + stack.push(newScope) + } + + /** + * Completely overwrite the top scope state with an unnamed plan output. + * + * See [[NameScope.+=]] for more details. + */ + def overwriteTop(attributes: Seq[Attribute]): Unit = { + val newScope = new NameScope + newScope += attributes + + stack.pop() + stack.push(newScope) + } + + /** + * Execute `body` in a context of a fresh scope. It's used during the [[Project]] or the + * [[Aggregate]] resolution to avoid calling [[push]] and [[pop]] explicitly. + */ + def withNewScope[R](body: => R): R = { + push() + try { + body + } finally { + pop() + } + } + + /** + * Push a new scope to the stack. Introduced by the [[Project]] or the [[Aggregate]]. + */ + private def push(): Unit = { + stack.push(new NameScope) + } + + /** + * Pop a scope from the stack. Called when the resolution process for the pushed scope is done. + */ + private def pop(): Unit = { + stack.pop() + } +} + +/** + * [[PlanOutput]] represents a sequence of attributes from a plan ([[NamedRelation]], [[Project]], + * [[Aggregate]], etc). + * + * It is created from `attributes`, which is an output of a named plan, optional plan `name` and a + * resolver provided by the [[NameScopeStack]]. + * + * @param attributes Plan output. Can contain duplicate names. + * @param name Plan name. Non-empty for named plans like [[NamedRelation]] or [[SubqueryAlias]], + * `None` otherwise. + */ +class PlanOutput( + val attributes: Seq[Attribute], + val name: Option[String], + val nameComparator: NameComparator) { + + /** + * attributesForResolution is an [[AttributeSeq]] that is used for resolution of + * multipart attribute names. It's created from the `attributes` when [[NameScope]] is updated. + */ + private val attributesForResolution: AttributeSeq = + AttributeSeq.fromNormalOutput(attributes) + + /** + * Find attributes by the multipart name. + * + * See [[NameScope.matchMultipartName]] for more details. + * + * @param multipartName Multipart attribute name. + * @return Matched attributes or [[Seq.empty]] otherwise. + */ + def matchMultipartName(multipartName: Seq[String]): NameTarget = { + val (candidates, nestedFields) = + attributesForResolution.getCandidatesForResolution(multipartName, nameComparator) + val resolvedCandidates = attributesForResolution.resolveCandidates( + multipartName, + nameComparator, + candidates, + nestedFields + ) + resolvedCandidates match { + case Seq(Alias(child, aliasName)) => + NameTarget(Seq(child), Some(aliasName)) + case other => + NameTarget(other, None) + } + } + + /** + * Method to expand an unresolved star. See [[NameScope.expandStar]] for more details. + * + * @param unresolvedStar Star to resolve. + * @return Attributes expanded from the star. + */ + def expandStar(unresolvedStar: UnresolvedStar): Seq[NamedExpression] = { + unresolvedStar.expandStar( + childOperatorOutput = attributes, + childOperatorMetadataOutput = Seq.empty, + resolve = + (nameParts, nameComparator) => attributesForResolution.resolve(nameParts, nameComparator), + suggestedAttributes = attributes, + resolver = nameComparator, + cleanupNestedAliasesDuringStructExpansion = true + ) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameTarget.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameTarget.scala new file mode 100644 index 0000000000000..3b31c9b1a9110 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameTarget.scala @@ -0,0 +1,81 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.catalyst.util.StringUtils.orderSuggestedIdentifiersBySimilarity +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * Class that represents results of name resolution or star expansion. It encapsulates: + * - `candidates` - A list of candidates that are possible matches for a given name. + * - `aliasName` - If the candidates size is 1 and it's type is `ExtractValue` (which means that + * it's a recursive type), then the `aliasName` should be the name with which the candidate is + * aliased. Otherwise, `aliasName` should be `None`. + * - `allAttributes` - A list of all attributes which is used to generate suggestions for + * unresolved column error. + * + * Example: + * + * - Attribute resolution: + * {{{ SELECT col1 FROM VALUES (1); }}} will have a [[NameTarget]] with a single candidate `col1`. + * `aliasName` would be `None` in this case because the column is not of recursive type. + * + * - Recursive attribute resolution: + * {{{ SELECT col1.col1 FROM VALUES(STRUCT(1,2), 3) }}} will have a [[NameTarget]] with a + * single candidate `col1` and an `aliasName` of `Some("col1")`. + */ +case class NameTarget( + candidates: Seq[Expression], + aliasName: Option[String] = None, + allAttributes: Seq[Attribute] = Seq.empty) { + + /** + * Picks a candidate from the list of candidates based on the given unresolved attribute. + * Its behavior is as follows (based on the number of candidates): + * + * - If there is only one candidate, it will be returned. + * + * - If there are multiple candidates, an ambiguous reference error will be thrown. + * + * - If there are no candidates, an unresolved column error will be thrown. + */ + def pickCandidate(unresolvedAttribute: UnresolvedAttribute): Expression = { + candidates match { + case Seq() => + throwUnresolvedColumnError(unresolvedAttribute) + case Seq(candidate) => + candidate + case _ => + throw QueryCompilationErrors.ambiguousReferenceError( + unresolvedAttribute.name, + candidates.collect { case attribute: AttributeReference => attribute } + ) + } + } + + private def throwUnresolvedColumnError(unresolvedAttribute: UnresolvedAttribute): Nothing = + throw QueryCompilationErrors.unresolvedColumnError( + unresolvedAttribute.name, + proposal = orderSuggestedIdentifiersBySimilarity( + unresolvedAttribute.name, + candidates = allAttributes.map(attribute => attribute.qualifier :+ attribute.name) + ) + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala new file mode 100644 index 0000000000000..fcf1eab0c04a9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala @@ -0,0 +1,58 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.LogKeys.QUERY_PLAN +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.sideBySide +import org.apache.spark.sql.internal.SQLConf + +/** + * [[PlanLogger]] is used by the [[Resolver]] to log intermediate resolution results. + */ +class PlanLogger extends Logging { + private val logLevel = SQLConf.get.planChangeLogLevel + + /** + * Logs the transition from the `unresolvedPlan` to the `resolvedPlan`. + */ + def log(unresolvedPlan: LogicalPlan, resolvedPlan: LogicalPlan): Unit = { + logBasedOnLevel(() => createMessage(unresolvedPlan, resolvedPlan)) + } + + private def createMessage( + unresolvedPlan: LogicalPlan, + resolvedPlan: LogicalPlan): MessageWithContext = + log""" + |=== Unresolved/resolved operator subtree === + |${MDC( + QUERY_PLAN, + sideBySide(unresolvedPlan.treeString, resolvedPlan.treeString).mkString("\n") + )} + """.stripMargin + + private def logBasedOnLevel(createMessage: () => MessageWithContext): Unit = logLevel match { + case "TRACE" => logTrace(createMessage().message) + case "DEBUG" => logDebug(createMessage().message) + case "INFO" => logInfo(createMessage()) + case "WARN" => logWarning(createMessage()) + case "ERROR" => logError(createMessage()) + case _ => logTrace(createMessage().message) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PredicateResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PredicateResolver.scala new file mode 100644 index 0000000000000..d94559496d04e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PredicateResolver.scala @@ -0,0 +1,112 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{ + AnsiStringPromotionTypeCoercion, + AnsiTypeCoercion, + ApplyCharTypePaddingHelper, + BooleanEqualityTypeCoercion, + CollationTypeCoercion, + DecimalPrecisionTypeCoercion, + DivisionTypeCoercion, + IntegralDivisionTypeCoercion, + StringPromotionTypeCoercion, + TypeCoercion +} +import org.apache.spark.sql.catalyst.expressions.{Expression, Predicate, StringRPad} +import org.apache.spark.sql.internal.SQLConf + +/** + * Resolver class for resolving all [[Predicate]] expressions. Recursively resolves all children + * and applies selected type coercions to the expression. + */ +class PredicateResolver( + expressionResolver: ExpressionResolver, + timezoneAwareExpressionResolver: TimezoneAwareExpressionResolver) + extends TreeNodeResolver[Predicate, Expression] + with ResolvesExpressionChildren { + + private val typeCoercionRules = if (conf.ansiEnabled) { + PredicateResolver.ANSI_TYPE_COERCION_RULES + } else { + PredicateResolver.TYPE_COERCION_RULES + } + private val typeCoercionResolver = + new TypeCoercionResolver(timezoneAwareExpressionResolver, typeCoercionRules) + + override def resolve(unresolvedPredicate: Predicate): Expression = { + val predicateWithResolvedChildren = + withResolvedChildren(unresolvedPredicate, expressionResolver.resolve) + val predicateWithTypeCoercion = typeCoercionResolver.resolve(predicateWithResolvedChildren) + val predicateWithCharTypePadding = { + ApplyCharTypePaddingHelper.singleNodePaddingForStringComparison( + predicateWithTypeCoercion, + !conf.getConf(SQLConf.LEGACY_NO_CHAR_PADDING_IN_PREDICATE) + ) + } + predicateWithCharTypePadding.children.collectFirst { + case rpad: StringRPad => rpad + } match { + // In the fixed-point Analyzer [[ApplyCharTypePadding]] is called after [[ResolveAliases]] + // and therefore padding doesn't affect the alias. In single-pass resolver we need to call + // this code before we resolve the alias, which will cause the alias to include the pad in + // its name: + // + // fixed-point: + // expression: rpad('12', 3, ' ') = '12 ' + // alias: '12' = '12 ' + // + // single-pass: + // expression: rpad('12', 3, ' ') = '12 ' + // alias: rpad('12', 3, ' ') = '12 ' + // + // Disabling this case until the aliasing is fixed. + case Some(_) => throw new ExplicitlyUnsupportedResolverFeature("CharTypePaddingAliasing") + + case _ => predicateWithCharTypePadding + } + } +} + +object PredicateResolver { + // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]]. + private val TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + CollationTypeCoercion.apply, + TypeCoercion.InTypeCoercion.apply, + StringPromotionTypeCoercion.apply, + DecimalPrecisionTypeCoercion.apply, + BooleanEqualityTypeCoercion.apply, + DivisionTypeCoercion.apply, + IntegralDivisionTypeCoercion.apply, + TypeCoercion.ImplicitTypeCoercion.apply, + TypeCoercion.DateTimeOperationsTypeCoercion.apply + ) + + // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]]. + private val ANSI_TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + CollationTypeCoercion.apply, + AnsiTypeCoercion.InTypeCoercion.apply, + AnsiStringPromotionTypeCoercion.apply, + DecimalPrecisionTypeCoercion.apply, + DivisionTypeCoercion.apply, + IntegralDivisionTypeCoercion.apply, + AnsiTypeCoercion.ImplicitTypeCoercion.apply, + AnsiTypeCoercion.AnsiDateTimeOperationsTypeCoercion.apply + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala new file mode 100644 index 0000000000000..8d85804a93634 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala @@ -0,0 +1,52 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.expressions.Expression + +/** + * A mixin trait for expression resolvers that as part of their resolution, replace single node + * with a subtree of nodes. This step is necessary because the underlying legacy code that is being + * called produces partially-unresolved subtrees. In order to resolve the subtree a callback + * resolver is called recursively. This callback must ensure that no node is resolved twice in + * order to not break the single-pass invariant. This is done by tagging the limits of this + * traversal with [[ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY]] tag. This tag is applied to + * the original expression's children, which are guaranteed to be resolved at the time of given + * expression's resolution. When callback resolver encounters the node that is tagged, it should + * return identity instead of trying to resolve it. + */ +trait ProducesUnresolvedSubtree extends ResolvesExpressionChildren { + + /** + * Helper method used to resolve a subtree that is generated as part of the resolution of some + * node. Method ensures that the downwards traversal never visits previously resolved nodes by + * tracking the limits of the traversal with a tag. Invokes a resolver callback to resolve + * children, but DOES NOT resolve the root of the subtree. + */ + protected def withResolvedSubtree( + expression: Expression, + expressionResolver: Expression => Expression)(body: => Expression): Expression = { + expression.children.foreach { child => + child.setTagValue(ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY, ()) + } + + val result = body + + withResolvedChildren(result, expressionResolver) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationId.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationId.scala new file mode 100644 index 0000000000000..eab97dd8345cb --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationId.scala @@ -0,0 +1,31 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** + * The [[RelationId]] is a unique identifier for a relation. It is used to lookup the relations + * which were processed by the [[MetadataResolver]] to substitute the unresolved relations in single + * pass during the analysis phase. + */ +case class RelationId( + multipartIdentifier: Seq[String], + options: CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty, + isStreaming: Boolean = false +) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationMetadataProvider.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationMetadataProvider.scala new file mode 100644 index 0000000000000..cf352842fd106 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RelationMetadataProvider.scala @@ -0,0 +1,86 @@ +/* + * 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.catalyst.analysis.resolver + +import java.util.HashMap + +import org.apache.spark.sql.catalyst.analysis.{withPosition, RelationResolution, UnresolvedRelation} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.LookupCatalog +import org.apache.spark.util.ArrayImplicits._ + +/** + * [[RelationMetadataProvider]] provides relations with resolved metadata based on the + * corresponding [[UnresolvedRelation]]s. It is used by [[Resolver]] to replace + * [[UnresolvedRelation]] with a specific [[LogicalPlan]] with resolved metadata, e.g. with + * [[UnresolvedCatalogRelation]] or [[View]]. + */ +trait RelationMetadataProvider extends LookupCatalog { + type RelationsWithResolvedMetadata = HashMap[RelationId, LogicalPlan] + + /** + * [[relationResolution]] is used by the [[RelationMetadataProvider]] to expand relation + * identifiers in [[relationIdFromUnresolvedRelation]]. + */ + protected val relationResolution: RelationResolution + + /** + * [[relationsWithResolvedMetadata]] is a map from relation ID to the specific [[LogicalPlan]] + * with resolved metadata, like [[UnresolvedCatalogRelation]] or [[View]]. It's filled by the + * specific [[RelationMetadataProvider]] implementation and is queried in + * [[getRelationWithResolvedMetadata]]. + */ + protected val relationsWithResolvedMetadata: RelationsWithResolvedMetadata + + /** + * Get the [[LogicalPlan]] with resolved metadata for the given [[UnresolvedRelation]]. + * + * [[java.util.HashMap]] returns `null` if the key is not found, so we wrap it in an [[Option]]. + */ + def getRelationWithResolvedMetadata( + unresolvedRelation: UnresolvedRelation): Option[LogicalPlan] = { + Option( + relationsWithResolvedMetadata.get( + relationIdFromUnresolvedRelation(unresolvedRelation) + ) + ) + } + + /** + * Returns the [[RelationId]] for the given [[UnresolvedRelation]]. Here we use + * [[relationResolution]] to expand the [[UnresolvedRelation]] identifier fully, so that our + * [[RelationId]] uniquely identifies the [[unresolvedRelation]]. + * + * This method is public, because it's used in [[MetadataResolverSuite]]. + */ + def relationIdFromUnresolvedRelation(unresolvedRelation: UnresolvedRelation): RelationId = { + relationResolution.expandIdentifier(unresolvedRelation.multipartIdentifier) match { + case CatalogAndIdentifier(catalog, ident) => + RelationId( + multipartIdentifier = + Seq(catalog.name()) ++ ident.namespace().toImmutableArraySeq ++ Seq(ident.name()), + options = unresolvedRelation.options, + isStreaming = unresolvedRelation.isStreaming + ) + case _ => + withPosition(unresolvedRelation) { + unresolvedRelation.tableNotFound(unresolvedRelation.multipartIdentifier) + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala new file mode 100644 index 0000000000000..6c4de2e6e58d7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala @@ -0,0 +1,153 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, ResolvedInlineTable} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.{ + Filter, + GlobalLimit, + LocalLimit, + LocalRelation, + LogicalPlan, + OneRowRelation, + Project, + SubqueryAlias +} +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.types.BooleanType + +/** + * The [[ResolutionValidator]] performs the validation work after the logical plan tree is + * resolved by the [[Resolver]]. Each `resolve*` method in the [[Resolver]] must + * have its `validate*` counterpart in the [[ResolutionValidator]]. The validation code asserts the + * conditions that must never be false no matter which SQL query or DataFrame program was provided. + * The validation approach is single-pass, post-order, complementary to the resolution process. + */ +class ResolutionValidator { + private val expressionResolutionValidator = new ExpressionResolutionValidator(this) + + private[resolver] var attributeScopeStack = new AttributeScopeStack + + /** + * Validate the resolved logical `plan` - assert invariants that should never be false no + * matter which SQL query or DataFrame program was provided. New operators must be added here as + * soon as [[Resolver]] supports them. We check this by throwing an exception for + * unknown operators. + */ + def validatePlan(plan: LogicalPlan): Unit = wrapErrors(plan) { + validate(plan) + } + + private def validate(operator: LogicalPlan): Unit = { + operator match { + case project: Project => + validateProject(project) + case filter: Filter => + validateFilter(filter) + case subqueryAlias: SubqueryAlias => + validateSubqueryAlias(subqueryAlias) + case globalLimit: GlobalLimit => + validateGlobalLimit(globalLimit) + case localLimit: LocalLimit => + validateLocalLimit(localLimit) + case inlineTable: ResolvedInlineTable => + validateInlineTable(inlineTable) + case localRelation: LocalRelation => + validateRelation(localRelation) + case oneRowRelation: OneRowRelation => + validateRelation(oneRowRelation) + // [[LogicalRelation]], [[HiveTableRelation]] and other specific relations can't be imported + // because of a potential circular dependency, so we match a generic Catalyst + // [[MultiInstanceRelation]] instead. + case multiInstanceRelation: MultiInstanceRelation => + validateRelation(multiInstanceRelation) + } + } + + private def validateProject(project: Project): Unit = { + attributeScopeStack.withNewScope { + validate(project.child) + expressionResolutionValidator.validateProjectList(project.projectList) + } + + handleOperatorOutput(project) + } + + private def validateFilter(filter: Filter): Unit = { + validate(filter.child) + + assert( + filter.condition.dataType == BooleanType, + s"Output type of a filter must be a boolean, but got: ${filter.condition.dataType.typeName}" + ) + expressionResolutionValidator.validate(filter.condition) + } + + private def validateSubqueryAlias(subqueryAlias: SubqueryAlias): Unit = { + validate(subqueryAlias.child) + + handleOperatorOutput(subqueryAlias) + } + + private def validateGlobalLimit(globalLimit: GlobalLimit): Unit = { + validate(globalLimit.child) + expressionResolutionValidator.validate(globalLimit.limitExpr) + } + + private def validateLocalLimit(localLimit: LocalLimit): Unit = { + validate(localLimit.child) + expressionResolutionValidator.validate(localLimit.limitExpr) + } + + private def validateInlineTable(inlineTable: ResolvedInlineTable): Unit = { + inlineTable.rows.foreach(row => { + row.foreach(expression => { + expressionResolutionValidator.validate(expression) + }) + }) + + handleOperatorOutput(inlineTable) + } + + private def validateRelation(relation: LogicalPlan): Unit = { + handleOperatorOutput(relation) + } + + private def handleOperatorOutput(operator: LogicalPlan): Unit = { + attributeScopeStack.overwriteTop(operator.output) + + operator.output.foreach(attribute => { + assert( + attribute.isInstanceOf[AttributeReference], + s"Output of an operator must be a reference to an attribute, but got: " + + s"${attribute.getClass.getSimpleName}" + ) + expressionResolutionValidator.validate(attribute) + }) + } + + private def wrapErrors[R](plan: LogicalPlan)(body: => R): Unit = { + try { + body + } catch { + case ex: Throwable => + throw QueryCompilationErrors.resolutionValidationError(ex, plan) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala new file mode 100644 index 0000000000000..b0e6828a97a08 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala @@ -0,0 +1,364 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.EvaluateUnresolvedInlineTable +import org.apache.spark.sql.catalyst.analysis.{ + withPosition, + FunctionResolution, + NamedRelation, + RelationResolution, + ResolvedInlineTable, + UnresolvedInlineTable, + UnresolvedRelation +} +import org.apache.spark.sql.catalyst.plans.logical.{ + Filter, + GlobalLimit, + LocalLimit, + LocalRelation, + LogicalPlan, + OneRowRelation, + Project, + SubqueryAlias +} +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} +import org.apache.spark.sql.types.BooleanType + +/** + * The Resolver implements a single-pass bottom-up analysis algorithm in the Catalyst. + * + * The functions here generally traverse the [[LogicalPlan]] nodes recursively, + * constructing and returning the resolved [[LogicalPlan]] nodes bottom-up. + * This is the primary entry point for implementing SQL and DataFrame plan analysis, + * wherein the [[resolve]] method accepts a fully unresolved [[LogicalPlan]] and returns + * a fully resolved [[LogicalPlan]] in response with all data types and attribute + * reference ID assigned for valid requests. This resolver also takes responsibility + * to detect any errors in the initial SQL query or DataFrame and return appropriate + * error messages including precise parse locations wherever possible. + * + * The Resolver is a one-shot object per each SQL/DataFrame logical plan, the calling code must + * re-create it for every new analysis run. + * + * @param catalogManager [[CatalogManager]] for relation and identifier resolution. + * @param extensions A list of [[ResolverExtension]] that can resolve external operators. + */ +class Resolver( + catalogManager: CatalogManager, + override val extensions: Seq[ResolverExtension] = Seq.empty, + metadataResolverExtensions: Seq[ResolverExtension] = Seq.empty) + extends TreeNodeResolver[LogicalPlan, LogicalPlan] + with QueryErrorsBase + with ResolvesOperatorChildren + with TracksResolvedNodes[LogicalPlan] + with DelegatesResolutionToExtensions { + private val scopes = new NameScopeStack + private val relationResolution = Resolver.createRelationResolution(catalogManager) + private val functionResolution = new FunctionResolution(catalogManager, relationResolution) + private val expressionResolver = new ExpressionResolver(this, scopes, functionResolution) + private val limitExpressionResolver = new LimitExpressionResolver(expressionResolver) + private val planLogger = new PlanLogger + + /** + * [[relationMetadataProvider]] is used to resolve metadata for relations. It's initialized with + * the default implementation [[MetadataResolver]] here and is called in + * [[lookupMetadataAndResolve]] on the unresolved logical plan to visit it (both operators and + * expressions) to resolve the metadata and populate its internal state. It's later queried by + * [[resolveRelation]] to get the plan with resolved metadata (for example, a [[View]] or an + * [[UnresolvedCatalogRelation]]) based on the [[UnresolvedRelation]]. + * + * If the [[AnalyzerBridgeState]] is provided, we reset this provider to the + * [[BridgedRelationMetadataProvider]] and later stick to it forever without resorting to the + * actual blocking metadata resolution. + */ + private var relationMetadataProvider: RelationMetadataProvider = new MetadataResolver( + catalogManager, + relationResolution, + metadataResolverExtensions + ) + + /** + * This method is an analysis entry point. It resolves the metadata and invokes [[resolve]], + * which does most of the analysis work. + */ + def lookupMetadataAndResolve( + unresolvedPlan: LogicalPlan, + analyzerBridgeState: Option[AnalyzerBridgeState] = None): LogicalPlan = { + relationMetadataProvider = analyzerBridgeState match { + case Some(analyzerBridgeState) => + new BridgedRelationMetadataProvider( + catalogManager, + relationResolution, + analyzerBridgeState + ) + case None => + relationMetadataProvider + } + + relationMetadataProvider match { + case metadataResolver: MetadataResolver => + metadataResolver.resolve(unresolvedPlan) + case _ => + } + + resolve(unresolvedPlan) + } + + /** + * This method takes an unresolved [[LogicalPlan]] and chooses the right `resolve*` method using + * pattern matching on the `unresolvedPlan` type. This pattern matching enumerates all the + * operator node types that are supported by the single-pass analysis. + * + * When developers introduce a new unresolved node type to the Catalyst, they should implement + * a corresponding `resolve*` method in the [[Resolver]] and add it to this pattern match + * list. + * + * [[resolve]] will be called recursively during the unresolved plan traversal eventually + * producing a fully resolved plan or a descriptive error message. + */ + override def resolve(unresolvedPlan: LogicalPlan): LogicalPlan = { + throwIfNodeWasResolvedEarlier(unresolvedPlan) + + val resolvedPlan = + unresolvedPlan match { + case unresolvedProject: Project => + resolveProject(unresolvedProject) + case unresolvedFilter: Filter => + resolveFilter(unresolvedFilter) + case unresolvedSubqueryAlias: SubqueryAlias => + resolveSubqueryAlias(unresolvedSubqueryAlias) + case unresolvedGlobalLimit: GlobalLimit => + resolveGlobalLimit(unresolvedGlobalLimit) + case unresolvedLocalLimit: LocalLimit => + resolveLocalLimit(unresolvedLocalLimit) + case unresolvedRelation: UnresolvedRelation => + resolveRelation(unresolvedRelation) + case unresolvedInlineTable: UnresolvedInlineTable => + resolveInlineTable(unresolvedInlineTable) + // See the reason why we have to match both [[LocalRelation]] and [[ResolvedInlineTable]] + // in the [[resolveInlineTable]] scaladoc + case resolvedInlineTable: ResolvedInlineTable => + updateNameScopeWithPlanOutput(resolvedInlineTable) + case localRelation: LocalRelation => + updateNameScopeWithPlanOutput(localRelation) + case unresolvedOneRowRelation: OneRowRelation => + updateNameScopeWithPlanOutput(unresolvedOneRowRelation) + case _ => + tryDelegateResolutionToExtension(unresolvedPlan).getOrElse { + handleUnmatchedOperator(unresolvedPlan) + } + } + + markNodeAsResolved(resolvedPlan) + planLogger.log(unresolvedPlan, resolvedPlan) + resolvedPlan + } + + /** + * [[Project]] introduces a new scope to resolve its subtree and project list expressions. After + * those are resolved in the child scope we overwrite current scope with resolved [[Project]]'s + * output to expose new names to the parent operators. + */ + private def resolveProject(unresolvedProject: Project): LogicalPlan = { + val resolvedProject = scopes.withNewScope { + val resolvedChild = resolve(unresolvedProject.child) + val resolvedProjectList = + expressionResolver.resolveProjectList(unresolvedProject.projectList) + Project(resolvedProjectList, resolvedChild) + } + + withPosition(unresolvedProject) { + scopes.overwriteTop(resolvedProject.output) + } + + resolvedProject + } + + /** + * [[Filter]] has a single child and a single condition and we resolve them in this respective + * order. + */ + private def resolveFilter(unresolvedFilter: Filter): LogicalPlan = { + val resolvedChild = resolve(unresolvedFilter.child) + val resolvedCondition = expressionResolver.resolve(unresolvedFilter.condition) + + val resolvedFilter = Filter(resolvedCondition, resolvedChild) + if (resolvedFilter.condition.dataType != BooleanType) { + withPosition(unresolvedFilter) { + throwDatatypeMismatchFilterNotBoolean(resolvedFilter) + } + } + + resolvedFilter + } + + /** + * [[SubqueryAlias]] has a single child and an identifier. We need to resolve the child and update + * the scope with the output, since upper expressions can reference [[SubqueryAlias]]es output by + * its identifier. + */ + private def resolveSubqueryAlias(unresolvedSubqueryAlias: SubqueryAlias): LogicalPlan = { + val resolvedSubqueryAlias = + SubqueryAlias(unresolvedSubqueryAlias.identifier, resolve(unresolvedSubqueryAlias.child)) + withPosition(unresolvedSubqueryAlias) { + scopes.overwriteTop(unresolvedSubqueryAlias.alias, resolvedSubqueryAlias.output) + } + resolvedSubqueryAlias + } + + /** + * Resolve [[GlobalLimit]]. We have to resolve its child and resolve and validate its limit + * expression. + */ + private def resolveGlobalLimit(unresolvedGlobalLimit: GlobalLimit): LogicalPlan = { + val resolvedChild = resolve(unresolvedGlobalLimit.child) + + val resolvedLimitExpr = withPosition(unresolvedGlobalLimit) { + limitExpressionResolver.resolve(unresolvedGlobalLimit.limitExpr) + } + + GlobalLimit(resolvedLimitExpr, resolvedChild) + } + + /** + * Resolve [[LocalLimit]]. We have to resolve its child and resolve and validate its limit + * expression. + */ + private def resolveLocalLimit(unresolvedLocalLimit: LocalLimit): LogicalPlan = { + val resolvedChild = resolve(unresolvedLocalLimit.child) + + val resolvedLimitExpr = withPosition(unresolvedLocalLimit) { + limitExpressionResolver.resolve(unresolvedLocalLimit.limitExpr) + } + + LocalLimit(resolvedLimitExpr, resolvedChild) + } + + /** + * [[UnresolvedRelation]] was previously looked up by the [[MetadataResolver]] and now we need to: + * - Get the specific relation with metadata from `relationsWithResolvedMetadata`, like + * [[UnresolvedCatalogRelation]], or throw an error if it wasn't found + * - Resolve it further, usually using extensions, like [[DataSourceResolver]] + */ + private def resolveRelation(unresolvedRelation: UnresolvedRelation): LogicalPlan = { + relationMetadataProvider.getRelationWithResolvedMetadata(unresolvedRelation) match { + case Some(relationWithResolvedMetadata) => + planLogger.log(unresolvedRelation, relationWithResolvedMetadata) + + withPosition(unresolvedRelation) { + resolve(relationWithResolvedMetadata) + } + case None => + withPosition(unresolvedRelation) { + unresolvedRelation.tableNotFound(unresolvedRelation.multipartIdentifier) + } + } + } + + /** + * [[UnresolvedInlineTable]] resolution requires all the rows to be resolved first. After that we + * use [[EvaluateUnresolvedInlineTable]] and try to evaluate the row expressions if possible to + * get [[LocalRelation]] right away. Sometimes it's not possible because of expressions like + * `current_date()` which are evaluated in the optimizer (SPARK-46380). + * + * Note: By default if all the inline table expressions can be evaluated eagerly, the parser + * would produce a [[LocalRelation]] and the analysis would just skip this step and go straight + * to `resolveLocalRelation` (SPARK-48967, SPARK-49269). + */ + private def resolveInlineTable(unresolvedInlineTable: UnresolvedInlineTable): LogicalPlan = { + val withResolvedExpressions = UnresolvedInlineTable( + unresolvedInlineTable.names, + unresolvedInlineTable.rows.map(row => { + row.map(expressionResolver.resolve(_)) + }) + ) + + val resolvedRelation = EvaluateUnresolvedInlineTable + .evaluateUnresolvedInlineTable(withResolvedExpressions) + + withPosition(unresolvedInlineTable) { + resolve(resolvedRelation) + } + } + + /** + * To finish the operator resolution we add its output to the current scope. This is usually + * done for relations. [[NamedRelation]]'s output should be added to the scope under its name. + */ + private def updateNameScopeWithPlanOutput(relation: LogicalPlan): LogicalPlan = { + withPosition(relation) { + relation match { + case namedRelation: NamedRelation => + scopes.top.update(namedRelation.name, namedRelation.output) + case _ => + scopes.top += relation.output + } + } + relation + } + + override def tryDelegateResolutionToExtension( + unresolvedOperator: LogicalPlan): Option[LogicalPlan] = { + val resolutionResult = super.tryDelegateResolutionToExtension(unresolvedOperator) + resolutionResult.map { resolvedOperator => + updateNameScopeWithPlanOutput(resolvedOperator) + } + } + + /** + * Check if the unresolved operator is explicitly unsupported and throw + * [[ExplicitlyUnsupportedResolverFeature]] in that case. Otherwise, throw + * [[QueryCompilationErrors.unsupportedSinglePassAnalyzerFeature]]. + */ + private def handleUnmatchedOperator(unresolvedOperator: LogicalPlan): Nothing = { + if (ExplicitlyUnsupportedResolverFeature.OPERATORS.contains( + unresolvedOperator.getClass.getName + )) { + throw new ExplicitlyUnsupportedResolverFeature( + s"unsupported operator: ${unresolvedOperator.getClass.getName}" + ) + } + throw QueryCompilationErrors + .unsupportedSinglePassAnalyzerFeature( + s"${unresolvedOperator.getClass} operator resolution" + ) + .withPosition(unresolvedOperator.origin) + } + + private def throwDatatypeMismatchFilterNotBoolean(filter: Filter): Nothing = + throw new AnalysisException( + errorClass = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", + messageParameters = Map( + "sqlExpr" -> filter.expressions.map(toSQLExpr).mkString(","), + "filter" -> toSQLExpr(filter.condition), + "type" -> toSQLType(filter.condition.dataType) + ) + ) +} + +object Resolver { + + /** + * Create a new instance of the [[RelationResolution]]. + */ + def createRelationResolution(catalogManager: CatalogManager): RelationResolution = { + new RelationResolution(catalogManager) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverExtension.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverExtension.scala new file mode 100644 index 0000000000000..8bed881ec97a1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverExtension.scala @@ -0,0 +1,43 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * The [[ResolverExtension]] is a main interface for single-pass analysis extensions in Catalyst. + * External code that needs specific node types to be resolved has to implement this trait and + * inject the implementation into the [[Analyzer.singlePassResolverExtensions]]. + * + * Note that resolver extensions are responsible for creating attribute references with IDs that + * are unique from any other subplans. This should be straightforward in most cases because + * creating new attribute references will assign [[NamedExpression.newExprId]] by default. + */ +trait ResolverExtension { + + /** + * Resolve the operator if it's supported by this extension. This method is called by the + * single-pass [[Resolver]] on all the configured extensions when it exhausted its match list + * for the known node types. + * + * Guarantees: + * - The implementation can rely on children being resolved + * - We commit to performing the partial function check only at most once per unresolved operator + */ + def resolveOperator: PartialFunction[LogicalPlan, LogicalPlan] +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala new file mode 100644 index 0000000000000..b3b3d4def602d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala @@ -0,0 +1,283 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.{ + ResolvedInlineTable, + UnresolvedAlias, + UnresolvedAttribute, + UnresolvedFunction, + UnresolvedInlineTable, + UnresolvedRelation, + UnresolvedStar +} +import org.apache.spark.sql.catalyst.expressions.{ + Alias, + AttributeReference, + BinaryArithmetic, + Cast, + ConditionalExpression, + CreateNamedStruct, + Expression, + Literal, + Predicate, + SubqueryExpression +} +import org.apache.spark.sql.catalyst.plans.logical.{ + Filter, + GlobalLimit, + LocalLimit, + LocalRelation, + LogicalPlan, + OneRowRelation, + Project, + SubqueryAlias +} +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode + +/** + * [[ResolverGuard]] is a class that checks if the operator that is yet to be analyzed + * only consists of operators and expressions that are currently supported by the + * single-pass analyzer. + * + * This is a one-shot object and should not be reused after [[apply]] call. + */ +class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper { + + /** + * Check the top level operator of the parsed operator. + */ + def apply(operator: LogicalPlan): Boolean = + checkConfValues() && checkVariables() && checkOperator(operator) + + /** + * Check if all the operators are supported. For implemented ones, recursively check + * their children. For unimplemented ones, return false. + */ + private def checkOperator(operator: LogicalPlan): Boolean = operator match { + case project: Project => + checkProject(project) + case filter: Filter => + checkFilter(filter) + case subqueryAlias: SubqueryAlias => + checkSubqueryAlias(subqueryAlias) + case globalLimit: GlobalLimit => + checkGlobalLimit(globalLimit) + case localLimit: LocalLimit => + checkLocalLimit(localLimit) + case unresolvedRelation: UnresolvedRelation => + checkUnresolvedRelation(unresolvedRelation) + case unresolvedInlineTable: UnresolvedInlineTable => + checkUnresolvedInlineTable(unresolvedInlineTable) + case resolvedInlineTable: ResolvedInlineTable => + checkResolvedInlineTable(resolvedInlineTable) + case localRelation: LocalRelation => + checkLocalRelation(localRelation) + case oneRowRelation: OneRowRelation => + checkOneRowRelation(oneRowRelation) + case _ => + false + } + + /** + * Method used to check if expressions are supported by the new analyzer. + * For LeafNode types, we return true or false. For other ones, check their children. + */ + private def checkExpression(expression: Expression): Boolean = { + expression match { + case alias: Alias => + checkAlias(alias) + case unresolvedBinaryArithmetic: BinaryArithmetic => + checkUnresolvedBinaryArithmetic(unresolvedBinaryArithmetic) + case unresolvedConditionalExpression: ConditionalExpression => + checkUnresolvedConditionalExpression(unresolvedConditionalExpression) + case unresolvedCast: Cast => + checkUnresolvedCast(unresolvedCast) + case unresolvedStar: UnresolvedStar => + checkUnresolvedStar(unresolvedStar) + case unresolvedAlias: UnresolvedAlias => + checkUnresolvedAlias(unresolvedAlias) + case unresolvedAttribute: UnresolvedAttribute => + checkUnresolvedAttribute(unresolvedAttribute) + case unresolvedPredicate: Predicate => + checkUnresolvedPredicate(unresolvedPredicate) + case literal: Literal => + checkLiteral(literal) + case attributeReference: AttributeReference => + checkAttributeReference(attributeReference) + case createNamedStruct: CreateNamedStruct => + checkCreateNamedStruct(createNamedStruct) + case unresolvedFunction: UnresolvedFunction => + checkUnresolvedFunction(unresolvedFunction) + case _ => + false + } + } + + private def checkProject(project: Project) = { + checkOperator(project.child) && project.projectList.forall(checkExpression) + } + + private def checkFilter(unresolvedFilter: Filter) = + checkOperator(unresolvedFilter.child) && checkExpression(unresolvedFilter.condition) + + private def checkSubqueryAlias(subqueryAlias: SubqueryAlias) = + subqueryAlias.identifier.qualifier.isEmpty && checkOperator(subqueryAlias.child) + + private def checkGlobalLimit(globalLimit: GlobalLimit) = + checkOperator(globalLimit.child) && checkExpression(globalLimit.limitExpr) + + private def checkLocalLimit(localLimit: LocalLimit) = + checkOperator(localLimit.child) && checkExpression(localLimit.limitExpr) + + private def checkUnresolvedInlineTable(unresolvedInlineTable: UnresolvedInlineTable) = + unresolvedInlineTable.rows.forall(_.forall(checkExpression)) + + private def checkUnresolvedRelation(unresolvedRelation: UnresolvedRelation) = true + + private def checkResolvedInlineTable(resolvedInlineTable: ResolvedInlineTable) = + resolvedInlineTable.rows.forall(_.forall(checkExpression)) + + // Usually we don't check outputs of operators in unresolved plans, but in this case + // [[LocalRelation]] is resolved in the parser. + private def checkLocalRelation(localRelation: LocalRelation) = + localRelation.output.forall(checkExpression) + + private def checkOneRowRelation(oneRowRelation: OneRowRelation) = true + + private def checkAlias(alias: Alias) = checkExpression(alias.child) + + private def checkUnresolvedBinaryArithmetic(unresolvedBinaryArithmetic: BinaryArithmetic) = + checkExpression(unresolvedBinaryArithmetic.left) && + checkExpression(unresolvedBinaryArithmetic.right) + + private def checkUnresolvedConditionalExpression( + unresolvedConditionalExpression: ConditionalExpression) = + unresolvedConditionalExpression.children.forall(checkExpression) + + private def checkUnresolvedCast(cast: Cast) = checkExpression(cast.child) + + private def checkUnresolvedStar(unresolvedStar: UnresolvedStar) = true + + private def checkUnresolvedAlias(unresolvedAlias: UnresolvedAlias) = + checkExpression(unresolvedAlias.child) + + private def checkUnresolvedAttribute(unresolvedAttribute: UnresolvedAttribute) = + !ResolverGuard.UNSUPPORTED_ATTRIBUTE_NAMES.contains(unresolvedAttribute.nameParts.head) + + private def checkUnresolvedPredicate(unresolvedPredicate: Predicate) = { + unresolvedPredicate match { + case _: SubqueryExpression => false + case other => + other.children.forall(checkExpression) + } + } + + private def checkAttributeReference(attributeReference: AttributeReference) = true + + private def checkCreateNamedStruct(createNamedStruct: CreateNamedStruct) = { + createNamedStruct.children.forall(checkExpression) + } + + private def checkUnresolvedFunction(unresolvedFunction: UnresolvedFunction) = + ResolverGuard.SUPPORTED_FUNCTION_NAMES.contains( + unresolvedFunction.nameParts.head + ) && unresolvedFunction.children.forall(checkExpression) + + private def checkLiteral(literal: Literal) = true + + private def checkConfValues() = + // Case sensitive analysis is not supported. + !conf.caseSensitiveAnalysis && + // Case-sensitive inference is not supported for Hive table schema. + conf.caseSensitiveInferenceMode == HiveCaseSensitiveInferenceMode.NEVER_INFER + + private def checkVariables() = catalogManager.tempVariableManager.isEmpty +} + +object ResolverGuard { + + private val UNSUPPORTED_ATTRIBUTE_NAMES = { + val map = new IdentifierMap[Unit]() + + /** + * Some SQL functions can be called without the braces and thus they are found in the + * parsed operator as UnresolvedAttributes. This list contains the names of those functions + * so we can reject them. Find more information in [[ColumnResolutionHelper.literalFunctions]]. + */ + map += ("current_date", ()) + map += ("current_timestamp", ()) + map += ("current_user", ()) + map += ("user", ()) + map += ("session_user", ()) + map += ("grouping__id", ()) + + /** + * Metadata column resolution is not supported for now + */ + map += ("_metadata", ()) + + map + } + + /** + * Most of the functions are not supported, but we allow some explicitly supported ones. + */ + private val SUPPORTED_FUNCTION_NAMES = { + val map = new IdentifierMap[Unit]() + map += ("array", ()) + // map += ("array_agg", ()) - until aggregate expressions are supported + map += ("array_append", ()) + map += ("array_compact", ()) + map += ("array_contains", ()) + map += ("array_distinct", ()) + map += ("array_except", ()) + map += ("array_insert", ()) + map += ("array_intersect", ()) + map += ("array_join", ()) + map += ("array_max", ()) + map += ("array_min", ()) + map += ("array_position", ()) + map += ("array_prepend", ()) + map += ("array_remove", ()) + map += ("array_repeat", ()) + map += ("array_size", ()) + // map += ("array_sort", ()) - until lambda functions are supported + map += ("array_union", ()) + map += ("arrays_overlap", ()) + map += ("arrays_zip", ()) + map += ("coalesce", ()) + map += ("if", ()) + map += ("map", ()) + map += ("map_concat", ()) + map += ("map_contains_key", ()) + map += ("map_entries", ()) + // map += ("map_filter", ()) - until lambda functions are supported + map += ("map_from_arrays", ()) + map += ("map_from_entries", ()) + map += ("map_keys", ()) + map += ("map_values", ()) + // map += ("map_zip_with", ()) - until lambda functions are supported + map += ("named_struct", ()) + map += ("sort_array", ()) + map += ("str_to_map", ()) + map + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesExpressionChildren.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesExpressionChildren.scala new file mode 100644 index 0000000000000..c170941ce5348 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesExpressionChildren.scala @@ -0,0 +1,33 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.expressions.Expression + +trait ResolvesExpressionChildren { + + /** + * Resolves generic [[Expression]] children and returns its copy with children resolved. + */ + protected def withResolvedChildren[ExpressionType <: Expression]( + unresolvedExpression: ExpressionType, + resolveChild: Expression => Expression): ExpressionType = { + val newChildren = unresolvedExpression.children.map(resolveChild(_)) + unresolvedExpression.withNewChildren(newChildren).asInstanceOf[ExpressionType] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesOperatorChildren.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesOperatorChildren.scala new file mode 100644 index 0000000000000..0f548c3c55858 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesOperatorChildren.scala @@ -0,0 +1,36 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * A mixin trait for all operator resolvers that need to resolve their children. + */ +trait ResolvesOperatorChildren { + + /** + * Resolves generic [[LogicalPlan]] children and returns its copy with children resolved. + */ + protected def withResolvedChildren[OperatorType <: LogicalPlan]( + unresolvedOperator: OperatorType, + resolve: LogicalPlan => LogicalPlan): OperatorType = { + val newChildren = unresolvedOperator.children.map(resolve(_)) + unresolvedOperator.withNewChildren(newChildren).asInstanceOf[OperatorType] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimeAddResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimeAddResolver.scala new file mode 100644 index 0000000000000..bf27f64598723 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimeAddResolver.scala @@ -0,0 +1,72 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{ + AnsiStringPromotionTypeCoercion, + AnsiTypeCoercion, + StringPromotionTypeCoercion, + TypeCoercion +} +import org.apache.spark.sql.catalyst.expressions.{Expression, TimeAdd} + +/** + * Helper resolver for [[TimeAdd]] which is produced by resolving [[BinaryArithmetic]] nodes. + */ +class TimeAddResolver( + expressionResolver: ExpressionResolver, + timezoneAwareExpressionResolver: TimezoneAwareExpressionResolver) + extends TreeNodeResolver[TimeAdd, Expression] + with ResolvesExpressionChildren { + + private val typeCoercionRules: Seq[Expression => Expression] = + if (conf.ansiEnabled) { + TimeAddResolver.ANSI_TYPE_COERCION_RULES + } else { + TimeAddResolver.TYPE_COERCION_RULES + } + private val typeCoercionResolver: TypeCoercionResolver = + new TypeCoercionResolver(timezoneAwareExpressionResolver, typeCoercionRules) + + override def resolve(unresolvedTimeAdd: TimeAdd): Expression = { + val timeAddWithResolvedChildren: TimeAdd = + withResolvedChildren(unresolvedTimeAdd, expressionResolver.resolve) + val timeAddWithTypeCoercion: Expression = typeCoercionResolver + .resolve(timeAddWithResolvedChildren) + timezoneAwareExpressionResolver.withResolvedTimezone( + timeAddWithTypeCoercion, + conf.sessionLocalTimeZone + ) + } +} + +object TimeAddResolver { + // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]]. + private val TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + StringPromotionTypeCoercion.apply, + TypeCoercion.ImplicitTypeCoercion.apply, + TypeCoercion.DateTimeOperationsTypeCoercion.apply + ) + + // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]]. + private val ANSI_TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + AnsiStringPromotionTypeCoercion.apply, + AnsiTypeCoercion.ImplicitTypeCoercion.apply, + AnsiTypeCoercion.AnsiDateTimeOperationsTypeCoercion.apply + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala new file mode 100644 index 0000000000000..a45e9e41cbfb1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala @@ -0,0 +1,73 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.expressions.{Expression, TimeZoneAwareExpression} + +/** + * Resolves [[TimeZoneAwareExpressions]] by applying the session's local timezone. + * + * This class is responsible for resolving [[TimeZoneAwareExpression]]s by first resolving their + * children and then applying the session's local timezone. Additionally, ensures that any tags from + * the original expression are preserved during the resolution process. + * + * @constructor Creates a new TimezoneAwareExpressionResolver with the given expression resolver. + * @param expressionResolver The [[ExpressionResolver]] used to resolve child expressions. + */ +class TimezoneAwareExpressionResolver(expressionResolver: TreeNodeResolver[Expression, Expression]) + extends TreeNodeResolver[TimeZoneAwareExpression, Expression] + with ResolvesExpressionChildren { + + /** + * Resolves a [[TimeZoneAwareExpression]] by resolving its children and applying a timezone. + * + * @param unresolvedTimezoneExpression The [[TimeZoneAwareExpression]] to resolve. + * @return A resolved [[Expression]] with the session's local timezone applied. + */ + override def resolve(unresolvedTimezoneExpression: TimeZoneAwareExpression): Expression = { + val expressionWithResolvedChildren = + withResolvedChildren(unresolvedTimezoneExpression, expressionResolver.resolve) + withResolvedTimezoneCopyTags(expressionWithResolvedChildren, conf.sessionLocalTimeZone) + } + + /** + * Applies a timezone to a [[TimeZoneAwareExpression]] while preserving original tags. + * + * This method is particularly useful for cases like resolving [[Cast]] expressions where tags + * such as [[USER_SPECIFIED_CAST]] need to be preserved. + * + * @param expression The [[TimeZoneAwareExpression]] to apply the timezone to. + * @param timeZoneId The timezone ID to apply. + * @return A new [[TimeZoneAwareExpression]] with the specified timezone and original tags. + */ + def withResolvedTimezoneCopyTags(expression: Expression, timeZoneId: String): Expression = { + val withTimeZone = withResolvedTimezone(expression, timeZoneId) + withTimeZone.copyTagsFrom(expression) + withTimeZone + } + + /** + * Apply timezone to [[TimeZoneAwareExpression]] expressions. + */ + def withResolvedTimezone(expression: Expression, timeZoneId: String): Expression = + expression match { + case timezoneExpression: TimeZoneAwareExpression if timezoneExpression.timeZoneId.isEmpty => + timezoneExpression.withTimeZone(timeZoneId) + case other => other + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TracksResolvedNodes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TracksResolvedNodes.scala new file mode 100644 index 0000000000000..dd86bf843b4ec --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TracksResolvedNodes.scala @@ -0,0 +1,51 @@ +/* + * 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.catalyst.analysis.resolver + +import java.util.IdentityHashMap + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.internal.SQLConf + +/** + * Trait for top-level resolvers that is used to keep track of resolved nodes and throw an error if + * a node is resolved more than once. This is only used in tests because of the memory overhead of + * using a set to track resolved nodes. + */ +trait TracksResolvedNodes[TreeNodeType <: TreeNode[TreeNodeType]] extends SQLConfHelper { + // Using Map because IdentityHashSet is not available in Scala + private val seenResolvedNodes = new IdentityHashMap[TreeNodeType, Unit] + + private val shouldTrackResolvedNodes = + conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_TRACK_RESOLVED_NODES_ENABLED) + + protected def throwIfNodeWasResolvedEarlier(node: TreeNodeType): Unit = + if (shouldTrackResolvedNodes && seenResolvedNodes.containsKey(node)) { + throw SparkException.internalError( + s"Single-pass resolver attempted to resolve the same node more than once: $node" + ) + } + + protected def markNodeAsResolved(node: TreeNodeType): Unit = { + if (shouldTrackResolvedNodes) { + seenResolvedNodes.put(node, ()) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TreeNodeResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TreeNodeResolver.scala new file mode 100644 index 0000000000000..5991585995cad --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TreeNodeResolver.scala @@ -0,0 +1,30 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.trees.TreeNode + +/** + * Base class for [[TreeNode]] resolvers. All resolvers should extend this class with + * specific [[UnresolvedNode]] and [[ResolvedNode]] types. + */ +trait TreeNodeResolver[UnresolvedNode <: TreeNode[_], ResolvedNode <: TreeNode[_]] + extends SQLConfHelper { + def resolve(unresolvedNode: UnresolvedNode): ResolvedNode +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolver.scala new file mode 100644 index 0000000000000..cf4c2ef0d7504 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolver.scala @@ -0,0 +1,50 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} + +/** + * [[TypeCoercionResolver]] is used by other resolvers to uniformly apply type coercions to all + * expressions. [[TypeCoercionResolver]] takes in a sequence of type coercion transformations that + * should be applied to an expression and applies them in order. Finally, [[TypeCoercionResolver]] + * applies timezone to expression's children, as a child could be replaced with Cast(child, type), + * therefore [[Cast]] resolution is needed. Timezone is applied only on children that have been + * re-instantiated by [[TypeCoercionResolver]], because otherwise children have already been + * resolved. + */ +class TypeCoercionResolver( + timezoneAwareExpressionResolver: TimezoneAwareExpressionResolver, + typeCoercionRules: Seq[Expression => Expression]) + extends TreeNodeResolver[Expression, Expression] { + + override def resolve(expression: Expression): Expression = { + val oldChildren = expression.children + + val withTypeCoercion = typeCoercionRules.foldLeft(expression) { + case (expr, rule) => rule.apply(expr) + } + + val newChildren = withTypeCoercion.children.zip(oldChildren).map { + case (newChild: Cast, oldChild) if !newChild.eq(oldChild) => + timezoneAwareExpressionResolver.withResolvedTimezone(newChild, conf.sessionLocalTimeZone) + case (newChild, _) => newChild + } + withTypeCoercion.withNewChildren(newChildren) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnaryMinusResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnaryMinusResolver.scala new file mode 100644 index 0000000000000..739d7cf43c183 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnaryMinusResolver.scala @@ -0,0 +1,60 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.sql.catalyst.analysis.{AnsiTypeCoercion, TypeCoercion} +import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryMinus} + +/** + * Resolver for [[UnaryMinus]]. Resolves children and applies type coercion to target node. + */ +class UnaryMinusResolver( + expressionResolver: ExpressionResolver, + timezoneAwareExpressionResolver: TimezoneAwareExpressionResolver) + extends TreeNodeResolver[UnaryMinus, Expression] + with ResolvesExpressionChildren { + + private val typeCoercionRules: Seq[Expression => Expression] = + if (conf.ansiEnabled) { + UnaryMinusResolver.ANSI_TYPE_COERCION_RULES + } else { + UnaryMinusResolver.TYPE_COERCION_RULES + } + private val typeCoercionResolver: TypeCoercionResolver = + new TypeCoercionResolver(timezoneAwareExpressionResolver, typeCoercionRules) + + override def resolve(unresolvedUnaryMinus: UnaryMinus): Expression = { + val unaryMinusWithResolvedChildren: UnaryMinus = + withResolvedChildren(unresolvedUnaryMinus, expressionResolver.resolve) + typeCoercionResolver.resolve(unaryMinusWithResolvedChildren) + } +} + +object UnaryMinusResolver { + // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]]. + private val TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + TypeCoercion.ImplicitTypeCoercion.apply, + TypeCoercion.DateTimeOperationsTypeCoercion.apply + ) + + // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]]. + private val ANSI_TYPE_COERCION_RULES: Seq[Expression => Expression] = Seq( + AnsiTypeCoercion.ImplicitTypeCoercion.apply, + AnsiTypeCoercion.AnsiDateTimeOperationsTypeCoercion.apply + ) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 0a73b6b856740..b14f4be534a38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -507,7 +507,9 @@ abstract class UnresolvedStarBase(target: Option[Seq[String]]) extends Star with childOperatorMetadataOutput: Seq[Attribute], resolve: (Seq[String], Resolver) => Option[NamedExpression], suggestedAttributes: Seq[Attribute], - resolver: Resolver): Seq[NamedExpression] = { + resolver: Resolver, + cleanupNestedAliasesDuringStructExpansion: Boolean = false + ): Seq[NamedExpression] = { // If there is no table specified, use all non-hidden input attributes. if (target.isEmpty) return childOperatorOutput @@ -528,11 +530,22 @@ abstract class UnresolvedStarBase(target: Option[Seq[String]]) extends Star with // (i.e. [name].* is both a table and a struct), the struct path can always be qualified. val attribute = resolve(target.get, resolver) if (attribute.isDefined) { + // If cleanupNestedAliasesDuringStructExpansion is true, we remove nested aliases during + // struct expansion. This is something which is done in the CleanupAliases rule but for the + // single-pass analyzer it has to be done here to avoid additional tree traversals. + val normalizedAttribute = if (cleanupNestedAliasesDuringStructExpansion) { + attribute.get match { + case a: Alias => a.child + case other => other + } + } else { + attribute.get + } // This target resolved to an attribute in child. It must be a struct. Expand it. - attribute.get.dataType match { + normalizedAttribute.dataType match { case s: StructType => s.zipWithIndex.map { case (f, i) => - val extract = GetStructField(attribute.get, i) + val extract = GetStructField(normalizedAttribute, i) Alias(extract, f.name)() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/TempVariableManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/TempVariableManager.scala index abe6cede0c550..2c262da1f4449 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/TempVariableManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/TempVariableManager.scala @@ -63,6 +63,10 @@ class TempVariableManager extends DataTypeErrorsBase { def clear(): Unit = synchronized { variables.clear() } + + def isEmpty: Boolean = synchronized { + variables.isEmpty + } } case class VariableDefinition(defaultValueSQL: String, currentValue: Literal) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index b673d5a043153..e55d4d1f95236 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4136,6 +4136,83 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat ) } + def unsupportedSinglePassAnalyzerFeature(feature: String): AnalysisException = { + new AnalysisException( + errorClass = "UNSUPPORTED_SINGLE_PASS_ANALYZER_FEATURE", + messageParameters = Map("feature" -> feature) + ) + } + + def ambiguousResolverExtension( + operator: LogicalPlan, + extensionNames: Seq[String]): AnalysisException = { + new AnalysisException( + errorClass = "AMBIGUOUS_RESOLVER_EXTENSION", + messageParameters = Map( + "operator" -> operator.getClass.getName, + "extensions" -> extensionNames.mkString(", ") + ) + ) + } + + def fixedPointFailedSinglePassSucceeded( + singlePassResult: LogicalPlan, + fixedPointException: Throwable): Throwable = { + new ExtendedAnalysisException( + new AnalysisException( + errorClass = "HYBRID_ANALYZER_EXCEPTION.FIXED_POINT_FAILED_SINGLE_PASS_SUCCEEDED", + messageParameters = Map("singlePassOutput" -> singlePassResult.toString), + cause = Some(fixedPointException) + ), + plan = singlePassResult + ) + } + + def hybridAnalyzerOutputSchemaComparisonMismatch( + fixedPointOutputSchema: StructType, + singlePassOutputSchema: StructType): Throwable = { + + def structToString(struct: StructType) = + struct.fields.map(structFieldToStringWithMetadata(_)).mkString(",") + + def structFieldToStringWithMetadata(sf: StructField) = + s"(${sf.name},${sf.dataType},${sf.nullable},${sf.metadata})" + + new AnalysisException( + errorClass = "HYBRID_ANALYZER_EXCEPTION.OUTPUT_SCHEMA_COMPARISON_MISMATCH", + messageParameters = Map( + "fixedPointOutputSchema" -> structToString(fixedPointOutputSchema), + "singlePassOutputSchema" -> structToString(singlePassOutputSchema) + ) + ) + } + + def hybridAnalyzerLogicalPlanComparisonMismatch( + fixedPointOutput: LogicalPlan, + singlePassOutput: LogicalPlan): Throwable = { + new AnalysisException( + errorClass = "HYBRID_ANALYZER_EXCEPTION.LOGICAL_PLAN_COMPARISON_MISMATCH", + messageParameters = Map( + "fixedPointOutput" -> fixedPointOutput.toString, + "singlePassOutput" -> singlePassOutput.toString + ) + ) + } + + def resolutionValidationError(cause: Throwable, plan: LogicalPlan): Throwable = { + new ExtendedAnalysisException( + new AnalysisException( + errorClass = "INTERNAL_ERROR", + cause = Some(cause), + messageParameters = Map( + "message" -> ("The analysis phase failed with an internal error. Reason: " + + cause.getMessage) + ) + ), + plan = plan + ) + } + def avroNotLoadedSqlFunctionsUnusable(functionName: String): Throwable = { new AnalysisException( errorClass = "AVRO_NOT_LOADED_SQL_FUNCTIONS_UNUSABLE", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e69fa996f6f29..31282d43bbced 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -247,6 +247,76 @@ object SQLConf { .intConf .createWithDefault(100) + val ANALYZER_SINGLE_PASS_RESOLVER_ENABLED = + buildConf("spark.sql.analyzer.singlePassResolver.enabled") + .internal() + .doc( + "When true, use the single-pass Resolver instead of the fixed-point Analyzer. " + + "This is an alternative Analyzer framework, which resolves the parsed logical plan in a " + + "single post-order traversal. It uses ExpressionResolver to resolve expressions and " + + "NameScope to control the visibility of names. In contrast to the current fixed-point " + + "framework, subsequent in-tree traversals are disallowed. Most of the fixed-point " + + "Analyzer code is reused in the form of specific node transformation functions " + + "(AliasResolution.resolve, FunctionResolution.resolveFunction, etc)." + + "This feature is currently under development." + ) + .version("4.0.0") + .booleanConf + .createWithDefault(false) + + val ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER = + buildConf("spark.sql.analyzer.singlePassResolver.dualRunWithLegacy") + .internal() + .doc("When true, run both analyzers to check if single-pass Analyzer correctly produces" + + " the same analyzed plan as the fixed-point Analyzer for the existing set of features" + + " defined in the ResolverGuard") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + + val ANALYZER_SINGLE_PASS_RESOLVER_VALIDATION_ENABLED = + buildConf("spark.sql.analyzer.singlePassResolver.validationEnabled") + .internal() + .doc( + "When true, validate the Resolver output with ResolutionValidator. " + + "The ResolutionValidator validates the resolved logical plan tree in one pass " + + "and asserts the internal contracts. It uses the ExpressionResolutionValidator " + + "internally to validate resolved expression trees in the same manner." + ) + .version("4.0.0") + .booleanConf + .createWithDefault(true) + + val ANALYZER_SINGLE_PASS_TRACK_RESOLVED_NODES_ENABLED = + buildConf("spark.sql.analyzer.singlePassResolver.trackResolvedNodes.enabled") + .internal() + .doc( + "When true, keep track of resolved nodes in order to assert that the single-pass " + + "invariant is never broken. While true, if a resolver attempts to resolve the same node " + + "twice, INTERNAL_ERROR exception is thrown. Used only for testing due to memory impact " + + "of storing each node in a HashSet." + ) + .version("4.0.0") + .booleanConf + .createWithDefault(false) + + val ANALYZER_SINGLE_PASS_RESOLVER_RELATION_BRIDGING_ENABLED = + buildConf("spark.sql.analyzer.singlePassResolver.relationBridging.enabled") + .internal() + .doc( + "When set to true, the single-pass Resolver will reuse the relation metadata that was " + + "previously resolved in fixed-point run. This makes sense only in " + + "ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER mode. In that case HybridAnalyzer " + + "enables the AnalyzerBridgeState and passes it to the single-pass Analyzer after the " + + "fixed-point run is complete. Single-pass Resolver uses this AnalyzerBridgeState to " + + "construct a special RelationMetadataProvider implementation - " + + "BridgedRelationMetadataProvider. This component simply reuses cached relation metadata " + + "and avoids any blocking calls (catalog RPCs or table metadata reads)." + ) + .version("4.0.0") + .booleanConf + .createWithDefault(Utils.isTesting) + val MULTI_COMMUTATIVE_OP_OPT_THRESHOLD = buildConf("spark.sql.analyzer.canonicalization.multiCommutativeOpMemoryOptThreshold") .internal() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolverSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolverSuite.scala new file mode 100644 index 0000000000000..fdab4df379a71 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolverSuite.scala @@ -0,0 +1,94 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, Expression, Literal} +import org.apache.spark.sql.errors.QueryErrorsBase +import org.apache.spark.sql.types.IntegerType + +class LimitExpressionResolverSuite extends SparkFunSuite with QueryErrorsBase { + + private class IdentityExpressionResolver extends TreeNodeResolver[Expression, Expression] { + override def resolve(expression: Expression): Expression = expression + } + + private val expressionResolver = new IdentityExpressionResolver + private val limitExpressionResolver = new LimitExpressionResolver(expressionResolver) + + test("Basic LIMIT without errors") { + val expr = Literal(42, IntegerType) + assert(limitExpressionResolver.resolve(expr) == expr) + } + + test("Unfoldable LIMIT") { + val col = AttributeReference(name = "foo", dataType = IntegerType)() + checkError( + exception = intercept[AnalysisException] { + limitExpressionResolver.resolve(col) + }, + condition = "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + parameters = Map("name" -> "limit", "expr" -> toSQLExpr(col)) + ) + } + + test("LIMIT with non-integer") { + val anyNonInteger = Literal("42") + checkError( + exception = intercept[AnalysisException] { + limitExpressionResolver.resolve(anyNonInteger) + }, + condition = "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", + parameters = Map( + "name" -> "limit", + "expr" -> toSQLExpr(anyNonInteger), + "dataType" -> toSQLType(anyNonInteger.dataType) + ) + ) + } + + test("LIMIT with null") { + val expr = Cast(Literal(null), IntegerType) + checkError( + exception = intercept[AnalysisException] { + limitExpressionResolver.resolve(expr) + }, + condition = "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL", + parameters = Map( + "name" -> "limit", + "expr" -> toSQLExpr(expr) + ) + ) + } + + test("LIMIT with negative integer") { + val expr = Literal(-1, IntegerType) + checkError( + exception = intercept[AnalysisException] { + limitExpressionResolver.resolve(expr) + }, + condition = "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", + parameters = Map( + "name" -> "limit", + "expr" -> toSQLExpr(expr), + "v" -> toSQLValue(-1, IntegerType) + ) + ) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidatorSuite.scala new file mode 100644 index 0000000000000..922e94ea442b3 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidatorSuite.scala @@ -0,0 +1,265 @@ +/* + * 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.catalyst.analysis.resolver + +import scala.collection.immutable +import scala.reflect.runtime.universe.typeOf + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.expressions.{ + Add, + Alias, + AttributeReference, + Cast, + GreaterThan, + Literal, + NamedExpression, + TimeAdd +} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan, Project} +import org.apache.spark.sql.types.{ + BooleanType, + DayTimeIntervalType, + DecimalType, + IntegerType, + StringType, + TimestampType +} + +class ResolutionValidatorSuite extends SparkFunSuite with SQLConfHelper { + private val resolveMethodNamesToIgnore = Seq( + // [[Resolver]] accepts [[UnresolvedInlineTable]], [[ResolvedInlineTable]] and + // [[LocalRelation]], but produces only [[ResolvedInlineTable]] and [[LocalRelation]], so + // we omit one of them here. + // See [[Resolver.resolveInlineTable]] scaladoc for more info. + "resolveResolvedInlineTable" + ) + + private val colInteger = AttributeReference(name = "colInteger", dataType = IntegerType)() + private val colBoolean = AttributeReference(name = "colBoolean", dataType = BooleanType)() + private val colTimestamp = AttributeReference(name = "colTimestamp", dataType = TimestampType)() + + test("All resolve* methods must have validate* counterparts") { + val actualMethodNames = typeOf[ResolutionValidator].decls + .collect { + case decl if decl.isMethod => decl.name.toString + } + .filter(name => { + name.startsWith("validate") + }) + val actualMethodNamesSet = immutable.HashSet(actualMethodNames.toSeq: _*) + + val resolveMethodNamesToIgnoreSet = immutable.HashSet(resolveMethodNamesToIgnore: _*) + + typeOf[Resolver].decls + .collect { + case decl if decl.isMethod => decl.name.toString + } + .filter(name => { + name.startsWith("resolve") && !resolveMethodNamesToIgnoreSet.contains(name) + }) + .map(name => { + "validate" + name.stripPrefix("resolve") + }) + .foreach(name => { + assert(actualMethodNamesSet.contains(name), name) + }) + } + + test("Project") { + validate( + Project( + projectList = Seq(colInteger, colBoolean, colInteger), + child = LocalRelation(output = Seq(colInteger, colBoolean)) + ) + ) + validate( + Project( + projectList = Seq(colInteger), + child = LocalRelation(output = colBoolean) + ), + error = Some("Project list contains nonexisting attribute") + ) + } + + test("Filter") { + validate( + Project( + projectList = Seq(colBoolean), + child = Filter( + condition = colBoolean, + child = LocalRelation(output = colBoolean) + ) + ) + ) + validate( + Project( + projectList = Seq(colInteger), + child = Filter( + condition = colInteger, + child = LocalRelation(output = colInteger) + ) + ), + error = Some("Non-boolean condition") + ) + validate( + Project( + projectList = Seq(colBoolean), + child = Filter( + condition = AttributeReference(name = "colBooleanOther", dataType = BooleanType)(), + child = LocalRelation(output = colBoolean) + ) + ), + error = Some("Condition references nonexisting attribute") + ) + } + + test("Predicate") { + validate( + Project( + projectList = Seq(colInteger), + child = Filter( + condition = GreaterThan(colInteger, colInteger), + child = LocalRelation(output = colInteger) + ) + ) + ) + validate( + Project( + projectList = Seq(colInteger), + child = Filter( + condition = GreaterThan(colInteger, colBoolean), + child = LocalRelation(output = Seq(colInteger, colBoolean)) + ) + ), + error = Some("Input data types mismatch") + ) + } + + test("BinaryExpression") { + validate( + Project( + projectList = Seq( + Alias( + child = Add( + left = Literal(5), + right = Literal(1) + ), + "Add" + )(NamedExpression.newExprId) + ), + child = LocalRelation(output = colInteger) + ) + ) + validate( + Project( + projectList = Seq( + Alias( + child = Add( + left = Literal(5), + right = Literal("1") + ), + "AddWrongInputTypes" + )(NamedExpression.newExprId) + ), + child = LocalRelation(output = colInteger) + ), + error = Some("checkInputDataTypes mismatch") + ) + validate( + Project( + projectList = Seq( + Alias( + child = TimeAdd( + start = Cast( + child = Literal("2024-10-01"), + dataType = TimestampType, + timeZoneId = Option(conf.sessionLocalTimeZone) + ), + interval = Cast( + child = Literal(1), + dataType = DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.DAY), + timeZoneId = Option(conf.sessionLocalTimeZone) + ) + ), + "AddNoTimezone" + )(NamedExpression.newExprId) + ), + child = LocalRelation(output = colInteger) + ), + error = Some("TimezoneId is not set for TimeAdd") + ) + } + + test("TimeZoneAwareExpression") { + validate( + Project( + projectList = Seq( + Alias( + Cast( + child = colInteger, + dataType = DecimalType.USER_DEFAULT, + timeZoneId = Option(conf.sessionLocalTimeZone) + ), + "withTimezone" + )(NamedExpression.newExprId) + ), + child = LocalRelation(output = colInteger) + ) + ) + validate( + Project( + projectList = Seq( + Alias( + Cast( + child = colTimestamp, + dataType = StringType + ), + "withoutTimezone" + )(NamedExpression.newExprId) + ), + child = LocalRelation(output = colTimestamp) + ), + error = Some("TimezoneId is not set") + ) + } + + def validate(plan: LogicalPlan, error: Option[String] = None): Unit = { + def errorWrapper(error: String)(body: => Unit): Unit = { + withClue(error) { + intercept[Throwable] { + body + } + } + } + + def noopWrapper(body: => Unit) = { + body + } + + val wrapper = error + .map(error => { errorWrapper(error) _ }) + .getOrElse { noopWrapper _ } + + val validator = new ResolutionValidator + wrapper { + validator.validatePlan(plan) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolverSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolverSuite.scala new file mode 100644 index 0000000000000..d5c5387d4b763 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolverSuite.scala @@ -0,0 +1,58 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.{ + AttributeReference, + Cast, + Expression, + TimeZoneAwareExpression +} +import org.apache.spark.sql.types.{IntegerType, StringType} + +class TimezoneAwareExpressionResolverSuite extends SparkFunSuite { + + class HardCodedExpressionResolver(resolvedExpression: Expression) + extends TreeNodeResolver[Expression, Expression] { + override def resolve(expression: Expression): Expression = resolvedExpression + } + + private val unresolvedChild = + AttributeReference(name = "unresolvedChild", dataType = StringType)() + private val resolvedChild = AttributeReference(name = "resolvedChild", dataType = IntegerType)() + private val castExpression = Cast(child = unresolvedChild, dataType = IntegerType) + private val expressionResolver = new HardCodedExpressionResolver(resolvedChild) + private val timezoneAwareExpressionResolver = new TimezoneAwareExpressionResolver( + expressionResolver + ) + + test("TimeZoneAwareExpression resolution") { + assert(castExpression.children.head == unresolvedChild) + assert(castExpression.timeZoneId.isEmpty) + assert(castExpression.getTagValue(Cast.USER_SPECIFIED_CAST).isEmpty) + + castExpression.setTagValue(Cast.USER_SPECIFIED_CAST, ()) + val resolvedExpression = + timezoneAwareExpressionResolver.resolve(castExpression).asInstanceOf[TimeZoneAwareExpression] + + assert(resolvedExpression.children.head == resolvedChild) + assert(resolvedExpression.timeZoneId.nonEmpty) + assert(resolvedExpression.getTagValue(Cast.USER_SPECIFIED_CAST).nonEmpty) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolverSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolverSuite.scala new file mode 100644 index 0000000000000..7e0107147c9ac --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/resolver/TypeCoercionResolverSuite.scala @@ -0,0 +1,68 @@ +/* + * 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.catalyst.analysis.resolver + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.AnsiTypeCoercion +import org.apache.spark.sql.catalyst.expressions.{Add, Cast, Expression, Literal} +import org.apache.spark.sql.types.{DoubleType, IntegerType} + +class TypeCoercionResolverSuite extends SparkFunSuite with SQLConfHelper { + + class HardCodedExpressionResolver(resolvedExpression: Expression) + extends TreeNodeResolver[Expression, Expression] { + override def resolve(expression: Expression): Expression = resolvedExpression + } + + private val integerChild = Literal(1, IntegerType) + private val doubleChild = Literal(1.1, DoubleType) + private val castIntegerChild = Cast(child = integerChild, dataType = DoubleType) + private val expressionResolver = new HardCodedExpressionResolver(castIntegerChild) + private val timezoneAwareExpressionResolver = new TimezoneAwareExpressionResolver( + expressionResolver + ) + private val typeCoercionRules = Seq( + AnsiTypeCoercion.ImplicitTypeCasts.transform + ) + private val typeCoercionResolver = + new TypeCoercionResolver(timezoneAwareExpressionResolver, typeCoercionRules) + + test("TypeCoercion resolution - with children reinstantiation") { + val expression = Add(left = doubleChild, right = integerChild) + val resolvedExpression = typeCoercionResolver.resolve(expression).asInstanceOf[Add] + // left child remains the same + assert(resolvedExpression.left == doubleChild) + // right first gets resolved to castIntegerChild. However, after the Cast gets + // re-resolved with timezone, it won't be equal to castIntegerChild because of re-instantiation + assert(resolvedExpression.right.isInstanceOf[Cast]) + val newRightChild = resolvedExpression.right.asInstanceOf[Cast] + assert(newRightChild != castIntegerChild) + assert(newRightChild.timeZoneId.nonEmpty) + // not a user-specified cast + assert(newRightChild.getTagValue(Cast.USER_SPECIFIED_CAST).isEmpty) + } + + test("TypeCoercion resolution - no children reinstantiation") { + val expression = Add(left = doubleChild, right = castIntegerChild) + val resolvedExpression = typeCoercionResolver.resolve(expression).asInstanceOf[Add] + assert(resolvedExpression.left == doubleChild) + // Cast that isn't a product of type coercion resolution won't be re-instantiated with timezone + assert(resolvedExpression.right == castIntegerChild) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala new file mode 100644 index 0000000000000..3a2a3207a01f9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala @@ -0,0 +1,67 @@ +/* + * 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 + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.resolver.{ + ExplicitlyUnsupportedResolverFeature, + ResolverExtension +} +import org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 +import org.apache.spark.sql.execution.streaming.StreamingRelation + +/** + * The [[DataSourceResolver]] is a [[Resolver]] extension that resolves nodes defined in the + * [[datasources]] package. We have it as an extension to avoid cyclic dependencies between + * [[resolver]] and [[datasources]] packages. + */ +class DataSourceResolver(sparkSession: SparkSession) extends ResolverExtension { + private val findDataSourceTable = new FindDataSourceTable(sparkSession) + + /** + * Resolve [[UnresolvedCatalogRelation]]: + * - Reuse [[FindDataSourceTable]] code to resolve [[UnresolvedCatalogRelation]] + * - Create a new instance of [[LogicalRelation]] to regenerate the expression IDs + * - Explicitly disallow [[StreamingRelation]] and [[StreamingRelationV2]] for now + * - [[FileResolver]], which is a [[ResolverExtension]], introduces a new [[LogicalPlan]] node + * which resolution has to be handled here (further resolution of it doesn't need any specific + * resolution except adding it's attributes to the scope). + */ + override def resolveOperator: PartialFunction[LogicalPlan, LogicalPlan] = { + case unresolvedCatalogRelation: UnresolvedCatalogRelation => + val result = findDataSourceTable.resolveUnresolvedCatalogRelation(unresolvedCatalogRelation) + result match { + case logicalRelation: LogicalRelation => + logicalRelation.newInstance() + case streamingRelation: StreamingRelation => + throw new ExplicitlyUnsupportedResolverFeature( + s"unsupported operator: ${streamingRelation.getClass.getName}" + ) + case streamingRelationV2: StreamingRelationV2 => + throw new ExplicitlyUnsupportedResolverFeature( + s"unsupported operator: ${streamingRelationV2.getClass.getName}" + ) + case other => + other + } + case logicalRelation: LogicalRelation => + logicalRelation.newInstance() + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileResolver.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileResolver.scala new file mode 100644 index 0000000000000..44102da752c2e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileResolver.scala @@ -0,0 +1,55 @@ +/* + * 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 + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension +import org.apache.spark.sql.catalyst.plans.logical.{AnalysisHelper, LogicalPlan} + +/** + * The [[FileResolver]] is a [[MetadataResolver]] extension that resolves [[UnresolvedRelation]] + * which is created out of file. It reuses the code from [[ResolveSQLOnFile]] to resolve it + * properly. + * + * We have it as an extension to avoid cyclic dependencies between [[resolver]] and [[datasources]] + * packages. + */ +class FileResolver(sparkSession: SparkSession) extends ResolverExtension { + private val resolveSQLOnFile = new ResolveSQLOnFile(sparkSession) + + /** + * [[ResolveSQLOnFile]] code that is reused to resolve [[UnresolvedRelation]] has + * [[ExpressionEncoder.resolveAndBind]] on its path which introduces another call to + * the analyzer which is acceptable as it is called on the leaf node of the plan. That's why we + * have to allow invoking transforms in the single-pass analyzer. + */ + object UnresolvedRelationResolution { + def unapply(operator: LogicalPlan): Option[LogicalPlan] = + AnalysisHelper.allowInvokingTransformsInAnalyzer { + resolveSQLOnFile.UnresolvedRelationResolution.unapply(operator) + } + } + + /** + * Reuse [[ResolveSQLOnFile]] code to resolve [[UnresolvedRelation]] made out of file. + */ + override def resolveOperator: PartialFunction[LogicalPlan, LogicalPlan] = { + case UnresolvedRelationResolution(resolvedRelation) => + resolvedRelation + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index f1033c5cded18..c78f9702557cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -53,7 +53,7 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { object UnresolvedRelationResolution { def unapply(plan: LogicalPlan): Option[LogicalPlan] = { - plan match { + val result = plan match { case u: UnresolvedRelation if maybeSQLFile(u) => try { val ds = resolveDataSource(u) @@ -75,6 +75,17 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { case _ => None } + result.foreach(resolvedRelation => plan match { + case unresolvedRelation: UnresolvedRelation => + // We put the resolved relation into the [[AnalyzerBridgeState]] for + // it to be later reused by the single-pass [[Resolver]] to avoid resolving the + // relation metadata twice. + AnalysisContext.get.getSinglePassResolverBridgeState.map { bridgeState => + bridgeState.relationsWithResolvedMetadata.put(unresolvedRelation, resolvedRelation) + } + case _ => + }) + result } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 59a873ef982fe..bfd5915529118 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -20,6 +20,7 @@ import org.apache.spark.annotation.Unstable import org.apache.spark.sql.{ExperimentalMethods, SparkSession, UDFRegistration, _} import org.apache.spark.sql.artifact.ArtifactManager import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, FunctionRegistry, InvokeProcedures, ReplaceCharWithVarchar, ResolveSessionCatalog, ResolveTranspose, TableFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{FunctionExpressionBuilder, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.optimizer.Optimizer @@ -199,6 +200,15 @@ abstract class BaseSessionStateBuilder( protected def analyzer: Analyzer = new Analyzer(catalogManager) { override val hintResolutionRules: Seq[Rule[LogicalPlan]] = customHintResolutionRules + + override val singlePassResolverExtensions: Seq[ResolverExtension] = Seq( + new DataSourceResolver(session) + ) + + override val singlePassMetadataResolverExtensions: Seq[ResolverExtension] = Seq( + new FileResolver(session) + ) + override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ExplicitlyUnsupportedResolverFeatureSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ExplicitlyUnsupportedResolverFeatureSuite.scala new file mode 100644 index 0000000000000..7fd7d570ecfc1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ExplicitlyUnsupportedResolverFeatureSuite.scala @@ -0,0 +1,91 @@ +/* + * 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.analysis.resolver + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.analysis.resolver.Resolver +import org.apache.spark.sql.test.SharedSparkSession + +class ExplicitlyUnsupportedResolverFeatureSuite extends QueryTest with SharedSparkSession { + test("Unsupported table types") { + withTable("csv_table") { + spark.sql("CREATE TABLE csv_table (col1 INT) USING CSV;").collect() + checkResolution("SELECT * FROM csv_table;", shouldPass = true) + } + withTable("json_table") { + spark.sql("CREATE TABLE json_table (col1 INT) USING JSON;").collect() + checkResolution("SELECT * FROM json_table;", shouldPass = true) + } + withTable("parquet_table") { + spark.sql("CREATE TABLE parquet_table (col1 INT) USING PARQUET;").collect() + checkResolution("SELECT * FROM parquet_table;", shouldPass = true) + } + withTable("orc_table") { + spark.sql("CREATE TABLE orc_table (col1 INT) USING ORC;").collect() + checkResolution("SELECT * FROM orc_table;", shouldPass = true) + } + } + + test("Unsupported view types") { + withTable("src_table") { + spark.sql("CREATE TABLE src_table (col1 INT) USING PARQUET;").collect() + + withView("temporary_view") { + spark.sql("CREATE TEMPORARY VIEW temporary_view AS SELECT * FROM src_table;").collect() + checkResolution("SELECT * FROM temporary_view;") + } + + withView("persistent_view") { + spark.sql("CREATE VIEW persistent_view AS SELECT * FROM src_table;").collect() + checkResolution("SELECT * FROM persistent_view;") + } + } + } + + test("Unsupported char type padding") { + withTable("char_type_padding") { + spark.sql(s"CREATE TABLE t1 (c1 CHAR(3), c2 STRING) USING PARQUET") + checkResolution("SELECT c1 = '12', c1 = '12 ', c1 = '12 ' FROM t1 WHERE c2 = '12'") + } + } + + test("Unsupported lateral column alias") { + checkResolution("SELECT 1 AS a, a AS b") + checkResolution("SELECT sum(1), `sum(1)` + 1 AS a") + } + + private def checkResolution(sqlText: String, shouldPass: Boolean = false): Unit = { + def noopWrapper(body: => Unit) = body + + val wrapper = if (shouldPass) { + noopWrapper _ + } else { + intercept[Throwable] _ + } + + val unresolvedPlan = spark.sql(sqlText).queryExecution.logical + + val resolver = new Resolver( + spark.sessionState.catalogManager, + extensions = spark.sessionState.analyzer.singlePassResolverExtensions + ) + wrapper { + resolver.lookupMetadataAndResolve(unresolvedPlan) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/HybridAnalyzerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/HybridAnalyzerSuite.scala new file mode 100644 index 0000000000000..587725093f0e5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/HybridAnalyzerSuite.scala @@ -0,0 +1,404 @@ +/* + * 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.analysis.resolver + +import org.scalactic.source.Position +import org.scalatest.Tag + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.{ + AliasIdentifier, + ExtendedAnalysisException, + QueryPlanningTracker +} +import org.apache.spark.sql.catalyst.analysis.{ + AnalysisContext, + Analyzer, + UnresolvedAttribute, + UnresolvedStar +} +import org.apache.spark.sql.catalyst.analysis.resolver.{ + AnalyzerBridgeState, + ExplicitlyUnsupportedResolverFeature, + HybridAnalyzer, + Resolver, + ResolverGuard +} +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.plans.logical.{ + LocalRelation, + LogicalPlan, + Project, + SubqueryAlias +} +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, MetadataBuilder} + +class HybridAnalyzerSuite extends QueryTest with SharedSparkSession { + private val col1Integer = AttributeReference("col1", IntegerType)() + private val col2Integer = AttributeReference("col2", IntegerType)() + private val col2IntegerWithMetadata = AttributeReference( + "col2", + IntegerType, + metadata = (new MetadataBuilder).putString("comment", "this is an integer").build() + )() + + private def validateSinglePassResolverBridgeState(bridgeRelations: Boolean): Unit = { + assert(bridgeRelations == AnalysisContext.get.getSinglePassResolverBridgeState.isDefined) + } + + private class BrokenResolver(ex: Throwable, bridgeRelations: Boolean) + extends Resolver(spark.sessionState.catalogManager) { + override def lookupMetadataAndResolve( + plan: LogicalPlan, + analyzerBridgeState: Option[AnalyzerBridgeState] = None): LogicalPlan = { + validateSinglePassResolverBridgeState(bridgeRelations) + throw ex + } + } + + private class ValidatingResolver(bridgeRelations: Boolean) + extends Resolver(spark.sessionState.catalogManager) { + override def lookupMetadataAndResolve( + plan: LogicalPlan, + analyzerBridgeState: Option[AnalyzerBridgeState] = None): LogicalPlan = { + validateSinglePassResolverBridgeState(bridgeRelations) + super.lookupMetadataAndResolve(plan, analyzerBridgeState) + } + } + + private class HardCodedResolver(resolvedPlan: LogicalPlan, bridgeRelations: Boolean) + extends Resolver(spark.sessionState.catalogManager) { + override def lookupMetadataAndResolve( + plan: LogicalPlan, + analyzerBridgeState: Option[AnalyzerBridgeState] = None): LogicalPlan = { + validateSinglePassResolverBridgeState(bridgeRelations) + resolvedPlan + } + } + + private class ValidatingAnalyzer(bridgeRelations: Boolean) + extends Analyzer(spark.sessionState.catalogManager) { + override def executeAndTrack(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { + validateSinglePassResolverBridgeState(bridgeRelations) + super.executeAndTrack(plan, tracker) + } + } + + private class BrokenAnalyzer(ex: Throwable, bridgeRelations: Boolean) + extends Analyzer(spark.sessionState.catalogManager) { + override def executeAndTrack(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { + validateSinglePassResolverBridgeState(bridgeRelations) + throw ex + } + } + + private class CustomAnalyzer(customCode: () => Unit, bridgeRelations: Boolean) + extends Analyzer(spark.sessionState.catalogManager) { + override def executeAndTrack(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { + validateSinglePassResolverBridgeState(bridgeRelations) + customCode() + super.executeAndTrack(plan, tracker) + } + } + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)( + implicit pos: Position): Unit = { + super.test(testName) { + withSQLConf( + SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER.key -> "true" + ) { + testFun + } + } + } + + test("Both fixed-point and single-pass analyzers pass") { + val plan: LogicalPlan = { + Project( + Seq(UnresolvedStar(None)), + LocalRelation(col1Integer) + ) + } + val resolvedPlan = + Project( + Seq(col1Integer), + LocalRelation(Seq(col1Integer)) + ) + assert( + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = true), + new ResolverGuard(spark.sessionState.catalogManager), + new ValidatingResolver(bridgeRelations = true) + ).apply(plan, null) + == + resolvedPlan + ) + } + + test("Fixed-point analyzer passes, single-pass analyzer fails") { + val plan: LogicalPlan = + Project(Seq(UnresolvedStar(None)), LocalRelation(col1Integer)) + checkError( + exception = intercept[AnalysisException]( + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = true), + new ResolverGuard(spark.sessionState.catalogManager), + new BrokenResolver( + QueryCompilationErrors.unsupportedSinglePassAnalyzerFeature("test"), + bridgeRelations = true + ) + ).apply(plan, null) + ), + condition = "UNSUPPORTED_SINGLE_PASS_ANALYZER_FEATURE", + parameters = Map("feature" -> "test") + ) + } + + test("Fixed-point analyzer fails, single-pass analyzer passes") { + val plan: LogicalPlan = + Project( + Seq(UnresolvedAttribute("nonexistent_col")), + LocalRelation(col1Integer) + ) + val resolvedPlan = + Project( + Seq(col1Integer), + LocalRelation(Seq(col1Integer)) + ) + checkError( + exception = intercept[AnalysisException]( + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = true), + new ResolverGuard(spark.sessionState.catalogManager), + new HardCodedResolver(resolvedPlan, bridgeRelations = true) + ).apply(plan, null) + ), + condition = "HYBRID_ANALYZER_EXCEPTION.FIXED_POINT_FAILED_SINGLE_PASS_SUCCEEDED", + parameters = Map("singlePassOutput" -> resolvedPlan.toString) + ) + } + + test("Both fixed-point and single-pass analyzers fail") { + val plan: LogicalPlan = + Project( + Seq(UnresolvedAttribute("nonexistent_col")), + LocalRelation(col1Integer) + ) + checkError( + exception = intercept[ExtendedAnalysisException]( + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = true), + new ResolverGuard(spark.sessionState.catalogManager), + new ValidatingResolver(bridgeRelations = true) + ).apply(plan, null) + ), + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map( + "objectName" -> "`nonexistent_col`", + "proposal" -> "`col1`" + ) + ) + } + + test("Plan mismatch") { + val plan: LogicalPlan = + Project( + Seq(UnresolvedAttribute("col1")), + SubqueryAlias( + AliasIdentifier("t", Seq.empty), + LocalRelation(Seq(col1Integer)) + ) + ) + val resolvedPlan = + Project( + Seq(col1Integer), + LocalRelation(Seq(col1Integer)) + ) + val expectedResolvedPlan = + Project( + Seq(col1Integer), + SubqueryAlias( + AliasIdentifier("t", Seq.empty), + LocalRelation(Seq(col1Integer)) + ) + ) + checkError( + exception = intercept[AnalysisException]( + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = true), + new ResolverGuard(spark.sessionState.catalogManager), + new HardCodedResolver(resolvedPlan, bridgeRelations = true) + ).apply(plan, null) + ), + condition = "HYBRID_ANALYZER_EXCEPTION.LOGICAL_PLAN_COMPARISON_MISMATCH", + parameters = Map( + "singlePassOutput" -> resolvedPlan.toString, + "fixedPointOutput" -> expectedResolvedPlan.toString + ) + ) + } + + test("Missing metadata in output schema") { + val plan: LogicalPlan = + Project( + Seq(UnresolvedAttribute("col2")), + LocalRelation(col2IntegerWithMetadata) + ) + val resolvedPlan = + Project( + Seq(col2Integer), + LocalRelation(Seq(col2Integer)) + ) + checkError( + exception = intercept[AnalysisException]( + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = true), + new ResolverGuard(spark.sessionState.catalogManager), + new HardCodedResolver(resolvedPlan, bridgeRelations = true) + ).apply(plan, null) + ), + condition = "HYBRID_ANALYZER_EXCEPTION.OUTPUT_SCHEMA_COMPARISON_MISMATCH", + parameters = Map( + "singlePassOutputSchema" -> "(col2,IntegerType,true,{})", + "fixedPointOutputSchema" -> "(col2,IntegerType,true,{\"comment\":\"this is an integer\"})" + ) + ) + } + + test("Explicitly unsupported resolver feature") { + val plan: LogicalPlan = { + Project( + Seq(UnresolvedStar(None)), + LocalRelation(col1Integer) + ) + } + checkAnswer( + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = true), + new ResolverGuard(spark.sessionState.catalogManager), + new BrokenResolver( + new ExplicitlyUnsupportedResolverFeature("FAILURE"), + bridgeRelations = true + ) + ).apply(plan, null), + plan + ) + } + + test("Fixed-point only run") { + val plan = Project( + Seq(UnresolvedStar(None)), + LocalRelation(col1Integer) + ) + val resolvedPlan = Project( + Seq(col1Integer), + LocalRelation(Seq(col1Integer)) + ) + assert( + withSQLConf( + SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER.key -> "false" + ) { + new HybridAnalyzer( + new ValidatingAnalyzer(bridgeRelations = false), + new ResolverGuard(spark.sessionState.catalogManager), + new BrokenResolver( + new Exception("Single-pass resolver should not be invoked"), + bridgeRelations = false + ) + ).apply(plan, null) + } + == + resolvedPlan + ) + } + + test("Single-pass only run") { + val plan = Project( + Seq(UnresolvedStar(None)), + LocalRelation(col1Integer) + ) + val resolvedPlan = Project( + Seq(col1Integer), + LocalRelation(Seq(col1Integer)) + ) + assert( + withSQLConf( + SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER.key -> "false", + SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_ENABLED.key -> "true" + ) { + new HybridAnalyzer( + new BrokenAnalyzer( + new Exception("Fixed-point analyzer should not be invoked"), + bridgeRelations = false + ), + new ResolverGuard(spark.sessionState.catalogManager), + new ValidatingResolver(bridgeRelations = false) + ).apply(plan, null) + } + == + resolvedPlan + ) + } + + test("Nested invocations") { + val plan = Project( + Seq(UnresolvedStar(None)), + LocalRelation(col1Integer) + ) + val resolvedPlan = Project( + Seq(col1Integer), + LocalRelation(Seq(col1Integer)) + ) + + val nestedAnalysis = () => { + assert( + withSQLConf( + SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER.key -> "false", + SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_ENABLED.key -> "true" + ) { + new HybridAnalyzer( + new BrokenAnalyzer( + new Exception("Fixed-point analyzer should not be invoked"), + bridgeRelations = false + ), + new ResolverGuard(spark.sessionState.catalogManager), + new ValidatingResolver(bridgeRelations = false) + ).apply(plan, null) + } + == + resolvedPlan + ) + } + + assert( + new HybridAnalyzer( + new CustomAnalyzer( + customCode = () => { nestedAnalysis() }, + bridgeRelations = true + ), + new ResolverGuard(spark.sessionState.catalogManager), + new ValidatingResolver(bridgeRelations = true) + ).apply(plan, null) + == + resolvedPlan + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/MetadataResolverSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/MetadataResolverSuite.scala new file mode 100644 index 0000000000000..5fd21d7543b33 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/MetadataResolverSuite.scala @@ -0,0 +1,277 @@ +/* + * 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.analysis.resolver + +import scala.collection.mutable + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.analysis.resolver.{ + AnalyzerBridgeState, + BridgedRelationMetadataProvider, + MetadataResolver, + RelationId, + Resolver +} +import org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation +import org.apache.spark.sql.catalyst.expressions.{Expression, PlanExpression} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.execution.datasources.{FileResolver, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class MetadataResolverSuite extends QueryTest with SharedSparkSession with SQLTestUtils { + private val keyValueTableSchema = StructType( + Seq( + StructField("key", IntegerType, true), + StructField("value", StringType, true) + ) + ) + private val fileTableSchema = StructType( + Seq( + StructField("id", LongType, true) + ) + ) + + test("Single CSV relation") { + withTable("src_csv") { + spark.sql("CREATE TABLE src_csv (key INT, value STRING) USING CSV;").collect() + + checkResolveUnresolvedCatalogRelation( + sqlText = "SELECT * FROM src_csv", + expectedTableData = Seq(createTableData("src_csv")) + ) + } + } + + test("Single ORC relation") { + withTable("src_orc") { + spark.sql("CREATE TABLE src_orc (key INT, value STRING) USING ORC;").collect() + + checkResolveUnresolvedCatalogRelation( + sqlText = "SELECT * FROM src_orc", + expectedTableData = Seq(createTableData("src_orc")) + ) + } + } + + test("Relation inside an EXISTS subquery") { + withTable("src") { + spark.sql("CREATE TABLE src (key INT, value STRING) USING PARQUET;").collect() + + checkResolveUnresolvedCatalogRelation( + sqlText = "SELECT * FROM VALUES (1) WHERE EXISTS (SELECT col1 FROM src)", + expectedTableData = Seq(createTableData("src")) + ) + } + } + + test("Relation inside an IN subquery") { + withTable("src") { + spark.sql("CREATE TABLE src (key INT, value STRING) USING PARQUET;").collect() + + checkResolveUnresolvedCatalogRelation( + sqlText = "SELECT * FROM VALUES (1) WHERE col1 IN (SELECT col1 FROM src)", + expectedTableData = Seq(createTableData("src")) + ) + } + } + + test("Relation inside a nested subquery expression") { + withTable("src") { + spark.sql("CREATE TABLE src (key INT, value STRING) USING PARQUET;").collect() + + checkResolveUnresolvedCatalogRelation( + sqlText = """ + SELECT + col1 + ( + SELECT 35 * ( + SELECT key FROM src LIMIT 1 + ) * col1 FROM VALUES (2) + ) + FROM + VALUES (1) + """, + expectedTableData = Seq(createTableData("src")) + ) + } + } + + test("Relation from a file") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.json(f.getCanonicalPath) + checkResolveLogicalRelation( + sqlText = s"select id from json.`${f.getCanonicalPath}`", + expectedTableData = Seq( + RelationId( + multipartIdentifier = Seq("spark_catalog", "json", s"${f.getCanonicalPath}") + ) -> TestTableData( + name = s"file:${f.getCanonicalPath}", + schema = fileTableSchema + ) + ) + ) + }) + } + + test("Relation bridged from legacy Analyzer") { + withTable("src") { + spark.sql("CREATE TABLE src (key INT, value STRING) USING PARQUET;").collect() + + val analyzerBridgeState = new AnalyzerBridgeState + analyzerBridgeState.relationsWithResolvedMetadata.put( + UnresolvedRelation(Seq("src")), + createUnresolvedCatalogRelation("src") + ) + + checkResolveUnresolvedCatalogRelation( + sqlText = "SELECT * FROM src", + expectedTableData = Seq(createTableData("src")), + analyzerBridgeState = Some(analyzerBridgeState) + ) + } + } + + test("Relation not bridged from legacy Analyzer") { + withTable("src") { + spark.sql("CREATE TABLE src (key INT, value STRING) USING PARQUET;").collect() + + checkResolveUnresolvedCatalogRelation( + sqlText = "SELECT * FROM src", + expectedTableData = Seq.empty, + analyzerBridgeState = Some(new AnalyzerBridgeState) + ) + } + } + + private def checkResolveUnresolvedCatalogRelation( + sqlText: String, + expectedTableData: Seq[(RelationId, TestTableData)], + analyzerBridgeState: Option[AnalyzerBridgeState] = None): Unit = { + checkResolve( + sqlText, + expectedTableData, + relation => + relation.asInstanceOf[UnresolvedCatalogRelation].tableMeta.identifier.unquotedString, + relation => relation.asInstanceOf[UnresolvedCatalogRelation].tableMeta.schema, + analyzerBridgeState + ) + } + + private def checkResolveLogicalRelation( + sqlText: String, + expectedTableData: Seq[(RelationId, TestTableData)], + analyzerBridgeState: Option[AnalyzerBridgeState] = None): Unit = { + checkResolve( + sqlText, + expectedTableData, + relation => + relation + .asInstanceOf[LogicalRelation] + .relation + .asInstanceOf[HadoopFsRelation] + .location + .rootPaths + .mkString(","), + relation => relation.asInstanceOf[LogicalRelation].relation.schema, + analyzerBridgeState + ) + } + + private def checkResolve( + sqlText: String, + expectedTableData: Seq[(RelationId, TestTableData)], + getTableName: LogicalPlan => String, + getTableSchema: LogicalPlan => StructType, + analyzerBridgeState: Option[AnalyzerBridgeState]): Unit = { + val unresolvedPlan = spark.sql(sqlText).queryExecution.logical + + val metadataResolver = analyzerBridgeState match { + case Some(analyzerBridgeState) => + new BridgedRelationMetadataProvider( + spark.sessionState.catalogManager, + Resolver.createRelationResolution(spark.sessionState.catalogManager), + analyzerBridgeState + ) + case None => + val metadataResolver = new MetadataResolver( + spark.sessionState.catalogManager, + Resolver.createRelationResolution(spark.sessionState.catalogManager), + Seq(new FileResolver(spark)) + ) + metadataResolver.resolve(unresolvedPlan) + metadataResolver + } + + val actualTableData = new mutable.HashMap[RelationId, TestTableData] + + def findUnresolvedRelations(unresolvedPlan: LogicalPlan): Unit = unresolvedPlan.foreach { + case unresolvedRelation: UnresolvedRelation => + metadataResolver.getRelationWithResolvedMetadata(unresolvedRelation) match { + case Some(plan) => + val relationId = metadataResolver.relationIdFromUnresolvedRelation(unresolvedRelation) + val relation = plan match { + case SubqueryAlias(_, relation) => relation + case relation => relation + } + + actualTableData(relationId) = + TestTableData(getTableName(relation), getTableSchema(relation)) + case None => + } + case unresolvedPlan => + def traverseExpressions(expression: Expression): Unit = expression match { + case planExpression: PlanExpression[_] => + planExpression.plan match { + case plan: LogicalPlan => + findUnresolvedRelations(plan) + case _ => + } + case expression => + expression.children.foreach(traverseExpressions) + } + + unresolvedPlan.expressions.foreach(traverseExpressions) + } + + findUnresolvedRelations(unresolvedPlan) + + assert(actualTableData == mutable.HashMap(expectedTableData: _*)) + } + + private def createTableData(name: String) = + RelationId( + multipartIdentifier = Seq("spark_catalog", "default", name) + ) -> TestTableData( + name = s"spark_catalog.default.$name", + schema = keyValueTableSchema + ) + + private def createUnresolvedCatalogRelation(name: String) = SubqueryAlias( + AliasIdentifier(name), + UnresolvedCatalogRelation( + spark.sessionState.catalog.getTableMetadata(TableIdentifier(name)), + CaseInsensitiveStringMap.empty + ) + ) + + private case class TestTableData(name: String, schema: StructType) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/NameScopeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/NameScopeSuite.scala new file mode 100644 index 0000000000000..ec744af89f000 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/NameScopeSuite.scala @@ -0,0 +1,659 @@ +/* + * 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.analysis.resolver + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.UnresolvedStar +import org.apache.spark.sql.catalyst.analysis.resolver.{NameScope, NameScopeStack, NameTarget} +import org.apache.spark.sql.catalyst.expressions.{ + AttributeReference, + GetArrayItem, + GetArrayStructFields, + GetMapValue, + GetStructField, + Literal +} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.types.{ + ArrayType, + BooleanType, + IntegerType, + MapType, + StringType, + StructField, + StructType +} + +class NameScopeSuite extends PlanTest with SQLConfHelper { + private val col1Integer = AttributeReference(name = "col1", dataType = IntegerType)() + private val col1IntegerOther = AttributeReference(name = "col1", dataType = IntegerType)() + private val col2Integer = AttributeReference(name = "col2", dataType = IntegerType)() + private val col3Boolean = AttributeReference(name = "col3", dataType = BooleanType)() + private val col4String = AttributeReference(name = "col4", dataType = StringType)() + private val col5String = AttributeReference(name = "col5", dataType = StringType)() + private val col6IntegerWithQualifier = AttributeReference( + name = "col6", + dataType = IntegerType + )(qualifier = Seq("catalog", "database", "table")) + private val col6IntegerOtherWithQualifier = AttributeReference( + name = "col6", + dataType = IntegerType + )(qualifier = Seq("catalog", "database", "table")) + private val col7StringWithQualifier = AttributeReference( + name = "col7", + dataType = IntegerType + )(qualifier = Seq("catalog", "database", "table")) + private val col8Struct = AttributeReference( + name = "col8", + dataType = StructType(Seq(StructField("field", IntegerType, true))) + )() + private val col9NestedStruct = AttributeReference( + name = "col9", + dataType = StructType( + Seq( + StructField( + "field", + StructType( + Seq( + StructField("subfield", IntegerType) + ) + ) + ) + ) + ) + )() + private val col10Map = AttributeReference( + name = "col10", + dataType = MapType(StringType, IntegerType) + )() + private val col11MapWithStruct = AttributeReference( + name = "col11", + dataType = MapType( + StringType, + StructType(Seq(StructField("field", StringType))) + ) + )() + private val col12Array = AttributeReference( + name = "col12", + dataType = ArrayType(IntegerType) + )() + private val col13ArrayWithStruct = AttributeReference( + name = "col13", + dataType = ArrayType( + StructType(Seq(StructField("field", StringType))) + ) + )() + + test("Empty scope") { + val nameScope = new NameScope + + assert(nameScope.getAllAttributes.isEmpty) + + assert(nameScope.matchMultipartName(Seq("col1")) == NameTarget(candidates = Seq.empty)) + } + + test("Single unnamed plan") { + val nameScope = new NameScope + + nameScope += Seq(col1Integer, col2Integer, col3Boolean) + + assert(nameScope.getAllAttributes == Seq(col1Integer, col2Integer, col3Boolean)) + + assert( + nameScope.matchMultipartName(Seq("col1")) == NameTarget( + candidates = Seq(col1Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col2")) == NameTarget( + candidates = Seq(col2Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col3")) == NameTarget( + candidates = Seq(col3Boolean), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col4")) == NameTarget( + candidates = Seq.empty, + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + } + + test("Several unnamed plans") { + val nameScope = new NameScope + + nameScope += Seq(col1Integer) + nameScope += Seq(col2Integer, col3Boolean) + nameScope += Seq(col4String) + + assert(nameScope.getAllAttributes == Seq(col1Integer, col2Integer, col3Boolean, col4String)) + + assert( + nameScope.matchMultipartName(Seq("col1")) == NameTarget( + candidates = Seq(col1Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col2")) == NameTarget( + candidates = Seq(col2Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col3")) == NameTarget( + candidates = Seq(col3Boolean), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col4")) == NameTarget( + candidates = Seq(col4String), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col5")) == NameTarget( + candidates = Seq.empty, + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + } + + test("Single named plan") { + val nameScope = new NameScope + + nameScope("table1") = Seq(col1Integer, col2Integer, col3Boolean) + + assert(nameScope.getAllAttributes == Seq(col1Integer, col2Integer, col3Boolean)) + + assert( + nameScope.matchMultipartName(Seq("col1")) == NameTarget( + candidates = Seq(col1Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col2")) == NameTarget( + candidates = Seq(col2Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col3")) == NameTarget( + candidates = Seq(col3Boolean), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col4")) == NameTarget( + candidates = Seq.empty, + allAttributes = Seq(col1Integer, col2Integer, col3Boolean) + ) + ) + } + + test("Several named plans") { + val nameScope = new NameScope + + nameScope("table1") = Seq(col1Integer) + nameScope("table2") = Seq(col2Integer, col3Boolean) + nameScope("table2") = Seq(col4String) + nameScope("table3") = Seq(col5String) + + assert( + nameScope.getAllAttributes == Seq( + col1Integer, + col2Integer, + col3Boolean, + col4String, + col5String + ) + ) + + assert( + nameScope.matchMultipartName(Seq("col1")) == NameTarget( + candidates = Seq(col1Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String, col5String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col2")) == NameTarget( + candidates = Seq(col2Integer), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String, col5String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col3")) == NameTarget( + candidates = Seq(col3Boolean), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String, col5String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col4")) == NameTarget( + candidates = Seq(col4String), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String, col5String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col5")) == NameTarget( + candidates = Seq(col5String), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String, col5String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col6")) == NameTarget( + candidates = Seq.empty, + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String, col5String) + ) + ) + } + + test("Named and unnamed plans with case insensitive comparison") { + val col1Integer = AttributeReference(name = "Col1", dataType = IntegerType)() + val col2Integer = AttributeReference(name = "col2", dataType = IntegerType)() + val col3Boolean = AttributeReference(name = "coL3", dataType = BooleanType)() + val col4String = AttributeReference(name = "Col4", dataType = StringType)() + + val nameScope = new NameScope + + nameScope("TaBle1") = Seq(col1Integer) + nameScope("table2") = Seq(col2Integer, col3Boolean) + nameScope += Seq(col4String) + + assert(nameScope.getAllAttributes == Seq(col1Integer, col2Integer, col3Boolean, col4String)) + + assert( + nameScope.matchMultipartName(Seq("cOL1")) == NameTarget( + candidates = Seq(col1Integer.withName("cOL1")), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("CoL2")) == NameTarget( + candidates = Seq(col2Integer.withName("CoL2")), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col3")) == NameTarget( + candidates = Seq(col3Boolean.withName("col3")), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("COL4")) == NameTarget( + candidates = Seq(col4String.withName("COL4")), + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + assert( + nameScope.matchMultipartName(Seq("col5")) == NameTarget( + candidates = Seq.empty, + allAttributes = Seq(col1Integer, col2Integer, col3Boolean, col4String) + ) + ) + } + + test("Duplicate attribute names from one plan") { + val nameScope = new NameScope + + nameScope("table1") = Seq(col1Integer, col1Integer) + nameScope("table1") = Seq(col1IntegerOther) + + assert(nameScope.getAllAttributes == Seq(col1Integer, col1Integer, col1IntegerOther)) + + nameScope.matchMultipartName(Seq("col1")) == NameTarget( + candidates = Seq(col1Integer, col1IntegerOther) + ) + } + + test("Duplicate attribute names from several plans") { + val nameScope = new NameScope + + nameScope("table1") = Seq(col1Integer, col1IntegerOther) + nameScope("table2") = Seq(col1Integer, col1IntegerOther) + + assert( + nameScope.getAllAttributes == Seq( + col1Integer, + col1IntegerOther, + col1Integer, + col1IntegerOther + ) + ) + + nameScope.matchMultipartName(Seq("col1")) == NameTarget( + candidates = Seq( + col1Integer, + col1IntegerOther, + col1Integer, + col1IntegerOther + ) + ) + } + + test("Expand star") { + val nameScope = new NameScope + + nameScope("table") = + Seq(col6IntegerWithQualifier, col6IntegerOtherWithQualifier, col7StringWithQualifier) + + Seq(Seq("table"), Seq("database", "table"), Seq("catalog", "database", "table")) + .foreach(tableQualifier => { + assert( + nameScope.expandStar(UnresolvedStar(Some(tableQualifier))) + == Seq(col6IntegerWithQualifier, col6IntegerOtherWithQualifier, col7StringWithQualifier) + ) + }) + + checkError( + exception = intercept[AnalysisException]( + nameScope.expandStar(UnresolvedStar(Some(Seq("database", "table_fail")))) + ), + condition = "CANNOT_RESOLVE_STAR_EXPAND", + parameters = Map( + "targetString" -> "`database`.`table_fail`", + "columns" -> "`col6`, `col6`, `col7`" + ) + ) + + nameScope("table2") = Seq(col6IntegerWithQualifier) + + checkError( + exception = intercept[AnalysisException]( + nameScope.expandStar(UnresolvedStar(Some(Seq("table2")))) + ), + condition = "INVALID_USAGE_OF_STAR_OR_REGEX", + parameters = Map( + "elem" -> "'*'", + "prettyName" -> "query" + ) + ) + } + + test("Multipart attribute names") { + val nameScope = new NameScope + + nameScope("table") = Seq(col6IntegerWithQualifier) + + for (multipartIdentifier <- Seq( + Seq("catalog", "database", "table", "col6"), + Seq("database", "table", "col6"), + Seq("table", "col6") + )) { + assert( + nameScope.matchMultipartName(multipartIdentifier) == NameTarget( + candidates = Seq( + col6IntegerWithQualifier + ), + allAttributes = Seq(col6IntegerWithQualifier) + ) + ) + } + + for (multipartIdentifier <- Seq( + Seq("catalog.database.table", "col6"), + Seq("`database`.`table`.`col6`"), + Seq("table.col6") + )) { + assert( + nameScope.matchMultipartName(multipartIdentifier) == NameTarget( + candidates = Seq.empty, + allAttributes = Seq(col6IntegerWithQualifier) + ) + ) + } + } + + test("Nested fields") { + val nameScope = new NameScope + + nameScope("table") = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct + ) + + var matchedStructs = nameScope.matchMultipartName(Seq("col8", "field")) + assert( + matchedStructs == NameTarget( + candidates = Seq( + GetStructField(col8Struct, 0, Some("field")) + ), + aliasName = Some("field"), + allAttributes = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct + ) + ) + ) + + matchedStructs = nameScope.matchMultipartName(Seq("col9", "field", "subfield")) + assert( + matchedStructs == NameTarget( + candidates = Seq( + GetStructField( + GetStructField( + col9NestedStruct, + 0, + Some("field") + ), + 0, + Some("subfield") + ) + ), + aliasName = Some("subfield"), + allAttributes = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct + ) + ) + ) + + var matchedMaps = nameScope.matchMultipartName(Seq("col10", "key")) + assert( + matchedMaps == NameTarget( + candidates = Seq(GetMapValue(col10Map, Literal("key"))), + aliasName = Some("key"), + allAttributes = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct + ) + ) + ) + + matchedMaps = nameScope.matchMultipartName(Seq("col11", "key")) + assert( + matchedMaps == NameTarget( + candidates = Seq(GetMapValue(col11MapWithStruct, Literal("key"))), + aliasName = Some("key"), + allAttributes = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct + ) + ) + ) + + var matchedArrays = nameScope.matchMultipartName(Seq("col12", "element")) + assert( + matchedArrays == NameTarget( + candidates = Seq(GetArrayItem(col12Array, Literal("element"))), + aliasName = Some("element"), + allAttributes = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct + ) + ) + ) + + matchedArrays = nameScope.matchMultipartName(Seq("col13", "field")) + assert( + matchedArrays == NameTarget( + candidates = Seq( + GetArrayStructFields( + col13ArrayWithStruct, + StructField("field", StringType, true), + 0, + 1, + true + ) + ), + aliasName = Some("field"), + allAttributes = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct + ) + ) + ) + + nameScope("table2") = Seq(col8Struct) + matchedStructs = nameScope.matchMultipartName(Seq("col8", "field")) + assert( + matchedStructs == NameTarget( + candidates = Seq( + GetStructField( + col8Struct, + 0, + Some("field") + ), + GetStructField( + col8Struct, + 0, + Some("field") + ) + ), + aliasName = Some("field"), + allAttributes = Seq( + col8Struct, + col9NestedStruct, + col10Map, + col11MapWithStruct, + col12Array, + col13ArrayWithStruct, + col8Struct + ) + ) + ) + } +} + +class NameScopeStackSuite extends PlanTest { + private val col1Integer = AttributeReference(name = "col1", dataType = IntegerType)() + private val col2String = AttributeReference(name = "col2", dataType = StringType)() + private val col3Integer = AttributeReference(name = "col3", dataType = IntegerType)() + private val col4String = AttributeReference(name = "col4", dataType = StringType)() + + test("Empty stack") { + val stack = new NameScopeStack + + assert(stack.top.getAllAttributes.isEmpty) + } + + test("Overwrite top of the stack containing single scope") { + val stack = new NameScopeStack + + stack.top.update("table1", Seq(col1Integer, col2String)) + assert(stack.top.getAllAttributes == Seq(col1Integer, col2String)) + + stack.overwriteTop("table2", Seq(col3Integer, col4String)) + assert(stack.top.getAllAttributes == Seq(col3Integer, col4String)) + + stack.overwriteTop(Seq(col2String)) + assert(stack.top.getAllAttributes == Seq(col2String)) + } + + test("Overwrite top of the stack containing several scopes") { + val stack = new NameScopeStack + + stack.top.update("table2", Seq(col3Integer)) + + stack.withNewScope { + assert(stack.top.getAllAttributes.isEmpty) + + stack.top.update("table1", Seq(col1Integer, col2String)) + assert(stack.top.getAllAttributes == Seq(col1Integer, col2String)) + + stack.overwriteTop("table2", Seq(col3Integer, col4String)) + assert(stack.top.getAllAttributes == Seq(col3Integer, col4String)) + + stack.overwriteTop(Seq(col2String)) + assert(stack.top.getAllAttributes == Seq(col2String)) + } + } + + test("Scope stacking") { + val stack = new NameScopeStack + + stack.top.update("table1", Seq(col1Integer)) + + stack.withNewScope { + stack.top.update("table2", Seq(col2String)) + + stack.withNewScope { + stack.top.update("table3", Seq(col3Integer)) + + stack.withNewScope { + stack.top.update("table4", Seq(col4String)) + + assert(stack.top.getAllAttributes == Seq(col4String)) + } + + assert(stack.top.getAllAttributes == Seq(col3Integer)) + } + + assert(stack.top.getAllAttributes == Seq(col2String)) + } + + assert(stack.top.getAllAttributes == Seq(col1Integer)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverGuardSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverGuardSuite.scala new file mode 100644 index 0000000000000..d512adbb0af37 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverGuardSuite.scala @@ -0,0 +1,207 @@ +/* + * 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.analysis.resolver + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.analysis.resolver.ResolverGuard +import org.apache.spark.sql.test.SharedSparkSession + +class ResolverGuardSuite extends QueryTest with SharedSparkSession { + + // Queries that should pass the OperatorResolverGuard + + test("Select * from an inline table") { + checkResolverGuard("SELECT * FROM VALUES(1,2,3)", shouldPass = true) + } + + test("Select the named parameters from an inline table") { + checkResolverGuard("SELECT col1,col2,col3 FROM VALUES(1,2,3)", shouldPass = true) + } + + test("Inline table as a top level operator") { + checkResolverGuard("VALUES(1,2,3)", shouldPass = true) + } + + test("Select one row") { + checkResolverGuard("SELECT 'Hello world!'", shouldPass = true) + } + + test("Where clause with a literal") { + checkResolverGuard( + "SELECT * FROM VALUES(1, 2, false), (3, 4, true) WHERE true", + shouldPass = true + ) + } + + test("Where clause with an attribute") { + checkResolverGuard( + "SELECT * FROM VALUES(1, 2, false), (3, 4, true) WHERE col3", + shouldPass = true + ) + } + + test("Explicit cast with auto-alias") { + checkResolverGuard( + "SELECT CAST(1 AS DECIMAL(3,2))", + shouldPass = true + ) + } + + test("Multipart attribute name") { + checkResolverGuard("SELECT table.col1 FROM VALUES(1) AS table", shouldPass = true) + } + + test("Predicates") { + checkResolverGuard("SELECT true and false", shouldPass = true) + checkResolverGuard("SELECT true or false", shouldPass = true) + checkResolverGuard( + "SELECT col1 from VALUES(1,2) where true and false or true", + shouldPass = true + ) + checkResolverGuard("SELECT 1 = 2", shouldPass = true) + checkResolverGuard("SELECT 1 != 2", shouldPass = true) + checkResolverGuard("SELECT 1 IN (1,2,3)", shouldPass = true) + checkResolverGuard("SELECT 1 NOT IN (1,2,3)", shouldPass = true) + checkResolverGuard("SELECT 1 IS NULL", shouldPass = true) + checkResolverGuard("SELECT 1 IS NOT NULL", shouldPass = true) + checkResolverGuard("SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR", shouldPass = true) + } + + test("Star target") { + checkResolverGuard("SELECT table.* FROM VALUES(1) as table", shouldPass = true) + } + + test("Binary arithmetic") { + checkResolverGuard("SELECT col1+col2 FROM VALUES(1,2)", shouldPass = true) + checkResolverGuard("SELECT 1 + 2.3 / 2 - 3 DIV 2 + 3.0 * 10.0", shouldPass = true) + checkResolverGuard( + "SELECT TIMESTAMP'2011-11-11 11:11:11' - TIMESTAMP'2011-11-11 11:11:10'", + shouldPass = true + ) + checkResolverGuard( + "SELECT DATE'2020-01-01' - TIMESTAMP'2019-10-06 10:11:12.345678'", + shouldPass = true + ) + checkResolverGuard("SELECT DATE'2012-01-01' - INTERVAL 3 HOURS", shouldPass = true) + checkResolverGuard( + "SELECT DATE'2012-01-01' + INTERVAL '12:12:12' HOUR TO SECOND", + shouldPass = true + ) + checkResolverGuard("SELECT DATE'2012-01-01' + 1", shouldPass = true) + checkResolverGuard("SELECT 2 * INTERVAL 2 YEAR", shouldPass = true) + } + + test("Supported recursive types") { + Seq("ARRAY", "MAP", "STRUCT").foreach { typeName => + checkResolverGuard( + s"SELECT col1 FROM VALUES($typeName(1,2),3)", + shouldPass = true + ) + } + } + + test("Recursive types related functions") { + checkResolverGuard("SELECT NAMED_STRUCT('a', 1)", shouldPass = true) + checkResolverGuard("SELECT MAP_CONTAINS_KEY(MAP(1, 'a', 2, 'b'), 2)", shouldPass = true) + checkResolverGuard("SELECT ARRAY_CONTAINS(ARRAY(1, 2, 3), 2);", shouldPass = true) + } + + test("Conditional expressions") { + checkResolverGuard("SELECT COALESCE(NULL, 1)", shouldPass = true) + checkResolverGuard("SELECT col1, IF(col1 > 1, 1, 0) FROM VALUES(1,2),(2,3)", shouldPass = true) + checkResolverGuard( + "SELECT col1, CASE WHEN col1 > 1 THEN 1 ELSE 0 END FROM VALUES(1,2),(2,3)", + shouldPass = true + ) + } + + test("User specified alias") { + checkResolverGuard("SELECT 1 AS alias", shouldPass = true) + } + + // Queries that shouldn't pass the OperatorResolverGuard + + test("Select from table") { + withTable("test_table") { + sql("CREATE TABLE test_table (col1 INT, col2 INT)") + checkResolverGuard("SELECT * FROM test_table", shouldPass = true) + } + } + + test("Single-layer subquery") { + checkResolverGuard("SELECT * FROM (SELECT * FROM VALUES(1))", shouldPass = true) + } + + test("Multi-layer subquery") { + checkResolverGuard("SELECT * FROM (SELECT * FROM (SELECT * FROM VALUES(1)))", shouldPass = true) + } + + test("Scalar subquery") { + checkResolverGuard("SELECT (SELECT * FROM VALUES(1))", shouldPass = false) + } + + test("EXISTS subquery") { + checkResolverGuard( + "SELECT * FROM VALUES (1) WHERE EXISTS (SELECT * FROM VALUES(1))", + shouldPass = false + ) + } + + test("IN subquery") { + checkResolverGuard( + "SELECT * FROM VALUES (1) WHERE col1 IN (SELECT * FROM VALUES(1))", + shouldPass = false + ) + } + + test("Function") { + checkResolverGuard("SELECT current_date()", shouldPass = false) + } + + test("Function without the braces") { + checkResolverGuard("SELECT current_date", shouldPass = false) + } + + test("Session variables") { + withSessionVariable { + checkResolverGuard("SELECT session_variable", shouldPass = false) + } + } + + test("Case sensitive analysis") { + withSQLConf("spark.sql.caseSensitive" -> "true") { + checkResolverGuard("SELECT 1", shouldPass = false) + } + } + + private def checkResolverGuard(query: String, shouldPass: Boolean): Unit = { + val resolverGuard = new ResolverGuard(spark.sessionState.catalogManager) + assert( + resolverGuard.apply(sql(query).queryExecution.logical) == shouldPass + ) + } + + private def withSessionVariable(body: => Unit): Unit = { + sql("DECLARE session_variable = 1;") + try { + body + } finally { + sql("DROP TEMPORARY VARIABLE session_variable;") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala new file mode 100644 index 0000000000000..057724758d332 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/ResolverSuite.scala @@ -0,0 +1,157 @@ +/* + * 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.analysis.resolver + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.analysis.resolver.{Resolver, ResolverExtension} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.IntegerType + +class ResolverSuite extends QueryTest with SharedSparkSession { + private val col1Integer = AttributeReference("col1", IntegerType)() + + test("Node matched the extension") { + val resolver = createResolver( + Seq( + new NoopResolver, + new TestRelationResolver + ) + ) + + val result = resolver.lookupMetadataAndResolve( + Project( + Seq(UnresolvedAttribute("col1")), + TestRelation(resolutionDone = false, output = Seq(col1Integer)) + ) + ) + assert( + result == Project( + Seq(col1Integer), + TestRelation(resolutionDone = true, output = Seq(col1Integer)) + ) + ) + } + + test("Node didn't match the extension") { + val resolver = createResolver( + Seq( + new NoopResolver, + new TestRelationResolver + ) + ) + + checkError( + exception = intercept[AnalysisException]( + resolver.lookupMetadataAndResolve( + Project( + Seq(UnresolvedAttribute("col1")), + UnknownRelation(output = Seq(col1Integer)) + ) + ) + ), + condition = "UNSUPPORTED_SINGLE_PASS_ANALYZER_FEATURE", + parameters = Map( + "feature" -> ("class " + + "org.apache.spark.sql.analysis.resolver.ResolverSuite$UnknownRelation operator resolution") + ) + ) + } + + test("Ambiguous extensions") { + val resolver = createResolver( + Seq( + new NoopResolver, + new TestRelationResolver, + new TestRelationBrokenResolver + ) + ) + + checkError( + exception = intercept[AnalysisException]( + resolver.lookupMetadataAndResolve( + Project( + Seq(UnresolvedAttribute("col1")), + TestRelation(resolutionDone = false, output = Seq(col1Integer)) + ) + ) + ), + condition = "AMBIGUOUS_RESOLVER_EXTENSION", + parameters = Map( + "operator" -> "org.apache.spark.sql.analysis.resolver.ResolverSuite$TestRelation", + "extensions" -> "TestRelationResolver, TestRelationBrokenResolver" + ) + ) + } + + private def createResolver(extensions: Seq[ResolverExtension] = Seq.empty): Resolver = { + new Resolver(spark.sessionState.catalogManager, extensions) + } + + private class TestRelationResolver extends ResolverExtension { + var timesCalled = 0 + + override def resolveOperator: PartialFunction[LogicalPlan, LogicalPlan] = { + case testNode: TestRelation if countTimesCalled() => + testNode.copy(resolutionDone = true) + } + + private def countTimesCalled(): Boolean = { + timesCalled += 1 + assert(timesCalled == 1) + true + } + } + + private class TestRelationBrokenResolver extends ResolverExtension { + override def resolveOperator: PartialFunction[LogicalPlan, LogicalPlan] = { + case testNode: TestRelation => + assert(false) + testNode + } + } + + private class NoopResolver extends ResolverExtension { + override def resolveOperator: PartialFunction[LogicalPlan, LogicalPlan] = { + case node: LogicalPlan if false => + assert(false) + node + } + } + + private case class TestRelation( + resolutionDone: Boolean, + override val output: Seq[Attribute], + override val children: Seq[LogicalPlan] = Seq.empty) + extends LogicalPlan { + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): TestRelation = + copy(children = newChildren) + } + + private case class UnknownRelation( + override val output: Seq[Attribute], + override val children: Seq[LogicalPlan] = Seq.empty) + extends LogicalPlan { + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): UnknownRelation = + copy(children = newChildren) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala new file mode 100644 index 0000000000000..28ccebc89bc52 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala @@ -0,0 +1,133 @@ +/* + * 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.analysis.resolver + +import org.apache.spark.SparkException +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.analysis.FunctionResolution +import org.apache.spark.sql.catalyst.analysis.resolver.{ + ExpressionResolver, + NameScopeStack, + Resolver +} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, ExprId} +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{BooleanType, StringType} + +class TracksResolvedNodesSuite extends QueryTest with SharedSparkSession { + + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set(SQLConf.ANALYZER_SINGLE_PASS_TRACK_RESOLVED_NODES_ENABLED.key, "true") + } + + test("Single-pass contract preserved for equal expressions with different memory addresses") { + val expressionResolver = createExpressionResolver() + val columnObjFirst = + AttributeReference(name = "column", dataType = BooleanType)(exprId = ExprId(0)) + val columnObjSecond = + AttributeReference(name = "column", dataType = BooleanType)(exprId = ExprId(0)) + + expressionResolver.resolve(columnObjFirst) + expressionResolver.resolve(columnObjSecond) + } + + test("Single-pass contract broken for operators") { + val resolver = createResolver() + + val project = Project( + projectList = Seq(), + child = Project( + projectList = Seq(), + child = OneRowRelation() + ) + ) + + val resolvedProject = resolver.lookupMetadataAndResolve(project) + + checkError( + exception = intercept[SparkException]({ + resolver.lookupMetadataAndResolve(resolvedProject.children.head) + }), + condition = "INTERNAL_ERROR", + parameters = Map( + "message" -> ("Single-pass resolver attempted to resolve the same " + + "node more than once: Project\n+- OneRowRelation\n") + ) + ) + checkError( + exception = intercept[SparkException]({ + resolver.lookupMetadataAndResolve(resolvedProject) + }), + condition = "INTERNAL_ERROR", + parameters = Map( + "message" -> ("Single-pass resolver attempted to resolve the same " + + "node more than once: Project\n+- Project\n +- OneRowRelation\n") + ) + ) + } + + test("Single-pass contract broken for expressions") { + val expressionResolver = createExpressionResolver() + + val cast = Cast( + child = AttributeReference(name = "column", dataType = BooleanType)(exprId = ExprId(0)), + dataType = StringType + ) + + val resolvedCast = expressionResolver.resolve(cast) + + checkError( + exception = intercept[SparkException]({ + expressionResolver.resolve(resolvedCast.children.head) + }), + condition = "INTERNAL_ERROR", + parameters = Map( + "message" -> ("Single-pass resolver attempted " + + "to resolve the same node more than once: column#0") + ) + ) + checkError( + exception = intercept[SparkException]({ + expressionResolver.resolve(resolvedCast) + }), + condition = "INTERNAL_ERROR", + parameters = Map( + "message" -> ("Single-pass resolver attempted " + + "to resolve the same node more than once: cast(column#0 as string)") + ) + ) + } + + private def createResolver(): Resolver = { + new Resolver(spark.sessionState.catalogManager) + } + + private def createExpressionResolver(): ExpressionResolver = { + new ExpressionResolver( + createResolver(), + new NameScopeStack, + new FunctionResolution( + spark.sessionState.catalogManager, + Resolver.createRelationResolution(spark.sessionState.catalogManager) + ) + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceResolverSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceResolverSuite.scala new file mode 100644 index 0000000000000..016c1e2f5457d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceResolverSuite.scala @@ -0,0 +1,119 @@ +/* + * 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 + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.analysis.resolver.{MetadataResolver, Resolver} +import org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +class DataSourceResolverSuite extends QueryTest with SharedSparkSession { + private val keyValueTableSchema = StructType( + Seq( + StructField("key", IntegerType, true), + StructField("value", StringType, true) + ) + ) + + test("CSV relation") { + withTable("src_csv") { + spark.sql("CREATE TABLE src_csv (key INT, value STRING) USING CSV;").collect() + + checkResolveOperator( + sqlText = "SELECT * FROM src_csv", + expectedTableName = "spark_catalog.default.src_csv", + expectedTableSchema = keyValueTableSchema + ) + } + } + + test("JSON relation") { + withTable("src_json") { + spark.sql("CREATE TABLE src_json (key INT, value STRING) USING JSON;").collect() + + checkResolveOperator( + sqlText = "SELECT * FROM src_json", + expectedTableName = "spark_catalog.default.src_json", + expectedTableSchema = keyValueTableSchema + ) + } + } + + test("PARQUET relation") { + withTable("src_parquet") { + spark.sql("CREATE TABLE src_parquet (key INT, value STRING) USING PARQUET;").collect() + + checkResolveOperator( + sqlText = "SELECT * FROM src_parquet", + expectedTableName = "spark_catalog.default.src_parquet", + expectedTableSchema = keyValueTableSchema + ) + } + } + + test("ORC relation") { + withTable("src_orc") { + spark.sql("CREATE TABLE src_orc (key INT, value STRING) USING ORC;").collect() + + checkResolveOperator( + sqlText = "SELECT * FROM src_orc", + expectedTableName = "spark_catalog.default.src_orc", + expectedTableSchema = keyValueTableSchema + ) + } + } + + private def checkResolveOperator( + sqlText: String, + expectedTableName: String, + expectedTableSchema: StructType) = { + val metadataResolver = new MetadataResolver( + spark.sessionState.catalogManager, + Resolver.createRelationResolution(spark.sessionState.catalogManager) + ) + val dataSourceResolver = new DataSourceResolver(spark) + + val unresolvedPlan = spark.sql(sqlText).queryExecution.logical + + metadataResolver.resolve(unresolvedPlan) + + val unresolvedRelations = unresolvedPlan.collect { + case unresolvedRelation: UnresolvedRelation => unresolvedRelation + } + assert(unresolvedRelations.size == 1) + + val partiallyResolvedRelation = metadataResolver + .getRelationWithResolvedMetadata(unresolvedRelations.head) + .get + .asInstanceOf[SubqueryAlias] + .child + assert(partiallyResolvedRelation.isInstanceOf[UnresolvedCatalogRelation]) + + val result = dataSourceResolver.resolveOperator(partiallyResolvedRelation) + + val logicalRelation = result.asInstanceOf[LogicalRelation] + assert( + logicalRelation.catalogTable.get.identifier.unquotedString + == expectedTableName + ) + assert(logicalRelation.relation.schema == expectedTableSchema) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileResolverSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileResolverSuite.scala new file mode 100644 index 0000000000000..1d1b228028bdb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileResolverSuite.scala @@ -0,0 +1,97 @@ +/* + * 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 + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{LongType, StringType, StructType} + +class FileResolverSuite extends QueryTest with SharedSparkSession { + private val tableSchema = new StructType().add("id", LongType) + private val csvTableSchema = new StructType().add("_c0", StringType) + + test("JSON file format") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.json(f.getCanonicalPath) + checkResolveOperator( + sqlText = s"select id from json.`${f.getCanonicalPath}`", + expectedTablePath = s"file:${f.getCanonicalPath}", + expectedTableSchema = tableSchema + ) + }) + } + + test("PARQUET file format") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.parquet(f.getCanonicalPath) + checkResolveOperator( + sqlText = s"select id from parquet.`${f.getCanonicalPath}`", + expectedTablePath = s"file:${f.getCanonicalPath}", + expectedTableSchema = tableSchema + ) + }) + } + + test("ORC file format") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.orc(f.getCanonicalPath) + checkResolveOperator( + sqlText = s"select id from ORC.`${f.getCanonicalPath}`", + expectedTablePath = s"file:${f.getCanonicalPath}", + expectedTableSchema = tableSchema + ) + }) + } + + test("CSV file format") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.csv(f.getCanonicalPath) + checkResolveOperator( + sqlText = s"select _c0 from csv.`${f.getCanonicalPath}`", + expectedTablePath = s"file:${f.getCanonicalPath}", + expectedTableSchema = csvTableSchema + ) + }) + } + + private def checkResolveOperator( + sqlText: String, + expectedTablePath: String, + expectedTableSchema: StructType) = { + val fileResolver = new FileResolver(spark) + + val unresolvedPlan = spark.sql(sqlText).queryExecution.logical + + val result = fileResolver.resolveOperator( + unresolvedPlan.asInstanceOf[Project].child.asInstanceOf[UnresolvedRelation] + ) + + val logicalRelation = result.asInstanceOf[LogicalRelation] + assert( + logicalRelation.relation.asInstanceOf[HadoopFsRelation].location.rootPaths.mkString(",") == + expectedTablePath + ) + assert(logicalRelation.relation.schema == expectedTableSchema) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolver.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolver.scala new file mode 100644 index 0000000000000..842faba66cc30 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolver.scala @@ -0,0 +1,57 @@ +/* + * 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.hive + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.{DataSourceResolver, LogicalRelation} + +/** + * [[DataSourceWithHiveResolver]] is a [[DataSourceResolver]] that additionally handles + * [[HiveTableRelation]] conversion using [[RelationConversions]]. + */ +class DataSourceWithHiveResolver(sparkSession: SparkSession, hiveCatalog: HiveSessionCatalog) + extends DataSourceResolver(sparkSession) { + private val relationConversions = RelationConversions(hiveCatalog) + + /** + * Invoke [[DataSourceResolver]] to resolve the input operator. If [[DataSourceResolver]] produces + * [[HiveTableRelation]], convert it to [[LogicalRelation]] if possible. + */ + override def resolveOperator: PartialFunction[LogicalPlan, LogicalPlan] = { + case operator: LogicalPlan if super.resolveOperator.isDefinedAt(operator) => + val relationAfterDataSourceResolver = super.resolveOperator(operator) + + relationAfterDataSourceResolver match { + case hiveTableRelation: HiveTableRelation => + resolveHiveTableRelation(hiveTableRelation) + case other => other + } + } + + private def resolveHiveTableRelation(hiveTableRelation: HiveTableRelation): LogicalPlan = { + if (relationConversions.doConvertHiveTableRelationForRead(hiveTableRelation)) { + val logicalRelation: LogicalRelation = + relationConversions.convertHiveTableRelationForRead(hiveTableRelation) + logicalRelation.newInstance() + } else { + hiveTableRelation.newInstance() + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index dbeb8607facc2..a1cf27510838f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, Gener import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, InvokeProcedures, ReplaceCharWithVarchar, ResolveSessionCatalog, ResolveTranspose} +import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogWithListener, InvalidUDFClassException} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -84,6 +85,14 @@ class HiveSessionStateBuilder( * A logical query plan `Analyzer` with rules specific to Hive. */ override protected def analyzer: Analyzer = new Analyzer(catalogManager) { + override val singlePassResolverExtensions: Seq[ResolverExtension] = Seq( + new DataSourceWithHiveResolver(session, catalog) + ) + + override val singlePassMetadataResolverExtensions: Seq[ResolverExtension] = Seq( + new FileResolver(session) + ) + override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolverSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolverSuite.scala new file mode 100644 index 0000000000000..cb26354521b02 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DataSourceWithHiveResolverSuite.scala @@ -0,0 +1,107 @@ +/* + * 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.hive + +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.analysis.resolver.{MetadataResolver, Resolver} +import org.apache.spark.sql.catalyst.catalog.{HiveTableRelation, UnresolvedCatalogRelation} +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +class DataSourceWithHiveResolverSuite extends TestHiveSingleton with SQLTestUtils { + private val keyValueTableSchema = StructType( + Seq( + StructField("key", IntegerType, true), + StructField("value", StringType, true) + ) + ) + + test("ORC table resolution") { + withTable("src_orc") { + spark.sql("CREATE TABLE src_orc (key INT, value STRING) STORED AS ORC") + + checkResolveOperator( + sqlText = "SELECT * FROM src_orc", + expectedTableName = "spark_catalog.default.src_orc", + expectedTableSchema = keyValueTableSchema, + convertedToLogicalRelation = true + ) + } + } + + test("ORC table resolution without conversion") { + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false") { + withTable("src_orc_no_conversion") { + spark.sql("CREATE TABLE src_orc_no_conversion (key INT, value STRING) STORED AS ORC") + + checkResolveOperator( + sqlText = "SELECT * FROM src_orc_no_conversion", + expectedTableName = "spark_catalog.default.src_orc_no_conversion", + expectedTableSchema = keyValueTableSchema, + convertedToLogicalRelation = false + ) + } + } + } + + private def checkResolveOperator( + sqlText: String, + expectedTableName: String, + expectedTableSchema: StructType, + convertedToLogicalRelation: Boolean) = { + val metadataResolver = new MetadataResolver( + spark.sessionState.catalogManager, + Resolver.createRelationResolution(spark.sessionState.catalogManager) + ) + val dataSourceWithHiveResolver = new DataSourceWithHiveResolver( + spark, + spark.sessionState.catalog.asInstanceOf[HiveSessionCatalog] + ) + + val unresolvedPlan = spark.sql(sqlText).queryExecution.logical + + metadataResolver.resolve(unresolvedPlan) + + val unresolvedRelations = unresolvedPlan.collect { + case unresolvedRelation: UnresolvedRelation => unresolvedRelation + } + assert(unresolvedRelations.size == 1) + + val partiallyResolvedRelation = metadataResolver + .getRelationWithResolvedMetadata(unresolvedRelations.head) + .get + .asInstanceOf[SubqueryAlias] + .child + assert(partiallyResolvedRelation.isInstanceOf[UnresolvedCatalogRelation]) + + dataSourceWithHiveResolver.resolveOperator(partiallyResolvedRelation) match { + case logicalRelation: LogicalRelation => + assert(convertedToLogicalRelation) + assert(logicalRelation.catalogTable.get.identifier.unquotedString == expectedTableName) + assert(logicalRelation.relation.schema == expectedTableSchema) + case hiveTableRelation: HiveTableRelation => + assert(!convertedToLogicalRelation) + assert(hiveTableRelation.tableMeta.identifier.unquotedString == expectedTableName) + assert(hiveTableRelation.tableMeta.schema == expectedTableSchema) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala index f7e453a1dbdec..b67370f6eb9f6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala @@ -34,6 +34,9 @@ class PartitionedTablePerfStatsSuite override def beforeEach(): Unit = { super.beforeEach() + // Hive operation counters are doubled in dual-analyzer mode. + hiveContext.sparkSession.conf.set( + SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER.key, "false") FileStatusCache.resetForTesting() } From 5ef99bda98636c8bdc98796ff8c8d0a6e731977d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 18 Dec 2024 17:34:53 +0800 Subject: [PATCH 208/438] [SPARK-50606][CONNECT] Fix NPE on uninitiated SessionHolder ### What changes were proposed in this pull request? This PR fixes NPE like, ```scala {"ts":"2024-12-18T05:41:52.977Z","level":"ERROR","msg":"Exception while executing runnable org.sparkproject.connect.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable418f1536","exception":{"class":"java.lang.NullPointerException","msg":"Cannot invoke \"org.apache.spark.sql.connect.service.SessionHolder.userId()\" because the return value of \"org.apache.spark.sql.connect.service.SparkConnectAddArtifactsHandler.holder()\" is null","stacktrace":[{"class":"org.apache.spark.sql.connect.service.SparkConnectAddArtifactsHandler","method":"onNext","file":"SparkConnectAddArtifactsHandler.scala","line":90},{"class":"org.apache.spark.sql.connect.service.SparkConnectAddArtifactsHandler","method":"onNext","file":"SparkConnectAddArtifactsHandler.scala","line":42},{"class":"org.sparkproject.connect.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener","method":"onMessage","file":"ServerCalls.java","line":262},{"class":"org.sparkproject.connect.grpc.internal.ServerCallImpl$ServerStreamListenerImpl","method":"messagesAvailableInternal","file":"ServerCallImpl.java","line":334},{"class":"org.sparkproject.connect.grpc.internal.ServerCallImpl$ServerStreamListenerImpl","method":"messagesAvailable","file":"ServerCallImpl.java","line":319},{"class":"org.sparkproject.connect.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable","method":"runInContext","file":"ServerImpl.java","line":834},{"class":"org.sparkproject.connect.grpc.internal.ContextRunnable","method":"run","file":"ContextRunnable.java","line":37},{"class":"org.sparkproject.connect.grpc.internal.SerializingExecutor","method":"run","file":"SerializingExecutor.java","line":133},{"class":"java.util.concurrent.ThreadPoolExecutor","method":"runWorker","file":"ThreadPoolExecutor.java","line":1136},{"class":"java.util.concurrent.ThreadPoolExecutor$Worker","method":"run","file":"ThreadPoolExecutor.java","line":635},{"class":"java.lang.Thread","method":"run","file":"Thread.java","line":840}]},"logger":"SerializingExecutor"} ``` When I simply run case like, ``` scala> spark.sql("SHOW TABLES").schema org.apache.spark.SparkException: org.sparkproject.io.grpc.StatusRuntimeException: UNKNOWN: Application error processing RPC org.apache.spark.sql.connect.client.ArtifactManager.addArtifacts(ArtifactManager.scala:242) org.apache.spark.sql.connect.client.ArtifactManager.uploadAllClassFileArtifacts(ArtifactManager.scala:205) org.apache.spark.sql.connect.client.SparkConnectClient.execute(SparkConnectClient.scala:121) org.apache.spark.sql.SparkSession.$anonfun$sql$3(SparkSession.scala:247) org.apache.spark.sql.SparkSession.$anonfun$sql$3$adapted(SparkSession.scala:243) org.apache.spark.sql.SparkSession.newDataset(SparkSession.scala:337) org.apache.spark.sql.SparkSession.newDataFrame(SparkSession.scala:329) org.apache.spark.sql.SparkSession.sql(SparkSession.scala:243) org.apache.spark.sql.SparkSession.sql(SparkSession.scala:220) org.apache.spark.sql.SparkSession.sql(SparkSession.scala:240) ammonite.$sess.cmd6$Helper.(cmd6.sc:1) ammonite.$sess.cmd6$.(cmd6.sc:7) ``` I believe it suppressed the actual cause ### Why are the changes needed? bugfix ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests, (I do not know how to trigger this NPE in UT) ### Was this patch authored or co-authored using generative AI tooling? no Closes #49222 from yaooqinn/SPARK-50606. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../sql/connect/service/SparkConnectAddArtifactsHandler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala index 72403016404c8..3ba79402e99ef 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala @@ -87,8 +87,8 @@ class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddAr ErrorUtils.handleError( "addArtifacts.onNext", responseObserver, - holder.userId, - holder.sessionId, + req.getUserContext.getUserId, + req.getSessionId, None, false, Some(() => { From 0e1fa6facd4d65e4e5d9c21a1fd0819d0e2a5c2f Mon Sep 17 00:00:00 2001 From: Jovan Pavlovic Date: Wed, 18 Dec 2024 17:49:47 +0300 Subject: [PATCH 209/438] [SPARK-49661][SQL] Removing redudant checks of collation in binary collation hashing expression ### What changes were proposed in this pull request? Removing redudant check of collation in binary collation hashing expression ### Why are the changes needed? Code cleanup, trying to speed up joins. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests should keep passing as join is already well covered with tests. ### Was this patch authored or co-authored using generative AI tooling? No. --> Closes #49186 from jovanpavl-db/move-collation-checks-from-critical-path. Authored-by: Jovan Pavlovic Signed-off-by: Max Gekk --- .../main/scala/org/apache/spark/sql/types/StringType.scala | 3 ++- .../org/apache/spark/sql/catalyst/expressions/hash.scala | 6 +++--- .../org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala | 3 +-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index 735df2bd3fb19..02dd139f073d2 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -43,7 +43,8 @@ class StringType private[sql] ( * equality and hashing). */ private[sql] def supportsBinaryEquality: Boolean = - CollationFactory.fetchCollation(collationId).supportsBinaryEquality + collationId == CollationFactory.UTF8_BINARY_COLLATION_ID || + CollationFactory.fetchCollation(collationId).supportsBinaryEquality private[sql] def supportsLowercaseEquality: Boolean = CollationFactory.fetchCollation(collationId).supportsLowercaseEquality diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 79879dc0edb4c..89d2259ea5c28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -419,7 +419,7 @@ abstract class HashExpression[E] extends Expression { protected def genHashString( ctx: CodegenContext, stringType: StringType, input: String, result: String): String = { - if (stringType.supportsBinaryEquality && !stringType.usesTrimCollation) { + if (stringType.supportsBinaryEquality) { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" @@ -570,7 +570,7 @@ abstract class InterpretedHashFunction { hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) case s: UTF8String => val st = dataType.asInstanceOf[StringType] - if (st.supportsBinaryEquality && !st.usesTrimCollation) { + if (st.supportsBinaryEquality) { hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) } else { val stringHash = CollationFactory @@ -821,7 +821,7 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] { override protected def genHashString( ctx: CodegenContext, stringType: StringType, input: String, result: String): String = { - if (stringType.supportsBinaryEquality && !stringType.usesTrimCollation) { + if (stringType.supportsBinaryEquality) { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala index 118dd92c3ed54..f2925314e2e2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala @@ -206,8 +206,7 @@ object UnsafeRowUtils { */ def isBinaryStable(dataType: DataType): Boolean = !dataType.existsRecursively { case st: StringType => - val collation = CollationFactory.fetchCollation(st.collationId) - (!collation.supportsBinaryEquality) + !st.supportsBinaryEquality case _ => false } } From 0dd90d99be2c227d93417e348343af613823a83f Mon Sep 17 00:00:00 2001 From: mihailoale-db Date: Wed, 18 Dec 2024 17:52:47 +0300 Subject: [PATCH 210/438] [SPARK-50611][SQL] Improve HybridAnalyzer so it throws ExplicitlyUnsupportedResolverFeature in specific cases ### What changes were proposed in this pull request? Introduction of `checkSupportedSinglePassFeatures` which indicates that `HybridAnalyzer` should be run in a mode where we get specific metrics of the single-pass analyzer. ### Why are the changes needed? These changes are needed in order to get better metrics in specific cases for the single-pass analyzer (when `checkSupportedSinglePassFeatures` is set to false). ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? CIs ### Was this patch authored or co-authored using generative AI tooling? No Closes #49229 from mihailoale-db/mihailoale-db/checkSupportedSinglePassFeatures. Authored-by: mihailoale-db Signed-off-by: Max Gekk --- .../analysis/resolver/HybridAnalyzer.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala index 8a1a8602368e0..4ebcb3e06b178 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala @@ -50,7 +50,11 @@ import org.apache.spark.sql.internal.SQLConf * plans and output schemas, and return the resolved plan from the fixed-point Analyzer. * - Otherwise we run the legacy analyzer. * */ -class HybridAnalyzer(legacyAnalyzer: Analyzer, resolverGuard: ResolverGuard, resolver: Resolver) +class HybridAnalyzer( + legacyAnalyzer: Analyzer, + resolverGuard: ResolverGuard, + resolver: Resolver, + checkSupportedSinglePassFeatures: Boolean = true) extends SQLConfHelper { private var singlePassResolutionDuration: Option[Long] = None private var fixedPointResolutionDuration: Option[Long] = None @@ -58,7 +62,7 @@ class HybridAnalyzer(legacyAnalyzer: Analyzer, resolverGuard: ResolverGuard, res def apply(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { val dualRun = conf.getConf(SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER) && - resolverGuard.apply(plan) + checkResolverGuard(plan) withTrackedAnalyzerBridgeState(dualRun) { if (dualRun) { @@ -71,6 +75,9 @@ class HybridAnalyzer(legacyAnalyzer: Analyzer, resolverGuard: ResolverGuard, res } } + private def checkResolverGuard(plan: LogicalPlan): Boolean = + !checkSupportedSinglePassFeatures || resolverGuard.apply(plan) + def getSinglePassResolutionDuration: Option[Long] = singlePassResolutionDuration def getFixedPointResolutionDuration: Option[Long] = fixedPointResolutionDuration @@ -152,7 +159,8 @@ class HybridAnalyzer(legacyAnalyzer: Analyzer, resolverGuard: ResolverGuard, res } case None => singlePassException match { - case Some(singlePassEx: ExplicitlyUnsupportedResolverFeature) => + case Some(singlePassEx: ExplicitlyUnsupportedResolverFeature) + if checkSupportedSinglePassFeatures => fixedPointResult.get case Some(singlePassEx) => throw singlePassEx From 788fa5ad1fe8286ae762321087aae8a1a5b0fe0d Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 18 Dec 2024 17:55:22 +0300 Subject: [PATCH 211/438] [SPARK-50609][INFRA] Eliminate warnings in docker image building ### What changes were proposed in this pull request? The pr aims to eliminate warnings in docker image building ```shell - LegacyKeyValueFormat: "ENV key=value" should be used instead of legacy "ENV key value" format (line 75) - LegacyKeyValueFormat: "ENV key=value" should be used instead of legacy "ENV key value" format (line 27) - LegacyKeyValueFormat: "ENV key=value" should be used instead of legacy "ENV key value" format (line 29) - LegacyKeyValueFormat: "ENV key=value" should be used instead of legacy "ENV key value" format (line 30) ``` ### Why are the changes needed? - Eliminate warnings - Alignment of environment variable declarations for other `dockerfile` files in the code repository https://github.com/apache/spark/blob/229118ca7a127753635543909efdb27601985d42/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile#L63 https://github.com/apache/spark/blob/229118ca7a127753635543909efdb27601985d42/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile#L40 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA. - Manually check. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49224 from panbingkun/SPARK-50609. Authored-by: panbingkun Signed-off-by: Max Gekk --- binder/Dockerfile | 4 ++-- dev/create-release/spark-rm/Dockerfile | 8 ++++---- dev/infra/Dockerfile | 8 ++++---- dev/spark-test-image/docs/Dockerfile | 8 ++++---- dev/spark-test-image/lint/Dockerfile | 8 ++++---- dev/spark-test-image/pypy-310/Dockerfile | 6 +++--- dev/spark-test-image/python-309/Dockerfile | 6 +++--- dev/spark-test-image/python-310/Dockerfile | 6 +++--- dev/spark-test-image/python-311/Dockerfile | 6 +++--- dev/spark-test-image/python-312/Dockerfile | 6 +++--- dev/spark-test-image/python-313/Dockerfile | 6 +++--- dev/spark-test-image/sparkr/Dockerfile | 8 ++++---- 12 files changed, 40 insertions(+), 40 deletions(-) diff --git a/binder/Dockerfile b/binder/Dockerfile index 6e3dd9155fb7a..2d5c30a9a92e0 100644 --- a/binder/Dockerfile +++ b/binder/Dockerfile @@ -22,8 +22,8 @@ RUN pip install --no-cache notebook jupyterlab # create user with a home directory ARG NB_USER ARG NB_UID -ENV USER ${NB_USER} -ENV HOME /home/${NB_USER} +ENV USER=${NB_USER} +ENV HOME=/home/${NB_USER} RUN adduser --disabled-password \ --gecos "Default user" \ diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index ae6b7cfea6f22..190c82429e29e 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -23,10 +23,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Release Manager Image" # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20240318 +ENV FULL_REFRESH_DATE=20240318 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ @@ -88,7 +88,7 @@ RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" # See more in SPARK-39735 -ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" +ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" RUN add-apt-repository ppa:pypy/ppa diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 99cd887c6b6ea..9cd6031023ca5 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image" # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241119 +ENV FULL_REFRESH_DATE=20241119 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ @@ -82,7 +82,7 @@ RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" # See more in SPARK-39735 -ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" +ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" RUN add-apt-repository ppa:pypy/ppa diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index 0f3559fa2d4cf..6b087f08281c3 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image for Documentat # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241029 +ENV FULL_REFRESH_DATE=20241029 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ @@ -72,7 +72,7 @@ RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', 'rmarkdown', Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" # See more in SPARK-39735 -ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" +ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" # Install Python 3.9 RUN add-apt-repository ppa:deadsnakes/ppa diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index f9ea3124291b1..c663c3dac3b9f 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image for Linter" # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241112 +ENV FULL_REFRESH_DATE=20241112 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ @@ -63,7 +63,7 @@ RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', 'rmarkdown', && Rscript -e "devtools::install_version('lintr', version='2.0.1', repos='https://cloud.r-project.org')" \ # See more in SPARK-39735 -ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" +ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" # Install Python 3.9 RUN add-apt-repository ppa:deadsnakes/ppa diff --git a/dev/spark-test-image/pypy-310/Dockerfile b/dev/spark-test-image/pypy-310/Dockerfile index 7d6ff26566de3..6a309d38f1d55 100644 --- a/dev/spark-test-image/pypy-310/Dockerfile +++ b/dev/spark-test-image/pypy-310/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark wi # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241212 +ENV FULL_REFRESH_DATE=20241212 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ diff --git a/dev/spark-test-image/python-309/Dockerfile b/dev/spark-test-image/python-309/Dockerfile index 2b4d5bc258012..bfe23bf572add 100644 --- a/dev/spark-test-image/python-309/Dockerfile +++ b/dev/spark-test-image/python-309/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark wi # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241205 +ENV FULL_REFRESH_DATE=20241205 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index 26748dd26b060..b9875ba969f8d 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark wi # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241205 +ENV FULL_REFRESH_DATE=20241205 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 33956b05b2d31..48f1fede03c05 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark wi # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241212 +ENV FULL_REFRESH_DATE=20241212 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index 29ec14290c4b7..0a8d5b79a361a 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark wi # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241206 +ENV FULL_REFRESH_DATE=20241206 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index 50d634b72602d..473f3df8fdb7c 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark wi # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241210 +ENV FULL_REFRESH_DATE=20241210 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ diff --git a/dev/spark-test-image/sparkr/Dockerfile b/dev/spark-test-image/sparkr/Dockerfile index 43260c714a550..3312c0852bd77 100644 --- a/dev/spark-test-image/sparkr/Dockerfile +++ b/dev/spark-test-image/sparkr/Dockerfile @@ -24,10 +24,10 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image for SparkR" # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241114 +ENV FULL_REFRESH_DATE=20241114 -ENV DEBIAN_FRONTEND noninteractive -ENV DEBCONF_NONINTERACTIVE_SEEN true +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true RUN apt-get update && apt-get install -y \ build-essential \ @@ -74,4 +74,4 @@ RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" # See more in SPARK-39735 -ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" +ENV R_LIBS_SITE="/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" From 332efb2eabb2b9383cfcfb0bf633089f38cdb398 Mon Sep 17 00:00:00 2001 From: Mihailo Milosevic Date: Wed, 18 Dec 2024 17:58:13 +0300 Subject: [PATCH 212/438] [SPARK-49636][SQL] Remove the ANSI config suggestion in INVALID_ARRAY_INDEX and INVALID_ARRAY_INDEX_IN_ELEMENT_AT ### What changes were proposed in this pull request? Removal of ansi turn off suggestion for INVALID_ARRAY_INDEX and INVALID_ARRAY_INDEX_IN_ELEMENT_AT. ### Why are the changes needed? Now that in Spark 4.0.0 we have moved to ANSI mode on by default, we want to keep suggestions of this kind to the minimum. The good thing is that both errors have other suggestions which are sufficient. ### Does this PR introduce _any_ user-facing change? Yes, error message has changed. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49213 from mihailom-db/array_index. Authored-by: Mihailo Milosevic Signed-off-by: Max Gekk --- .../utils/src/main/resources/error/error-conditions.json | 4 ++-- .../apache/spark/sql/errors/QueryExecutionErrors.scala | 6 ++---- .../src/test/resources/sql-tests/results/array.sql.out | 7 ------- .../spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala | 8 ++++---- 4 files changed, 8 insertions(+), 17 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 55265fa6dfc9c..8c2fc8a038b06 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2131,13 +2131,13 @@ }, "INVALID_ARRAY_INDEX" : { "message" : [ - "The index is out of bounds. The array has elements. Use the SQL function `get()` to tolerate accessing element at invalid index and return NULL instead. If necessary set to \"false\" to bypass this error." + "The index is out of bounds. The array has elements. Use the SQL function `get()` to tolerate accessing element at invalid index and return NULL instead." ], "sqlState" : "22003" }, "INVALID_ARRAY_INDEX_IN_ELEMENT_AT" : { "message" : [ - "The index is out of bounds. The array has elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set to \"false\" to bypass this error." + "The index is out of bounds. The array has elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead." ], "sqlState" : "22003" }, 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 95e2f30bbf2db..86137fc1c3c0e 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 @@ -225,8 +225,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE errorClass = "INVALID_ARRAY_INDEX", messageParameters = Map( "indexValue" -> toSQLValue(index, IntegerType), - "arraySize" -> toSQLValue(numElements, IntegerType), - "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), + "arraySize" -> toSQLValue(numElements, IntegerType)), context = getQueryContext(context), summary = getSummary(context)) } @@ -239,8 +238,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE errorClass = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", messageParameters = Map( "indexValue" -> toSQLValue(index, IntegerType), - "arraySize" -> toSQLValue(numElements, IntegerType), - "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), + "arraySize" -> toSQLValue(numElements, IntegerType)), context = getQueryContext(context), summary = getSummary(context)) } diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 7394e428091c7..b5dabfb47f439 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -185,7 +185,6 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", "sqlState" : "22003", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", "indexValue" : "5" }, @@ -209,7 +208,6 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", "sqlState" : "22003", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", "indexValue" : "-5" }, @@ -252,7 +250,6 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException "errorClass" : "INVALID_ARRAY_INDEX", "sqlState" : "22003", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "2", "indexValue" : "4" }, @@ -276,7 +273,6 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException "errorClass" : "INVALID_ARRAY_INDEX", "sqlState" : "22003", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "2", "indexValue" : "0" }, @@ -300,7 +296,6 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException "errorClass" : "INVALID_ARRAY_INDEX", "sqlState" : "22003", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "2", "indexValue" : "-1" }, @@ -356,7 +351,6 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException "errorClass" : "INVALID_ARRAY_INDEX", "sqlState" : "22003", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", "indexValue" : "5" }, @@ -380,7 +374,6 @@ org.apache.spark.SparkArrayIndexOutOfBoundsException "errorClass" : "INVALID_ARRAY_INDEX", "sqlState" : "22003", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "arraySize" : "3", "indexValue" : "-1" }, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala index f07d2d6620f72..a9ff7c308c153 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala @@ -145,7 +145,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest sql("select array(1, 2, 3, 4, 5)[8]").collect() }, condition = "INVALID_ARRAY_INDEX", - parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), + parameters = Map("indexValue" -> "8", "arraySize" -> "5"), context = ExpectedContext(fragment = "array(1, 2, 3, 4, 5)[8]", start = 7, stop = 29)) checkError( @@ -153,7 +153,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest OneRowRelation().select(lit(Array(1, 2, 3, 4, 5))(8)).collect() }, condition = "INVALID_ARRAY_INDEX", - parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), + parameters = Map("indexValue" -> "8", "arraySize" -> "5"), context = ExpectedContext( fragment = "apply", callSitePattern = getCurrentClassCallSitePattern)) @@ -165,7 +165,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest sql("select element_at(array(1, 2, 3, 4, 5), 8)").collect() }, condition = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", - parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), + parameters = Map("indexValue" -> "8", "arraySize" -> "5"), context = ExpectedContext( fragment = "element_at(array(1, 2, 3, 4, 5), 8)", start = 7, @@ -176,7 +176,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest OneRowRelation().select(element_at(lit(Array(1, 2, 3, 4, 5)), 8)).collect() }, condition = "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", - parameters = Map("indexValue" -> "8", "arraySize" -> "5", "ansiConfig" -> ansiConf), + parameters = Map("indexValue" -> "8", "arraySize" -> "5"), context = ExpectedContext(fragment = "element_at", callSitePattern = getCurrentClassCallSitePattern)) } From 2b1369b4abed643d986ee6a22ec86371eadb831e Mon Sep 17 00:00:00 2001 From: Sebastian Hillig <101127480+sebastianhillig-db@users.noreply.github.com> Date: Thu, 19 Dec 2024 08:47:21 +0900 Subject: [PATCH 213/438] [SPARK-50499][PYTHON] Expose metrics from BasePythonRunner ### What changes were proposed in this pull request? Expose additional metrics from BasePythonRunner for all Python UDF executions: bootTime, initTime, and totalTime. ### Why are the changes needed? Metrics are currently only exposed as logs, but can be helpful to debug slow queries. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Extended existing testcases. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49076 from sebastianhillig-db/add-basepythonrunner-metrics. Lead-authored-by: Sebastian Hillig <101127480+sebastianhillig-db@users.noreply.github.com> Co-authored-by: Sebastian Hillig Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/api/python/PythonRunner.scala | 8 ++++++-- .../datasources/v2/python/PythonCustomMetric.scala | 2 ++ .../python/ApplyInPandasWithStatePythonRunner.scala | 3 ++- .../spark/sql/execution/python/ArrowPythonRunner.scala | 2 +- .../sql/execution/python/ArrowPythonUDTFRunner.scala | 2 +- .../execution/python/CoGroupedArrowPythonRunner.scala | 2 +- .../spark/sql/execution/python/PythonSQLMetrics.scala | 10 ++++++++++ .../spark/sql/execution/python/PythonUDFRunner.scala | 2 +- .../TransformWithStateInPandasPythonRunner.scala | 3 ++- .../spark/sql/execution/python/PythonUDFSuite.scala | 5 ++++- 10 files changed, 30 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index b7fb22bab844a..a34cce980ae44 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -109,7 +109,8 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( protected val funcs: Seq[ChainedPythonFunctions], protected val evalType: Int, protected val argOffsets: Array[Array[Int]], - protected val jobArtifactUUID: Option[String]) + protected val jobArtifactUUID: Option[String], + protected val metrics: Map[String, AccumulatorV2[Long, Long]]) extends Logging { require(funcs.length == argOffsets.length, "argOffsets should have the same length as funcs") @@ -522,6 +523,9 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( log"boot = ${MDC(LogKeys.BOOT_TIME, boot)}, " + log"init = ${MDC(LogKeys.INIT_TIME, init)}, " + log"finish = ${MDC(LogKeys.FINISH_TIME, finish)}") + metrics.get("pythonBootTime").foreach(_.add(boot)) + metrics.get("pythonInitTime").foreach(_.add(init)) + metrics.get("pythonTotalTime").foreach(_.add(total)) val memoryBytesSpilled = stream.readLong() val diskBytesSpilled = stream.readLong() context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) @@ -824,7 +828,7 @@ private[spark] object PythonRunner { private[spark] class PythonRunner( funcs: Seq[ChainedPythonFunctions], jobArtifactUUID: Option[String]) extends BasePythonRunner[Array[Byte], Array[Byte]]( - funcs, PythonEvalType.NON_UDF, Array(Array(0)), jobArtifactUUID) { + funcs, PythonEvalType.NON_UDF, Array(Array(0)), jobArtifactUUID, Map.empty) { protected override def newWriter( env: SparkEnv, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonCustomMetric.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonCustomMetric.scala index bca1cbed7e70b..7551cd04f20f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonCustomMetric.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonCustomMetric.scala @@ -45,6 +45,8 @@ object PythonCustomMetric { // See also `UserDefinedPythonDataSource.createPythonMetrics`. PythonSQLMetrics.pythonSizeMetricsDesc.keys .map(_ -> new SQLMetric("size", -1)).toMap ++ + PythonSQLMetrics.pythonTimingMetricsDesc.keys + .map(_ -> new SQLMetric("timing", -1)).toMap ++ PythonSQLMetrics.pythonOtherMetricsDesc.keys .map(_ -> new SQLMetric("sum", -1)).toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ApplyInPandasWithStatePythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ApplyInPandasWithStatePythonRunner.scala index ae982f2f87f2e..d704638b85e8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ApplyInPandasWithStatePythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ApplyInPandasWithStatePythonRunner.scala @@ -65,7 +65,8 @@ class ApplyInPandasWithStatePythonRunner( stateValueSchema: StructType, override val pythonMetrics: Map[String, SQLMetric], jobArtifactUUID: Option[String]) - extends BasePythonRunner[InType, OutType](funcs.map(_._1), evalType, argOffsets, jobArtifactUUID) + extends BasePythonRunner[InType, OutType]( + funcs.map(_._1), evalType, argOffsets, jobArtifactUUID, pythonMetrics) with PythonArrowInput[InType] with PythonArrowOutput[OutType] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 72e9c5210194a..579b496046852 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -38,7 +38,7 @@ abstract class BaseArrowPythonRunner( override val pythonMetrics: Map[String, SQLMetric], jobArtifactUUID: Option[String]) extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch]( - funcs.map(_._1), evalType, argOffsets, jobArtifactUUID) + funcs.map(_._1), evalType, argOffsets, jobArtifactUUID, pythonMetrics) with BasicPythonArrowInput with BasicPythonArrowOutput { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala index f52b01b6646ac..99a9e706c6620 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala @@ -43,7 +43,7 @@ class ArrowPythonUDTFRunner( jobArtifactUUID: Option[String]) extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch]( Seq(ChainedPythonFunctions(Seq(udtf.func))), evalType, Array(argMetas.map(_.offset)), - jobArtifactUUID) + jobArtifactUUID, pythonMetrics) with BasicPythonArrowInput with BasicPythonArrowOutput { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala index 5670cad67e7b0..c5e86d010938d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala @@ -51,7 +51,7 @@ class CoGroupedArrowPythonRunner( profiler: Option[String]) extends BasePythonRunner[ (Iterator[InternalRow], Iterator[InternalRow]), ColumnarBatch]( - funcs.map(_._1), evalType, argOffsets, jobArtifactUUID) + funcs.map(_._1), evalType, argOffsets, jobArtifactUUID, pythonMetrics) with BasicPythonArrowOutput { override val pythonExec: String = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonSQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonSQLMetrics.scala index 4df6d821c014f..bd22739613eef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonSQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonSQLMetrics.scala @@ -24,6 +24,8 @@ trait PythonSQLMetrics { self: SparkPlan => protected val pythonMetrics: Map[String, SQLMetric] = { PythonSQLMetrics.pythonSizeMetricsDesc.map { case (k, v) => k -> SQLMetrics.createSizeMetric(sparkContext, v) + } ++ PythonSQLMetrics.pythonTimingMetricsDesc.map { case (k, v) => + k -> SQLMetrics.createTimingMetric(sparkContext, v) } ++ PythonSQLMetrics.pythonOtherMetricsDesc.map { case (k, v) => k -> SQLMetrics.createMetric(sparkContext, v) } @@ -40,6 +42,14 @@ object PythonSQLMetrics { ) } + val pythonTimingMetricsDesc: Map[String, String] = { + Map( + "pythonBootTime" -> "total time to start Python workers", + "pythonInitTime" -> "total time to initialize Python workers", + "pythonTotalTime" -> "total time to run Python workers" + ) + } + val pythonOtherMetricsDesc: Map[String, String] = { Map("pythonNumRowsReceived" -> "number of output rows") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala index 87ff5a0ec4333..92310aa755db8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala @@ -36,7 +36,7 @@ abstract class BasePythonUDFRunner( pythonMetrics: Map[String, SQLMetric], jobArtifactUUID: Option[String]) extends BasePythonRunner[Array[Byte], Array[Byte]]( - funcs.map(_._1), evalType, argOffsets, jobArtifactUUID) { + funcs.map(_._1), evalType, argOffsets, jobArtifactUUID, pythonMetrics) { override val pythonExec: String = SQLConf.get.pysparkWorkerPythonExecutable.getOrElse( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala index c5980012124fe..641e49657ca95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala @@ -173,7 +173,8 @@ abstract class TransformWithStateInPandasPythonBaseRunner[I]( groupingKeySchema: StructType, batchTimestampMs: Option[Long], eventTimeWatermarkForEviction: Option[Long]) - extends BasePythonRunner[I, ColumnarBatch](funcs.map(_._1), evalType, argOffsets, jobArtifactUUID) + extends BasePythonRunner[I, ColumnarBatch]( + funcs.map(_._1), evalType, argOffsets, jobArtifactUUID, pythonMetrics) with PythonArrowInput[I] with BasicPythonArrowOutput with Logging { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala index 4b46331be107a..2f44994c301b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala @@ -91,7 +91,10 @@ class PythonUDFSuite extends QueryTest with SharedSparkSession { val pythonSQLMetrics = List( "data sent to Python workers", "data returned from Python workers", - "number of output rows") + "number of output rows", + "total time to initialize Python workers", + "total time to start Python workers", + "total time to run Python workers") val df = base.groupBy(pythonTestUDF(base("a") + 1)) .agg(pythonTestUDF(pythonTestUDF(base("a") + 1))) From 2cb66e6a86ee223dc97bf68579ae426af954af0d Mon Sep 17 00:00:00 2001 From: Mihailo Milosevic Date: Thu, 19 Dec 2024 08:48:42 +0900 Subject: [PATCH 214/438] [SPARK-49632][SQL] Remove the ANSI config suggestion in CANNOT_PARSE_TIMESTAMP ### What changes were proposed in this pull request? This PR changes the message returned on a failure of `ToTimestamp` family of expressions. ### Why are the changes needed? CANNOT_PARSE_TIMESTAMP contains suggested fix for turning off ANSI mode. Now that in Spark 4.0.0 we have moved to ANSI mode on by default, we want to keep suggestions of this kind to the minimum. ### Does this PR introduce _any_ user-facing change? Yes, error message is changing. ### How was this patch tested? Existing tests span the error message change. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49227 from mihailom-db/cannot_parse_timestamp. Authored-by: Mihailo Milosevic Signed-off-by: Hyukjin Kwon --- .../resources/error/error-conditions.json | 2 +- .../sql/errors/QueryExecutionErrors.scala | 3 +-- .../resources/sql-tests/results/date.sql.out | 1 - .../sql-tests/results/datetime-legacy.sql.out | 24 ------------------- .../results/datetime-parsing-invalid.sql.out | 24 ------------------- .../sql-tests/results/timestamp.sql.out | 6 ----- .../timestampNTZ/timestamp-ansi.sql.out | 6 ----- .../native/stringCastAndExpressions.sql.out | 2 -- .../QueryExecutionAnsiErrorsSuite.scala | 4 +--- 9 files changed, 3 insertions(+), 69 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 8c2fc8a038b06..695f89d741c19 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -377,7 +377,7 @@ }, "CANNOT_PARSE_TIMESTAMP" : { "message" : [ - ". If necessary set to \"false\" to bypass this error." + ". Use `try_to_timestamp` to tolerate invalid input string and return NULL instead." ], "sqlState" : "22007" }, 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 86137fc1c3c0e..061eaf45cffb9 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 @@ -269,8 +269,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE new SparkDateTimeException( errorClass = "CANNOT_PARSE_TIMESTAMP", messageParameters = Map( - "message" -> e.getMessage, - "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), + "message" -> e.getMessage), context = Array.empty, summary = "") } diff --git a/sql/core/src/test/resources/sql-tests/results/date.sql.out b/sql/core/src/test/resources/sql-tests/results/date.sql.out index aa283d3249617..8b892b1795a17 100644 --- a/sql/core/src/test/resources/sql-tests/results/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date.sql.out @@ -207,7 +207,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 5635196efc2e5..9a40d4fd13169 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -207,7 +207,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"02-29\"" } } @@ -1585,7 +1584,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.\"" } } @@ -1601,7 +1599,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.0\"" } } @@ -1617,7 +1614,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.1\"" } } @@ -1633,7 +1629,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" } } @@ -1649,7 +1644,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.123UTC\"" } } @@ -1665,7 +1659,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" } } @@ -1681,7 +1674,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.12345CST\"" } } @@ -1697,7 +1689,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.123456PST\"" } } @@ -1713,7 +1704,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.1234567PST\"" } } @@ -1729,7 +1719,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"123456 2019-10-06 10:11:12.123456PST\"" } } @@ -1745,7 +1734,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"223456 2019-10-06 10:11:12.123456PST\"" } } @@ -1761,7 +1749,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" } } @@ -1777,7 +1764,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.123\"" } } @@ -1793,7 +1779,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12\"" } } @@ -1809,7 +1794,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" } } @@ -1825,7 +1809,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 10:11\"" } } @@ -1841,7 +1824,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06S10:11:12.12345\"" } } @@ -1857,7 +1839,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"12.12342019-10-06S10:11\"" } } @@ -1873,7 +1854,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" } } @@ -1889,7 +1869,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" } } @@ -1905,7 +1884,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"12.1234019-10-06S10:11\"" } } @@ -1977,7 +1955,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"02-29\"" } } @@ -2208,7 +2185,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Unparseable date: \"2019-10-06 A\"" } } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out index 0708a523900ff..9e3b0e2d1e0f5 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -18,7 +18,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '1' could not be parsed at index 0" } } @@ -34,7 +33,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '-12' could not be parsed at index 0" } } @@ -50,7 +48,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '123' could not be parsed, unparsed text found at index 2" } } @@ -66,7 +63,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '1' could not be parsed at index 0" } } @@ -99,7 +95,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" } } @@ -115,7 +110,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '9' could not be parsed at index 0" } } @@ -131,7 +125,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" } } @@ -147,7 +140,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '9' could not be parsed at index 0" } } @@ -163,7 +155,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '99' could not be parsed at index 0" } } @@ -179,7 +170,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." } } @@ -195,7 +185,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31." } } @@ -211,7 +200,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year" } } @@ -227,7 +215,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." } } @@ -243,7 +230,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30" } } @@ -259,7 +245,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15" } } @@ -275,7 +260,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'" } } @@ -299,7 +283,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -315,7 +298,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -331,7 +313,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -347,7 +328,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -363,7 +343,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -379,7 +358,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -395,7 +373,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -411,7 +388,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text 'Unparseable' could not be parsed at index 0" } } diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out index e3cf1a1549228..69025271601e9 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out @@ -395,7 +395,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" } } @@ -467,7 +466,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" } } @@ -491,7 +489,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" } } @@ -563,7 +560,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" } } @@ -579,7 +575,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" } } @@ -659,7 +654,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out index 681306ba9f405..14835ae0c5926 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out @@ -409,7 +409,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" } } @@ -481,7 +480,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" } } @@ -505,7 +503,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" } } @@ -577,7 +574,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" } } @@ -593,7 +589,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" } } @@ -673,7 +668,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index 781fff4835c51..0b043c70cc235 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -376,7 +376,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text 'aa' could not be parsed at index 0" } } @@ -409,7 +408,6 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", "message" : "Text 'aa' could not be parsed at index 0" } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala index a9ff7c308c153..52ef662080fb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala @@ -239,9 +239,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest sql("select to_timestamp('abc', 'yyyy-MM-dd HH:mm:ss')").collect() }, condition = "CANNOT_PARSE_TIMESTAMP", - parameters = Map( - "message" -> "Text 'abc' could not be parsed at index 0", - "ansiConfig" -> ansiConf) + parameters = Map("message" -> "Text 'abc' could not be parsed at index 0") ) } From 7f6d554a493330744113fa7934236d0dc5a90bc0 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 19 Dec 2024 09:03:20 +0900 Subject: [PATCH 215/438] [SPARK-49436][CONNECT][SQL] Common interface for SQLContext ### What changes were proposed in this pull request? This PR adds an abstraction for `SQLContext` in the `spark-api` package. Both sides (Classic and Connect) maintain their own implementation. ### Why are the changes needed? To unify the API interface and make `SQLContext` available to Spark Connect. ### Does this PR introduce _any_ user-facing change? Yes. Connect users are now able to call `sparkSession.sqlContext` and the APIs it provides. ### How was this patch tested? Not needed. All new methods are mirrored from SparkSession except `tables()`, which is covered by existing tests in `ListTablesSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48958 from xupefei/api-sqlcontext. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../resources/error/error-conditions.json | 5 - .../org/apache/spark/sql/SQLContext.scala | 336 ++++++ .../org/apache/spark/sql/SparkSession.scala | 3 +- .../ConnectClientUnsupportedErrors.scala | 3 - .../spark/sql/UnsupportedFeaturesSuite.scala | 4 - .../CheckConnectJvmClientCompatibility.scala | 7 +- .../org/apache/spark/sql/api/SQLContext.scala | 1022 ++++++++++++++++ .../apache/spark/sql/api/SparkSession.scala | 2 +- .../main/scala/org/apache/spark/shims.scala | 1 - .../org/apache/spark/sql/SQLContext.scala | 1063 ++++------------- .../apache/spark/sql/SQLContextSuite.scala | 23 + 11 files changed, 1592 insertions(+), 877 deletions(-) create mode 100644 connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLContext.scala create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/api/SQLContext.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 695f89d741c19..deb62866f072e 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5090,11 +5090,6 @@ "message" : [ "Access to the SparkContext." ] - }, - "SESSION_SQL_CONTEXT" : { - "message" : [ - "Access to the SparkSession SQL Context." - ] } }, "sqlState" : "0A000" diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLContext.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLContext.scala new file mode 100644 index 0000000000000..3603eb6ea508d --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -0,0 +1,336 @@ +/* + * 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 + +import java.util.{List => JList, Map => JMap, Properties} + +import scala.jdk.CollectionConverters.PropertiesHasAsScala +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Stable +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.connect.ConnectClientUnsupportedErrors +import org.apache.spark.sql.connect.ConnectConversions._ +import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.streaming.{DataStreamReader, StreamingQueryManager} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ExecutionListenerManager + +@Stable +class SQLContext private[sql] (override val sparkSession: SparkSession) + extends api.SQLContext(sparkSession) { + + /** @inheritdoc */ + def newSession(): SQLContext = sparkSession.newSession().sqlContext + + /** @inheritdoc */ + def listenerManager: ExecutionListenerManager = sparkSession.listenerManager + + /** @inheritdoc */ + def setConf(props: Properties): Unit = sparkSession.conf.synchronized { + props.asScala.foreach { case (k, v) => sparkSession.conf.set(k, v) } + } + + /** @inheritdoc */ + def experimental: ExperimentalMethods = sparkSession.experimental + + /** @inheritdoc */ + def udf: UDFRegistration = sparkSession.udf + + // scalastyle:off + // Disable style checker so "implicits" object can start with lowercase i + + /** @inheritdoc */ + object implicits extends SQLImplicits { + + /** @inheritdoc */ + override protected def session: SparkSession = sparkSession + } + + // scalastyle:on + + /** @inheritdoc */ + def read: DataFrameReader = sparkSession.read + + /** @inheritdoc */ + def readStream: DataStreamReader = sparkSession.readStream + + /** + * Returns a `StreamingQueryManager` that allows managing all the + * [[org.apache.spark.sql.streaming.StreamingQuery StreamingQueries]] active on `this` context. + * + * @since 4.0.0 + */ + def streams: StreamingQueryManager = sparkSession.streams + + /** @inheritdoc */ + override def sparkContext: SparkContext = { + throw ConnectClientUnsupportedErrors.sparkContext() + } + + /** @inheritdoc */ + override def emptyDataFrame: Dataset[Row] = super.emptyDataFrame + + /** @inheritdoc */ + override def createDataFrame[A <: Product: TypeTag](rdd: RDD[A]): Dataset[Row] = + super.createDataFrame(rdd) + + /** @inheritdoc */ + override def createDataFrame[A <: Product: TypeTag](data: Seq[A]): Dataset[Row] = + super.createDataFrame(data) + + /** @inheritdoc */ + override def baseRelationToDataFrame(baseRelation: BaseRelation): Dataset[Row] = + super.baseRelationToDataFrame(baseRelation) + + /** @inheritdoc */ + override def createDataFrame(rowRDD: RDD[Row], schema: StructType): Dataset[Row] = + super.createDataFrame(rowRDD, schema) + + /** @inheritdoc */ + override def createDataset[T: Encoder](data: Seq[T]): Dataset[T] = super.createDataset(data) + + /** @inheritdoc */ + override def createDataset[T: Encoder](data: RDD[T]): Dataset[T] = super.createDataset(data) + + /** @inheritdoc */ + override def createDataset[T: Encoder](data: JList[T]): Dataset[T] = + super.createDataset(data) + + /** @inheritdoc */ + override def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): Dataset[Row] = + super.createDataFrame(rowRDD, schema) + + /** @inheritdoc */ + override def createDataFrame(rows: JList[Row], schema: StructType): Dataset[Row] = + super.createDataFrame(rows, schema) + + /** @inheritdoc */ + override def createDataFrame(rdd: RDD[_], beanClass: Class[_]): Dataset[Row] = + super.createDataFrame(rdd, beanClass) + + /** @inheritdoc */ + override def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): Dataset[Row] = + super.createDataFrame(rdd, beanClass) + + /** @inheritdoc */ + override def createDataFrame(data: JList[_], beanClass: Class[_]): Dataset[Row] = + super.createDataFrame(data, beanClass) + + /** @inheritdoc */ + override def createExternalTable(tableName: String, path: String): Dataset[Row] = + super.createExternalTable(tableName, path) + + /** @inheritdoc */ + override def createExternalTable( + tableName: String, + path: String, + source: String): Dataset[Row] = { + super.createExternalTable(tableName, path, source) + } + + /** @inheritdoc */ + override def createExternalTable( + tableName: String, + source: String, + options: JMap[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, options) + } + + /** @inheritdoc */ + override def createExternalTable( + tableName: String, + source: String, + options: Map[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, options) + } + + /** @inheritdoc */ + override def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: JMap[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, schema, options) + } + + /** @inheritdoc */ + override def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: Map[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, schema, options) + } + + /** @inheritdoc */ + override def range(end: Long): Dataset[Row] = super.range(end) + + /** @inheritdoc */ + override def range(start: Long, end: Long): Dataset[Row] = super.range(start, end) + + /** @inheritdoc */ + override def range(start: Long, end: Long, step: Long): Dataset[Row] = + super.range(start, end, step) + + /** @inheritdoc */ + override def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[Row] = + super.range(start, end, step, numPartitions) + + /** @inheritdoc */ + override def sql(sqlText: String): Dataset[Row] = super.sql(sqlText) + + /** @inheritdoc */ + override def table(tableName: String): Dataset[Row] = super.table(tableName) + + /** @inheritdoc */ + override def tables(): Dataset[Row] = super.tables() + + /** @inheritdoc */ + override def tables(databaseName: String): Dataset[Row] = super.tables(databaseName) + + /** @inheritdoc */ + override def applySchema(rowRDD: RDD[Row], schema: StructType): Dataset[Row] = + super.applySchema(rowRDD, schema) + + /** @inheritdoc */ + override def applySchema(rowRDD: JavaRDD[Row], schema: StructType): Dataset[Row] = + super.applySchema(rowRDD, schema) + + /** @inheritdoc */ + override def applySchema(rdd: RDD[_], beanClass: Class[_]): Dataset[Row] = + super.applySchema(rdd, beanClass) + + /** @inheritdoc */ + override def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): Dataset[Row] = + super.applySchema(rdd, beanClass) + + /** @inheritdoc */ + @scala.annotation.varargs + override def parquetFile(paths: String*): Dataset[Row] = super.parquetFile(paths: _*) + + /** @inheritdoc */ + override def jsonFile(path: String): Dataset[Row] = super.jsonFile(path) + + /** @inheritdoc */ + override def jsonFile(path: String, schema: StructType): Dataset[Row] = + super.jsonFile(path, schema) + + /** @inheritdoc */ + override def jsonFile(path: String, samplingRatio: Double): Dataset[Row] = + super.jsonFile(path, samplingRatio) + + /** @inheritdoc */ + override def jsonRDD(json: RDD[String]): Dataset[Row] = super.jsonRDD(json) + + /** @inheritdoc */ + override def jsonRDD(json: JavaRDD[String]): Dataset[Row] = super.jsonRDD(json) + + /** @inheritdoc */ + override def jsonRDD(json: RDD[String], schema: StructType): Dataset[Row] = + super.jsonRDD(json, schema) + + /** @inheritdoc */ + override def jsonRDD(json: JavaRDD[String], schema: StructType): Dataset[Row] = + super.jsonRDD(json, schema) + + /** @inheritdoc */ + override def jsonRDD(json: RDD[String], samplingRatio: Double): Dataset[Row] = + super.jsonRDD(json, samplingRatio) + + /** @inheritdoc */ + override def jsonRDD(json: JavaRDD[String], samplingRatio: Double): Dataset[Row] = + super.jsonRDD(json, samplingRatio) + + /** @inheritdoc */ + override def load(path: String): Dataset[Row] = super.load(path) + + /** @inheritdoc */ + override def load(path: String, source: String): Dataset[Row] = super.load(path, source) + + /** @inheritdoc */ + override def load(source: String, options: JMap[String, String]): Dataset[Row] = + super.load(source, options) + + /** @inheritdoc */ + override def load(source: String, options: Map[String, String]): Dataset[Row] = + super.load(source, options) + + /** @inheritdoc */ + override def load( + source: String, + schema: StructType, + options: JMap[String, String]): Dataset[Row] = { + super.load(source, schema, options) + } + + /** @inheritdoc */ + override def load( + source: String, + schema: StructType, + options: Map[String, String]): Dataset[Row] = { + super.load(source, schema, options) + } + + /** @inheritdoc */ + override def jdbc(url: String, table: String): Dataset[Row] = super.jdbc(url, table) + + /** @inheritdoc */ + override def jdbc( + url: String, + table: String, + columnName: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): Dataset[Row] = { + super.jdbc(url, table, columnName, lowerBound, upperBound, numPartitions) + } + + /** @inheritdoc */ + override def jdbc(url: String, table: String, theParts: Array[String]): Dataset[Row] = { + super.jdbc(url, table, theParts) + } +} +object SQLContext extends api.SQLContextCompanion { + + override private[sql] type SQLContextImpl = SQLContext + override private[sql] type SparkContextImpl = SparkContext + + /** + * Get the singleton SQLContext if it exists or create a new one. + * + * This function can be used to create a singleton SQLContext object that can be shared across + * the JVM. + * + * If there is an active SQLContext for current thread, it will be returned instead of the + * global one. + * + * @param sparkContext + * The SparkContext. This parameter is not used in Spark Connect. + * + * @since 4.0.0 + */ + def getOrCreate(sparkContext: SparkContext): SQLContext = { + SparkSession.builder().getOrCreate().sqlContext + } + + /** @inheritdoc */ + override def setActive(sqlContext: SQLContext): Unit = super.setActive(sqlContext) +} diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 939a1341a8911..b6bba8251913f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -188,8 +188,7 @@ class SparkSession private[sql] ( throw ConnectClientUnsupportedErrors.sessionState() /** @inheritdoc */ - override def sqlContext: SQLContext = - throw ConnectClientUnsupportedErrors.sqlContext() + override val sqlContext: SQLContext = new SQLContext(this) /** @inheritdoc */ override def listenerManager: ExecutionListenerManager = diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/ConnectClientUnsupportedErrors.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/ConnectClientUnsupportedErrors.scala index e73bcb8a0059d..5783a20348d75 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/ConnectClientUnsupportedErrors.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/ConnectClientUnsupportedErrors.scala @@ -53,7 +53,4 @@ private[sql] object ConnectClientUnsupportedErrors { def sparkContext(): SparkUnsupportedOperationException = unsupportedFeatureException("SESSION_SPARK_CONTEXT") - - def sqlContext(): SparkUnsupportedOperationException = - unsupportedFeatureException("SESSION_SQL_CONTEXT") } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UnsupportedFeaturesSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UnsupportedFeaturesSuite.scala index 6a26cf581751d..42ae6987c9f36 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UnsupportedFeaturesSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UnsupportedFeaturesSuite.scala @@ -79,10 +79,6 @@ class UnsupportedFeaturesSuite extends ConnectFunSuite { _.listenerManager } - testUnsupportedFeature("SparkSession.sqlContext", "SESSION_SQL_CONTEXT") { - _.sqlContext - } - testUnsupportedFeature( "SparkSession.baseRelationToDataFrame", "SESSION_BASE_RELATION_TO_DATAFRAME") { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index b5ea973aa1d70..4ec84a4087eb7 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -176,8 +176,6 @@ object CheckConnectJvmClientCompatibility { // Skip unsupported classes ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ExperimentalMethods"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$*"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SparkSessionExtensions"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.SparkSessionExtensionsProvider"), @@ -331,6 +329,11 @@ object CheckConnectJvmClientCompatibility { ProblemFilters.exclude[DirectMissingMethodProblem]( "org.apache.spark.sql.SparkSession#Builder.interceptor"), + // Private case class in SQLContext + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$ListTableRow"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.SQLContext$ListTableRow$"), + // SQLImplicits ProblemFilters.exclude[Problem]("org.apache.spark.sql.SQLImplicits.session"), diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/SQLContext.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/SQLContext.scala new file mode 100644 index 0000000000000..50590fffa1521 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/SQLContext.scala @@ -0,0 +1,1022 @@ +/* + * 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.api + +import scala.collection.immutable +import scala.reflect.runtime.universe.TypeTag + +import _root_.java.util.{List => JList, Map => JMap, Properties} + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Encoder, Encoders, ExperimentalMethods, Row} +import org.apache.spark.sql.api.SQLImplicits +import org.apache.spark.sql.catalog.Table +import org.apache.spark.sql.functions.{array_size, coalesce, col, lit, when} +import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ExecutionListenerManager + +/** + * The entry point for working with structured data (rows and columns) in Spark 1.x. + * + * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class here + * for backward compatibility. + * + * @groupname basic Basic Operations + * @groupname ddl_ops Persistent Catalog DDL + * @groupname cachemgmt Cached Table Management + * @groupname genericdata Generic Data Sources + * @groupname specificdata Specific Data Sources + * @groupname config Configuration + * @groupname dataframes Custom DataFrame Creation + * @groupname dataset Custom Dataset Creation + * @groupname Ungrouped Support functions for language integrated queries + * @since 1.0.0 + */ +@Stable +abstract class SQLContext private[sql] (val sparkSession: SparkSession) + extends Logging + with Serializable { + + // Note: Since Spark 2.0 this class has become a wrapper of SparkSession, where the + // real functionality resides. This class remains mainly for backward compatibility. + + def sparkContext: SparkContext = sparkSession.sparkContext + + /** + * Returns a [[SQLContext]] as new session, with separated SQL configurations, temporary tables, + * registered functions, but sharing the same `SparkContext`, cached data and other things. + * + * @since 1.6.0 + */ + def newSession(): SQLContext + + /** + * An interface to register custom QueryExecutionListener that listen for execution metrics. + */ + def listenerManager: ExecutionListenerManager + + /** + * Set Spark SQL configuration properties. + * + * @group config + * @since 1.0.0 + */ + def setConf(props: Properties): Unit + + /** + * 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) + } + + /** + * 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) + } + + /** + * Return the value of Spark SQL configuration property for the given key. If the key is not set + * yet, return `defaultValue`. + * + * @group config + * @since 1.0.0 + */ + def getConf(key: String, defaultValue: String): String = { + sparkSession.conf.get(key, defaultValue) + } + + /** + * Return all the configuration properties that have been set (i.e. not the default). This + * creates a new copy of the config properties in the form of a Map. + * + * @group config + * @since 1.0.0 + */ + def getAllConfs: immutable.Map[String, String] = { + sparkSession.conf.getAll + } + + /** + * :: Experimental :: A collection of methods that are considered experimental, but can be used + * to hook into the query planner for advanced functionality. + * + * @group basic + * @since 1.3.0 + */ + @Experimental + @transient + @Unstable + def experimental: ExperimentalMethods + + /** + * Returns a `DataFrame` with no rows or columns. + * + * @group basic + * @since 1.3.0 + */ + def emptyDataFrame: Dataset[Row] = sparkSession.emptyDataFrame + + /** + * A collection of methods for registering user-defined functions (UDF). + * + * The following example registers a Scala closure as UDF: + * {{{ + * sqlContext.udf.register("myUDF", (arg1: Int, arg2: String) => arg2 + arg1) + * }}} + * + * The following example registers a UDF in Java: + * {{{ + * sqlContext.udf().register("myUDF", + * (Integer arg1, String arg2) -> arg2 + arg1, + * DataTypes.StringType); + * }}} + * + * @note + * The user-defined functions must be deterministic. Due to optimization, duplicate + * invocations may be eliminated or the function may even be invoked more times than it is + * present in the query. + * + * @group basic + * @since 1.3.0 + */ + def udf: UDFRegistration + + /** + * (Scala-specific) Implicit methods available in Scala for converting common Scala objects into + * `DataFrame`s. + * + * {{{ + * val sqlContext = new SQLContext(sc) + * import sqlContext.implicits._ + * }}} + * + * @group basic + * @since 1.3.0 + */ + val implicits: SQLImplicits + + /** + * Returns true if the table is currently cached in-memory. + * @group cachemgmt + * @since 1.3.0 + */ + def isCached(tableName: String): Boolean = { + sparkSession.catalog.isCached(tableName) + } + + /** + * Caches the specified table in-memory. + * @group cachemgmt + * @since 1.3.0 + */ + def cacheTable(tableName: String): Unit = { + sparkSession.catalog.cacheTable(tableName) + } + + /** + * Removes the specified table from the in-memory cache. + * @group cachemgmt + * @since 1.3.0 + */ + def uncacheTable(tableName: String): Unit = { + sparkSession.catalog.uncacheTable(tableName) + } + + /** + * Removes all cached tables from the in-memory cache. + * @since 1.3.0 + */ + def clearCache(): Unit = { + sparkSession.catalog.clearCache() + } + + /** + * 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]): Dataset[Row] = { + sparkSession.createDataFrame(rdd) + } + + /** + * Creates a DataFrame from a local Seq of Product. + * + * @group dataframes + * @since 1.3.0 + */ + def createDataFrame[A <: Product: TypeTag](data: Seq[A]): Dataset[Row] = { + sparkSession.createDataFrame(data) + } + + /** + * Convert a `BaseRelation` created for external data sources into a `DataFrame`. + * + * @group dataframes + * @since 1.3.0 + */ + def baseRelationToDataFrame(baseRelation: BaseRelation): Dataset[Row] = { + sparkSession.baseRelationToDataFrame(baseRelation) + } + + /** + * :: DeveloperApi :: Creates a `DataFrame` from an `RDD` containing + * [[org.apache.spark.sql.Row Row]]s using the given schema. It is important to make sure that + * the structure of every [[org.apache.spark.sql.Row Row]] of the provided RDD matches the + * provided schema. Otherwise, there will be runtime exception. Example: + * {{{ + * import org.apache.spark.sql._ + * import org.apache.spark.sql.types._ + * val sqlContext = new org.apache.spark.sql.SQLContext(sc) + * + * val schema = + * StructType( + * StructField("name", StringType, false) :: + * StructField("age", IntegerType, true) :: Nil) + * + * val people = + * sc.textFile("examples/src/main/resources/people.txt").map( + * _.split(",")).map(p => Row(p(0), p(1).trim.toInt)) + * val dataFrame = sqlContext.createDataFrame(people, schema) + * dataFrame.printSchema + * // root + * // |-- name: string (nullable = false) + * // |-- age: integer (nullable = true) + * + * dataFrame.createOrReplaceTempView("people") + * sqlContext.sql("select name from people").collect.foreach(println) + * }}} + * + * @group dataframes + * @since 1.3.0 + */ + @DeveloperApi + def createDataFrame(rowRDD: RDD[Row], schema: StructType): Dataset[Row] = { + sparkSession.createDataFrame(rowRDD, schema) + } + + /** + * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL + * representation) that is generally created automatically through implicits from a + * `SparkSession`, or can be created explicitly by calling static methods on + * [[org.apache.spark.sql.Encoders Encoders]]. + * + * ==Example== + * + * {{{ + * + * import spark.implicits._ + * case class Person(name: String, age: Long) + * val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19)) + * val ds = spark.createDataset(data) + * + * ds.show() + * // +-------+---+ + * // | name|age| + * // +-------+---+ + * // |Michael| 29| + * // | Andy| 30| + * // | Justin| 19| + * // +-------+---+ + * }}} + * + * @since 2.0.0 + * @group dataset + */ + def createDataset[T: Encoder](data: Seq[T]): Dataset[T] = { + sparkSession.createDataset(data) + } + + /** + * Creates a [[Dataset]] from an RDD of a given type. This method requires an encoder (to + * convert a JVM object of type `T` to and from the internal Spark SQL representation) that is + * generally created automatically through implicits from a `SparkSession`, or can be created + * explicitly by calling static methods on [[org.apache.spark.sql.Encoders Encoders]]. + * + * @since 2.0.0 + * @group dataset + */ + def createDataset[T: Encoder](data: RDD[T]): Dataset[T] = { + sparkSession.createDataset(data) + } + + /** + * Creates a [[Dataset]] from a `JList` of a given type. This method requires an encoder (to + * convert a JVM object of type `T` to and from the internal Spark SQL representation) that is + * generally created automatically through implicits from a `SparkSession`, or can be created + * explicitly by calling static methods on [[org.apache.spark.sql.Encoders Encoders]]. + * + * ==Java Example== + * + * {{{ + * List data = Arrays.asList("hello", "world"); + * Dataset ds = spark.createDataset(data, Encoders.STRING()); + * }}} + * + * @since 2.0.0 + * @group dataset + */ + def createDataset[T: Encoder](data: JList[T]): Dataset[T] = { + sparkSession.createDataset(data) + } + + /** + * :: DeveloperApi :: Creates a `DataFrame` from a `JavaRDD` containing + * [[org.apache.spark.sql.Row Row]]s using the given schema. It is important to make sure that + * the structure of every [[org.apache.spark.sql.Row Row]] of the provided RDD matches the + * provided schema. Otherwise, there will be runtime exception. + * + * @group dataframes + * @since 1.3.0 + */ + @DeveloperApi + def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): Dataset[Row] = { + sparkSession.createDataFrame(rowRDD, schema) + } + + /** + * :: DeveloperApi :: Creates a `DataFrame` from a `JList` containing + * [[org.apache.spark.sql.Row Row]]s using the given schema. It is important to make sure that + * the structure of every [[org.apache.spark.sql.Row Row]] of the provided List matches the + * provided schema. Otherwise, there will be runtime exception. + * + * @group dataframes + * @since 1.6.0 + */ + @DeveloperApi + def createDataFrame(rows: JList[Row], schema: StructType): Dataset[Row] = { + sparkSession.createDataFrame(rows, schema) + } + + /** + * Applies a schema to an RDD of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, SELECT * queries + * will return the columns in an undefined order. + * @group dataframes + * @since 1.3.0 + */ + def createDataFrame(rdd: RDD[_], beanClass: Class[_]): Dataset[Row] = { + sparkSession.createDataFrame(rdd, beanClass) + } + + /** + * Applies a schema to an RDD of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, SELECT * queries + * will return the columns in an undefined order. + * @group dataframes + * @since 1.3.0 + */ + def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): Dataset[Row] = { + sparkSession.createDataFrame(rdd, beanClass) + } + + /** + * Applies a schema to a List of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, SELECT * queries + * will return the columns in an undefined order. + * @group dataframes + * @since 1.6.0 + */ + def createDataFrame(data: JList[_], beanClass: Class[_]): Dataset[Row] = { + sparkSession.createDataFrame(data, beanClass) + } + + /** + * 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 + + /** + * Returns a `DataStreamReader` that can be used to read streaming data in as a `DataFrame`. + * {{{ + * sparkSession.readStream.parquet("/path/to/directory/of/parquet/files") + * sparkSession.readStream.schema(schema).json("/path/to/directory/of/json/files") + * }}} + * + * @since 2.0.0 + */ + def readStream: DataStreamReader + + /** + * Creates an external table from the given path and returns the corresponding DataFrame. It + * will use the default data source configured by spark.sql.sources.default. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable(tableName: String, path: String): Dataset[Row] = { + sparkSession.catalog.createTable(tableName, path) + } + + /** + * Creates an external table from the given path based on a data source and returns the + * corresponding DataFrame. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable(tableName: String, path: String, source: String): Dataset[Row] = { + sparkSession.catalog.createTable(tableName, path, source) + } + + /** + * Creates an external table from the given path based on a data source and a set of options. + * Then, returns the corresponding DataFrame. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + options: JMap[String, String]): Dataset[Row] = { + sparkSession.catalog.createTable(tableName, source, options) + } + + /** + * (Scala-specific) Creates an external table from the given path based on a data source and a + * set of options. Then, returns the corresponding DataFrame. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + options: Map[String, String]): Dataset[Row] = { + sparkSession.catalog.createTable(tableName, source, options) + } + + /** + * Create an external table from the given path based on a data source, a schema and a set of + * options. Then, returns the corresponding DataFrame. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: JMap[String, String]): Dataset[Row] = { + sparkSession.catalog.createTable(tableName, source, schema, options) + } + + /** + * (Scala-specific) Create an external table from the given path based on a data source, a + * schema and a set of options. Then, returns the corresponding DataFrame. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: Map[String, String]): Dataset[Row] = { + sparkSession.catalog.createTable(tableName, source, schema, options) + } + + /** + * Drops the temporary table with the given table name in the catalog. If the table has been + * cached/persisted before, it's also unpersisted. + * + * @param tableName + * the name of the table to be unregistered. + * @group basic + * @since 1.3.0 + */ + def dropTempTable(tableName: String): Unit = { + sparkSession.catalog.dropTempView(tableName) + } + + /** + * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements in a + * range from 0 to `end` (exclusive) with step value 1. + * + * @since 1.4.1 + * @group dataframe + */ + def range(end: Long): Dataset[Row] = sparkSession.range(end).toDF() + + /** + * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements in a + * range from `start` to `end` (exclusive) with step value 1. + * + * @since 1.4.0 + * @group dataframe + */ + def range(start: Long, end: Long): Dataset[Row] = sparkSession.range(start, end).toDF() + + /** + * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements in a + * range from `start` to `end` (exclusive) with a step value. + * + * @since 2.0.0 + * @group dataframe + */ + def range(start: Long, end: Long, step: Long): Dataset[Row] = { + sparkSession.range(start, end, step).toDF() + } + + /** + * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements in an + * range from `start` to `end` (exclusive) with an step value, with partition number specified. + * + * @since 1.4.0 + * @group dataframe + */ + def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[Row] = { + sparkSession.range(start, end, step, numPartitions).toDF() + } + + /** + * Executes a SQL query using Spark, returning the result as a `DataFrame`. This API eagerly + * runs DDL/DML commands, but not for SELECT queries. + * + * @group basic + * @since 1.3.0 + */ + def sql(sqlText: String): Dataset[Row] = sparkSession.sql(sqlText) + + /** + * Returns the specified table as a `DataFrame`. + * + * @group ddl_ops + * @since 1.3.0 + */ + def table(tableName: String): Dataset[Row] = { + sparkSession.table(tableName) + } + + /** + * Returns a `DataFrame` containing names of existing tables in the current database. The + * returned DataFrame has three columns, database, tableName and isTemporary (a Boolean + * indicating if a table is a temporary one or not). + * + * @group ddl_ops + * @since 1.3.0 + */ + def tables(): Dataset[Row] = { + mapTableDatasetOutput(sparkSession.catalog.listTables()) + } + + /** + * Returns a `DataFrame` containing names of existing tables in the given database. The returned + * DataFrame has three columns, database, tableName and isTemporary (a Boolean indicating if a + * table is a temporary one or not). + * + * @group ddl_ops + * @since 1.3.0 + */ + def tables(databaseName: String): Dataset[Row] = { + mapTableDatasetOutput(sparkSession.catalog.listTables(databaseName)) + } + + private def mapTableDatasetOutput(tables: Dataset[Table]): Dataset[Row] = { + tables + .select( + // Re-implement `org.apache.spark.sql.catalog.Table.database` method. + // Abusing `coalesce` to tell Spark all these columns are not nullable. + when( + coalesce(array_size(col("namespace")), lit(0)).equalTo(lit(1)), + coalesce(col("namespace")(0), lit(""))) + .otherwise(lit("")) + .as("namespace"), + coalesce(col("name"), lit("")).as("tableName"), + col("isTemporary")) + } + + /** + * Returns a `StreamingQueryManager` that allows managing all the + * [[org.apache.spark.sql.api.StreamingQuery StreamingQueries]] active on `this` context. + * + * @since 2.0.0 + */ + def streams: StreamingQueryManager + + /** + * Returns the names of tables in the current database as an array. + * + * @group ddl_ops + * @since 1.3.0 + */ + def tableNames(): Array[String] = { + tableNames(sparkSession.catalog.currentDatabase) + } + + /** + * Returns the names of tables in the given database as an array. + * + * @group ddl_ops + * @since 1.3.0 + */ + def tableNames(databaseName: String): Array[String] = { + sparkSession.catalog + .listTables(databaseName) + .select(col("name")) + .as(Encoders.STRING) + .collect() + } + + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + // Deprecated methods + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + + /** + * @deprecated + * As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: RDD[Row], schema: StructType): Dataset[Row] = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated + * As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): Dataset[Row] = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated + * As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: RDD[_], beanClass: Class[_]): Dataset[Row] = { + createDataFrame(rdd, beanClass) + } + + /** + * @deprecated + * As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): Dataset[Row] = { + createDataFrame(rdd, beanClass) + } + + /** + * Loads a Parquet file, returning the result as a `DataFrame`. This function returns an empty + * `DataFrame` if no paths are passed in. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().parquet()`. + */ + @deprecated("Use read.parquet() instead.", "1.4.0") + @scala.annotation.varargs + def parquetFile(paths: String*): Dataset[Row] = { + if (paths.isEmpty) { + emptyDataFrame + } else { + read.parquet(paths: _*) + } + } + + /** + * Loads a JSON file (one object per line), returning the result as a `DataFrame`. It goes + * through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String): Dataset[Row] = { + read.json(path) + } + + /** + * Loads a JSON file (one object per line) and applies the given schema, returning the result as + * a `DataFrame`. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, schema: StructType): Dataset[Row] = { + read.schema(schema).json(path) + } + + /** + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, samplingRatio: Double): Dataset[Row] = { + read.option("samplingRatio", samplingRatio.toString).json(path) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * `DataFrame`. It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String]): Dataset[Row] = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * `DataFrame`. It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String]): Dataset[Row] = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], schema: StructType): Dataset[Row] = { + read.schema(schema).json(json) + } + + /** + * Loads an JavaRDD[String] storing JSON objects (one object per record) and applies the given + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], schema: StructType): Dataset[Row] = { + read.schema(schema).json(json) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record) inferring the schema, + * returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], samplingRatio: Double): Dataset[Row] = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Loads a JavaRDD[String] storing JSON objects (one object per record) inferring the schema, + * returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], samplingRatio: Double): Dataset[Row] = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Returns the dataset stored at path as a DataFrame, using the default data source configured + * by spark.sql.sources.default. + * + * @group genericdata + * @deprecated + * As of 1.4.0, replaced by `read().load(path)`. + */ + @deprecated("Use read.load(path) instead.", "1.4.0") + def load(path: String): Dataset[Row] = { + read.load(path) + } + + /** + * Returns the dataset stored at path as a DataFrame, using the given data source. + * + * @group genericdata + * @deprecated + * As of 1.4.0, replaced by `read().format(source).load(path)`. + */ + @deprecated("Use read.format(source).load(path) instead.", "1.4.0") + def load(path: String, source: String): Dataset[Row] = { + read.format(source).load(path) + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and a set of options + * as a DataFrame. + * + * @group genericdata + * @deprecated + * As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: JMap[String, String]): Dataset[Row] = { + read.options(options).format(source).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and a set of options + * as a DataFrame. + * + * @group genericdata + * @deprecated + * As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: Map[String, String]): Dataset[Row] = { + read.options(options).format(source).load() + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and a set of options + * as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated + * As of 1.4.0, replaced by `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load(source: String, schema: StructType, options: JMap[String, String]): Dataset[Row] = { + read.format(source).schema(schema).options(options).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and a set of options + * as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated + * As of 1.4.0, replaced by `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load(source: String, schema: StructType, options: Map[String, String]): Dataset[Row] = { + read.format(source).schema(schema).options(options).load() + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL url named + * table. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String): Dataset[Row] = { + read.jdbc(url, table, new Properties) + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL url named + * table. Partitions of the table will be retrieved in parallel based on the parameters passed + * to this function. + * + * @param columnName + * the name of a column of integral type that will be used for partitioning. + * @param lowerBound + * the minimum value of `columnName` used to decide partition stride + * @param upperBound + * the maximum value of `columnName` used to decide partition stride + * @param numPartitions + * the number of partitions. the range `minValue`-`maxValue` will be split evenly into this + * many partitions + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc( + url: String, + table: String, + columnName: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): Dataset[Row] = { + read.jdbc(url, table, columnName, lowerBound, upperBound, numPartitions, new Properties) + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL url named + * table. The theParts parameter gives a list expressions suitable for inclusion in WHERE + * clauses; each one defines one partition of the `DataFrame`. + * + * @group specificdata + * @deprecated + * As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String, theParts: Array[String]): Dataset[Row] = { + read.jdbc(url, table, theParts, new Properties) + } +} + +/** + * This SQLContext object contains utility functions to create a singleton SQLContext instance, or + * to get the created SQLContext instance. + * + * It also provides utility functions to support preference for threads in multiple sessions + * scenario, setActive could set a SQLContext for current thread, which will be returned by + * getOrCreate instead of the global one. + */ +trait SQLContextCompanion { + private[sql] type SQLContextImpl <: SQLContext + private[sql] type SparkContextImpl <: SparkContext + + /** + * Get the singleton SQLContext if it exists or create a new one using the given SparkContext. + * + * This function can be used to create a singleton SQLContext object that can be shared across + * the JVM. + * + * If there is an active SQLContext for current thread, it will be returned instead of the + * global one. + * + * @since 1.5.0 + */ + @deprecated("Use SparkSession.builder instead", "2.0.0") + def getOrCreate(sparkContext: SparkContextImpl): SQLContextImpl + + /** + * Changes the SQLContext that will be returned in this thread and its children when + * SQLContext.getOrCreate() is called. This can be used to ensure that a given thread receives a + * SQLContext with an isolated session, instead of the global (first created) context. + * + * @since 1.6.0 + */ + @deprecated("Use SparkSession.setActiveSession instead", "2.0.0") + def setActive(sqlContext: SQLContextImpl): Unit = { + SparkSession.setActiveSession(sqlContext.sparkSession) + } + + /** + * Clears the active SQLContext for current thread. Subsequent calls to getOrCreate will return + * the first created context instead of a thread-local override. + * + * @since 1.6.0 + */ + @deprecated("Use SparkSession.clearActiveSession instead", "2.0.0") + def clearActive(): Unit = { + SparkSession.clearActiveSession() + } +} diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala index 35f74497b96f4..af2144cb9eb41 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala @@ -30,7 +30,7 @@ import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Encoder, ExperimentalMethods, Row, RuntimeConfig, SparkSessionExtensions, SQLContext} +import org.apache.spark.sql.{Encoder, ExperimentalMethods, Row, RuntimeConfig, SparkSessionExtensions} import org.apache.spark.sql.internal.{SessionState, SharedState} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType diff --git a/sql/connect/shims/src/main/scala/org/apache/spark/shims.scala b/sql/connect/shims/src/main/scala/org/apache/spark/shims.scala index ad8771a03b287..9c5fb515580a7 100644 --- a/sql/connect/shims/src/main/scala/org/apache/spark/shims.scala +++ b/sql/connect/shims/src/main/scala/org/apache/spark/shims.scala @@ -32,7 +32,6 @@ package rdd { package sql { class ExperimentalMethods class SparkSessionExtensions - class SQLContext package execution { class QueryExecution diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 636899a7acb06..1318563f8c93b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -17,21 +17,18 @@ package org.apache.spark.sql -import java.util.Properties +import java.util.{List => JList, Map => JMap, Properties} -import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} -import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.analysis.{CurrentNamespace, UnresolvedNamespace} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.ShowTables +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.streaming.{DataStreamReader, StreamingQueryManager} @@ -41,8 +38,8 @@ import org.apache.spark.sql.util.ExecutionListenerManager /** * The entry point for working with structured data (rows and columns) in Spark 1.x. * - * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class - * here for backward compatibility. + * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class here + * for backward compatibility. * * @groupname basic Basic Operations * @groupname ddl_ops Persistent Catalog DDL @@ -56,8 +53,8 @@ import org.apache.spark.sql.util.ExecutionListenerManager * @since 1.0.0 */ @Stable -class SQLContext private[sql](val sparkSession: SparkSession) - extends Logging with Serializable { +class SQLContext private[sql] (override val sparkSession: SparkSession) + extends api.SQLContext(sparkSession) { self => @@ -77,980 +74,325 @@ class SQLContext private[sql](val sparkSession: SparkSession) // TODO: move this logic into SparkSession private[sql] def sessionState: SessionState = sparkSession.sessionState + private[sql] def sharedState: SharedState = sparkSession.sharedState + @deprecated("Use SparkSession.sessionState.conf instead", "4.0.0") private[sql] def conf: SQLConf = sessionState.conf - def sparkContext: SparkContext = sparkSession.sparkContext - - /** - * Returns a [[SQLContext]] as new session, with separated SQL configurations, temporary - * tables, registered functions, but sharing the same `SparkContext`, cached data and - * other things. - * - * @since 1.6.0 - */ - def newSession(): SQLContext = sparkSession.newSession().sqlContext - - /** - * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s - * that listen for execution metrics. - */ + /** @inheritdoc */ def listenerManager: ExecutionListenerManager = sparkSession.listenerManager - /** - * Set Spark SQL configuration properties. - * - * @group config - * @since 1.0.0 - */ + /** @inheritdoc */ def setConf(props: Properties): Unit = { sessionState.conf.setConf(props) } - /** - * Set the given Spark SQL configuration property. - */ private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = { sessionState.conf.setConf(entry, value) } - /** - * 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) - } - - /** - * 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) - } - - /** - * Return the value of Spark SQL configuration property for the given key. If the key is not set - * yet, return `defaultValue`. - * - * @group config - * @since 1.0.0 - */ - def getConf(key: String, defaultValue: String): String = { - sparkSession.conf.get(key, defaultValue) - } - - /** - * Return all the configuration properties that have been set (i.e. not the default). - * This creates a new copy of the config properties in the form of a Map. - * - * @group config - * @since 1.0.0 - */ - def getAllConfs: immutable.Map[String, String] = { - sparkSession.conf.getAll - } - - /** - * :: Experimental :: - * A collection of methods that are considered experimental, but can be used to hook into - * the query planner for advanced functionality. - * - * @group basic - * @since 1.3.0 - */ + /** @inheritdoc */ @Experimental @transient @Unstable def experimental: ExperimentalMethods = sparkSession.experimental - /** - * Returns a `DataFrame` with no rows or columns. - * - * @group basic - * @since 1.3.0 - */ - def emptyDataFrame: DataFrame = sparkSession.emptyDataFrame - - /** - * A collection of methods for registering user-defined functions (UDF). - * - * The following example registers a Scala closure as UDF: - * {{{ - * sqlContext.udf.register("myUDF", (arg1: Int, arg2: String) => arg2 + arg1) - * }}} - * - * The following example registers a UDF in Java: - * {{{ - * sqlContext.udf().register("myUDF", - * (Integer arg1, String arg2) -> arg2 + arg1, - * DataTypes.StringType); - * }}} - * - * @note The user-defined functions must be deterministic. Due to optimization, - * duplicate invocations may be eliminated or the function may even be invoked more times than - * it is present in the query. - * - * @group basic - * @since 1.3.0 - */ + /** @inheritdoc */ def udf: UDFRegistration = sparkSession.udf - /** - * Returns true if the table is currently cached in-memory. - * @group cachemgmt - * @since 1.3.0 - */ - def isCached(tableName: String): Boolean = { - sparkSession.catalog.isCached(tableName) - } - - /** - * Caches the specified table in-memory. - * @group cachemgmt - * @since 1.3.0 - */ - def cacheTable(tableName: String): Unit = { - sparkSession.catalog.cacheTable(tableName) - } - - /** - * Removes the specified table from the in-memory cache. - * @group cachemgmt - * @since 1.3.0 - */ - def uncacheTable(tableName: String): Unit = { - sparkSession.catalog.uncacheTable(tableName) - } - - /** - * Removes all cached tables from the in-memory cache. - * @since 1.3.0 - */ - def clearCache(): Unit = { - sparkSession.catalog.clearCache() - } - // scalastyle:off // Disable style checker so "implicits" object can start with lowercase i - /** - * (Scala-specific) Implicit methods available in Scala for converting - * common Scala objects into `DataFrame`s. - * - * {{{ - * val sqlContext = new SQLContext(sc) - * import sqlContext.implicits._ - * }}} - * - * @group basic - * @since 1.3.0 - */ + + /** @inheritdoc */ object implicits extends SQLImplicits { + /** @inheritdoc */ override protected def session: SparkSession = sparkSession } + // scalastyle:on /** - * Creates a DataFrame from an RDD of Product (e.g. case classes, tuples). - * - * @group dataframes - * @since 1.3.0 + * Creates a DataFrame from an RDD[Row]. User can specify whether the input rows should be + * converted to Catalyst rows. */ - def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { - sparkSession.createDataFrame(rdd) + private[sql] def internalCreateDataFrame( + catalystRows: RDD[InternalRow], + schema: StructType, + isStreaming: Boolean = false): DataFrame = { + sparkSession.internalCreateDataFrame(catalystRows, schema, isStreaming) } - /** - * Creates a DataFrame from a local Seq of Product. - * - * @group dataframes - * @since 1.3.0 - */ - def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { - sparkSession.createDataFrame(data) - } + /** @inheritdoc */ + def read: DataFrameReader = sparkSession.read - /** - * Convert a `BaseRelation` created for external data sources into a `DataFrame`. - * - * @group dataframes - * @since 1.3.0 - */ - def baseRelationToDataFrame(baseRelation: BaseRelation): DataFrame = { - sparkSession.baseRelationToDataFrame(baseRelation) - } + /** @inheritdoc */ + def readStream: DataStreamReader = sparkSession.readStream /** - * :: DeveloperApi :: - * Creates a `DataFrame` from an `RDD` containing [[Row]]s using the given schema. - * It is important to make sure that the structure of every [[Row]] of the provided RDD matches - * the provided schema. Otherwise, there will be runtime exception. - * Example: - * {{{ - * import org.apache.spark.sql._ - * import org.apache.spark.sql.types._ - * val sqlContext = new org.apache.spark.sql.SQLContext(sc) - * - * val schema = - * StructType( - * StructField("name", StringType, false) :: - * StructField("age", IntegerType, true) :: Nil) - * - * val people = - * sc.textFile("examples/src/main/resources/people.txt").map( - * _.split(",")).map(p => Row(p(0), p(1).trim.toInt)) - * val dataFrame = sqlContext.createDataFrame(people, schema) - * dataFrame.printSchema - * // root - * // |-- name: string (nullable = false) - * // |-- age: integer (nullable = true) - * - * dataFrame.createOrReplaceTempView("people") - * sqlContext.sql("select name from people").collect.foreach(println) - * }}} - * - * @group dataframes - * @since 1.3.0 + * Registers the given `DataFrame` as a temporary table in the catalog. Temporary tables exist + * only during the lifetime of this instance of SQLContext. */ - @DeveloperApi - def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { - sparkSession.createDataFrame(rowRDD, schema) + private[sql] def registerDataFrameAsTable(df: DataFrame, tableName: String): Unit = { + df.createOrReplaceTempView(tableName) } /** - * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an - * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) - * that is generally created automatically through implicits from a `SparkSession`, or can be - * created explicitly by calling static methods on [[Encoders]]. - * - * == Example == - * - * {{{ - * - * import spark.implicits._ - * case class Person(name: String, age: Long) - * val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19)) - * val ds = spark.createDataset(data) - * - * ds.show() - * // +-------+---+ - * // | name|age| - * // +-------+---+ - * // |Michael| 29| - * // | Andy| 30| - * // | Justin| 19| - * // +-------+---+ - * }}} + * Returns a `StreamingQueryManager` that allows managing all the + * [[org.apache.spark.sql.streaming.StreamingQuery StreamingQueries]] active on `this` context. * * @since 2.0.0 - * @group dataset */ - def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { - sparkSession.createDataset(data) - } + def streams: StreamingQueryManager = sparkSession.streams - /** - * Creates a [[Dataset]] from an RDD of a given type. This method requires an - * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) - * that is generally created automatically through implicits from a `SparkSession`, or can be - * created explicitly by calling static methods on [[Encoders]]. - * - * @since 2.0.0 - * @group dataset - */ - def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { - sparkSession.createDataset(data) - } + /** @inheritdoc */ + override def sparkContext: SparkContext = super.sparkContext - /** - * Creates a [[Dataset]] from a `java.util.List` of a given type. This method requires an - * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) - * that is generally created automatically through implicits from a `SparkSession`, or can be - * created explicitly by calling static methods on [[Encoders]]. - * - * == Java Example == - * - * {{{ - * List data = Arrays.asList("hello", "world"); - * Dataset ds = spark.createDataset(data, Encoders.STRING()); - * }}} - * - * @since 2.0.0 - * @group dataset - */ - def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { - sparkSession.createDataset(data) - } + /** @inheritdoc */ + override def newSession(): SQLContext = sparkSession.newSession().sqlContext - /** - * Creates a DataFrame from an RDD[Row]. User can specify whether the input rows should be - * converted to Catalyst rows. - */ - private[sql] - def internalCreateDataFrame( - catalystRows: RDD[InternalRow], - schema: StructType, - isStreaming: Boolean = false) = { - sparkSession.internalCreateDataFrame(catalystRows, schema, isStreaming) - } + /** @inheritdoc */ + override def emptyDataFrame: Dataset[Row] = super.emptyDataFrame - /** - * :: DeveloperApi :: - * Creates a `DataFrame` from a `JavaRDD` containing [[Row]]s using the given schema. - * It is important to make sure that the structure of every [[Row]] of the provided RDD matches - * the provided schema. Otherwise, there will be runtime exception. - * - * @group dataframes - * @since 1.3.0 - */ - @DeveloperApi - def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { - sparkSession.createDataFrame(rowRDD, schema) - } + /** @inheritdoc */ + override def createDataFrame[A <: Product: TypeTag](rdd: RDD[A]): Dataset[Row] = + super.createDataFrame(rdd) - /** - * :: DeveloperApi :: - * Creates a `DataFrame` from a `java.util.List` containing [[Row]]s using the given schema. - * It is important to make sure that the structure of every [[Row]] of the provided List matches - * the provided schema. Otherwise, there will be runtime exception. - * - * @group dataframes - * @since 1.6.0 - */ + /** @inheritdoc */ + override def createDataFrame[A <: Product: TypeTag](data: Seq[A]): Dataset[Row] = + super.createDataFrame(data) + + /** @inheritdoc */ + override def baseRelationToDataFrame(baseRelation: BaseRelation): Dataset[Row] = + super.baseRelationToDataFrame(baseRelation) + + /** @inheritdoc */ @DeveloperApi - def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { - sparkSession.createDataFrame(rows, schema) - } + override def createDataFrame(rowRDD: RDD[Row], schema: StructType): Dataset[Row] = + super.createDataFrame(rowRDD, schema) - /** - * Applies a schema to an RDD of Java Beans. - * - * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, - * SELECT * queries will return the columns in an undefined order. - * @group dataframes - * @since 1.3.0 - */ - def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame = { - sparkSession.createDataFrame(rdd, beanClass) - } + /** @inheritdoc */ + override def createDataset[T: Encoder](data: Seq[T]): Dataset[T] = super.createDataset(data) - /** - * Applies a schema to an RDD of Java Beans. - * - * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, - * SELECT * queries will return the columns in an undefined order. - * @group dataframes - * @since 1.3.0 - */ - def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { - sparkSession.createDataFrame(rdd, beanClass) - } + /** @inheritdoc */ + override def createDataset[T: Encoder](data: RDD[T]): Dataset[T] = super.createDataset(data) - /** - * Applies a schema to a List of Java Beans. - * - * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, - * SELECT * queries will return the columns in an undefined order. - * @group dataframes - * @since 1.6.0 - */ - def createDataFrame(data: java.util.List[_], beanClass: Class[_]): DataFrame = { - sparkSession.createDataFrame(data, beanClass) - } + /** @inheritdoc */ + override def createDataset[T: Encoder](data: JList[T]): Dataset[T] = + super.createDataset(data) - /** - * 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 + /** @inheritdoc */ + @DeveloperApi + override def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): Dataset[Row] = + super.createDataFrame(rowRDD, schema) + /** @inheritdoc */ + @DeveloperApi + override def createDataFrame(rows: JList[Row], schema: StructType): Dataset[Row] = + super.createDataFrame(rows, schema) - /** - * Returns a `DataStreamReader` that can be used to read streaming data in as a `DataFrame`. - * {{{ - * sparkSession.readStream.parquet("/path/to/directory/of/parquet/files") - * sparkSession.readStream.schema(schema).json("/path/to/directory/of/json/files") - * }}} - * - * @since 2.0.0 - */ - def readStream: DataStreamReader = sparkSession.readStream + /** @inheritdoc */ + override def createDataFrame(rdd: RDD[_], beanClass: Class[_]): Dataset[Row] = + super.createDataFrame(rdd, beanClass) + /** @inheritdoc */ + override def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): Dataset[Row] = + super.createDataFrame(rdd, beanClass) - /** - * Creates an external table from the given path and returns the corresponding DataFrame. - * It will use the default data source configured by spark.sql.sources.default. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable(tableName: String, path: String): DataFrame = { - sparkSession.catalog.createTable(tableName, path) - } + /** @inheritdoc */ + override def createDataFrame(data: JList[_], beanClass: Class[_]): Dataset[Row] = + super.createDataFrame(data, beanClass) - /** - * Creates an external table from the given path based on a data source - * and returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( + /** @inheritdoc */ + override def createExternalTable(tableName: String, path: String): Dataset[Row] = + super.createExternalTable(tableName, path) + + /** @inheritdoc */ + override def createExternalTable( tableName: String, path: String, - source: String): DataFrame = { - sparkSession.catalog.createTable(tableName, path, source) + source: String): Dataset[Row] = { + super.createExternalTable(tableName, path, source) } - /** - * Creates an external table from the given path based on a data source and a set of options. - * Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( + /** @inheritdoc */ + override def createExternalTable( tableName: String, source: String, - options: java.util.Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, options) + options: JMap[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, options) } - /** - * (Scala-specific) - * Creates an external table from the given path based on a data source and a set of options. - * Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( + /** @inheritdoc */ + override def createExternalTable( tableName: String, source: String, - options: Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, options) + options: Map[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, options) } - /** - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( + /** @inheritdoc */ + override def createExternalTable( tableName: String, source: String, schema: StructType, - options: java.util.Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, schema, options) + options: JMap[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, schema, options) } - /** - * (Scala-specific) - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( + /** @inheritdoc */ + override def createExternalTable( tableName: String, source: String, schema: StructType, - options: Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, schema, options) - } - - /** - * Registers the given `DataFrame` as a temporary table in the catalog. Temporary tables exist - * only during the lifetime of this instance of SQLContext. - */ - private[sql] def registerDataFrameAsTable(df: DataFrame, tableName: String): Unit = { - df.createOrReplaceTempView(tableName) + options: Map[String, String]): Dataset[Row] = { + super.createExternalTable(tableName, source, schema, options) } - /** - * Drops the temporary table with the given table name in the catalog. If the table has been - * cached/persisted before, it's also unpersisted. - * - * @param tableName the name of the table to be unregistered. - * @group basic - * @since 1.3.0 - */ - def dropTempTable(tableName: String): Unit = { - sparkSession.catalog.dropTempView(tableName) - } + /** @inheritdoc */ + override def range(end: Long): Dataset[Row] = super.range(end) - /** - * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements - * in a range from 0 to `end` (exclusive) with step value 1. - * - * @since 1.4.1 - * @group dataframe - */ - def range(end: Long): DataFrame = sparkSession.range(end).toDF() - - /** - * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements - * in a range from `start` to `end` (exclusive) with step value 1. - * - * @since 1.4.0 - * @group dataframe - */ - def range(start: Long, end: Long): DataFrame = sparkSession.range(start, end).toDF() - - /** - * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements - * in a range from `start` to `end` (exclusive) with a step value. - * - * @since 2.0.0 - * @group dataframe - */ - def range(start: Long, end: Long, step: Long): DataFrame = { - sparkSession.range(start, end, step).toDF() - } - - /** - * Creates a `DataFrame` with a single `LongType` column named `id`, containing elements - * in an range from `start` to `end` (exclusive) with an step value, with partition number - * specified. - * - * @since 1.4.0 - * @group dataframe - */ - def range(start: Long, end: Long, step: Long, numPartitions: Int): DataFrame = { - sparkSession.range(start, end, step, numPartitions).toDF() - } - - /** - * Executes a SQL query using Spark, returning the result as a `DataFrame`. - * This API eagerly runs DDL/DML commands, but not for SELECT queries. - * - * @group basic - * @since 1.3.0 - */ - def sql(sqlText: String): DataFrame = sparkSession.sql(sqlText) - - /** - * Returns the specified table as a `DataFrame`. - * - * @group ddl_ops - * @since 1.3.0 - */ - def table(tableName: String): DataFrame = { - sparkSession.table(tableName) - } + /** @inheritdoc */ + override def range(start: Long, end: Long): Dataset[Row] = super.range(start, end) - /** - * Returns a `DataFrame` containing names of existing tables in the current database. - * The returned DataFrame has three columns, database, tableName and isTemporary (a Boolean - * indicating if a table is a temporary one or not). - * - * @group ddl_ops - * @since 1.3.0 - */ - def tables(): DataFrame = { - Dataset.ofRows(sparkSession, ShowTables(CurrentNamespace, None)) - } + /** @inheritdoc */ + override def range(start: Long, end: Long, step: Long): Dataset[Row] = + super.range(start, end, step) - /** - * Returns a `DataFrame` containing names of existing tables in the given database. - * The returned DataFrame has three columns, database, tableName and isTemporary (a Boolean - * indicating if a table is a temporary one or not). - * - * @group ddl_ops - * @since 1.3.0 - */ - def tables(databaseName: String): DataFrame = { - Dataset.ofRows(sparkSession, ShowTables(UnresolvedNamespace(Seq(databaseName)), None)) - } + /** @inheritdoc */ + override def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[Row] = + super.range(start, end, step, numPartitions) - /** - * 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 + /** @inheritdoc */ + override def sql(sqlText: String): Dataset[Row] = super.sql(sqlText) - /** - * Returns the names of tables in the current database as an array. - * - * @group ddl_ops - * @since 1.3.0 - */ - def tableNames(): Array[String] = { - tableNames(sparkSession.catalog.currentDatabase) - } + /** @inheritdoc */ + override def table(tableName: String): Dataset[Row] = super.table(tableName) - /** - * Returns the names of tables in the given database as an array. - * - * @group ddl_ops - * @since 1.3.0 - */ - def tableNames(databaseName: String): Array[String] = { - sessionState.catalog.listTables(databaseName).map(_.table).toArray - } + /** @inheritdoc */ + override def tables(): DataFrame = super.tables() - //////////////////////////////////////////////////////////////////////////// - //////////////////////////////////////////////////////////////////////////// - // Deprecated methods - //////////////////////////////////////////////////////////////////////////// - //////////////////////////////////////////////////////////////////////////// + /** @inheritdoc */ + override def tables(databaseName: String): DataFrame = super.tables(databaseName) - /** - * @deprecated As of 1.3.0, replaced by `createDataFrame()`. - */ - @deprecated("Use createDataFrame instead.", "1.3.0") - def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { - createDataFrame(rowRDD, schema) - } + /** @inheritdoc */ + override def applySchema(rowRDD: RDD[Row], schema: StructType): Dataset[Row] = + super.applySchema(rowRDD, schema) - /** - * @deprecated As of 1.3.0, replaced by `createDataFrame()`. - */ - @deprecated("Use createDataFrame instead.", "1.3.0") - def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { - createDataFrame(rowRDD, schema) - } + /** @inheritdoc */ + override def applySchema(rowRDD: JavaRDD[Row], schema: StructType): Dataset[Row] = + super.applySchema(rowRDD, schema) - /** - * @deprecated As of 1.3.0, replaced by `createDataFrame()`. - */ - @deprecated("Use createDataFrame instead.", "1.3.0") - def applySchema(rdd: RDD[_], beanClass: Class[_]): DataFrame = { - createDataFrame(rdd, beanClass) - } + /** @inheritdoc */ + override def applySchema(rdd: RDD[_], beanClass: Class[_]): Dataset[Row] = + super.applySchema(rdd, beanClass) - /** - * @deprecated As of 1.3.0, replaced by `createDataFrame()`. - */ - @deprecated("Use createDataFrame instead.", "1.3.0") - def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { - createDataFrame(rdd, beanClass) - } + /** @inheritdoc */ + override def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): Dataset[Row] = + super.applySchema(rdd, beanClass) - /** - * Loads a Parquet file, returning the result as a `DataFrame`. This function returns an empty - * `DataFrame` if no paths are passed in. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().parquet()`. - */ - @deprecated("Use read.parquet() instead.", "1.4.0") + /** @inheritdoc */ @scala.annotation.varargs - def parquetFile(paths: String*): DataFrame = { - if (paths.isEmpty) { - emptyDataFrame - } else { - read.parquet(paths : _*) - } - } + override def parquetFile(paths: String*): Dataset[Row] = super.parquetFile(paths: _*) - /** - * Loads a JSON file (one object per line), returning the result as a `DataFrame`. - * It goes through the entire dataset once to determine the schema. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonFile(path: String): DataFrame = { - read.json(path) - } + /** @inheritdoc */ + override def jsonFile(path: String): Dataset[Row] = super.jsonFile(path) - /** - * Loads a JSON file (one object per line) and applies the given schema, - * returning the result as a `DataFrame`. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonFile(path: String, schema: StructType): DataFrame = { - read.schema(schema).json(path) - } + /** @inheritdoc */ + override def jsonFile(path: String, schema: StructType): Dataset[Row] = + super.jsonFile(path, schema) - /** - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonFile(path: String, samplingRatio: Double): DataFrame = { - read.option("samplingRatio", samplingRatio.toString).json(path) - } + /** @inheritdoc */ + override def jsonFile(path: String, samplingRatio: Double): Dataset[Row] = + super.jsonFile(path, samplingRatio) - /** - * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a - * `DataFrame`. - * It goes through the entire dataset once to determine the schema. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonRDD(json: RDD[String]): DataFrame = read.json(json) + /** @inheritdoc */ + override def jsonRDD(json: RDD[String]): Dataset[Row] = read.json(json) - /** - * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a - * `DataFrame`. - * It goes through the entire dataset once to determine the schema. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonRDD(json: JavaRDD[String]): DataFrame = read.json(json) + /** @inheritdoc */ + override def jsonRDD(json: JavaRDD[String]): Dataset[Row] = read.json(json) - /** - * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, - * returning the result as a `DataFrame`. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonRDD(json: RDD[String], schema: StructType): DataFrame = { - read.schema(schema).json(json) - } + /** @inheritdoc */ + override def jsonRDD(json: RDD[String], schema: StructType): Dataset[Row] = + super.jsonRDD(json, schema) - /** - * Loads an JavaRDD[String] storing JSON objects (one object per record) and applies the given - * schema, returning the result as a `DataFrame`. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonRDD(json: JavaRDD[String], schema: StructType): DataFrame = { - read.schema(schema).json(json) - } + /** @inheritdoc */ + override def jsonRDD(json: JavaRDD[String], schema: StructType): Dataset[Row] = + super.jsonRDD(json, schema) - /** - * Loads an RDD[String] storing JSON objects (one object per record) inferring the - * schema, returning the result as a `DataFrame`. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonRDD(json: RDD[String], samplingRatio: Double): DataFrame = { - read.option("samplingRatio", samplingRatio.toString).json(json) - } + /** @inheritdoc */ + override def jsonRDD(json: RDD[String], samplingRatio: Double): Dataset[Row] = + super.jsonRDD(json, samplingRatio) - /** - * Loads a JavaRDD[String] storing JSON objects (one object per record) inferring the - * schema, returning the result as a `DataFrame`. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().json()`. - */ - @deprecated("Use read.json() instead.", "1.4.0") - def jsonRDD(json: JavaRDD[String], samplingRatio: Double): DataFrame = { - read.option("samplingRatio", samplingRatio.toString).json(json) - } + /** @inheritdoc */ + override def jsonRDD(json: JavaRDD[String], samplingRatio: Double): Dataset[Row] = + super.jsonRDD(json, samplingRatio) - /** - * Returns the dataset stored at path as a DataFrame, - * using the default data source configured by spark.sql.sources.default. - * - * @group genericdata - * @deprecated As of 1.4.0, replaced by `read().load(path)`. - */ - @deprecated("Use read.load(path) instead.", "1.4.0") - def load(path: String): DataFrame = { - read.load(path) - } + /** @inheritdoc */ + override def load(path: String): Dataset[Row] = super.load(path) - /** - * Returns the dataset stored at path as a DataFrame, using the given data source. - * - * @group genericdata - * @deprecated As of 1.4.0, replaced by `read().format(source).load(path)`. - */ - @deprecated("Use read.format(source).load(path) instead.", "1.4.0") - def load(path: String, source: String): DataFrame = { - read.format(source).load(path) - } + /** @inheritdoc */ + override def load(path: String, source: String): Dataset[Row] = super.load(path, source) - /** - * (Java-specific) Returns the dataset specified by the given data source and - * a set of options as a DataFrame. - * - * @group genericdata - * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. - */ - @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") - def load(source: String, options: java.util.Map[String, String]): DataFrame = { - read.options(options).format(source).load() - } + /** @inheritdoc */ + override def load(source: String, options: JMap[String, String]): Dataset[Row] = + super.load(source, options) - /** - * (Scala-specific) Returns the dataset specified by the given data source and - * a set of options as a DataFrame. - * - * @group genericdata - * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. - */ - @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") - def load(source: String, options: Map[String, String]): DataFrame = { - read.options(options).format(source).load() - } + /** @inheritdoc */ + override def load(source: String, options: Map[String, String]): Dataset[Row] = + super.load(source, options) - /** - * (Java-specific) Returns the dataset specified by the given data source and - * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. - * - * @group genericdata - * @deprecated As of 1.4.0, replaced by - * `read().format(source).schema(schema).options(options).load()`. - */ - @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") - def load( + /** @inheritdoc */ + override def load( source: String, schema: StructType, - options: java.util.Map[String, String]): DataFrame = { - read.format(source).schema(schema).options(options).load() + options: JMap[String, String]): Dataset[Row] = { + super.load(source, schema, options) } - /** - * (Scala-specific) Returns the dataset specified by the given data source and - * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. - * - * @group genericdata - * @deprecated As of 1.4.0, replaced by - * `read().format(source).schema(schema).options(options).load()`. - */ - @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") - def load(source: String, schema: StructType, options: Map[String, String]): DataFrame = { - read.format(source).schema(schema).options(options).load() + /** @inheritdoc */ + override def load( + source: String, + schema: StructType, + options: Map[String, String]): Dataset[Row] = { + super.load(source, schema, options) } - /** - * Construct a `DataFrame` representing the database table accessible via JDBC URL - * url named table. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().jdbc()`. - */ - @deprecated("Use read.jdbc() instead.", "1.4.0") - def jdbc(url: String, table: String): DataFrame = { - read.jdbc(url, table, new Properties) - } + /** @inheritdoc */ + override def jdbc(url: String, table: String): Dataset[Row] = super.jdbc(url, table) - /** - * Construct a `DataFrame` representing the database table accessible via JDBC URL - * url named table. Partitions of the table will be retrieved in parallel based on the parameters - * passed to this function. - * - * @param columnName the name of a column of integral type that will be used for partitioning. - * @param lowerBound the minimum value of `columnName` used to decide partition stride - * @param upperBound the maximum value of `columnName` used to decide partition stride - * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split - * evenly into this many partitions - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().jdbc()`. - */ - @deprecated("Use read.jdbc() instead.", "1.4.0") - def jdbc( + /** @inheritdoc */ + override def jdbc( url: String, table: String, columnName: String, lowerBound: Long, upperBound: Long, - numPartitions: Int): DataFrame = { - read.jdbc(url, table, columnName, lowerBound, upperBound, numPartitions, new Properties) + numPartitions: Int): Dataset[Row] = { + super.jdbc(url, table, columnName, lowerBound, upperBound, numPartitions) } - /** - * Construct a `DataFrame` representing the database table accessible via JDBC URL - * url named table. The theParts parameter gives a list expressions - * suitable for inclusion in WHERE clauses; each one defines one partition - * of the `DataFrame`. - * - * @group specificdata - * @deprecated As of 1.4.0, replaced by `read().jdbc()`. - */ - @deprecated("Use read.jdbc() instead.", "1.4.0") - def jdbc(url: String, table: String, theParts: Array[String]): DataFrame = { - read.jdbc(url, table, theParts, new Properties) - } + /** @inheritdoc */ + override def jdbc(url: String, table: String, theParts: Array[String]): Dataset[Row] = + super.jdbc(url, table, theParts) } -/** - * This SQLContext object contains utility functions to create a singleton SQLContext instance, - * or to get the created SQLContext instance. - * - * It also provides utility functions to support preference for threads in multiple sessions - * scenario, setActive could set a SQLContext for current thread, which will be returned by - * getOrCreate instead of the global one. - */ -object SQLContext { +object SQLContext extends api.SQLContextCompanion { - /** - * Get the singleton SQLContext if it exists or create a new one using the given SparkContext. - * - * This function can be used to create a singleton SQLContext object that can be shared across - * the JVM. - * - * If there is an active SQLContext for current thread, it will be returned instead of the global - * one. - * - * @since 1.5.0 - */ - @deprecated("Use SparkSession.builder instead", "2.0.0") + override private[sql] type SQLContextImpl = SQLContext + override private[sql] type SparkContextImpl = SparkContext + + /** @inheritdoc */ def getOrCreate(sparkContext: SparkContext): SQLContext = { SparkSession.builder().sparkContext(sparkContext).getOrCreate().sqlContext } - /** - * Changes the SQLContext that will be returned in this thread and its children when - * SQLContext.getOrCreate() is called. This can be used to ensure that a given thread receives - * a SQLContext with an isolated session, instead of the global (first created) context. - * - * @since 1.6.0 - */ - @deprecated("Use SparkSession.setActiveSession instead", "2.0.0") - def setActive(sqlContext: SQLContext): Unit = { - SparkSession.setActiveSession(sqlContext.sparkSession) - } - - /** - * Clears the active SQLContext for current thread. Subsequent calls to getOrCreate will - * return the first created context instead of a thread-local override. - * - * @since 1.6.0 - */ - @deprecated("Use SparkSession.clearActiveSession instead", "2.0.0") - def clearActive(): Unit = { - SparkSession.clearActiveSession() - } + /** @inheritdoc */ + override def setActive(sqlContext: SQLContext): Unit = super.setActive(sqlContext) /** - * Converts an iterator of Java Beans to InternalRow using the provided - * bean info & schema. This is not related to the singleton, but is a static - * method for internal use. + * Converts an iterator of Java Beans to InternalRow using the provided bean info & schema. This + * is not related to the singleton, but is a static method for internal use. */ private[sql] def beansToRows( data: Iterator[_], @@ -1058,7 +400,9 @@ object SQLContext { attrs: Seq[AttributeReference]): Iterator[InternalRow] = { def createStructConverter(cls: Class[_], fieldTypes: Seq[DataType]): Any => InternalRow = { val methodConverters = - JavaTypeInference.getJavaBeanReadableProperties(cls).zip(fieldTypes) + JavaTypeInference + .getJavaBeanReadableProperties(cls) + .zip(fieldTypes) .map { case (property, fieldType) => val method = property.getReadMethod method -> createConverter(method.getReturnType, fieldType) @@ -1067,16 +411,17 @@ object SQLContext { if (value == null) { null } else { - new GenericInternalRow( - methodConverters.map { case (method, converter) => - converter(method.invoke(value)) - }) + new GenericInternalRow(methodConverters.map { case (method, converter) => + converter(method.invoke(value)) + }) } } + def createConverter(cls: Class[_], dataType: DataType): Any => Any = dataType match { case struct: StructType => createStructConverter(cls, struct.map(_.dataType)) case _ => CatalystTypeConverters.createToCatalystConverter(dataType) } + val dataConverter = createStructConverter(beanClass, attrs.map(_.dataType)) data.map(dataConverter) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index d81768c0077eb..ea0d405d2a8f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -99,6 +99,29 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { assert(sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) } + test("get tables from a database") { + val sqlContext = SQLContext.getOrCreate(sc) + + try { + sqlContext.sql("CREATE DATABASE IF NOT EXISTS temp_db_1") + sqlContext.sql("CREATE TABLE temp_db_1.temp_table_1 (key int)") + sqlContext.sql("INSERT INTO temp_db_1.temp_table_1 VALUES (1)") + + assert(sqlContext.tableNames("temp_db_1").sameElements(Array("temp_table_1"))) + + assert(sqlContext.tables("temp_db_1").collect().toSeq == + Row("temp_db_1", "temp_table_1", false) :: Nil) + + assert(sqlContext.tables().collect().toSeq == Nil) + sqlContext.sql("USE temp_db_1") + assert(sqlContext.tableNames().sameElements(Array("temp_table_1"))) + assert(sqlContext.tables().collect().toSeq == Row("temp_db_1", "temp_table_1", false) :: Nil) + } finally { + sqlContext.sql("USE default") + sqlContext.sql("DROP DATABASE IF EXISTS temp_db_1 CASCADE") + } + } + test("getting all tables with a database name has no impact on returned table names") { val sqlContext = SQLContext.getOrCreate(sc) val df = sqlContext.range(10) From 3a61eeff340a0f979dfb2929aeff128c60f18a2c Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Thu, 19 Dec 2024 17:04:34 +0800 Subject: [PATCH 216/438] [SPARK-50619][SQL] Refactor VariantGet.cast to pack the cast arguments ### What changes were proposed in this pull request? As the title. It refactors the code for simplification. ### Why are the changes needed? The refactor will make it simpler for the shredded user to use `VariantGet.cast`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49239 from chenhao-db/VariantCastArgs. Authored-by: Chenhao Li Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/Cast.scala | 13 ++-- .../variant/variantExpressions.scala | 61 ++++++++----------- 2 files changed, 34 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index d4ebdf10ef11f..abd635e22f261 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -572,6 +572,11 @@ case class Cast( } } + private lazy val castArgs = variant.VariantCastArgs( + evalMode != EvalMode.TRY, + timeZoneId, + zoneId) + def needsTimeZone: Boolean = Cast.needsTimeZone(child.dataType, dataType) // [[func]] assumes the input is no longer null because eval already does the null check. @@ -1127,7 +1132,7 @@ case class Cast( _ => throw QueryExecutionErrors.cannotCastFromNullTypeError(to) } else if (from.isInstanceOf[VariantType]) { buildCast[VariantVal](_, v => { - variant.VariantGet.cast(v, to, evalMode != EvalMode.TRY, timeZoneId, zoneId) + variant.VariantGet.cast(v, to, castArgs) }) } else { to match { @@ -1225,12 +1230,10 @@ case class Cast( case _ if from.isInstanceOf[VariantType] => (c, evPrim, evNull) => val tmp = ctx.freshVariable("tmp", classOf[Object]) val dataTypeArg = ctx.addReferenceObj("dataType", to) - val zoneStrArg = ctx.addReferenceObj("zoneStr", timeZoneId) - val zoneIdArg = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val failOnError = evalMode != EvalMode.TRY + val castArgsArg = ctx.addReferenceObj("castArgs", castArgs) val cls = classOf[variant.VariantGet].getName code""" - Object $tmp = $cls.cast($c, $dataTypeArg, $failOnError, $zoneStrArg, $zoneIdArg); + Object $tmp = $cls.cast($c, $dataTypeArg, $castArgsArg); if ($tmp == null) { $evNull = true; } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala index 2fa0ce0f570c9..c19df82e6576b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala @@ -278,14 +278,13 @@ case class VariantGet( override def nullable: Boolean = true override def nullIntolerant: Boolean = true + private lazy val castArgs = VariantCastArgs( + failOnError, + timeZoneId, + zoneId) + protected override def nullSafeEval(input: Any, path: Any): Any = { - VariantGet.variantGet( - input.asInstanceOf[VariantVal], - parsedPath, - dataType, - failOnError, - timeZoneId, - zoneId) + VariantGet.variantGet(input.asInstanceOf[VariantVal], parsedPath, dataType, castArgs) } protected override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -293,15 +292,14 @@ case class VariantGet( val tmp = ctx.freshVariable("tmp", classOf[Object]) val parsedPathArg = ctx.addReferenceObj("parsedPath", parsedPath) val dataTypeArg = ctx.addReferenceObj("dataType", dataType) - val zoneStrArg = ctx.addReferenceObj("zoneStr", timeZoneId) - val zoneIdArg = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + val castArgsArg = ctx.addReferenceObj("castArgs", castArgs) val code = code""" ${childCode.code} boolean ${ev.isNull} = ${childCode.isNull}; ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { Object $tmp = org.apache.spark.sql.catalyst.expressions.variant.VariantGet.variantGet( - ${childCode.value}, $parsedPathArg, $dataTypeArg, $failOnError, $zoneStrArg, $zoneIdArg); + ${childCode.value}, $parsedPathArg, $dataTypeArg, $castArgsArg); if ($tmp == null) { ${ev.isNull} = true; } else { @@ -323,6 +321,12 @@ case class VariantGet( override def withTimeZone(timeZoneId: String): VariantGet = copy(timeZoneId = Option(timeZoneId)) } +// Several parameters used by `VariantGet.cast`. Packed together to simplify parameter passing. +case class VariantCastArgs( + failOnError: Boolean, + zoneStr: Option[String], + zoneId: ZoneId) + case object VariantGet { /** * Returns whether a data type can be cast into/from variant. For scalar types, we allow a subset @@ -347,9 +351,7 @@ case object VariantGet { input: VariantVal, parsedPath: Array[VariantPathParser.PathSegment], dataType: DataType, - failOnError: Boolean, - zoneStr: Option[String], - zoneId: ZoneId): Any = { + castArgs: VariantCastArgs): Any = { var v = new Variant(input.getValue, input.getMetadata) for (path <- parsedPath) { v = path match { @@ -359,21 +361,16 @@ case object VariantGet { } if (v == null) return null } - VariantGet.cast(v, dataType, failOnError, zoneStr, zoneId) + VariantGet.cast(v, dataType, castArgs) } /** * A simple wrapper of the `cast` function that takes `Variant` rather than `VariantVal`. The * `Cast` expression uses it and makes the implementation simpler. */ - def cast( - input: VariantVal, - dataType: DataType, - failOnError: Boolean, - zoneStr: Option[String], - zoneId: ZoneId): Any = { + def cast(input: VariantVal, dataType: DataType, castArgs: VariantCastArgs): Any = { val v = new Variant(input.getValue, input.getMetadata) - VariantGet.cast(v, dataType, failOnError, zoneStr, zoneId) + VariantGet.cast(v, dataType, castArgs) } /** @@ -383,15 +380,10 @@ case object VariantGet { * "hello" to int). If the cast fails, throw an exception when `failOnError` is true, or return a * SQL NULL when it is false. */ - def cast( - v: Variant, - dataType: DataType, - failOnError: Boolean, - zoneStr: Option[String], - zoneId: ZoneId): Any = { + def cast(v: Variant, dataType: DataType, castArgs: VariantCastArgs): Any = { def invalidCast(): Any = { - if (failOnError) { - throw QueryExecutionErrors.invalidVariantCast(v.toJson(zoneId), dataType) + if (castArgs.failOnError) { + throw QueryExecutionErrors.invalidVariantCast(v.toJson(castArgs.zoneId), dataType) } else { null } @@ -411,7 +403,7 @@ case object VariantGet { val input = variantType match { case Type.OBJECT | Type.ARRAY => return if (dataType.isInstanceOf[StringType]) { - UTF8String.fromString(v.toJson(zoneId)) + UTF8String.fromString(v.toJson(castArgs.zoneId)) } else { invalidCast() } @@ -457,7 +449,7 @@ case object VariantGet { } case _ => if (Cast.canAnsiCast(input.dataType, dataType)) { - val result = Cast(input, dataType, zoneStr, EvalMode.TRY).eval() + val result = Cast(input, dataType, castArgs.zoneStr, EvalMode.TRY).eval() if (result == null) invalidCast() else result } else { invalidCast() @@ -468,7 +460,7 @@ case object VariantGet { val size = v.arraySize() val array = new Array[Any](size) for (i <- 0 until size) { - array(i) = cast(v.getElementAtIndex(i), elementType, failOnError, zoneStr, zoneId) + array(i) = cast(v.getElementAtIndex(i), elementType, castArgs) } new GenericArrayData(array) } else { @@ -482,7 +474,7 @@ case object VariantGet { for (i <- 0 until size) { val field = v.getFieldAtIndex(i) keyArray(i) = UTF8String.fromString(field.key) - valueArray(i) = cast(field.value, valueType, failOnError, zoneStr, zoneId) + valueArray(i) = cast(field.value, valueType, castArgs) } ArrayBasedMapData(keyArray, valueArray) } else { @@ -495,8 +487,7 @@ case object VariantGet { val field = v.getFieldAtIndex(i) st.getFieldIndex(field.key) match { case Some(idx) => - row.update(idx, - cast(field.value, fields(idx).dataType, failOnError, zoneStr, zoneId)) + row.update(idx, cast(field.value, fields(idx).dataType, castArgs)) case _ => } } From f4c88ca597bacb14a6d3cb08b1690ea4ec5632ea Mon Sep 17 00:00:00 2001 From: Ole Sasse Date: Thu, 19 Dec 2024 19:21:13 +0800 Subject: [PATCH 217/438] [SPARK-50558][SQL] Add configurable logging limits for the number of elements in InSet and In ### What changes were proposed in this pull request? Introduce a new SQL conf `OPTIMIZER_INSET_MEMBER_LOGGING_LIMIT` that allows to limit the number of elements logged for In and InSet in toString. The current implementation does log arbitrarily many elements and in the case of InSet even sorts them before, which can lead to a lot of unwanted log volumes and have significant performance impact. If clients require all elements, they can set to conf to a very high value or preferably use the `sql` function. ### Why are the changes needed? I have observed performance impact of the logging ### Does this PR introduce _any_ user-facing change? Teh logging string changes. Given that there is also the `sql` function, clients should not depend on the output of the logging function. ### How was this patch tested? New tests have been added ### Was this patch authored or co-authored using generative AI tooling? No Closes #49177 from olaky/spark-50558-do-not-log-all-values-in-in-set-expressions. Authored-by: Ole Sasse Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/predicates.scala | 31 ++++++++++++++----- .../catalyst/expressions/PredicateSuite.scala | 10 ++++++ 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 5e402fa2b6caa..d8d81a9cc12f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project, Union} import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.util.{CollationFactory, TypeUtils} +import org.apache.spark.sql.catalyst.util.SparkStringUtils.truncatedString import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -487,7 +488,10 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { } } - override def toString: String = s"$value IN ${list.mkString("(", ",", ")")}" + override def simpleString(maxFields: Int): String = + s"$value IN ${truncatedString(list, "(", ",", ")", maxFields)}" + + override def toString: String = simpleString(Int.MaxValue) override def eval(input: InternalRow): Any = { if (list.isEmpty && !legacyNullInEmptyBehavior) { @@ -608,18 +612,29 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with require(hset != null, "hset could not be null") - override def toString: String = { + override def simpleString(maxFields: Int): String = { if (!child.resolved) { return s"$child INSET (values with unresolved data types)" } - val listString = hset.toSeq - .map(elem => Literal(elem, child.dataType).toString) - // Sort elements for deterministic behaviours - .sorted - .mkString(", ") - s"$child INSET $listString" + if (hset.size <= maxFields) { + val listString = hset.toSeq + .map(elem => Literal(elem, child.dataType).toString) + // Sort elements for deterministic behaviours + .sorted + .mkString(", ") + s"$child INSET $listString" + } else { + // Skip sorting if there are many elements. Do not use truncatedString because we would have + // to convert elements we do not print to Literals. + val listString = hset.take(maxFields).toSeq + .map(elem => Literal(elem, child.dataType).toString) + .mkString(", ") + s"$child INSET $listString, ... ${hset.size - maxFields} more fields" + } } + override def toString: String = simpleString(Int.MaxValue) + @transient private[this] lazy val hasNull: Boolean = hset.contains(null) @transient private[this] lazy val isNaN: Any => Boolean = child.dataType match { case DoubleType => (value: Any) => java.lang.Double.isNaN(value.asInstanceOf[java.lang.Double]) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index a0c75b703ade4..4a7bf807d1de9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -674,4 +674,14 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { checkInAndInSet(In(Literal(Double.NaN), Seq(Literal(Double.NaN), Literal(2d), Literal.create(null, DoubleType))), true) } + + test("In and InSet logging limits") { + assert(In(Literal(1), Seq(Literal(1), Literal(2))).simpleString(1) + === "1 IN (1,... 1 more fields)") + assert(In(Literal(1), Seq(Literal(1), Literal(2))).simpleString(2) === "1 IN (1,2)") + assert(In(Literal(1), Seq(Literal(1))).simpleString(1) === "1 IN (1)") + assert(InSet(Literal(1), Set(1, 2)).simpleString(1) === "1 INSET 1, ... 1 more fields") + assert(InSet(Literal(1), Set(1, 2)).simpleString(2) === "1 INSET 1, 2") + assert(InSet(Literal(1), Set(1)).simpleString(1) === "1 INSET 1") + } } From f593cda97ed2afc938d39f692077e9db512fa1cd Mon Sep 17 00:00:00 2001 From: Mihailo Timotic Date: Thu, 19 Dec 2024 21:50:56 +0300 Subject: [PATCH 218/438] [SPARK-50612][SQL] Normalize ordering in the project list of an inner Project ### What changes were proposed in this pull request? Normalize ordering in the project list of an inner Project. For example, following two plans should be the same after normalization: ``` Project[col1] Project[col1] :- Project[col1, col2] :- Project[col2, col1] ``` This is because only the top most ordering matters for comparing plans and inner project list can be computed in a non-deterministic way (i.e. via `HashSet`/`HashMap`) ### Why are the changes needed? For plan comparison in the single-pass resolver. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Added a new test suite. ### Was this patch authored or co-authored using generative AI tooling? Closes #49230 from mihailotim-db/mihailotim-db/normalize_plan. Authored-by: Mihailo Timotic Signed-off-by: Max Gekk --- .../sql/catalyst/plans/NormalizePlan.scala | 25 ++++++---- .../catalyst/plans/NormalizePlanSuite.scala | 46 +++++++++++++++++++ 2 files changed, 63 insertions(+), 8 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala index 3b691f4f87778..1cc876588550b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala @@ -95,19 +95,28 @@ object NormalizePlan extends PredicateHelper { .sortBy(_.hashCode()) .reduce(And) Join(left, right, newJoinType, Some(newCondition), hint) + case Project(outerProjectList, innerProject: Project) => + val normalizedInnerProjectList = normalizeProjectList(innerProject.projectList) + val orderedInnerProjectList = normalizedInnerProjectList.sortBy(_.name) + val newInnerProject = + Project(orderedInnerProjectList, innerProject.child) + Project(normalizeProjectList(outerProjectList), newInnerProject) case Project(projectList, child) => - val projList = projectList - .map { e => - e.transformUp { - case g: GetViewColumnByNameAndOrdinal => g.copy(viewDDL = None) - } - } - .asInstanceOf[Seq[NamedExpression]] - Project(projList, child) + Project(normalizeProjectList(projectList), child) case c: KeepAnalyzedQuery => c.storeAnalyzedQuery() } } + private def normalizeProjectList(projectList: Seq[NamedExpression]): Seq[NamedExpression] = { + projectList + .map { e => + e.transformUp { + case g: GetViewColumnByNameAndOrdinal => g.copy(viewDDL = None) + } + } + .asInstanceOf[Seq[NamedExpression]] + } + /** * Rewrite [[BinaryComparison]] operator to keep order. The following cases will be * equivalent: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala new file mode 100644 index 0000000000000..80cdf19e4b73b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala @@ -0,0 +1,46 @@ +/* + * 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.catalyst.plans + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation + +class NormalizePlanSuite extends SparkFunSuite{ + + test("Normalize Project") { + val baselineCol1 = $"col1".int + val testCol1 = baselineCol1.newInstance() + val baselinePlan = LocalRelation(baselineCol1).select(baselineCol1) + val testPlan = LocalRelation(testCol1).select(testCol1) + + assert(baselinePlan != testPlan) + assert(NormalizePlan(baselinePlan) == NormalizePlan(testPlan)) + } + + test("Normalize ordering in a project list of an inner Project") { + val baselinePlan = + LocalRelation($"col1".int, $"col2".string).select($"col1", $"col2").select($"col1") + val testPlan = + LocalRelation($"col1".int, $"col2".string).select($"col2", $"col1").select($"col1") + + assert(baselinePlan != testPlan) + assert(NormalizePlan(baselinePlan) == NormalizePlan(testPlan)) + } +} From f9aaeb4c1742a305d77c442bf74ee9967b5f5a06 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 19 Dec 2024 22:09:57 +0300 Subject: [PATCH 219/438] [SPARK-50586][BUILD] Use CommonJS format for ESLint configuration file ### What changes were proposed in this pull request? This PR proposes to use CommonJS format for ESLint configuration file instead of JSON format. ### Why are the changes needed? CommonJS format is just plain JavaScript so we can add comments to the configuration file. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Temporarily modified some `.js` files and ran `dev/lint-js`, then the linter detected violations. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49198 from sarutak/eslint-js. Authored-by: Kousuke Saruta Signed-off-by: Max Gekk --- dev/eslint.js | 45 +++++++++++++++++++++++++++++++++++++++++++++ dev/eslint.json | 28 ---------------------------- dev/lint-js | 2 +- 3 files changed, 46 insertions(+), 29 deletions(-) create mode 100644 dev/eslint.js delete mode 100644 dev/eslint.json diff --git a/dev/eslint.js b/dev/eslint.js new file mode 100644 index 0000000000000..24b5170b436a9 --- /dev/null +++ b/dev/eslint.js @@ -0,0 +1,45 @@ +/* + * 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. + */ + +module.exports = { + "env": { + "browser": true, + "es6": true, + "jest": true + }, + "extends": "eslint:recommended", + "rules": { + "indent": [ + "error", + 2, + { + "SwitchCase": 1, + "MemberExpression": "off" + } + ], + "no-unused-vars": ["error", {"argsIgnorePattern": "^_ignored_.*"}] + }, + "ignorePatterns": [ + "*.min.js", + "sorttable.js", + "jquery.mustache.js", + "dataTables.rowsGroup.js" + ], + "parserOptions": { + "sourceType": "module" + } +} diff --git a/dev/eslint.json b/dev/eslint.json deleted file mode 100644 index 29692696a6df2..0000000000000 --- a/dev/eslint.json +++ /dev/null @@ -1,28 +0,0 @@ -{ - "env": { - "browser": true, - "es6": true, - "jest": true - }, - "extends": "eslint:recommended", - "rules": { - "indent": [ - "error", - 2, - { - "SwitchCase": 1, - "MemberExpression": "off" - } - ], - "no-unused-vars": ["error", {"argsIgnorePattern": "^_ignored_.*"}] - }, - "ignorePatterns": [ - "*.min.js", - "sorttable.js", - "jquery.mustache.js", - "dataTables.rowsGroup.js" - ], - "parserOptions": { - "sourceType": "module" - } -} diff --git a/dev/lint-js b/dev/lint-js index f3f7eac4f6b8e..1a94348b7430a 100755 --- a/dev/lint-js +++ b/dev/lint-js @@ -45,7 +45,7 @@ if ! npm ls eslint > /dev/null; then npm ci eslint fi -npx eslint -c "$SPARK_ROOT_DIR/dev/eslint.json" ${LINT_TARGET_FILES[@]} | tee "$LINT_JS_REPORT_FILE_NAME" +npx eslint -c "$SPARK_ROOT_DIR/dev/eslint.js" ${LINT_TARGET_FILES[@]} | tee "$LINT_JS_REPORT_FILE_NAME" lint_status=$? if [ "$lint_status" = "0" ] ; then From 09d93ece2cf78ff265c305fcf79c5a4f4b5bd5a4 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Thu, 19 Dec 2024 14:11:07 -0800 Subject: [PATCH 220/438] [SPARK-50602][SQL] Fix transpose to show a proper error message when invalid index columns are specified ### What changes were proposed in this pull request? Fixes `transpose` to show a proper error message when invalid index columns are specified. ### Why are the changes needed? When invalid index columns are specified, it shows `INTERNAL_ERROR`. ```py >>> df = spark.range(10).transpose(sf.col("id") + 1) Traceback (most recent call last): ... py4j.protocol.Py4JJavaError: An error occurred while calling o40.transpose. : org.apache.spark.SparkException: [INTERNAL_ERROR] Found the unresolved operator: 'UnresolvedTranspose [unresolvedalias((id#0L + cast(1 as bigint)))] SQLSTATE: XX000 ... ``` ### Does this PR introduce _any_ user-facing change? Yes, the proper error message will be shown. ```py >>> df = spark.range(10).transpose(sf.col("id") + 1) Traceback (most recent call last): ... pyspark.errors.exceptions.captured.AnalysisException: [TRANSPOSE_INVALID_INDEX_COLUMN] Invalid index column for TRANSPOSE because: Index column must be an atomic attribute SQLSTATE: 42804 ``` ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49219 from ueshin/issues/SPARK-50602/fix_transpose. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- python/pyspark/sql/tests/test_dataframe.py | 12 ++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../spark/sql/DataFrameTransposeSuite.scala | 17 +++++++++++++++++ 3 files changed, 30 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index cd6a57429cfa9..a0234a527f639 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -1044,6 +1044,18 @@ def test_transpose(self): messageParameters={"dt1": '"STRING"', "dt2": '"BIGINT"'}, ) + def test_transpose_with_invalid_index_columns(self): + # SPARK-50602: invalid index columns + df = self.spark.createDataFrame([{"a": "x", "b": "y", "c": "z"}]) + + with self.assertRaises(AnalysisException) as pe: + df.transpose(col("a") + 1).collect() + self.check_error( + exception=pe.exception, + errorClass="TRANSPOSE_INVALID_INDEX_COLUMN", + messageParameters={"reason": "Index column must be an atomic attribute"}, + ) + class DataFrameTests(DataFrameTestsMixin, ReusedSQLTestCase): def test_query_execution_unsupported_in_classic(self): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index c5c9dde901c9c..c8c1bacfb9ded 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1013,7 +1013,7 @@ class Dataset[T] private[sql]( /** @inheritdoc */ def transpose(indexColumn: Column): DataFrame = withPlan { UnresolvedTranspose( - Seq(indexColumn.named), + Seq(indexColumn.expr), logicalPlan ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTransposeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTransposeSuite.scala index 51de8553216c6..ce1c8d7ceb64a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTransposeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTransposeSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -173,4 +174,20 @@ class DataFrameTransposeSuite extends QueryTest with SharedSparkSession { ) assertResult(Array("key", "A", "B"))(transposedDf.columns) } + + test("SPARK-50602: invalid index columns") { + val df = Seq( + ("A", 1, 2), + ("B", 3, 4), + (null, 5, 6) + ).toDF("id", "val1", "val2") + + checkError( + exception = intercept[AnalysisException] { + df.transpose($"id" + lit(1)) + }, + condition = "TRANSPOSE_INVALID_INDEX_COLUMN", + parameters = Map("reason" -> "Index column must be an atomic attribute") + ) + } } From d499400e8cd8515fbf0fb74fb1b107f0bb08447a Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Fri, 20 Dec 2024 09:09:58 +0900 Subject: [PATCH 221/438] [SPARK-50243][SQL][CONNECT] Cached classloader for ArtifactManager ### What changes were proposed in this pull request? This PR implements a caching mechanism for the classloader of `ArtifactManager`, to avoid re-generating a new one every time when a SQL query runs. This change also fixes a longstanding bug where the codegen cache was broken for Spark Connect, due to new classloaders causing cache misses: https://github.com/apache/spark/blob/05728e4ff64e6684d7c6501f8a079e3b9aded9ed/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala#L1487-L1490. The approach we use is to cache the generated classloader until a new artifact is added. ### Why are the changes needed? To improve performance and fix codegen caching. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added two new test cases. Also, some existing tests will fail when the config `ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER` is enabled: - AdaptiveQueryExecSuite: `SPARK-37753: Inhibit broadcast in left outer join when there are many empty partitions on outer/left side` - DatasetOptimizationSuite: `SPARK-27871: Dataset encoder should benefit from codegen cache` These tests will not fail after this PR. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49007 from xupefei/cache-artifact-classloader. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../spark/sql/artifact/ArtifactManager.scala | 18 +++++- .../spark/sql/DatasetOptimizationSuite.scala | 18 +++--- .../sql/artifact/ArtifactManagerSuite.scala | 55 +++++++++++++++++++ .../adaptive/AdaptiveQueryExecSuite.scala | 1 + 4 files changed, 83 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index 0aae5a43ca40f..65f361cbae3f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -87,6 +87,9 @@ class ArtifactManager(session: SparkSession) extends Logging { */ protected val sessionArtifactAdded = new AtomicBoolean(false) + @volatile + protected var cachedClassLoader: Option[ClassLoader] = None + private def withClassLoaderIfNeeded[T](f: => T): T = { val log = s" classloader for session ${session.sessionUUID} because " + s"alwaysApplyClassLoader=$alwaysApplyClassLoader, " + @@ -202,6 +205,7 @@ class ArtifactManager(session: SparkSession) extends Logging { allowOverwrite = true, deleteSource = deleteStagedFile) sessionArtifactAdded.set(true) + cachedClassLoader = None } else { val target = ArtifactUtils.concatenatePaths(artifactPath, normalizedRemoteRelativePath) // Disallow overwriting with modified version @@ -226,6 +230,7 @@ class ArtifactManager(session: SparkSession) extends Logging { (SparkContextResourceType.JAR, normalizedRemoteRelativePath, fragment)) jarsList.add(normalizedRemoteRelativePath) sessionArtifactAdded.set(true) + cachedClassLoader = None } else if (normalizedRemoteRelativePath.startsWith(s"pyfiles${File.separator}")) { session.sparkContext.addFile(uri) sparkContextRelativePaths.add( @@ -281,10 +286,18 @@ class ArtifactManager(session: SparkSession) extends Logging { } } + def classloader: ClassLoader = synchronized { + cachedClassLoader.getOrElse { + val loader = buildClassLoader + cachedClassLoader = Some(loader) + loader + } + } + /** * Returns a [[ClassLoader]] for session-specific jar/class file resources. */ - def classloader: ClassLoader = { + private def buildClassLoader: ClassLoader = { val urls = (getAddedJars :+ classDir.toUri.toURL).toArray val prefixes = SparkEnv.get.conf.get(CONNECT_SCALA_UDF_STUB_PREFIXES) val userClasspathFirst = SparkEnv.get.conf.get(EXECUTOR_USER_CLASS_PATH_FIRST) @@ -394,6 +407,9 @@ class ArtifactManager(session: SparkSession) extends Logging { pythonIncludeList.clear() cachedBlockIdList.clear() sparkContextRelativePaths.clear() + + // Removed cached classloader + cachedClassLoader = None } def uploadArtifactToFs( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala index 81d7de856f881..5db3990b67c8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala @@ -184,16 +184,18 @@ class DatasetOptimizationSuite extends QueryTest with SharedSparkSession { assert(count3 == count2) } - withClue("array type") { - checkCodegenCache(() => Seq(Seq("abc")).toDS()) - } + withSQLConf(SQLConf.ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER.key -> "true") { + withClue("array type") { + checkCodegenCache(() => Seq(Seq("abc")).toDS()) + } - withClue("map type") { - checkCodegenCache(() => Seq(Map("abc" -> 1)).toDS()) - } + withClue("map type") { + checkCodegenCache(() => Seq(Map("abc" -> 1)).toDS()) + } - withClue("array of map") { - checkCodegenCache(() => Seq(Seq(Map("abc" -> 1))).toDS()) + withClue("array of map") { + checkCodegenCache(() => Seq(Seq(Map("abc" -> 1))).toDS()) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala index e935af8b8bf8c..f09e896fb39a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala @@ -23,6 +23,7 @@ import java.nio.file.{Files, Path, Paths} import org.apache.commons.io.FileUtils import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf @@ -447,4 +448,58 @@ class ArtifactManagerSuite extends SharedSparkSession { assert(msg == "Hello Talon! Nice to meet you!") } } + + test("Codegen cache should be invalid when artifacts are added - class artifact") { + withTempDir { dir => + runCodegenTest("class artifact") { + val randomFilePath = dir.toPath.resolve("random.class") + val testBytes = "test".getBytes(StandardCharsets.UTF_8) + Files.write(randomFilePath, testBytes) + spark.addArtifact(randomFilePath.toString) + } + } + } + + test("Codegen cache should be invalid when artifacts are added - JAR artifact") { + withTempDir { dir => + runCodegenTest("JAR artifact") { + val randomFilePath = dir.toPath.resolve("random.jar") + val testBytes = "test".getBytes(StandardCharsets.UTF_8) + Files.write(randomFilePath, testBytes) + spark.addArtifact(randomFilePath.toString) + } + } + } + + private def getCodegenCount: Long = CodegenMetrics.METRIC_COMPILATION_TIME.getCount + + private def runCodegenTest(msg: String)(addOneArtifact: => Unit): Unit = { + withSQLConf(SQLConf.ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER.key -> "true") { + val s = spark + import s.implicits._ + + val count1 = getCodegenCount + // trigger codegen for Dataset + Seq(Seq("abc")).toDS().collect() + val count2 = getCodegenCount + // codegen happens + assert(count2 > count1, s"$msg: codegen should happen at the first time") + + // add one artifact, codegen cache should be invalid after this + addOneArtifact + + // trigger codegen for another Dataset of same type + Seq(Seq("abc")).toDS().collect() + // codegen cache should not work for Datasets of same type. + val count3 = getCodegenCount + assert(count3 > count2, s"$msg: codegen should happen again after adding artifact") + + // trigger again + Seq(Seq("abc")).toDS().collect() + // codegen should work now as classloader is not changed + val count4 = getCodegenCount + assert(count4 == count3, + s"$msg: codegen should not happen again as classloader is not changed") + } + } } 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 ad28fd5176d99..d298b98aaa8da 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 @@ -749,6 +749,7 @@ class AdaptiveQueryExecSuite // so retry several times here to avoid unit test failure. eventually(timeout(15.seconds), interval(500.milliseconds)) { withSQLConf( + SQLConf.ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5") { // `testData` is small enough to be broadcast but has empty partition ratio over the config. From 939f3dfc05513966b60df5dd2e15be60f7c9e9e0 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 20 Dec 2024 09:10:35 +0900 Subject: [PATCH 222/438] [SPARK-50621][PYTHON] Upgrade Cloudpickle to 3.1.0 ### What changes were proposed in this pull request? This PR proposes to upgrade Cloudpickle to 3.1.0 ### Why are the changes needed? To leverage bug fixes. ### Does this PR introduce _any_ user-facing change? Yes several bug fixes listed in https://github.com/cloudpipe/cloudpickle/blob/v3.1.0/CHANGES.md#310 ### How was this patch tested? Existing unittests should cover. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49241 from HyukjinKwon/upgrade-cloudpickle. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/cloudpickle/__init__.py | 2 +- python/pyspark/cloudpickle/cloudpickle.py | 98 ++++++++++++++----- .../pyspark/cloudpickle/cloudpickle_fast.py | 1 + python/pyspark/cloudpickle/compat.py | 18 ---- 4 files changed, 76 insertions(+), 43 deletions(-) delete mode 100644 python/pyspark/cloudpickle/compat.py diff --git a/python/pyspark/cloudpickle/__init__.py b/python/pyspark/cloudpickle/__init__.py index a3348e8b3da28..4d317916d4e46 100644 --- a/python/pyspark/cloudpickle/__init__.py +++ b/python/pyspark/cloudpickle/__init__.py @@ -3,7 +3,7 @@ __doc__ = cloudpickle.__doc__ -__version__ = "3.0.0" +__version__ = "3.1.0" __all__ = [ # noqa "__version__", diff --git a/python/pyspark/cloudpickle/cloudpickle.py b/python/pyspark/cloudpickle/cloudpickle.py index eb43a9676bbb1..8c50ba175e3e7 100644 --- a/python/pyspark/cloudpickle/cloudpickle.py +++ b/python/pyspark/cloudpickle/cloudpickle.py @@ -126,7 +126,7 @@ def _lookup_class_or_track(class_tracker_id, class_def): def register_pickle_by_value(module): - """Register a module to make it functions and classes picklable by value. + """Register a module to make its functions and classes picklable by value. By default, functions and classes that are attributes of an importable module are to be pickled by reference, that is relying on re-importing @@ -213,6 +213,7 @@ def _whichmodule(obj, name): # sys.modules if ( module_name == "__main__" + or module_name == "__mp_main__" or module is None or not isinstance(module, types.ModuleType) ): @@ -409,7 +410,10 @@ def _walk_global_ops(code): def _extract_class_dict(cls): """Retrieve a copy of the dict of a class without the inherited method.""" - clsdict = dict(cls.__dict__) # copy dict proxy to a dict + # Hack to circumvent non-predictable memoization caused by string interning. + # See the inline comment in _class_setstate for details. + clsdict = {"".join(k): cls.__dict__[k] for k in sorted(cls.__dict__)} + if len(cls.__bases__) == 1: inherited_dict = cls.__bases__[0].__dict__ else: @@ -533,9 +537,15 @@ class id will also reuse this class definition. The "extra" variable is meant to be a dict (or None) that can be used for forward compatibility shall the need arise. """ + # We need to intern the keys of the type_kwargs dict to avoid having + # different pickles for the same dynamic class depending on whether it was + # dynamically created or reconstructed from a pickled stream. + type_kwargs = {sys.intern(k): v for k, v in type_kwargs.items()} + skeleton_class = types.new_class( name, bases, {"metaclass": type_constructor}, lambda ns: ns.update(type_kwargs) ) + return _lookup_class_or_track(class_tracker_id, skeleton_class) @@ -694,8 +704,10 @@ def _function_getstate(func): # unpickling time by iterating over slotstate and calling setattr(func, # slotname, slotvalue) slotstate = { - "__name__": func.__name__, - "__qualname__": func.__qualname__, + # Hack to circumvent non-predictable memoization caused by string interning. + # See the inline comment in _class_setstate for details. + "__name__": "".join(func.__name__), + "__qualname__": "".join(func.__qualname__), "__annotations__": func.__annotations__, "__kwdefaults__": func.__kwdefaults__, "__defaults__": func.__defaults__, @@ -721,7 +733,9 @@ def _function_getstate(func): ) slotstate["__globals__"] = f_globals - state = func.__dict__ + # Hack to circumvent non-predictable memoization caused by string interning. + # See the inline comment in _class_setstate for details. + state = {"".join(k): v for k, v in func.__dict__.items()} return state, slotstate @@ -802,6 +816,19 @@ def _code_reduce(obj): # of the specific type from types, for example: # >>> from types import CodeType # >>> help(CodeType) + + # Hack to circumvent non-predictable memoization caused by string interning. + # See the inline comment in _class_setstate for details. + co_name = "".join(obj.co_name) + + # Create shallow copies of these tuple to make cloudpickle payload deterministic. + # When creating a code object during load, copies of these four tuples are + # created, while in the main process, these tuples can be shared. + # By always creating copies, we make sure the resulting payload is deterministic. + co_names = tuple(name for name in obj.co_names) + co_varnames = tuple(name for name in obj.co_varnames) + co_freevars = tuple(name for name in obj.co_freevars) + co_cellvars = tuple(name for name in obj.co_cellvars) if hasattr(obj, "co_exceptiontable"): # Python 3.11 and later: there are some new attributes # related to the enhanced exceptions. @@ -814,16 +841,16 @@ def _code_reduce(obj): obj.co_flags, obj.co_code, obj.co_consts, - obj.co_names, - obj.co_varnames, + co_names, + co_varnames, obj.co_filename, - obj.co_name, + co_name, obj.co_qualname, obj.co_firstlineno, obj.co_linetable, obj.co_exceptiontable, - obj.co_freevars, - obj.co_cellvars, + co_freevars, + co_cellvars, ) elif hasattr(obj, "co_linetable"): # Python 3.10 and later: obj.co_lnotab is deprecated and constructor @@ -837,14 +864,14 @@ def _code_reduce(obj): obj.co_flags, obj.co_code, obj.co_consts, - obj.co_names, - obj.co_varnames, + co_names, + co_varnames, obj.co_filename, - obj.co_name, + co_name, obj.co_firstlineno, obj.co_linetable, - obj.co_freevars, - obj.co_cellvars, + co_freevars, + co_cellvars, ) elif hasattr(obj, "co_nmeta"): # pragma: no cover # "nogil" Python: modified attributes from 3.9 @@ -859,15 +886,15 @@ def _code_reduce(obj): obj.co_flags, obj.co_code, obj.co_consts, - obj.co_varnames, + co_varnames, obj.co_filename, - obj.co_name, + co_name, obj.co_firstlineno, obj.co_lnotab, obj.co_exc_handlers, obj.co_jump_table, - obj.co_freevars, - obj.co_cellvars, + co_freevars, + co_cellvars, obj.co_free2reg, obj.co_cell2reg, ) @@ -882,14 +909,14 @@ def _code_reduce(obj): obj.co_flags, obj.co_code, obj.co_consts, - obj.co_names, - obj.co_varnames, + co_names, + co_varnames, obj.co_filename, - obj.co_name, + co_name, obj.co_firstlineno, obj.co_lnotab, - obj.co_freevars, - obj.co_cellvars, + co_freevars, + co_cellvars, ) return types.CodeType, args @@ -1127,7 +1154,30 @@ def _class_setstate(obj, state): if attrname == "_abc_impl": registry = attr else: + # Note: setting attribute names on a class automatically triggers their + # interning in CPython: + # https://github.com/python/cpython/blob/v3.12.0/Objects/object.c#L957 + # + # This means that to get deterministic pickling for a dynamic class that + # was initially defined in a different Python process, the pickler + # needs to ensure that dynamic class and function attribute names are + # systematically copied into a non-interned version to avoid + # unpredictable pickle payloads. + # + # Indeed the Pickler's memoizer relies on physical object identity to break + # cycles in the reference graph of the object being serialized. setattr(obj, attrname, attr) + + if sys.version_info >= (3, 13) and "__firstlineno__" in state: + # Set the Python 3.13+ only __firstlineno__ attribute one more time, as it + # will be automatically deleted by the `setattr(obj, attrname, attr)` call + # above when `attrname` is "__firstlineno__". We assume that preserving this + # information might be important for some users and that it not stale in the + # context of cloudpickle usage, hence legitimate to propagate. Furthermore it + # is necessary to do so to keep deterministic chained pickling as tested in + # test_deterministic_str_interning_for_chained_dynamic_class_pickling. + obj.__firstlineno__ = state["__firstlineno__"] + if registry is not None: for subclass in registry: obj.register(subclass) diff --git a/python/pyspark/cloudpickle/cloudpickle_fast.py b/python/pyspark/cloudpickle/cloudpickle_fast.py index 52d6732e44ebc..20280f0ca354a 100644 --- a/python/pyspark/cloudpickle/cloudpickle_fast.py +++ b/python/pyspark/cloudpickle/cloudpickle_fast.py @@ -6,6 +6,7 @@ See: tests/test_backward_compat.py """ + from . import cloudpickle diff --git a/python/pyspark/cloudpickle/compat.py b/python/pyspark/cloudpickle/compat.py deleted file mode 100644 index 5e9b52773d279..0000000000000 --- a/python/pyspark/cloudpickle/compat.py +++ /dev/null @@ -1,18 +0,0 @@ -import sys - - -if sys.version_info < (3, 8): - try: - import pickle5 as pickle # noqa: F401 - from pickle5 import Pickler # noqa: F401 - except ImportError: - import pickle # noqa: F401 - - # Use the Python pickler for old CPython versions - from pickle import _Pickler as Pickler # noqa: F401 -else: - import pickle # noqa: F401 - - # Pickler will the C implementation in CPython and the Python - # implementation in PyPy - from pickle import Pickler # noqa: F401 From 78592a070c279b2aa181c5a7f84adff8a2fc0e74 Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Fri, 20 Dec 2024 13:03:33 +0800 Subject: [PATCH 223/438] [SPARK-50615][SQL] Push variant into scan ### What changes were proposed in this pull request? It adds an optimizer rule to push variant into scan by rewriting the variant type with a struct type producing all requested fields and rewriting the variant extraction expressions by struct accesses. This will be the foundation of the variant shredding reader. The rule must be disabled at this point because the scan part is not yet able to recognize the special struct. ### Why are the changes needed? It is necessary for the performance of reading from shredded variant. With this rule (and the reader implemented), the scan only needs to fetch the necessary shredded columns required by the plan. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49235 from chenhao-db/PushVariantIntoScan. Authored-by: Chenhao Li Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 9 + .../spark/sql/execution/SparkOptimizer.scala | 5 +- .../datasources/PushVariantIntoScan.scala | 340 ++++++++++++++++++ .../PushVariantIntoScanSuite.scala | 178 +++++++++ 4 files changed, 530 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScanSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 31282d43bbced..306058fb3681e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4635,6 +4635,15 @@ object SQLConf { .booleanConf .createWithDefault(true) + val PUSH_VARIANT_INTO_SCAN = + buildConf("spark.sql.variant.pushVariantIntoScan") + .internal() + .doc("When true, replace variant type in the scan schema with a struct containing " + + "requested fields.") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + val LEGACY_CSV_ENABLE_DATE_TIME_PARSING_FALLBACK = buildConf("spark.sql.legacy.csv.enableDateTimeParsingFallback") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 6ceb363b41aef..a51870cfd7fdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.execution.datasources.{PruneFileSourcePartitions, SchemaPruning, V1Writes} +import org.apache.spark.sql.execution.datasources.{PruneFileSourcePartitions, PushVariantIntoScan, SchemaPruning, V1Writes} import org.apache.spark.sql.execution.datasources.v2.{GroupBasedRowLevelOperationScanPlanning, OptimizeMetadataOnlyDeleteFromTable, V2ScanPartitioningAndOrdering, V2ScanRelationPushDown, V2Writes} import org.apache.spark.sql.execution.dynamicpruning.{CleanupDynamicPruningFilters, PartitionPruning, RowLevelOperationRuntimeGroupFiltering} import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs, ExtractPythonUDTFs} @@ -43,7 +43,8 @@ class SparkOptimizer( V2ScanRelationPushDown, V2ScanPartitioningAndOrdering, V2Writes, - PruneFileSourcePartitions) + PruneFileSourcePartitions, + PushVariantIntoScan) override def preCBORules: Seq[Rule[LogicalPlan]] = Seq(OptimizeMetadataOnlyDeleteFromTable) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala new file mode 100644 index 0000000000000..83d219c28983b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala @@ -0,0 +1,340 @@ +/* + * 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 + +import scala.collection.mutable.HashMap + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.variant.{VariantGet, VariantPathParser} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project, Subquery} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +// A metadata class of a struct field. All struct fields in a struct must either all have this +// metadata, or all don't have it. +// We define a "variant struct" as: a special struct with its fields annotated with this metadata. +// It indicates that the struct should produce all requested fields of a variant type, and should be +// treated specially by the scan. +case class VariantMetadata( + // The `path` parameter of VariantGet. It has the same format as a JSON path, except that + // `[*]` is not supported. + path: String, + failOnError: Boolean, + timeZoneId: String) { + // Produce a metadata contain one key-value pair. The key is the special `METADATA_KEY`. + // The value contains three key-value pairs for `path`, `failOnError`, and `timeZoneId`. + def toMetadata: Metadata = + new MetadataBuilder().putMetadata( + VariantMetadata.METADATA_KEY, + new MetadataBuilder() + .putString(VariantMetadata.PATH_KEY, path) + .putBoolean(VariantMetadata.FAIL_ON_ERROR_KEY, failOnError) + .putString(VariantMetadata.TIME_ZONE_ID_KEY, timeZoneId) + .build() + ).build() + + def parsedPath(): Array[VariantPathParser.PathSegment] = { + VariantPathParser.parse(path).getOrElse { + val name = if (failOnError) "variant_get" else "try_variant_get" + throw QueryExecutionErrors.invalidVariantGetPath(path, name) + } + } +} + +object VariantMetadata { + val METADATA_KEY = "__VARIANT_METADATA_KEY" + val PATH_KEY = "path" + val FAIL_ON_ERROR_KEY = "failOnError" + val TIME_ZONE_ID_KEY = "timeZoneId" + + def isVariantStruct(s: StructType): Boolean = + s.fields.length > 0 && s.fields.forall(_.metadata.contains(METADATA_KEY)) + + def isVariantStruct(t: DataType): Boolean = t match { + case s: StructType => isVariantStruct(s) + case _ => false + } + + // Parse the `VariantMetadata` from a metadata produced by `toMetadata`. + def fromMetadata(metadata: Metadata): VariantMetadata = { + val value = metadata.getMetadata(METADATA_KEY) + VariantMetadata( + value.getString(PATH_KEY), + value.getBoolean(FAIL_ON_ERROR_KEY), + value.getString(TIME_ZONE_ID_KEY) + ) + } +} + +// Represent a requested field of a variant that the scan should produce. +// Each `RequestedVariantField` is corresponded to a variant path extraction in the plan. +case class RequestedVariantField(path: VariantMetadata, targetType: DataType) + +object RequestedVariantField { + def fullVariant: RequestedVariantField = + RequestedVariantField(VariantMetadata("$", failOnError = true, "UTC"), VariantType) + + def apply(v: VariantGet): RequestedVariantField = + RequestedVariantField( + VariantMetadata(v.path.eval().toString, v.failOnError, v.timeZoneId.get), v.dataType) + + def apply(c: Cast): RequestedVariantField = + RequestedVariantField( + VariantMetadata("$", c.evalMode != EvalMode.TRY, c.timeZoneId.get), c.dataType) +} + +// Extract a nested struct access path. Return the (root attribute id, a sequence of ordinals to +// access the field). For non-nested attribute access, the sequence is empty. +object StructPath { + def unapply(expr: Expression): Option[(ExprId, Seq[Int])] = expr match { + case GetStructField(StructPath(root, path), ordinal, _) => Some((root, path :+ ordinal)) + case a: Attribute => Some(a.exprId, Nil) + case _ => None + } +} + +// A collection of all eligible variants in a relation, which are in the root of the relation output +// schema, or only nested in struct types. +// The user should: +// 1. Call `addVariantFields` to add all eligible variants in a relation. +// 2. Call `collectRequestedFields` on all expressions depending on the relation. This process will +// add the requested fields of each variant and potentially remove non-eligible variants. See +// `collectRequestedFields` for details. +// 3. Call `rewriteType` to produce a new output schema for the relation. +// 4. Call `rewriteExpr` to rewrite the previously visited expressions by replacing variant +// extractions with struct accessed. +class VariantInRelation { + // First level key: root attribute id. + // Second level key: struct access paths to the variant type. + // Third level key: requested fields of a variant type. + // Final value: the ordinal of a requested field in the final struct of requested fields. + val mapping = new HashMap[ExprId, HashMap[Seq[Int], HashMap[RequestedVariantField, Int]]] + + // Extract the SQL-struct path where the leaf is a variant. + object StructPathToVariant { + def unapply(expr: Expression): Option[HashMap[RequestedVariantField, Int]] = expr match { + case StructPath(attrId, path) => + mapping.get(attrId).flatMap(_.get(path)) + case _ => None + } + } + + // Find eligible variants recursively. `attrId` is the root attribute id. + // `path` is the current struct access path. `dataType` is the child data type after extracting + // `path` from the root attribute struct. + def addVariantFields( + attrId: ExprId, + dataType: DataType, + defaultValue: Any, + path: Seq[Int]): Unit = { + dataType match { + // TODO(SHREDDING): non-null default value is not yet supported. + case _: VariantType if defaultValue == null => + mapping.getOrElseUpdate(attrId, new HashMap).put(path, new HashMap) + case s: StructType if !VariantMetadata.isVariantStruct(s) => + val row = defaultValue.asInstanceOf[InternalRow] + for ((field, idx) <- s.fields.zipWithIndex) { + val fieldDefault = if (row == null || row.isNullAt(idx)) { + null + } else { + row.get(idx, field.dataType) + } + addVariantFields(attrId, field.dataType, fieldDefault, path :+ idx) + } + case _ => + } + } + + def rewriteType(attrId: ExprId, dataType: DataType, path: Seq[Int]): DataType = { + dataType match { + case _: VariantType => + mapping.get(attrId).flatMap(_.get(path)) match { + case Some(fields) => + var requestedFields = fields.toArray.sortBy(_._2).map { case (field, ordinal) => + StructField(ordinal.toString, field.targetType, metadata = field.path.toMetadata) + } + // Avoid producing an empty struct of requested fields. This is intended to simplify the + // scan implementation, which may not be able to handle empty struct type. This happens + // if the variant is not used, or only used in `IsNotNull/IsNull` expressions. The value + // of the placeholder field doesn't matter, even if the scan source accidentally + // contains such a field. + if (requestedFields.isEmpty) { + val placeholder = VariantMetadata("$.__placeholder_field__", + failOnError = false, timeZoneId = "UTC") + requestedFields = Array(StructField("0", BooleanType, + metadata = placeholder.toMetadata)) + } + StructType(requestedFields) + case _ => dataType + } + case s: StructType if !VariantMetadata.isVariantStruct(s) => + val newFields = s.fields.zipWithIndex.map { case (field, idx) => + field.copy(dataType = rewriteType(attrId, field.dataType, path :+ idx)) + } + StructType(newFields) + case _ => dataType + } + } + + // Add a requested field to a variant column. + private def addField( + map: HashMap[RequestedVariantField, Int], + field: RequestedVariantField): Unit = { + val idx = map.size + map.getOrElseUpdate(field, idx) + } + + // Update `mapping` with any access to a variant. Add the requested fields of each variant and + // potentially remove non-eligible variants. + // If a struct containing a variant is directly used, this variant is not eligible for push down. + // This is because we need to replace the variant type with a struct producing all requested + // fields, which also changes the struct type containing it, and it is difficult to reconstruct + // the original struct value. This is not a big loss, because we need the full variant anyway. + def collectRequestedFields(expr: Expression): Unit = expr match { + case v@VariantGet(StructPathToVariant(fields), _, _, _, _) => + addField(fields, RequestedVariantField(v)) + case c@Cast(StructPathToVariant(fields), _, _, _) => addField(fields, RequestedVariantField(c)) + case IsNotNull(StructPath(_, _)) | IsNull(StructPath(_, _)) => + case StructPath(attrId, path) => + mapping.get(attrId) match { + case Some(variants) => + variants.get(path) match { + case Some(fields) => + addField(fields, RequestedVariantField.fullVariant) + case _ => + // Remove non-eligible variants. + variants.filterInPlace { case (key, _) => !key.startsWith(path) } + } + case _ => + } + case _ => expr.children.foreach(collectRequestedFields) + } + + def rewriteExpr( + expr: Expression, + attributeMap: Map[ExprId, AttributeReference]): Expression = { + def rewriteAttribute(expr: Expression): Expression = expr.transformDown { + case a: Attribute => attributeMap.getOrElse(a.exprId, a) + } + + // Rewrite patterns should be consistent with visit patterns in `collectRequestedFields`. + expr.transformDown { + case g@VariantGet(v@StructPathToVariant(fields), _, _, _, _) => + // Rewrite the attribute in advance, rather than depending on the last branch to rewrite it. + // Ww need to avoid the `v@StructPathToVariant(fields)` branch to rewrite the child again. + GetStructField(rewriteAttribute(v), fields(RequestedVariantField(g))) + case c@Cast(v@StructPathToVariant(fields), _, _, _) => + GetStructField(rewriteAttribute(v), fields(RequestedVariantField(c))) + case i@IsNotNull(StructPath(_, _)) => rewriteAttribute(i) + case i@IsNull(StructPath(_, _)) => rewriteAttribute(i) + case v@StructPathToVariant(fields) => + GetStructField(rewriteAttribute(v), fields(RequestedVariantField.fullVariant)) + case a: Attribute => attributeMap.getOrElse(a.exprId, a) + } + } +} + +// Push variant into scan by rewriting the variant type with a struct type producing all requested +// fields and rewriting the variant extraction expressions by struct accesses. +// For example, for an input plan: +// - Project [v:a::int, v:b::string, v] +// - Filter [v:a::int = 1] +// - Relation [v: variant] +// Rewrite it as: +// - Project [v.0, v.1, v.2] +// - Filter [v.0 = 1] +// - Relation [v: struct<0: int, 1: string, 2: variant>] +// The struct fields are annotated with `VariantMetadata` to indicate the extraction path. +object PushVariantIntoScan extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan match { + // A correlated subquery will be rewritten into join later, and will go through this rule + // eventually. + case s: Subquery if s.correlated => plan + case _ if !SQLConf.get.getConf(SQLConf.PUSH_VARIANT_INTO_SCAN) => plan + case _ => plan.transformDown { + case p@PhysicalOperation(projectList, filters, + relation @ LogicalRelationWithTable( + hadoopFsRelation@HadoopFsRelation(_, _, _, _, _: ParquetFileFormat, _), _)) => + rewritePlan(p, projectList, filters, relation, hadoopFsRelation) + } + } + + private def rewritePlan( + originalPlan: LogicalPlan, + projectList: Seq[NamedExpression], + filters: Seq[Expression], + relation: LogicalRelation, + hadoopFsRelation: HadoopFsRelation): LogicalPlan = { + val variants = new VariantInRelation + val defaultValues = ResolveDefaultColumns.existenceDefaultValues(hadoopFsRelation.schema) + // I'm not aware of any case that an attribute `relation.output` can have a different data type + // than the corresponding field in `hadoopFsRelation.schema`. Other code seems to prefer using + // the data type in `hadoopFsRelation.schema`, let's also stick to it. + val schemaWithAttributes = hadoopFsRelation.schema.fields.zip(relation.output) + for (((f, attr), defaultValue) <- schemaWithAttributes.zip(defaultValues)) { + variants.addVariantFields(attr.exprId, f.dataType, defaultValue, Nil) + } + if (variants.mapping.isEmpty) return originalPlan + + projectList.foreach(variants.collectRequestedFields) + filters.foreach(variants.collectRequestedFields) + // `collectRequestedFields` may have removed all variant columns. + if (variants.mapping.forall(_._2.isEmpty)) return originalPlan + + val (newFields, newOutput) = schemaWithAttributes.map { + case (f, attr) => + if (variants.mapping.get(attr.exprId).exists(_.nonEmpty)) { + val newType = variants.rewriteType(attr.exprId, f.dataType, Nil) + val newAttr = AttributeReference(f.name, newType, f.nullable, f.metadata)() + (f.copy(dataType = newType), newAttr) + } else { + (f, attr) + } + }.unzip + + val newHadoopFsRelation = hadoopFsRelation.copy(dataSchema = StructType(newFields))( + hadoopFsRelation.sparkSession) + val newRelation = relation.copy(relation = newHadoopFsRelation, output = newOutput.toIndexedSeq) + + val attributeMap = relation.output.zip(newOutput).map { + case (oldAttr, newAttr) => oldAttr.exprId -> newAttr + }.toMap + val withFilter = if (filters.nonEmpty) { + Filter(filters.map(variants.rewriteExpr(_, attributeMap)).reduce(And), newRelation) + } else { + newRelation + } + val newProjectList = projectList.map { e => + val rewritten = variants.rewriteExpr(e, attributeMap) + rewritten match { + case n: NamedExpression => n + // This is when the variant column is directly selected. We replace the attribute reference + // with a struct access, which is not a `NamedExpression` that `Project` requires. We wrap + // it with an `Alias`. + case _ => Alias(rewritten, e.name)(e.exprId, e.qualifier) + } + } + Project(newProjectList, withFilter) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScanSuite.scala new file mode 100644 index 0000000000000..2a866dcd66f06 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScanSuite.scala @@ -0,0 +1,178 @@ +/* + * 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 + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.variant._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ + +class PushVariantIntoScanSuite extends SharedSparkSession { + override def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.PUSH_VARIANT_INTO_SCAN.key, "true") + + private def localTimeZone = spark.sessionState.conf.sessionLocalTimeZone + + // Return a `StructField` with the expected `VariantMetadata`. + private def field(ordinal: Int, dataType: DataType, path: String, + failOnError: Boolean = true, timeZone: String = localTimeZone): StructField = + StructField(ordinal.toString, dataType, + metadata = VariantMetadata(path, failOnError, timeZone).toMetadata) + + // Validate an `Alias` expression has the expected name and child. + private def checkAlias(expr: Expression, expectedName: String, expected: Expression): Unit = { + expr match { + case Alias(child, name) => + assert(name == expectedName) + assert(child == expected) + case _ => fail() + } + } + + private def testOnFormats(fn: String => Unit): Unit = { + for (format <- Seq("PARQUET")) { + test("test - " + format) { + withTable("T") { + fn(format) + } + } + } + } + + testOnFormats { format => + sql("create table T (v variant, vs struct, " + + "va array, vd variant default parse_json('1')) " + + s"using $format") + + sql("select variant_get(v, '$.a', 'int') as a, v, cast(v as struct) as v from T") + .queryExecution.optimizedPlan match { + case Project(projectList, l: LogicalRelation) => + val output = l.output + val v = output(0) + checkAlias(projectList(0), "a", GetStructField(v, 0)) + checkAlias(projectList(1), "v", GetStructField(v, 1)) + checkAlias(projectList(2), "v", GetStructField(v, 2)) + assert(v.dataType == StructType(Array( + field(0, IntegerType, "$.a"), + field(1, VariantType, "$", timeZone = "UTC"), + field(2, StructType(Array(StructField("b", FloatType))), "$")))) + case _ => fail() + } + + sql("select 1 from T where isnotnull(v)") + .queryExecution.optimizedPlan match { + case Project(projectList, Filter(condition, l: LogicalRelation)) => + val output = l.output + val v = output(0) + checkAlias(projectList(0), "1", Literal(1)) + assert(condition == IsNotNull(v)) + assert(v.dataType == StructType(Array( + field(0, BooleanType, "$.__placeholder_field__", failOnError = false, timeZone = "UTC")))) + case _ => fail() + } + + sql("select variant_get(v, '$.a', 'int') + 1 as a, try_variant_get(v, '$.b', 'string') as b " + + "from T where variant_get(v, '$.a', 'int') = 1").queryExecution.optimizedPlan match { + case Project(projectList, Filter(condition, l: LogicalRelation)) => + val output = l.output + val v = output(0) + checkAlias(projectList(0), "a", Add(GetStructField(v, 0), Literal(1))) + checkAlias(projectList(1), "b", GetStructField(v, 1)) + assert(condition == And(IsNotNull(v), EqualTo(GetStructField(v, 0), Literal(1)))) + assert(v.dataType == StructType(Array( + field(0, IntegerType, "$.a"), + field(1, StringType, "$.b", failOnError = false)))) + case _ => fail() + } + + sql("select variant_get(vs.v1, '$.a', 'int') as a, variant_get(vs.v1, '$.b', 'int') as b, " + + "variant_get(vs.v2, '$.a', 'int') as a, vs.i from T").queryExecution.optimizedPlan match { + case Project(projectList, l: LogicalRelation) => + val output = l.output + val vs = output(1) + val v1 = GetStructField(vs, 0, Some("v1")) + val v2 = GetStructField(vs, 1, Some("v2")) + checkAlias(projectList(0), "a", GetStructField(v1, 0)) + checkAlias(projectList(1), "b", GetStructField(v1, 1)) + checkAlias(projectList(2), "a", GetStructField(v2, 0)) + checkAlias(projectList(3), "i", GetStructField(vs, 2, Some("i"))) + assert(vs.dataType == StructType(Array( + StructField("v1", StructType(Array( + field(0, IntegerType, "$.a"), field(1, IntegerType, "$.b")))), + StructField("v2", StructType(Array(field(0, IntegerType, "$.a")))), + StructField("i", IntegerType)))) + case _ => fail() + } + + def variantGet(child: Expression): Expression = VariantGet( + child, + path = Literal("$.a"), + targetType = VariantType, + failOnError = true, + timeZoneId = Some(localTimeZone)) + + // No push down if the struct containing variant is used. + sql("select vs, variant_get(vs.v1, '$.a') as a from T").queryExecution.optimizedPlan match { + case Project(projectList, l: LogicalRelation) => + val output = l.output + val vs = output(1) + assert(projectList(0) == vs) + checkAlias(projectList(1), "a", variantGet(GetStructField(vs, 0, Some("v1")))) + assert(vs.dataType == StructType(Array( + StructField("v1", VariantType), + StructField("v2", VariantType), + StructField("i", IntegerType)))) + case _ => fail() + } + + // No push down for variant in array. + sql("select variant_get(va[0], '$.a') as a from T").queryExecution.optimizedPlan match { + case Project(projectList, l: LogicalRelation) => + val output = l.output + val va = output(2) + checkAlias(projectList(0), "a", variantGet(GetArrayItem(va, Literal(0)))) + assert(va.dataType == ArrayType(VariantType)) + case _ => fail() + } + + // No push down if variant has default value. + sql("select variant_get(vd, '$.a') as a from T").queryExecution.optimizedPlan match { + case Project(projectList, l: LogicalRelation) => + val output = l.output + val vd = output(3) + checkAlias(projectList(0), "a", variantGet(vd)) + assert(vd.dataType == VariantType) + case _ => fail() + } + } + + test("No push down for JSON") { + withTable("T") { + sql("create table T (v variant) using JSON") + sql("select variant_get(v, '$.a') from T").queryExecution.optimizedPlan match { + case Project(_, l: LogicalRelation) => + val output = l.output + assert(output(0).dataType == VariantType) + case _ => fail() + } + } + } +} From 629fe3f4217303793b05338eaa371d92b4a7ac51 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 20 Dec 2024 23:21:46 +0800 Subject: [PATCH 224/438] [SPARK-50370][SQL] Codegen Support for `json_tuple` ### What changes were proposed in this pull request? The pr aims to add `Codegen` Support for `JsonTuple`(`json_tuple`). ### Why are the changes needed? - improve codegen coverage. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA & Existed UT (eg: JsonFunctionsSuite#`*json_tuple*`, JsonExpressionsSuite#`*json_tuple*`) - Updated existed UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48908 from panbingkun/SPARK-50370. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../json/JsonExpressionEvalUtils.scala | 120 ++++++++++++- .../expressions/jsonExpressions.scala | 158 +++++------------- .../expressions/JsonExpressionsSuite.scala | 5 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 24 +++ 4 files changed, 184 insertions(+), 123 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala index cc2bc19c4f779..7ff2bfe51729c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala @@ -16,10 +16,13 @@ */ package org.apache.spark.sql.catalyst.expressions.json -import java.io.CharArrayWriter +import java.io.{ByteArrayOutputStream, CharArrayWriter} +import com.fasterxml.jackson.core._ + +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow, SharedFactory} import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonGenerator, JacksonParser, JsonInferSchema, JSONOptions} import org.apache.spark.sql.catalyst.util.{ArrayData, FailFastMode, FailureSafeParser, MapData, PermissiveMode} @@ -162,3 +165,116 @@ case class SchemaOfJsonEvaluator(options: Map[String, String]) { UTF8String.fromString(dt.sql) } } + +/** + * The expression `JsonTuple` will utilize it to support codegen. + */ +case class JsonTupleEvaluator(foldableFieldNames: Array[Option[String]]) { + + import SharedFactory._ + + // If processing fails this shared value will be returned. + @transient private lazy val nullRow: Seq[InternalRow] = + new GenericInternalRow(Array.ofDim[Any](foldableFieldNames.length)) :: Nil + + // And count the number of foldable fields, we'll use this later to optimize evaluation. + @transient private lazy val constantFields: Int = foldableFieldNames.count(_ != null) + + private def getFieldNameStrings(fields: Array[UTF8String]): Array[String] = { + // Evaluate the field names as String rather than UTF8String to + // optimize lookups from the json token, which is also a String. + if (constantFields == fields.length) { + // Typically the user will provide the field names as foldable expressions + // so we can use the cached copy. + foldableFieldNames.map(_.orNull) + } else if (constantFields == 0) { + // None are foldable so all field names need to be evaluated from the input row. + fields.map { f => if (f != null) f.toString else null } + } else { + // If there is a mix of constant and non-constant expressions + // prefer the cached copy when available. + foldableFieldNames.zip(fields).map { + case (null, f) => if (f != null) f.toString else null + case (fieldName, _) => fieldName.orNull + } + } + } + + private def parseRow(parser: JsonParser, fieldNames: Array[String]): Seq[InternalRow] = { + // Only objects are supported. + if (parser.nextToken() != JsonToken.START_OBJECT) return nullRow + + val row = Array.ofDim[Any](fieldNames.length) + + // Start reading through the token stream, looking for any requested field names. + while (parser.nextToken() != JsonToken.END_OBJECT) { + if (parser.getCurrentToken == JsonToken.FIELD_NAME) { + // Check to see if this field is desired in the output. + val jsonField = parser.currentName + var idx = fieldNames.indexOf(jsonField) + if (idx >= 0) { + // It is, copy the child tree to the correct location in the output row. + val output = new ByteArrayOutputStream() + + // Write the output directly to UTF8 encoded byte array. + if (parser.nextToken() != JsonToken.VALUE_NULL) { + Utils.tryWithResource(jsonFactory.createGenerator(output, JsonEncoding.UTF8)) { + generator => copyCurrentStructure(generator, parser) + } + + val jsonValue = UTF8String.fromBytes(output.toByteArray) + + // SPARK-21804: json_tuple returns null values within repeated columns + // except the first one; so that we need to check the remaining fields. + do { + row(idx) = jsonValue + idx = fieldNames.indexOf(jsonField, idx + 1) + } while (idx >= 0) + } + } + } + + // Always skip children, it's cheap enough to do even if copyCurrentStructure was called. + parser.skipChildren() + } + new GenericInternalRow(row) :: Nil + } + + private def copyCurrentStructure(generator: JsonGenerator, parser: JsonParser): Unit = { + parser.getCurrentToken match { + // If the user requests a string field it needs to be returned without enclosing + // quotes which is accomplished via JsonGenerator.writeRaw instead of JsonGenerator.write. + case JsonToken.VALUE_STRING if parser.hasTextCharacters => + // Slight optimization to avoid allocating a String instance, though the characters + // still have to be decoded... Jackson doesn't have a way to access the raw bytes. + generator.writeRaw(parser.getTextCharacters, parser.getTextOffset, parser.getTextLength) + + case JsonToken.VALUE_STRING => + // The normal String case, pass it through to the output without enclosing quotes. + generator.writeRaw(parser.getText) + + case JsonToken.VALUE_NULL => + // A special case that needs to be handled outside of this method. + // If a requested field is null, the result must be null. The easiest + // way to achieve this is just by ignoring null tokens entirely. + throw SparkException.internalError("Do not attempt to copy a null field.") + + case _ => + // Handle other types including objects, arrays, booleans and numbers. + generator.copyCurrentStructure(parser) + } + } + + final def evaluate(json: UTF8String, fieldNames: Array[UTF8String]): IterableOnce[InternalRow] = { + if (json == null) return nullRow + try { + /* We know the bytes are UTF-8 encoded. Pass a Reader to avoid having Jackson + detect character encoding which could fail for some malformed strings. */ + Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => + parseRow(parser, getFieldNameStrings(fieldNames)) + } + } catch { + case _: JsonProcessingException => nullRow + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index bb8cbdd3e6f18..93eda307ec9c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -28,9 +28,9 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, CodegenFallback, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper -import org.apache.spark.sql.catalyst.expressions.json.{JsonExpressionUtils, JsonToStructsEvaluator, SchemaOfJsonEvaluator, StructsToJsonEvaluator} +import org.apache.spark.sql.catalyst.expressions.json.{JsonExpressionUtils, JsonToStructsEvaluator, JsonTupleEvaluator, SchemaOfJsonEvaluator, StructsToJsonEvaluator} import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.trees.TreePattern.{JSON_TO_STRUCT, RUNTIME_REPLACEABLE, TreePattern} @@ -106,7 +106,7 @@ private[this] object JsonPathParser extends RegexParsers { } } -private[this] object SharedFactory { +private[expressions] object SharedFactory { val jsonFactory = new JsonFactoryBuilder() // The two options below enabled for Hive compatibility .enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS) @@ -448,37 +448,27 @@ class GetJsonObjectEvaluator(cachedPath: UTF8String) { // scalastyle:on line.size.limit line.contains.tab case class JsonTuple(children: Seq[Expression]) extends Generator - with CodegenFallback with QueryErrorsBase { - import SharedFactory._ - override def nullable: Boolean = { - // a row is always returned + // A row is always returned. false } - // if processing fails this shared value will be returned - @transient private lazy val nullRow: Seq[InternalRow] = - new GenericInternalRow(Array.ofDim[Any](fieldExpressions.length)) :: Nil - - // the json body is the first child + // The json body is the first child. @transient private lazy val jsonExpr: Expression = children.head - // the fields to query are the remaining children + // The fields to query are the remaining children. @transient private lazy val fieldExpressions: Seq[Expression] = children.tail - // eagerly evaluate any foldable the field names - @transient private lazy val foldableFieldNames: IndexedSeq[Option[String]] = { + // Eagerly evaluate any foldable the field names. + @transient private lazy val foldableFieldNames: Array[Option[String]] = { fieldExpressions.map { case expr if expr.foldable => Option(expr.eval()).map(_.asInstanceOf[UTF8String].toString) case _ => null - }.toIndexedSeq + }.toArray } - // and count the number of foldable fields, we'll use this later to optimize evaluation - @transient private lazy val constantFields: Int = foldableFieldNames.count(_ != null) - override def elementSchema: StructType = StructType(fieldExpressions.zipWithIndex.map { case (_, idx) => StructField(s"c$idx", children.head.dataType, nullable = true) }) @@ -502,111 +492,41 @@ case class JsonTuple(children: Seq[Expression]) } } + @transient + private lazy val evaluator: JsonTupleEvaluator = JsonTupleEvaluator(foldableFieldNames) + override def eval(input: InternalRow): IterableOnce[InternalRow] = { val json = jsonExpr.eval(input).asInstanceOf[UTF8String] - if (json == null) { - return nullRow - } - - try { - /* We know the bytes are UTF-8 encoded. Pass a Reader to avoid having Jackson - detect character encoding which could fail for some malformed strings */ - Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => - parseRow(parser, input) - } - } catch { - case _: JsonProcessingException => - nullRow - } + val filedNames = fieldExpressions.map(_.eval(input).asInstanceOf[UTF8String]).toArray + evaluator.evaluate(json, filedNames) } - private def parseRow(parser: JsonParser, input: InternalRow): Seq[InternalRow] = { - // only objects are supported - if (parser.nextToken() != JsonToken.START_OBJECT) { - return nullRow - } - - // evaluate the field names as String rather than UTF8String to - // optimize lookups from the json token, which is also a String - val fieldNames = if (constantFields == fieldExpressions.length) { - // typically the user will provide the field names as foldable expressions - // so we can use the cached copy - foldableFieldNames.map(_.orNull) - } else if (constantFields == 0) { - // none are foldable so all field names need to be evaluated from the input row - fieldExpressions.map { expr => - Option(expr.eval(input)).map(_.asInstanceOf[UTF8String].toString).orNull - } - } else { - // if there is a mix of constant and non-constant expressions - // prefer the cached copy when available - foldableFieldNames.zip(fieldExpressions).map { - case (null, expr) => - Option(expr.eval(input)).map(_.asInstanceOf[UTF8String].toString).orNull - case (fieldName, _) => fieldName.orNull - } - } - - val row = Array.ofDim[Any](fieldNames.length) - - // start reading through the token stream, looking for any requested field names - while (parser.nextToken() != JsonToken.END_OBJECT) { - if (parser.getCurrentToken == JsonToken.FIELD_NAME) { - // check to see if this field is desired in the output - val jsonField = parser.currentName - var idx = fieldNames.indexOf(jsonField) - if (idx >= 0) { - // it is, copy the child tree to the correct location in the output row - val output = new ByteArrayOutputStream() - - // write the output directly to UTF8 encoded byte array - if (parser.nextToken() != JsonToken.VALUE_NULL) { - Utils.tryWithResource(jsonFactory.createGenerator(output, JsonEncoding.UTF8)) { - generator => copyCurrentStructure(generator, parser) - } - - val jsonValue = UTF8String.fromBytes(output.toByteArray) - - // SPARK-21804: json_tuple returns null values within repeated columns - // except the first one; so that we need to check the remaining fields. - do { - row(idx) = jsonValue - idx = fieldNames.indexOf(jsonField, idx + 1) - } while (idx >= 0) - } - } - } - - // always skip children, it's cheap enough to do even if copyCurrentStructure was called - parser.skipChildren() - } - - new GenericInternalRow(row) :: Nil - } - - private def copyCurrentStructure(generator: JsonGenerator, parser: JsonParser): Unit = { - parser.getCurrentToken match { - // if the user requests a string field it needs to be returned without enclosing - // quotes which is accomplished via JsonGenerator.writeRaw instead of JsonGenerator.write - case JsonToken.VALUE_STRING if parser.hasTextCharacters => - // slight optimization to avoid allocating a String instance, though the characters - // still have to be decoded... Jackson doesn't have a way to access the raw bytes - generator.writeRaw(parser.getTextCharacters, parser.getTextOffset, parser.getTextLength) - - case JsonToken.VALUE_STRING => - // the normal String case, pass it through to the output without enclosing quotes - generator.writeRaw(parser.getText) - - case JsonToken.VALUE_NULL => - // a special case that needs to be handled outside of this method. - // if a requested field is null, the result must be null. the easiest - // way to achieve this is just by ignoring null tokens entirely - throw SparkException.internalError("Do not attempt to copy a null field.") - - case _ => - // handle other types including objects, arrays, booleans and numbers - generator.copyCurrentStructure(parser) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val refEvaluator = ctx.addReferenceObj("evaluator", evaluator) + val jsonEval = jsonExpr.genCode(ctx) + val filedNamesTerm = ctx.freshName("fieldNames") + val fieldNamesEval = fieldExpressions.map(_.genCode(ctx)) + val wrapperClass = classOf[IterableOnce[_]].getName + val setFieldNames = fieldNamesEval.zipWithIndex.map { + case (fieldNameEval, idx) => + s""" + |if (${fieldNameEval.isNull}) { + | $filedNamesTerm[$idx] = null; + |} else { + | $filedNamesTerm[$idx] = ${fieldNameEval.value}; + |} + |""".stripMargin } + ev.copy(code = + code""" + |UTF8String[] $filedNamesTerm = new UTF8String[${fieldExpressions.length}]; + |${jsonEval.code} + |${fieldNamesEval.map(_.code).mkString("\n")} + |${setFieldNames.mkString("\n")} + |boolean ${ev.isNull} = false; + |$wrapperClass ${ev.value} = + | $refEvaluator.evaluate(${jsonEval.value}, $filedNamesTerm); + |""".stripMargin) } override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): JsonTuple = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 2c82ce712d957..0ec1a93b5cd29 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -273,8 +273,9 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with } test("json_tuple escaping") { - GenerateUnsafeProjection.generate( - JsonTuple(Literal("\"quote") :: Literal("\"quote") :: Nil) :: Nil) + checkJsonTuple( + JsonTuple(Literal("\"quote") :: Literal("\"quote") :: Nil), + InternalRow.fromSeq(Seq(null).map(UTF8String.fromString))) } test("json_tuple - hive key 1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 3803360f2da4b..ea185b6b4901a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -1455,4 +1455,28 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { assert(plan.isInstanceOf[WholeStageCodegenExec]) checkAnswer(df, Row(null)) } + + test("function json_tuple - field names foldable") { + withTempView("t") { + val json = """{"a":1, "b":2, "c":3}""" + val df = Seq((json, "a", "b", "c")).toDF("json", "c1", "c2", "c3") + df.createOrReplaceTempView("t") + + // Json and all field names are foldable. + val df1 = sql(s"SELECT json_tuple('$json', 'a', 'b', 'c') from t") + checkAnswer(df1, Row("1", "2", "3")) + + // All field names are foldable. + val df2 = sql("SELECT json_tuple(json, 'a', 'b', 'c') from t") + checkAnswer(df2, Row("1", "2", "3")) + + // The field names some foldable, some non-foldable. + val df3 = sql("SELECT json_tuple(json, 'a', c2, 'c') from t") + checkAnswer(df3, Row("1", "2", "3")) + + // All field names are non-foldable. + val df4 = sql("SELECT json_tuple(json, c1, c2, c3) from t") + checkAnswer(df4, Row("1", "2", "3")) + } + } } From bccdf1ffd467cb60ca6e100c20a1b659102eb304 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 20 Dec 2024 23:23:33 +0800 Subject: [PATCH 225/438] [SPARK-50483][SPARK-50545][DOC][FOLLOWUP] Mention behavior changes in migration guide ### What changes were proposed in this pull request? Update migration guide for SPARK-50483 and SPARK-50545 ### Why are the changes needed? Mention behavior changes in migration guide ### Does this PR introduce _any_ user-facing change? Yes, docs are updated. ### How was this patch tested? Review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49252 from pan3793/SPARK-50483-SPARK-50545-followup. Authored-by: Cheng Pan Signed-off-by: Wenchen Fan --- docs/core-migration-guide.md | 6 ++++++ docs/sql-migration-guide.md | 5 +++++ 2 files changed, 11 insertions(+) diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 958e442545dcd..49737392312a7 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -54,6 +54,12 @@ license: | - Since Spark 4.0, `spark.shuffle.unsafe.file.output.buffer` is deprecated though still works. Use `spark.shuffle.localDisk.file.output.buffer` instead. +- Since Spark 4.0, when reading files hits `org.apache.hadoop.security.AccessControlException` and `org.apache.hadoop.hdfs.BlockMissingException`, the exception will be thrown and fail the task, even if `spark.files.ignoreCorruptFiles` is set to `true`. + +## Upgrading from Core 3.5.3 to 3.5.4 + +- Since Spark 3.5.4, when reading files hits `org.apache.hadoop.security.AccessControlException` and `org.apache.hadoop.hdfs.BlockMissingException`, the exception will be thrown and fail the task, even if `spark.files.ignoreCorruptFiles` is set to `true`. + ## Upgrading from Core 3.4 to 3.5 - Since Spark 3.5, `spark.yarn.executor.failuresValidityInterval` is deprecated. Use `spark.executor.failuresValidityInterval` instead. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 717d27befef02..254c54a414a7e 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -29,6 +29,7 @@ license: | - Since Spark 4.0, the default behaviour when inserting elements in a map is changed to first normalize keys -0.0 to 0.0. The affected SQL functions are `create_map`, `map_from_arrays`, `map_from_entries`, and `map_concat`. To restore the previous behaviour, set `spark.sql.legacy.disableMapKeyNormalization` to `true`. - Since Spark 4.0, the default value of `spark.sql.maxSinglePartitionBytes` is changed from `Long.MaxValue` to `128m`. To restore the previous behavior, set `spark.sql.maxSinglePartitionBytes` to `9223372036854775807`(`Long.MaxValue`). - Since Spark 4.0, any read of SQL tables takes into consideration the SQL configs `spark.sql.files.ignoreCorruptFiles`/`spark.sql.files.ignoreMissingFiles` instead of the core config `spark.files.ignoreCorruptFiles`/`spark.files.ignoreMissingFiles`. +- Since Spark 4.0, when reading SQL tables hits `org.apache.hadoop.security.AccessControlException` and `org.apache.hadoop.hdfs.BlockMissingException`, the exception will be thrown and fail the task, even if `spark.sql.files.ignoreCorruptFiles` is set to `true`. - Since Spark 4.0, `spark.sql.hive.metastore` drops the support of Hive prior to 2.0.0 as they require JDK 8 that Spark does not support anymore. Users should migrate to higher versions. - Since Spark 4.0, `spark.sql.parquet.compression.codec` drops the support of codec name `lz4raw`, please use `lz4_raw` instead. - Since Spark 4.0, when overflowing during casting timestamp to byte/short/int under non-ansi mode, Spark will return null instead a wrapping value. @@ -63,6 +64,10 @@ license: | - Since Spark 4.0, The Storage-Partitioned Join feature flag `spark.sql.sources.v2.bucketing.pushPartValues.enabled` is set to `true`. To restore the previous behavior, set `spark.sql.sources.v2.bucketing.pushPartValues.enabled` to `false`. - Since Spark 4.0, the `sentences` function uses `Locale(language)` instead of `Locale.US` when `language` parameter is not `NULL` and `country` parameter is `NULL`. +## Upgrading from Spark SQL 3.5.3 to 3.5.4 + +- Since Spark 3.5.4, when reading SQL tables hits `org.apache.hadoop.security.AccessControlException` and `org.apache.hadoop.hdfs.BlockMissingException`, the exception will be thrown and fail the task, even if `spark.sql.files.ignoreCorruptFiles` is set to `true`. + ## Upgrading from Spark SQL 3.5.1 to 3.5.2 - Since 3.5.2, MySQL JDBC datasource will read TINYINT UNSIGNED as ShortType, while in 3.5.1, it was wrongly read as ByteType. From a2e3188b4997001f4dbc1eb364d61ca55d438208 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 20 Dec 2024 14:28:32 -0800 Subject: [PATCH 226/438] [SPARK-50640][CORE][TESTS] Update `ChecksumBenchmark` by removing `PureJavaCrc32C` and setting `Adler32` as a baseline ### What changes were proposed in this pull request? This PR aims to update `ChecksumBenchmark` by - Removing `PureJavaCrc32C` - Setting `Adler32` as a baseline ### Why are the changes needed? Not only Apache Spark, but also Apache Hadoop community doesn't use that legacy code on Java 9+ since 2018 from Hadoop 3.1.0 (HADOOP-15033). - https://github.com/apache/hadoop/pull/291 We can save our resources by removing obsolete code usage and focusing on our available options. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. I also attached Apple Silicon result. **Java 17** ``` [info] OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.3 [info] Apple M3 Max [info] Checksum Algorithms: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative [info] ---------------------------------------------------------------------------------------------------- [info] Adler32 8689 8709 28 0.0 8485001.2 1.0X [info] CRC32 3201 3205 4 0.0 3125877.4 2.7X [info] CRC32C 3199 3205 5 0.0 3124264.6 2.7X ``` **Java 21** ``` [info] OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Mac OS X 15.3 [info] Apple M3 Max [info] Checksum Algorithms: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative [info] --------------------------------------------------------------------------------------------------- [info] Adler32 9208 9226 20 0.0 8991732.4 1.0X [info] CRC32 3238 3357 105 0.0 3162007.9 2.8X [info] CRC32C 3224 3351 110 0.0 3147966.1 2.9X ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49258 from dongjoon-hyun/SPARK-50640. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/benchmarks/ChecksumBenchmark-jdk21-results.txt | 9 ++++----- core/benchmarks/ChecksumBenchmark-results.txt | 9 ++++----- .../org/apache/spark/shuffle/ChecksumBenchmark.scala | 11 +++-------- 3 files changed, 11 insertions(+), 18 deletions(-) diff --git a/core/benchmarks/ChecksumBenchmark-jdk21-results.txt b/core/benchmarks/ChecksumBenchmark-jdk21-results.txt index 85370450f355c..8230f15aa659c 100644 --- a/core/benchmarks/ChecksumBenchmark-jdk21-results.txt +++ b/core/benchmarks/ChecksumBenchmark-jdk21-results.txt @@ -2,13 +2,12 @@ Benchmark Checksum Algorithms ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Checksum Algorithms: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -CRC32 2743 2746 3 0.0 2678409.9 1.0X -CRC32C 1974 2055 70 0.0 1928129.2 1.4X -Adler32 12689 12709 17 0.0 12391425.9 0.2X -hadoop PureJavaCrc32C 23027 23041 13 0.0 22487098.9 0.1X +Adler32 11116 11123 7 0.0 10855585.4 1.0X +CRC32 2774 2777 4 0.0 2709448.1 4.0X +CRC32C 2083 2148 65 0.0 2034177.5 5.3X diff --git a/core/benchmarks/ChecksumBenchmark-results.txt b/core/benchmarks/ChecksumBenchmark-results.txt index cce5a61abf637..bf4e6182edfd0 100644 --- a/core/benchmarks/ChecksumBenchmark-results.txt +++ b/core/benchmarks/ChecksumBenchmark-results.txt @@ -2,13 +2,12 @@ Benchmark Checksum Algorithms ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Checksum Algorithms: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -CRC32 2757 2758 1 0.0 2692250.2 1.0X -CRC32C 2142 2244 116 0.0 2091901.8 1.3X -Adler32 12699 12712 15 0.0 12401205.6 0.2X -hadoop PureJavaCrc32C 23049 23066 15 0.0 22508320.3 0.1X +Adler32 11112 11130 20 0.0 10851949.2 1.0X +CRC32 2765 2767 2 0.0 2699749.0 4.0X +CRC32C 2101 2159 54 0.0 2051565.3 5.3X diff --git a/core/src/test/scala/org/apache/spark/shuffle/ChecksumBenchmark.scala b/core/src/test/scala/org/apache/spark/shuffle/ChecksumBenchmark.scala index 16a50fabb7ffd..4eb167930b0dd 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ChecksumBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ChecksumBenchmark.scala @@ -19,8 +19,6 @@ package org.apache.spark.shuffle import java.util.zip.{Adler32, CRC32, CRC32C} -import org.apache.hadoop.util.PureJavaCrc32C - import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} /** @@ -41,18 +39,15 @@ object ChecksumBenchmark extends BenchmarkBase { runBenchmark("Benchmark Checksum Algorithms") { val data: Array[Byte] = (1 until 32 * 1024 * 1024).map(_.toByte).toArray val benchmark = new Benchmark("Checksum Algorithms", N, 3, output = output) + benchmark.addCase(s"Adler32") { _ => + (1 to N).foreach(_ => new Adler32().update(data)) + } benchmark.addCase("CRC32") { _ => (1 to N).foreach(_ => new CRC32().update(data)) } benchmark.addCase(s"CRC32C") { _ => (1 to N).foreach(_ => new CRC32C().update(data)) } - benchmark.addCase(s"Adler32") { _ => - (1 to N).foreach(_ => new Adler32().update(data)) - } - benchmark.addCase(s"hadoop PureJavaCrc32C") { _ => - (1 to N).foreach(_ => new PureJavaCrc32C().update(data)) - } benchmark.run() } } From 482a27c82fbac46cde5a82d1e75cdb1811862d06 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Sun, 22 Dec 2024 22:31:51 +0300 Subject: [PATCH 227/438] [SPARK-50637][SQL] Fix code style for the single-pass Analyzer ### What changes were proposed in this pull request? Fix some nits that were left out from the initial single-pass Analyzer review. ### Why are the changes needed? To make code style better. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49254 from vladimirg-db/vladimirg-db/single-pass-analyzer/fix-code-style. Authored-by: Vladimir Golubev Signed-off-by: Max Gekk --- .../analysis/resolver/HybridAnalyzer.scala | 6 +++--- .../apache/spark/sql/internal/SQLConf.scala | 20 ++++++++++--------- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala index 4ebcb3e06b178..039c07f5edbc2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala @@ -75,9 +75,6 @@ class HybridAnalyzer( } } - private def checkResolverGuard(plan: LogicalPlan): Boolean = - !checkSupportedSinglePassFeatures || resolverGuard.apply(plan) - def getSinglePassResolutionDuration: Option[Long] = singlePassResolutionDuration def getFixedPointResolutionDuration: Option[Long] = fixedPointResolutionDuration @@ -218,6 +215,9 @@ class HybridAnalyzer( NormalizePlan(plan) } + private def checkResolverGuard(plan: LogicalPlan): Boolean = + !checkSupportedSinglePassFeatures || resolverGuard.apply(plan) + private def recordDuration[T](thunk: => T): (Long, T) = { val start = System.nanoTime() val res = thunk diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 306058fb3681e..be883b2112d19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -264,15 +264,17 @@ object SQLConf { .booleanConf .createWithDefault(false) - val ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER = - buildConf("spark.sql.analyzer.singlePassResolver.dualRunWithLegacy") - .internal() - .doc("When true, run both analyzers to check if single-pass Analyzer correctly produces" + - " the same analyzed plan as the fixed-point Analyzer for the existing set of features" + - " defined in the ResolverGuard") - .version("4.0.0") - .booleanConf - .createWithDefault(false) + val ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER = + buildConf("spark.sql.analyzer.singlePassResolver.dualRunWithLegacy") + .internal() + .doc( + "When true, run both analyzers to check if single-pass Analyzer correctly produces " + + "the same analyzed plan as the fixed-point Analyzer for the existing set of features " + + "defined in the ResolverGuard" + ) + .version("4.0.0") + .booleanConf + .createWithDefault(false) val ANALYZER_SINGLE_PASS_RESOLVER_VALIDATION_ENABLED = buildConf("spark.sql.analyzer.singlePassResolver.validationEnabled") From 827d2a016c54e2c89b35a8128aefaabf677b9a7b Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Sun, 22 Dec 2024 22:35:00 +0300 Subject: [PATCH 228/438] [SPARK-50638][SQL] Refactor the view resolution into the separate file to reuse it in the single-pass Analyzer ### What changes were proposed in this pull request? Refactor the view resolution into the separate file to reuse it in the single-pass Analyzer. ### Why are the changes needed? To reuse this code from the single-pass Analyzer. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49255 from vladimirg-db/vladimirg-db/refactor-view-resolution-to-a-separate-file. Authored-by: Vladimir Golubev Signed-off-by: Max Gekk --- .../sql/catalyst/analysis/Analyzer.scala | 23 +------- .../catalyst/analysis/ViewResolution.scala | 55 +++++++++++++++++++ 2 files changed, 58 insertions(+), 20 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7c97fccade018..35ae0125d141e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1036,26 +1036,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // If `AnalysisContext.catalogAndNamespace` is non-empty, analyzer will expand single-part names // with it, instead of current catalog and namespace. private def resolveViews(plan: LogicalPlan): LogicalPlan = plan match { - // The view's child should be a logical plan parsed from the `desc.viewText`, the variable - // `viewText` should be defined, or else we throw an error on the generation of the View - // operator. - case view @ View(desc, isTempView, child) if !child.resolved => - // Resolve all the UnresolvedRelations and Views in the child. - val newChild = AnalysisContext.withAnalysisContext(desc) { - val nestedViewDepth = AnalysisContext.get.nestedViewDepth - val maxNestedViewDepth = AnalysisContext.get.maxNestedViewDepth - if (nestedViewDepth > maxNestedViewDepth) { - throw QueryCompilationErrors.viewDepthExceedsMaxResolutionDepthError( - desc.identifier, maxNestedViewDepth, view) - } - SQLConf.withExistingConf(View.effectiveSQLConf(desc.viewSQLConfigs, isTempView)) { - executeSameContext(child) - } - } - // Fail the analysis eagerly because outside AnalysisContext, the unresolved operators - // inside a view maybe resolved incorrectly. - checkAnalysis(newChild) - view.copy(child = newChild) + case view: View if !view.child.resolved => + ViewResolution + .resolve(view, resolveChild = executeSameContext, checkAnalysis = checkAnalysis) case p @ SubqueryAlias(_, view: View) => p.copy(child = resolveViews(view)) case _ => plan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala new file mode 100644 index 0000000000000..89ef29ddaaf1c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala @@ -0,0 +1,55 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, View} +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf + +object ViewResolution { + def resolve( + view: View, + resolveChild: LogicalPlan => LogicalPlan, + checkAnalysis: LogicalPlan => Unit): View = { + // The view's child should be a logical plan parsed from the `desc.viewText`, the variable + // `viewText` should be defined, or else we throw an error on the generation of the View + // operator. + + // Resolve all the UnresolvedRelations and Views in the child. + val newChild = AnalysisContext.withAnalysisContext(view.desc) { + val nestedViewDepth = AnalysisContext.get.nestedViewDepth + val maxNestedViewDepth = AnalysisContext.get.maxNestedViewDepth + if (nestedViewDepth > maxNestedViewDepth) { + throw QueryCompilationErrors.viewDepthExceedsMaxResolutionDepthError( + view.desc.identifier, + maxNestedViewDepth, + view + ) + } + SQLConf.withExistingConf(View.effectiveSQLConf(view.desc.viewSQLConfigs, view.isTempView)) { + resolveChild(view.child) + } + } + + // Fail the analysis eagerly because outside AnalysisContext, the unresolved operators + // inside a view maybe resolved incorrectly. + checkAnalysis(newChild) + + view.copy(child = newChild) + } +} From 5ac42e2fe122c972cd3a2ecc87b11b1a70758f72 Mon Sep 17 00:00:00 2001 From: changgyoopark-db Date: Mon, 23 Dec 2024 09:03:47 +0900 Subject: [PATCH 229/438] [SPARK-50534][SPARK-50535][TEST][CONNECT] Fix sporadic test failures ### What changes were proposed in this pull request? Fix sporadic Spark Connect test failures. 1. SPARK-50534: VerifyEvents.this.listener.executeHolder was not declared "volatile", causing the thread to repeatedly read potentially outdated value. The data structure is only used by the test suite. 2. SPARK-50535: org.apache.spark.sql.connect.service.SparkConnectSessionManager.invalidateAllSessions is susceptible to system time synchronization (e.g., NTP), leaving stale sessions. invalidateAllSessions is only used by test suites. ### Why are the changes needed? Fix sporadic test failures. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Repeatedly ran testOnly org.apache.spark.sql.connect.planner.SparkConnectServiceSuite and org.apache.spark.sql.connect.service.SparkConnectServiceE2ESuite. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49253 from changgyoopark-db/SPARK-50534. Authored-by: changgyoopark-db Signed-off-by: Hyukjin Kwon --- .../sql/connect/service/SparkConnectSessionManager.scala | 6 ++++-- .../sql/connect/planner/SparkConnectServiceSuite.scala | 3 ++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala index a306856efa33c..b0b74a36e187b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala @@ -289,8 +289,10 @@ class SparkConnectSessionManager extends Logging { * Used for testing */ private[connect] def invalidateAllSessions(): Unit = { - periodicMaintenance(defaultInactiveTimeoutMs = 0L, ignoreCustomTimeout = true) - assert(sessionStore.isEmpty) + sessionStore.forEach((key, sessionHolder) => { + removeSessionHolder(key) + shutdownSessionHolder(sessionHolder) + }) closedSessionsCache.invalidateAll() } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala index d6d137e6d91aa..5e88725691656 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala @@ -919,7 +919,8 @@ class SparkConnectServiceSuite } class MockSparkListener() extends SparkListener { val semaphoreStarted = new Semaphore(0) - var executeHolder = Option.empty[ExecuteHolder] + // Accessed by multiple threads in parallel. + @volatile var executeHolder = Option.empty[ExecuteHolder] override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { case e: SparkListenerConnectOperationStarted => From f6f6dae726e5c5125baab5bb6a73535fa1494dfd Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 23 Dec 2024 09:51:25 +0800 Subject: [PATCH 230/438] [SPARK-50590][INFRA] Skip uncessary image build and push ### What changes were proposed in this pull request? Skip uncessary image build and push ### Why are the changes needed? for master, the build of `./dev/infra/` is not needed; for branch-3.5, the build of new images is not needed; skip those steps by branch, to make sure right images are used ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #49204 from zhengruifeng/infra_avoid_build. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index f827cb9a63731..758deca93c82e 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -393,6 +393,7 @@ jobs: if: >- fromJson(needs.precondition.outputs.required).pyspark == 'true' || fromJson(needs.precondition.outputs.required).lint == 'true' || + fromJson(needs.precondition.outputs.required).docs == 'true' || fromJson(needs.precondition.outputs.required).sparkr == 'true' runs-on: ubuntu-latest permissions: @@ -422,7 +423,8 @@ jobs: uses: docker/setup-qemu-action@v3 - name: Set up Docker Buildx uses: docker/setup-buildx-action@v3 - - name: Build and push + - name: Build and push for branch-3.5 + if: inputs.branch == 'branch-3.5' id: docker_build uses: docker/build-push-action@v6 with: @@ -433,7 +435,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-cache:${{ inputs.branch }} - name: Build and push (Documentation) - if: hashFiles('dev/spark-test-image/docs/Dockerfile') != '' + if: ${{ inputs.branch != 'branch-3.5' && hashFiles('dev/spark-test-image/docs/Dockerfile') != '' }} id: docker_build_docs uses: docker/build-push-action@v6 with: @@ -444,7 +446,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-docs-cache:${{ inputs.branch }} - name: Build and push (Linter) - if: hashFiles('dev/spark-test-image/lint/Dockerfile') != '' + if: ${{ inputs.branch != 'branch-3.5' && hashFiles('dev/spark-test-image/lint/Dockerfile') != '' }} id: docker_build_lint uses: docker/build-push-action@v6 with: @@ -455,7 +457,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-lint-cache:${{ inputs.branch }} - name: Build and push (SparkR) - if: hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' + if: ${{ inputs.branch != 'branch-3.5' && hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' }} id: docker_build_sparkr uses: docker/build-push-action@v6 with: From f8fd398439d02fae865be09bce81207f54a98efb Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 23 Dec 2024 11:06:07 +0800 Subject: [PATCH 231/438] [SPARK-50645][INFRA] Make more daily builds able to manually trigger ### What changes were proposed in this pull request? similar to https://github.com/apache/spark/pull/49207, make the rest daily builds able to be executed manually ### Why are the changes needed? re-run failed tests in a daily build won't fetch the latest code changes, so somethimes we don't have a quick way to test daily build in time. ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? will manually check after merge ### Was this patch authored or co-authored using generative AI tooling? no Closes #49264 from zhengruifeng/add_manually_trigger. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_coverage.yml | 1 + .github/workflows/build_infra_images_cache.yml | 1 + .github/workflows/build_java21.yml | 1 + .github/workflows/build_maven.yml | 1 + .github/workflows/build_maven_java21.yml | 1 + .github/workflows/build_maven_java21_macos15.yml | 1 + .github/workflows/build_non_ansi.yml | 1 + .github/workflows/build_python_3.10.yml | 1 + .github/workflows/build_python_3.11_macos.yml | 1 + .github/workflows/build_python_3.12.yml | 1 + .github/workflows/build_python_3.13.yml | 1 + .github/workflows/build_python_3.9.yml | 1 + .github/workflows/build_python_connect.yml | 1 + .github/workflows/build_python_connect35.yml | 1 + .github/workflows/build_python_pypy3.10.yml | 1 + .github/workflows/build_rockdb_as_ui_backend.yml | 1 + .github/workflows/build_sparkr_window.yml | 1 + 17 files changed, 17 insertions(+) diff --git a/.github/workflows/build_coverage.yml b/.github/workflows/build_coverage.yml index 8da65927f37f7..007d9ce99c847 100644 --- a/.github/workflows/build_coverage.yml +++ b/.github/workflows/build_coverage.yml @@ -22,6 +22,7 @@ name: "Build / Coverage (master, Scala 2.13, Hadoop 3, JDK 17)" on: schedule: - cron: '0 10 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index cd57e070e21b3..930932420bda0 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -39,6 +39,7 @@ on: - '.github/workflows/build_infra_images_cache.yml' # Create infra image when cutting down branches/tags create: + workflow_dispatch: jobs: main: if: github.repository == 'apache/spark' diff --git a/.github/workflows/build_java21.yml b/.github/workflows/build_java21.yml index 826e6f46e358a..51ece691f9284 100644 --- a/.github/workflows/build_java21.yml +++ b/.github/workflows/build_java21.yml @@ -22,6 +22,7 @@ name: "Build (master, Scala 2.13, Hadoop 3, JDK 21)" on: schedule: - cron: '0 4 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_maven.yml b/.github/workflows/build_maven.yml index b5546c61eb11b..e047390add6f9 100644 --- a/.github/workflows/build_maven.yml +++ b/.github/workflows/build_maven.yml @@ -22,6 +22,7 @@ name: "Build / Maven (master, Scala 2.13, Hadoop 3, JDK 17)" on: schedule: - cron: '0 13 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_maven_java21.yml b/.github/workflows/build_maven_java21.yml index 127904145464b..9fbc7b84383f0 100644 --- a/.github/workflows/build_maven_java21.yml +++ b/.github/workflows/build_maven_java21.yml @@ -22,6 +22,7 @@ name: "Build / Maven (master, Scala 2.13, Hadoop 3, JDK 21)" on: schedule: - cron: '0 14 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_maven_java21_macos15.yml b/.github/workflows/build_maven_java21_macos15.yml index cc6d0ea4e90da..377a67191ab49 100644 --- a/.github/workflows/build_maven_java21_macos15.yml +++ b/.github/workflows/build_maven_java21_macos15.yml @@ -22,6 +22,7 @@ name: "Build / Maven (master, Scala 2.13, Hadoop 3, JDK 21, MacOS-15)" on: schedule: - cron: '0 20 */2 * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_non_ansi.yml b/.github/workflows/build_non_ansi.yml index 32489ea8c858d..31654476ea3f8 100644 --- a/.github/workflows/build_non_ansi.yml +++ b/.github/workflows/build_non_ansi.yml @@ -22,6 +22,7 @@ name: "Build / Non-ANSI (master, Hadoop 3, JDK 17, Scala 2.13)" on: schedule: - cron: '0 1 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_python_3.10.yml b/.github/workflows/build_python_3.10.yml index 131f22935f111..9b0c90c5c7747 100644 --- a/.github/workflows/build_python_3.10.yml +++ b/.github/workflows/build_python_3.10.yml @@ -22,6 +22,7 @@ name: "Build / Python-only (master, Python 3.10)" on: schedule: - cron: '0 17 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_python_3.11_macos.yml b/.github/workflows/build_python_3.11_macos.yml index 4caae55b5fea8..57902e4871ffa 100644 --- a/.github/workflows/build_python_3.11_macos.yml +++ b/.github/workflows/build_python_3.11_macos.yml @@ -22,6 +22,7 @@ name: "Build / Python-only (master, Python 3.11, MacOS)" on: schedule: - cron: '0 21 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_python_3.12.yml b/.github/workflows/build_python_3.12.yml index 2503a2f158357..e0c04700554ca 100644 --- a/.github/workflows/build_python_3.12.yml +++ b/.github/workflows/build_python_3.12.yml @@ -22,6 +22,7 @@ name: "Build / Python-only (master, Python 3.12)" on: schedule: - cron: '0 19 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_python_3.13.yml b/.github/workflows/build_python_3.13.yml index 5c62a431e3f0d..e85b1577f323f 100644 --- a/.github/workflows/build_python_3.13.yml +++ b/.github/workflows/build_python_3.13.yml @@ -22,6 +22,7 @@ name: "Build / Python-only (master, Python 3.13)" on: schedule: - cron: '0 20 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_python_3.9.yml b/.github/workflows/build_python_3.9.yml index 744e18cc8db39..0df17699140ed 100644 --- a/.github/workflows/build_python_3.9.yml +++ b/.github/workflows/build_python_3.9.yml @@ -22,6 +22,7 @@ name: "Build / Python-only (master, Python 3.9)" on: schedule: - cron: '0 21 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index b15e15abed1c3..c2151a0093f7f 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -22,6 +22,7 @@ name: Build / Spark Connect Python-only (master, Python 3.11) on: schedule: - cron: '0 19 * * *' + workflow_dispatch: jobs: # Build: build Spark and run the tests for specified modules using SBT diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml index 080932c6c09c0..95ec5af719975 100644 --- a/.github/workflows/build_python_connect35.yml +++ b/.github/workflows/build_python_connect35.yml @@ -22,6 +22,7 @@ name: Build / Spark Connect Python-only (master-server, 35-client, Python 3.11) on: schedule: - cron: '0 21 * * *' + workflow_dispatch: jobs: # Build: build Spark and run the tests for specified modules using SBT diff --git a/.github/workflows/build_python_pypy3.10.yml b/.github/workflows/build_python_pypy3.10.yml index c590427f6f49e..0bd2ef03ce77c 100644 --- a/.github/workflows/build_python_pypy3.10.yml +++ b/.github/workflows/build_python_pypy3.10.yml @@ -22,6 +22,7 @@ name: "Build / Python-only (master, PyPy 3.10)" on: schedule: - cron: '0 15 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_rockdb_as_ui_backend.yml b/.github/workflows/build_rockdb_as_ui_backend.yml index a85edd09256ed..1d9a079e72643 100644 --- a/.github/workflows/build_rockdb_as_ui_backend.yml +++ b/.github/workflows/build_rockdb_as_ui_backend.yml @@ -22,6 +22,7 @@ name: "Build / RocksDB as UI Backend (master, Hadoop 3, JDK 17, Scala 2.13)" on: schedule: - cron: '0 6 * * *' + workflow_dispatch: jobs: run-build: diff --git a/.github/workflows/build_sparkr_window.yml b/.github/workflows/build_sparkr_window.yml index b97251a461715..b28e81908549f 100644 --- a/.github/workflows/build_sparkr_window.yml +++ b/.github/workflows/build_sparkr_window.yml @@ -21,6 +21,7 @@ name: "Build / SparkR-only (master, 4.4.2, windows-2022)" on: schedule: - cron: '0 17 * * *' + workflow_dispatch: jobs: build: From ab95c4e05fa986f42c4e1adcba9d5048c32f8d45 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 23 Dec 2024 12:16:00 +0800 Subject: [PATCH 232/438] [SPARK-50527][INFRA][FOLLOW-UP] Python 3.12 image clean up ### What changes were proposed in this pull request? Python 3.12 image clean up ### Why are the changes needed? `python3.12` was installed twice no similar issue were found in other new images ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? CI, will monitor the Python 3.12 daily build ### Was this patch authored or co-authored using generative AI tooling? no Closes #49266 from zhengruifeng/infra_12_cleanup. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- dev/spark-test-image/python-312/Dockerfile | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index 0a8d5b79a361a..090c20742e652 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -71,13 +71,10 @@ ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.1 googleapis-common-protos==1.65.0 graphviz==0.20.3" -# Install Python 3.12 at the last stage to avoid breaking the existing Python installations -RUN apt-get update && apt-get install -y \ - python3.12 \ - && rm -rf /var/lib/apt/lists/* +# Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 RUN python3.12 -m pip install --ignore-installed blinker>=1.6.2 # mlflow needs this RUN python3.12 -m pip install $BASIC_PIP_PKGS unittest-xml-reporting $CONNECT_PIP_PKGS lxml && \ python3.12 -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ python3.12 -m pip install torcheval && \ - python3.12 -m pip cache purge \ + python3.12 -m pip cache purge From 08675b16927080e8c305483327db8da9564a5bc5 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Mon, 23 Dec 2024 12:53:59 +0800 Subject: [PATCH 233/438] [SPARK-50630][SQL] Fix GROUP BY ordinal support for pipe SQL AGGREGATE operators ### What changes were proposed in this pull request? This PR fixes GROUP BY ordinal support for pipe SQL AGGREGATE operators. It adds a new `UnresolvedPipeAggregateOrdinal` expression to represent these ordinals. In this context, the ordinal refers to the one-based position of the column in the input relation. Note that this behavior is different from GROUP BY ordinals in regular SQL, wherein the ordinal refers to the one-based position of the column in the SELECT clause instead. For example: ``` select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 2, 3 > 4, 5, 4 select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 1, 2, 3 > 3, 4, 5, 4 ``` This PR also makes a small fix for `|> UNION` (and other set operations) to prefer future pipe operators to apply on the result of the entire union, rather than binding to the right leg of the union only (to allay reported confusion during testing). For example, `values (0, 1) s(x, y) |> union all values (2, 3) t(x, y) |> drop x` will succeed rather than report an error that the number of columns does not match. ### Why are the changes needed? The current implementation has a bug where the ordinals are sometimes mistakenly retained as literal integers. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds new golden file based test coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49248 from dtenedor/group-by-ordinals-pipe-aggregate. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 26 ++- .../sql/catalyst/analysis/unresolved.scala | 22 ++ .../sql/catalyst/parser/AstBuilder.scala | 33 ++- .../analyzer-results/pipe-operators.sql.out | 200 +++++++++++++++-- .../sql-tests/inputs/pipe-operators.sql | 54 ++++- .../sql-tests/results/pipe-operators.sql.out | 205 ++++++++++++++++-- 7 files changed, 496 insertions(+), 46 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 8ef7ab90c6ff7..e743aa2a744f7 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -1523,7 +1523,7 @@ operatorPipeRightSide | unpivotClause pivotClause? | sample | joinRelation - | operator=(UNION | EXCEPT | SETMINUS | INTERSECT) setQuantifier? right=queryTerm + | operator=(UNION | EXCEPT | SETMINUS | INTERSECT) setQuantifier? right=queryPrimary | queryOrganization | AGGREGATE namedExpressionSeq? aggregationClause? ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 35ae0125d141e..e8839148f51b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1887,10 +1887,14 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // Replace the index with the corresponding expression in aggregateExpressions. The index is // a 1-base position of aggregateExpressions, which is output columns (select expression) - case Aggregate(groups, aggs, child, hint) if aggs.forall(_.resolved) && + case Aggregate(groups, aggs, child, hint) + if aggs + .filter(!containUnresolvedPipeAggregateOrdinal(_)) + .forall(_.resolved) && groups.exists(containUnresolvedOrdinal) => - val newGroups = groups.map(resolveGroupByExpressionOrdinal(_, aggs)) - Aggregate(newGroups, aggs, child, hint) + val newAggs = aggs.map(resolvePipeAggregateExpressionOrdinal(_, child.output)) + val newGroups = groups.map(resolveGroupByExpressionOrdinal(_, newAggs)) + Aggregate(newGroups, newAggs, child, hint) } private def containUnresolvedOrdinal(e: Expression): Boolean = e match { @@ -1899,6 +1903,11 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case _ => false } + private def containUnresolvedPipeAggregateOrdinal(e: Expression): Boolean = e match { + case UnresolvedAlias(_: UnresolvedPipeAggregateOrdinal, _) => true + case _ => false + } + private def resolveGroupByExpressionOrdinal( expr: Expression, aggs: Seq[Expression]): Expression = expr match { @@ -1934,6 +1943,17 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } + private def resolvePipeAggregateExpressionOrdinal( + expr: NamedExpression, + inputs: Seq[Attribute]): NamedExpression = expr match { + case UnresolvedAlias(UnresolvedPipeAggregateOrdinal(index), _) => + // In this case, the user applied the SQL pipe aggregate operator ("|> AGGREGATE") and used + // ordinals in its GROUP BY clause. This expression then refers to the i-th attribute of the + // child operator (one-based). Here we resolve the ordinal to the corresponding attribute. + inputs(index - 1) + case other => + other + } /** * Checks whether a function identifier referenced by an [[UnresolvedFunction]] is defined in the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index b14f4be534a38..61b68b743a5c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -956,6 +956,28 @@ case class UnresolvedOrdinal(ordinal: Int) final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_ORDINAL) } +/** + * Represents an unresolved ordinal used in the GROUP BY clause of a SQL pipe aggregate operator + * ("|> AGGREGATE"). + * + * In this context, the ordinal refers to the one-based position of the column in the input + * relation. Note that this behavior is different from GROUP BY ordinals in regular SQL, wherein the + * ordinal refers to the one-based position of the column in the SELECT clause. + * + * For example: + * {{{ + * values ('abc', 'def') tab(x, y) + * |> aggregate sum(x) group by 2 + * }}} + * @param ordinal ordinal starts from 1, instead of 0 + */ +case class UnresolvedPipeAggregateOrdinal(ordinal: Int) + extends LeafExpression with Unevaluable with NonSQLExpression { + override def dataType: DataType = throw new UnresolvedException("dataType") + override def nullable: Boolean = throw new UnresolvedException("nullable") + override lazy val resolved = false +} + /** * Represents unresolved having clause, the child for it can be Aggregate, GroupingSets, Rollup * and Cube. It is turned by the analyzer into a Filter. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 64491264f3e9e..f95c0d6556bad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -6016,7 +6016,8 @@ class AstBuilder extends DataTypeAstBuilder // analyzer behave as if we had added the corresponding SQL clause after a table subquery // containing the input plan. def withSubqueryAlias(): LogicalPlan = left match { - case _: SubqueryAlias | _: UnresolvedRelation | _: Join | _: Filter => + case _: SubqueryAlias | _: UnresolvedRelation | _: Join | _: Filter | + _: GlobalLimit | _: LocalLimit | _: Offset | _: Sort => left case _ => SubqueryAlias(SubqueryAlias.generateSubqueryName(), left) @@ -6137,7 +6138,7 @@ class AstBuilder extends DataTypeAstBuilder "The AGGREGATE clause requires a list of aggregate expressions " + "or a list of grouping expressions, or both", ctx) } - // Visit each aggregate expression, and add a PipeAggregate expression on top of it to generate + // Visit each aggregate expression, and add a [[PipeExpression]] on top of it to generate // clear error messages if the expression does not contain at least one aggregate function. val aggregateExpressions: Seq[NamedExpression] = Option(ctx.namedExpressionSeq()).map { n: NamedExpressionSeqContext => @@ -6183,12 +6184,28 @@ class AstBuilder extends DataTypeAstBuilder a.aggregateExpressions.foreach(visit) // Prepend grouping keys to the list of aggregate functions, since operator pipe AGGREGATE // clause returns the GROUP BY expressions followed by the list of aggregate functions. - val namedGroupingExpressions: Seq[NamedExpression] = - a.groupingExpressions.map { - case n: NamedExpression => n - case e: Expression => UnresolvedAlias(e, None) - } - a.copy(aggregateExpressions = namedGroupingExpressions ++ a.aggregateExpressions) + val newGroupingExpressions = ArrayBuffer.empty[Expression] + val newAggregateExpressions = ArrayBuffer.empty[NamedExpression] + a.groupingExpressions.foreach { + case n: NamedExpression => + newGroupingExpressions += n + newAggregateExpressions += n + // If the grouping expression is an integer literal, create [[UnresolvedOrdinal]] and + // [[UnresolvedPipeAggregateOrdinal]] expressions to represent it in the final grouping + // and aggregate expressions, respectively. This will let the + // [[ResolveOrdinalInOrderByAndGroupBy]] rule detect the ordinal in the aggregate list + // and replace it with the corresponding attribute from the child operator. + case Literal(v: Int, IntegerType) if conf.groupByOrdinal => + newGroupingExpressions += UnresolvedOrdinal(newAggregateExpressions.length + 1) + newAggregateExpressions += UnresolvedAlias(UnresolvedPipeAggregateOrdinal(v), None) + case e: Expression => + newGroupingExpressions += e + newAggregateExpressions += UnresolvedAlias(e, None) + } + newAggregateExpressions.appendAll(a.aggregateExpressions) + a.copy( + groupingExpressions = newGroupingExpressions.toSeq, + aggregateExpressions = newAggregateExpressions.toSeq) } }.getOrElse { // This is a table aggregation with no grouping expressions. diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index 1121d8baf5dba..70de582fb7b25 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -1539,6 +1539,78 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +table t +|> select x, length(y) as z +|> limit 1000 +|> where x + length(y) < 4 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`x`, `z`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "y" + } ] +} + + +-- !query +table t +|> select x, length(y) as z +|> limit 1000 offset 1 +|> where x + length(y) < 4 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`x`, `z`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 80, + "stopIndex" : 80, + "fragment" : "y" + } ] +} + + +-- !query +table t +|> select x, length(y) as z +|> order by x, y +|> where x + length(y) < 4 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`x`, `z`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 52, + "fragment" : "y" + } ] +} + + -- !query (select x, sum(length(y)) as sum_len from t group by x) |> where sum(length(y)) = 3 @@ -2697,21 +2769,34 @@ Union false, false -- !query -values (0, 1) tab(x, y) +values (2, 'xyz') tab(x, y) |> union table t |> where x = 0 -- !query analysis -Distinct -+- Union false, false - :- Project [x#x, cast(y#x as bigint) AS y#xL] - : +- SubqueryAlias tab - : +- LocalRelation [x#x, y#x] - +- Project [x#x, cast(y#x as bigint) AS y#xL] - +- Filter (x#x = 0) +Filter (x#x = 0) ++- SubqueryAlias __auto_generated_subquery_name + +- Distinct + +- Union false, false + :- SubqueryAlias tab + : +- LocalRelation [x#x, y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv +-- !query +values (2, 'xyz') tab(x, y) +|> union table t +|> drop x +-- !query analysis +Project [y#x] ++- Distinct + +- Union false, false + :- SubqueryAlias tab + : +- LocalRelation [x#x, y#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + -- !query (select * from t) |> union all (select * from t) @@ -2878,10 +2963,9 @@ table t -- !query analysis GlobalLimit 1 +- LocalLimit 1 - +- SubqueryAlias __auto_generated_subquery_name - +- Sort [x#x ASC NULLS FIRST], true - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- Sort [x#x ASC NULLS FIRST], true + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -3109,11 +3193,101 @@ Aggregate [x#x, y#x], [x#x, y#x] select 3 as x, 4 as y |> aggregate group by 1, 2 -- !query analysis -Aggregate [1, 2], [1 AS 1#x, 2 AS 2#x] +Aggregate [x#x, y#x], [x#x, y#x] +- Project [3 AS x#x, 4 AS y#x] +- OneRowRelation +-- !query +values (3, 4) as tab(x, y) +|> aggregate sum(y) group by 1 +-- !query analysis +Aggregate [x#x], [x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- SubqueryAlias tab + +- LocalRelation [x#x, y#x] + + +-- !query +values (3, 4), (5, 4) as tab(x, y) +|> aggregate sum(y) group by 1 +-- !query analysis +Aggregate [x#x], [x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- SubqueryAlias tab + +- LocalRelation [x#x, y#x] + + +-- !query +select 3 as x, 4 as y +|> aggregate sum(y) group by 1, 1 +-- !query analysis +Aggregate [x#x, x#x], [x#x, x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- Project [3 AS x#x, 4 AS y#x] + +- OneRowRelation + + +-- !query +select 1 as `1`, 2 as `2` +|> aggregate sum(`2`) group by `1` +-- !query analysis +Aggregate [1#x], [1#x, pipeexpression(sum(2#x), true, AGGREGATE) AS pipeexpression(sum(2))#xL] ++- Project [1 AS 1#x, 2 AS 2#x] + +- OneRowRelation + + +-- !query +select 3 as x, 4 as y +|> aggregate sum(y) group by 2 +-- !query analysis +Aggregate [y#x], [y#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- Project [3 AS x#x, 4 AS y#x] + +- OneRowRelation + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2 +-- !query analysis +Aggregate [y#x], [y#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] + +- OneRowRelation + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 3 +-- !query analysis +Aggregate [z#x], [z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] + +- OneRowRelation + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2, 3 +-- !query analysis +Aggregate [y#x, z#x], [y#x, z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] + +- OneRowRelation + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 1, 2, 3 +-- !query analysis +Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] + +- OneRowRelation + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by x, 2, 3 +-- !query analysis +Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] ++- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] + +- OneRowRelation + + -- !query table t |> aggregate sum(x) diff --git a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql index 1299da3020d59..ec4afc6b23721 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql @@ -542,6 +542,21 @@ table t |> select x, length(y) as z |> where x + length(y) < 4; +table t +|> select x, length(y) as z +|> limit 1000 +|> where x + length(y) < 4; + +table t +|> select x, length(y) as z +|> limit 1000 offset 1 +|> where x + length(y) < 4; + +table t +|> select x, length(y) as z +|> order by x, y +|> where x + length(y) < 4; + -- If the WHERE clause wants to filter rows produced by an aggregation, it is not valid to try to -- refer to the aggregate functions directly; it is necessary to use aliases instead. (select x, sum(length(y)) as sum_len from t group by x) @@ -843,10 +858,17 @@ values (0, 'abc') tab(x, y) |> union all table t; -- Union distinct with a VALUES list. -values (0, 1) tab(x, y) +-- The |> WHERE operator applies to the result of the |> UNION operator, not to the "table t" input. +values (2, 'xyz') tab(x, y) |> union table t |> where x = 0; +-- Union distinct with a VALUES list. +-- The |> DROP operator applies to the result of the |> UNION operator, not to the "table t" input. +values (2, 'xyz') tab(x, y) +|> union table t +|> drop x; + -- Union all with a table subquery on both the source and target sides. (select * from t) |> union all (select * from t); @@ -998,6 +1020,36 @@ select 1 as x, 2 as y select 3 as x, 4 as y |> aggregate group by 1, 2; +values (3, 4) as tab(x, y) +|> aggregate sum(y) group by 1; + +values (3, 4), (5, 4) as tab(x, y) +|> aggregate sum(y) group by 1; + +select 3 as x, 4 as y +|> aggregate sum(y) group by 1, 1; + +select 1 as `1`, 2 as `2` +|> aggregate sum(`2`) group by `1`; + +select 3 as x, 4 as y +|> aggregate sum(y) group by 2; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 3; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2, 3; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 1, 2, 3; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by x, 2, 3; + -- Basic table aggregation. table t |> aggregate sum(x); diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index cc603903712a6..3dd212d889f90 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -1493,6 +1493,84 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException } +-- !query +table t +|> select x, length(y) as z +|> limit 1000 +|> where x + length(y) < 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`x`, `z`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "y" + } ] +} + + +-- !query +table t +|> select x, length(y) as z +|> limit 1000 offset 1 +|> where x + length(y) < 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`x`, `z`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 80, + "stopIndex" : 80, + "fragment" : "y" + } ] +} + + +-- !query +table t +|> select x, length(y) as z +|> order by x, y +|> where x + length(y) < 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`x`, `z`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 52, + "fragment" : "y" + } ] +} + + -- !query (select x, sum(length(y)) as sum_len from t group by x) |> where sum(length(y)) = 3 @@ -2362,29 +2440,25 @@ struct -- !query -values (0, 1) tab(x, y) +values (2, 'xyz') tab(x, y) |> union table t |> where x = 0 -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 55, - "fragment" : "values (0, 1) tab(x, y)\n|> union table t\n|> where x = 0" - } ] -} +0 abc + + +-- !query +values (2, 'xyz') tab(x, y) +|> union table t +|> drop x +-- !query schema +struct +-- !query output +abc +def +xyz -- !query @@ -2779,11 +2853,102 @@ struct select 3 as x, 4 as y |> aggregate group by 1, 2 -- !query schema -struct<1:int,2:int> +struct +-- !query output +3 4 + + +-- !query +values (3, 4) as tab(x, y) +|> aggregate sum(y) group by 1 +-- !query schema +struct +-- !query output +3 4 + + +-- !query +values (3, 4), (5, 4) as tab(x, y) +|> aggregate sum(y) group by 1 +-- !query schema +struct +-- !query output +3 4 +5 4 + + +-- !query +select 3 as x, 4 as y +|> aggregate sum(y) group by 1, 1 +-- !query schema +struct +-- !query output +3 3 4 + + +-- !query +select 1 as `1`, 2 as `2` +|> aggregate sum(`2`) group by `1` +-- !query schema +struct<1:int,pipeexpression(sum(2)):bigint> -- !query output 1 2 +-- !query +select 3 as x, 4 as y +|> aggregate sum(y) group by 2 +-- !query schema +struct +-- !query output +4 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2 +-- !query schema +struct +-- !query output +4 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 3 +-- !query schema +struct +-- !query output +5 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2, 3 +-- !query schema +struct +-- !query output +4 5 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 1, 2, 3 +-- !query schema +struct +-- !query output +3 4 5 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by x, 2, 3 +-- !query schema +struct +-- !query output +3 4 5 4 + + -- !query table t |> aggregate sum(x) From 876450c4130062a80f4ecb7b7afd232b5481a99f Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Mon, 23 Dec 2024 13:21:21 +0800 Subject: [PATCH 234/438] [SPARK-50646][PYTHON][DOCS] Document explicit style of pyspark plotting ### What changes were proposed in this pull request? Document explicit style of pyspark plotting. ### Why are the changes needed? To improve documentation by explicitly showing both chaining (df.plot.line(...)) and explicit (df.plot(kind="line", ...)) styles, ensuring clarity for users ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49265 from xinrong-meng/pyplot. Authored-by: Xinrong Meng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/dataframe.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ccb9806cc76dc..f88ca5348ff2d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -6785,6 +6785,9 @@ def plot(self) -> "PySparkPlotAccessor": Notes ----- This API is experimental. + It provides two ways to create plots: + 1. Chaining style (e.g., `df.plot.line(...)`). + 2. Explicit style (e.g., `df.plot(kind="line", ...)`). Examples -------- @@ -6794,6 +6797,7 @@ def plot(self) -> "PySparkPlotAccessor": >>> type(df.plot) >>> df.plot.line(x="category", y=["int_val", "float_val"]) # doctest: +SKIP + >>> df.plot(kind="line", x="category", y=["int_val", "float_val"]) # doctest: +SKIP """ ... From 7cd5c4a1d1eb56fa92c10696bdbd8450d357b128 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 23 Dec 2024 16:27:00 +0800 Subject: [PATCH 235/438] [SPARK-50641][SQL] Move `GetJsonObjectEvaluator` to `JsonExpressionEvalUtils` ### What changes were proposed in this pull request? The pr aims to move `GetJsonObjectEvaluator` to `JsonExpressionEvalUtils`. ### Why are the changes needed? Make code clearly. ### 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 #49259 from panbingkun/SPARK-50641. Authored-by: panbingkun Signed-off-by: panbingkun --- .../expressions/json/JsonExpressionUtils.java | 1 - .../json/JsonExpressionEvalUtils.scala | 302 ++++++++++++++++- .../expressions/jsonExpressions.scala | 306 +----------------- 3 files changed, 301 insertions(+), 308 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionUtils.java index 2bad67d426af6..38bdcbec2069d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionUtils.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionUtils.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; -import org.apache.spark.sql.catalyst.expressions.SharedFactory; import org.apache.spark.sql.catalyst.json.CreateJacksonParser; import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.unsafe.types.UTF8String; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala index 7ff2bfe51729c..c9d15e1eb2e4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala @@ -16,13 +16,16 @@ */ package org.apache.spark.sql.catalyst.expressions.json -import java.io.{ByteArrayOutputStream, CharArrayWriter} +import java.io.{ByteArrayOutputStream, CharArrayWriter, StringWriter} + +import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ +import com.fasterxml.jackson.core.json.JsonReadFeature import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow, SharedFactory} +import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonGenerator, JacksonParser, JsonInferSchema, JSONOptions} import org.apache.spark.sql.catalyst.util.{ArrayData, FailFastMode, FailureSafeParser, MapData, PermissiveMode} @@ -32,6 +35,79 @@ import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, St import org.apache.spark.unsafe.types.{UTF8String, VariantVal} import org.apache.spark.util.Utils +private[this] sealed trait PathInstruction +private[this] object PathInstruction { + private[expressions] case object Subscript extends PathInstruction + private[expressions] case object Wildcard extends PathInstruction + private[expressions] case object Key extends PathInstruction + private[expressions] case class Index(index: Long) extends PathInstruction + private[expressions] case class Named(name: String) extends PathInstruction +} + +private[this] sealed trait WriteStyle +private[this] object WriteStyle { + private[expressions] case object RawStyle extends WriteStyle + private[expressions] case object QuotedStyle extends WriteStyle + private[expressions] case object FlattenStyle extends WriteStyle +} + +private[this] object JsonPathParser extends RegexParsers { + import PathInstruction._ + + def root: Parser[Char] = '$' + + def long: Parser[Long] = "\\d+".r ^? { + case x => x.toLong + } + + // parse `[*]` and `[123]` subscripts + def subscript: Parser[List[PathInstruction]] = + for { + operand <- '[' ~> ('*' ^^^ Wildcard | long ^^ Index) <~ ']' + } yield { + Subscript :: operand :: Nil + } + + // parse `.name` or `['name']` child expressions + def named: Parser[List[PathInstruction]] = + for { + name <- '.' ~> "[^\\.\\[]+".r | "['" ~> "[^\\']+".r <~ "']" + } yield { + Key :: Named(name) :: Nil + } + + // child wildcards: `..`, `.*` or `['*']` + def wildcard: Parser[List[PathInstruction]] = + (".*" | "['*']") ^^^ List(Wildcard) + + def node: Parser[List[PathInstruction]] = + wildcard | + named | + subscript + + val expression: Parser[List[PathInstruction]] = { + phrase(root ~> rep(node) ^^ (x => x.flatten)) + } + + def parse(str: String): Option[List[PathInstruction]] = { + this.parseAll(expression, str) match { + case Success(result, _) => + Some(result) + + case _ => + None + } + } +} + +private[this] object SharedFactory { + val jsonFactory: JsonFactory = new JsonFactoryBuilder() + // The two options below enabled for Hive compatibility + .enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS) + .enable(JsonReadFeature.ALLOW_SINGLE_QUOTES) + .build() +} + case class JsonToStructsEvaluator( options: Map[String, String], nullableSchema: DataType, @@ -278,3 +354,225 @@ case class JsonTupleEvaluator(foldableFieldNames: Array[Option[String]]) { } } } + +/** + * The expression `GetJsonObject` will utilize it to support codegen. + */ +case class GetJsonObjectEvaluator(cachedPath: UTF8String) { + import com.fasterxml.jackson.core.JsonToken._ + import PathInstruction._ + import SharedFactory._ + import WriteStyle._ + + def this() = this(null) + + @transient + private lazy val parsedPath: Option[List[PathInstruction]] = parsePath(cachedPath) + + @transient + private var jsonStr: UTF8String = _ + + @transient + private var pathStr: UTF8String = _ + + def setJson(arg: UTF8String): Unit = { + jsonStr = arg + } + + def setPath(arg: UTF8String): Unit = { + pathStr = arg + } + + def evaluate(): Any = { + if (jsonStr == null) return null + + val parsed = if (cachedPath != null) { + parsedPath + } else { + parsePath(pathStr) + } + + if (parsed.isDefined) { + try { + /* We know the bytes are UTF-8 encoded. Pass a Reader to avoid having Jackson + detect character encoding which could fail for some malformed strings */ + Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, jsonStr)) { parser => + val output = new ByteArrayOutputStream() + val matched = Utils.tryWithResource( + jsonFactory.createGenerator(output, JsonEncoding.UTF8)) { generator => + parser.nextToken() + evaluatePath(parser, generator, RawStyle, parsed.get) + } + if (matched) { + UTF8String.fromBytes(output.toByteArray) + } else { + null + } + } + } catch { + case _: JsonProcessingException => null + } + } else { + null + } + } + + private def parsePath(path: UTF8String): Option[List[PathInstruction]] = { + if (path != null) { + JsonPathParser.parse(path.toString) + } else { + None + } + } + + // advance to the desired array index, assumes to start at the START_ARRAY token + private def arrayIndex(p: JsonParser, f: () => Boolean): Long => Boolean = { + case _ if p.getCurrentToken == END_ARRAY => + // terminate, nothing has been written + false + + case 0 => + // we've reached the desired index + val dirty = f() + + while (p.nextToken() != END_ARRAY) { + // advance the token stream to the end of the array + p.skipChildren() + } + + dirty + + case i if i > 0 => + // skip this token and evaluate the next + p.skipChildren() + p.nextToken() + arrayIndex(p, f)(i - 1) + } + + /** + * Evaluate a list of JsonPath instructions, returning a bool that indicates if any leaf nodes + * have been written to the generator + */ + private def evaluatePath( + p: JsonParser, + g: JsonGenerator, + style: WriteStyle, + path: List[PathInstruction]): Boolean = { + (p.getCurrentToken, path) match { + case (VALUE_STRING, Nil) if style == RawStyle => + // there is no array wildcard or slice parent, emit this string without quotes + if (p.hasTextCharacters) { + g.writeRaw(p.getTextCharacters, p.getTextOffset, p.getTextLength) + } else { + g.writeRaw(p.getText) + } + true + + case (START_ARRAY, Nil) if style == FlattenStyle => + // flatten this array into the parent + var dirty = false + while (p.nextToken() != END_ARRAY) { + dirty |= evaluatePath(p, g, style, Nil) + } + dirty + + case (_, Nil) => + // general case: just copy the child tree verbatim + g.copyCurrentStructure(p) + true + + case (START_OBJECT, Key :: xs) => + var dirty = false + while (p.nextToken() != END_OBJECT) { + if (dirty) { + // once a match has been found we can skip other fields + p.skipChildren() + } else { + dirty = evaluatePath(p, g, style, xs) + } + } + dirty + + case (START_ARRAY, Subscript :: Wildcard :: Subscript :: Wildcard :: xs) => + // special handling for the non-structure preserving double wildcard behavior in Hive + var dirty = false + g.writeStartArray() + while (p.nextToken() != END_ARRAY) { + dirty |= evaluatePath(p, g, FlattenStyle, xs) + } + g.writeEndArray() + dirty + + case (START_ARRAY, Subscript :: Wildcard :: xs) if style != QuotedStyle => + // retain Flatten, otherwise use Quoted... cannot use Raw within an array + val nextStyle = style match { + case RawStyle => QuotedStyle + case FlattenStyle => FlattenStyle + case QuotedStyle => throw SparkException.internalError("Unexpected the quoted style.") + } + + // temporarily buffer child matches, the emitted json will need to be + // modified slightly if there is only a single element written + val buffer = new StringWriter() + + var dirty = 0 + Utils.tryWithResource(jsonFactory.createGenerator(buffer)) { flattenGenerator => + flattenGenerator.writeStartArray() + + while (p.nextToken() != END_ARRAY) { + // track the number of array elements and only emit an outer array if + // we've written more than one element, this matches Hive's behavior + dirty += (if (evaluatePath(p, flattenGenerator, nextStyle, xs)) 1 else 0) + } + flattenGenerator.writeEndArray() + } + + val buf = buffer.getBuffer + if (dirty > 1) { + g.writeRawValue(buf.toString) + } else if (dirty == 1) { + // remove outer array tokens + g.writeRawValue(buf.substring(1, buf.length() - 1)) + } // else do not write anything + + dirty > 0 + + case (START_ARRAY, Subscript :: Wildcard :: xs) => + var dirty = false + g.writeStartArray() + while (p.nextToken() != END_ARRAY) { + // wildcards can have multiple matches, continually update the dirty count + dirty |= evaluatePath(p, g, QuotedStyle, xs) + } + g.writeEndArray() + + dirty + + case (START_ARRAY, Subscript :: Index(idx) :: (xs@Subscript :: Wildcard :: _)) => + p.nextToken() + // we're going to have 1 or more results, switch to QuotedStyle + arrayIndex(p, () => evaluatePath(p, g, QuotedStyle, xs))(idx) + + case (START_ARRAY, Subscript :: Index(idx) :: xs) => + p.nextToken() + arrayIndex(p, () => evaluatePath(p, g, style, xs))(idx) + + case (FIELD_NAME, Named(name) :: xs) if p.currentName == name => + // exact field match + if (p.nextToken() != JsonToken.VALUE_NULL) { + evaluatePath(p, g, style, xs) + } else { + false + } + + case (FIELD_NAME, Wildcard :: xs) => + // wildcard field match + p.nextToken() + evaluatePath(p, g, style, xs) + + case _ => + p.skipChildren() + false + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 93eda307ec9c1..e80f543f14eda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -17,20 +17,12 @@ package org.apache.spark.sql.catalyst.expressions -import java.io._ - -import scala.util.parsing.combinator.RegexParsers - -import com.fasterxml.jackson.core._ -import com.fasterxml.jackson.core.json.JsonReadFeature - -import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper -import org.apache.spark.sql.catalyst.expressions.json.{JsonExpressionUtils, JsonToStructsEvaluator, JsonTupleEvaluator, SchemaOfJsonEvaluator, StructsToJsonEvaluator} +import org.apache.spark.sql.catalyst.expressions.json.{GetJsonObjectEvaluator, JsonExpressionUtils, JsonToStructsEvaluator, JsonTupleEvaluator, SchemaOfJsonEvaluator, StructsToJsonEvaluator} import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.trees.TreePattern.{JSON_TO_STRUCT, RUNTIME_REPLACEABLE, TreePattern} @@ -39,80 +31,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.util.Utils - -private[this] sealed trait PathInstruction -private[this] object PathInstruction { - private[expressions] case object Subscript extends PathInstruction - private[expressions] case object Wildcard extends PathInstruction - private[expressions] case object Key extends PathInstruction - private[expressions] case class Index(index: Long) extends PathInstruction - private[expressions] case class Named(name: String) extends PathInstruction -} - -private[this] sealed trait WriteStyle -private[this] object WriteStyle { - private[expressions] case object RawStyle extends WriteStyle - private[expressions] case object QuotedStyle extends WriteStyle - private[expressions] case object FlattenStyle extends WriteStyle -} - -private[this] object JsonPathParser extends RegexParsers { - import PathInstruction._ - - def root: Parser[Char] = '$' - - def long: Parser[Long] = "\\d+".r ^? { - case x => x.toLong - } - - // parse `[*]` and `[123]` subscripts - def subscript: Parser[List[PathInstruction]] = - for { - operand <- '[' ~> ('*' ^^^ Wildcard | long ^^ Index) <~ ']' - } yield { - Subscript :: operand :: Nil - } - - // parse `.name` or `['name']` child expressions - def named: Parser[List[PathInstruction]] = - for { - name <- '.' ~> "[^\\.\\[]+".r | "['" ~> "[^\\']+".r <~ "']" - } yield { - Key :: Named(name) :: Nil - } - - // child wildcards: `..`, `.*` or `['*']` - def wildcard: Parser[List[PathInstruction]] = - (".*" | "['*']") ^^^ List(Wildcard) - - def node: Parser[List[PathInstruction]] = - wildcard | - named | - subscript - - val expression: Parser[List[PathInstruction]] = { - phrase(root ~> rep(node) ^^ (x => x.flatten)) - } - - def parse(str: String): Option[List[PathInstruction]] = { - this.parseAll(expression, str) match { - case Success(result, _) => - Some(result) - - case _ => - None - } - } -} - -private[expressions] object SharedFactory { - val jsonFactory = new JsonFactoryBuilder() - // The two options below enabled for Hive compatibility - .enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS) - .enable(JsonReadFeature.ALLOW_SINGLE_QUOTES) - .build() -} /** * Extracts json object from a json string based on json path specified, and returns json string @@ -213,228 +131,6 @@ case class GetJsonObject(json: Expression, path: Expression) copy(json = newLeft, path = newRight) } -class GetJsonObjectEvaluator(cachedPath: UTF8String) { - import com.fasterxml.jackson.core.JsonToken._ - import PathInstruction._ - import SharedFactory._ - import WriteStyle._ - - def this() = this(null) - - @transient - private lazy val parsedPath: Option[List[PathInstruction]] = - parsePath(cachedPath) - - @transient - private var jsonStr: UTF8String = null - - @transient - private var pathStr: UTF8String = null - - def setJson(arg: UTF8String): Unit = { - jsonStr = arg - } - - def setPath(arg: UTF8String): Unit = { - pathStr = arg - } - - def evaluate(): Any = { - if (jsonStr == null) { - return null - } - - val parsed = if (cachedPath != null) { - parsedPath - } else { - parsePath(pathStr) - } - - if (parsed.isDefined) { - try { - /* We know the bytes are UTF-8 encoded. Pass a Reader to avoid having Jackson - detect character encoding which could fail for some malformed strings */ - Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, jsonStr)) { parser => - val output = new ByteArrayOutputStream() - val matched = Utils.tryWithResource( - jsonFactory.createGenerator(output, JsonEncoding.UTF8)) { generator => - parser.nextToken() - evaluatePath(parser, generator, RawStyle, parsed.get) - } - if (matched) { - UTF8String.fromBytes(output.toByteArray) - } else { - null - } - } - } catch { - case _: JsonProcessingException => null - } - } else { - null - } - } - - private def parsePath(path: UTF8String): Option[List[PathInstruction]] = { - if (path != null) { - JsonPathParser.parse(path.toString) - } else { - None - } - } - - // advance to the desired array index, assumes to start at the START_ARRAY token - private def arrayIndex(p: JsonParser, f: () => Boolean): Long => Boolean = { - case _ if p.getCurrentToken == END_ARRAY => - // terminate, nothing has been written - false - - case 0 => - // we've reached the desired index - val dirty = f() - - while (p.nextToken() != END_ARRAY) { - // advance the token stream to the end of the array - p.skipChildren() - } - - dirty - - case i if i > 0 => - // skip this token and evaluate the next - p.skipChildren() - p.nextToken() - arrayIndex(p, f)(i - 1) - } - - /** - * Evaluate a list of JsonPath instructions, returning a bool that indicates if any leaf nodes - * have been written to the generator - */ - private def evaluatePath( - p: JsonParser, - g: JsonGenerator, - style: WriteStyle, - path: List[PathInstruction]): Boolean = { - (p.getCurrentToken, path) match { - case (VALUE_STRING, Nil) if style == RawStyle => - // there is no array wildcard or slice parent, emit this string without quotes - if (p.hasTextCharacters) { - g.writeRaw(p.getTextCharacters, p.getTextOffset, p.getTextLength) - } else { - g.writeRaw(p.getText) - } - true - - case (START_ARRAY, Nil) if style == FlattenStyle => - // flatten this array into the parent - var dirty = false - while (p.nextToken() != END_ARRAY) { - dirty |= evaluatePath(p, g, style, Nil) - } - dirty - - case (_, Nil) => - // general case: just copy the child tree verbatim - g.copyCurrentStructure(p) - true - - case (START_OBJECT, Key :: xs) => - var dirty = false - while (p.nextToken() != END_OBJECT) { - if (dirty) { - // once a match has been found we can skip other fields - p.skipChildren() - } else { - dirty = evaluatePath(p, g, style, xs) - } - } - dirty - - case (START_ARRAY, Subscript :: Wildcard :: Subscript :: Wildcard :: xs) => - // special handling for the non-structure preserving double wildcard behavior in Hive - var dirty = false - g.writeStartArray() - while (p.nextToken() != END_ARRAY) { - dirty |= evaluatePath(p, g, FlattenStyle, xs) - } - g.writeEndArray() - dirty - - case (START_ARRAY, Subscript :: Wildcard :: xs) if style != QuotedStyle => - // retain Flatten, otherwise use Quoted... cannot use Raw within an array - val nextStyle = style match { - case RawStyle => QuotedStyle - case FlattenStyle => FlattenStyle - case QuotedStyle => throw SparkException.internalError("Unexpected the quoted style.") - } - - // temporarily buffer child matches, the emitted json will need to be - // modified slightly if there is only a single element written - val buffer = new StringWriter() - - var dirty = 0 - Utils.tryWithResource(jsonFactory.createGenerator(buffer)) { flattenGenerator => - flattenGenerator.writeStartArray() - - while (p.nextToken() != END_ARRAY) { - // track the number of array elements and only emit an outer array if - // we've written more than one element, this matches Hive's behavior - dirty += (if (evaluatePath(p, flattenGenerator, nextStyle, xs)) 1 else 0) - } - flattenGenerator.writeEndArray() - } - - val buf = buffer.getBuffer - if (dirty > 1) { - g.writeRawValue(buf.toString) - } else if (dirty == 1) { - // remove outer array tokens - g.writeRawValue(buf.substring(1, buf.length() - 1)) - } // else do not write anything - - dirty > 0 - - case (START_ARRAY, Subscript :: Wildcard :: xs) => - var dirty = false - g.writeStartArray() - while (p.nextToken() != END_ARRAY) { - // wildcards can have multiple matches, continually update the dirty count - dirty |= evaluatePath(p, g, QuotedStyle, xs) - } - g.writeEndArray() - - dirty - - case (START_ARRAY, Subscript :: Index(idx) :: (xs@Subscript :: Wildcard :: _)) => - p.nextToken() - // we're going to have 1 or more results, switch to QuotedStyle - arrayIndex(p, () => evaluatePath(p, g, QuotedStyle, xs))(idx) - - case (START_ARRAY, Subscript :: Index(idx) :: xs) => - p.nextToken() - arrayIndex(p, () => evaluatePath(p, g, style, xs))(idx) - - case (FIELD_NAME, Named(name) :: xs) if p.currentName == name => - // exact field match - if (p.nextToken() != JsonToken.VALUE_NULL) { - evaluatePath(p, g, style, xs) - } else { - false - } - - case (FIELD_NAME, Wildcard :: xs) => - // wildcard field match - p.nextToken() - evaluatePath(p, g, style, xs) - - case _ => - p.skipChildren() - false - } - } -} - // scalastyle:off line.size.limit line.contains.tab @ExpressionDescription( usage = "_FUNC_(jsonStr, p1, p2, ..., pn) - Returns a tuple like the function get_json_object, but it takes multiple names. All the input parameters and output column types are string.", From a30a3fd0e74d36c744af26ac1931dfa3c2883552 Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Tue, 24 Dec 2024 09:47:51 +0900 Subject: [PATCH 236/438] [SPARK-49530][PYTHON] Support pie subplots in pyspark plotting ### What changes were proposed in this pull request? Support pie subplots in pyspark plotting. ### Why are the changes needed? API parity with pandas.DataFrame.plot.pie, see [here](https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.plot.pie.html) ### Does this PR introduce _any_ user-facing change? Pie subplots are supported as shown below: ```py >>> from datetime import datetime >>> data = [ ... (3, 5, 20, datetime(2018, 1, 31)), ... (2, 5, 42, datetime(2018, 2, 28)), ... (3, 6, 28, datetime(2018, 3, 31)), ... (9, 12, 62, datetime(2018, 4, 30)), ... ] >>> columns = ["sales", "signups", "visits", "date"] >>> df = spark.createDataFrame(data, columns) >>> fig = df.plot(kind="pie", x="date", subplots=True) >>> fig.show() ``` ![newplot (2)](https://github.com/user-attachments/assets/2b019c6a-82da-4c12-b1ff-096786801f56) ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49268 from xinrong-meng/pie_subplot. Lead-authored-by: Xinrong Meng Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/errors/error-conditions.json | 5 +++ python/pyspark/sql/plot/core.py | 22 +++-------- python/pyspark/sql/plot/plotly.py | 25 +++++++++++- .../sql/tests/plot/test_frame_plot_plotly.py | 39 +++++++++++++++++-- 4 files changed, 68 insertions(+), 23 deletions(-) diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index c4ad3f8d5feba..b7c1ec23c3af9 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -1103,6 +1103,11 @@ "Function `` should use only POSITIONAL or POSITIONAL OR KEYWORD arguments." ] }, + "UNSUPPORTED_PIE_PLOT_PARAM": { + "message": [ + "Pie plot requires either a `y` column or `subplots=True`." + ] + }, "UNSUPPORTED_PLOT_BACKEND": { "message": [ "`` is not supported, it should be one of the values from " diff --git a/python/pyspark/sql/plot/core.py b/python/pyspark/sql/plot/core.py index f7133bdb70ed6..e565a5d1ebf32 100644 --- a/python/pyspark/sql/plot/core.py +++ b/python/pyspark/sql/plot/core.py @@ -19,11 +19,10 @@ from typing import Any, TYPE_CHECKING, List, Optional, Union, Sequence from types import ModuleType -from pyspark.errors import PySparkTypeError, PySparkValueError +from pyspark.errors import PySparkValueError from pyspark.sql import Column, functions as F from pyspark.sql.internal import InternalFunction as SF from pyspark.sql.pandas.utils import require_minimum_pandas_version -from pyspark.sql.types import NumericType from pyspark.sql.utils import NumpyHelper, require_minimum_plotly_version if TYPE_CHECKING: @@ -295,7 +294,7 @@ def area(self, x: str, y: Union[str, list[str]], **kwargs: Any) -> "Figure": """ return self(kind="area", x=x, y=y, **kwargs) - def pie(self, x: str, y: str, **kwargs: Any) -> "Figure": + def pie(self, x: str, y: Optional[str], **kwargs: Any) -> "Figure": """ Generate a pie plot. @@ -306,8 +305,8 @@ def pie(self, x: str, y: str, **kwargs: Any) -> "Figure": ---------- x : str Name of column to be used as the category labels for the pie plot. - y : str - Name of the column to plot. + y : str, optional + Name of the column to plot. If not provided, `subplots=True` must be passed at `kwargs`. **kwargs Additional keyword arguments. @@ -327,19 +326,8 @@ def pie(self, x: str, y: str, **kwargs: Any) -> "Figure": >>> columns = ["sales", "signups", "visits", "date"] >>> df = spark.createDataFrame(data, columns) >>> df.plot.pie(x='date', y='sales') # doctest: +SKIP + >>> df.plot.pie(x='date', subplots=True) # doctest: +SKIP """ - schema = self.data.schema - - # Check if 'y' is a numerical column - y_field = schema[y] if y in schema.names else None - if y_field is None or not isinstance(y_field.dataType, NumericType): - raise PySparkTypeError( - errorClass="PLOT_NOT_NUMERIC_COLUMN_ARGUMENT", - messageParameters={ - "arg_name": "y", - "arg_type": str(y_field.dataType.__class__.__name__) if y_field else "None", - }, - ) return self(kind="pie", x=x, y=y, **kwargs) def box(self, column: Optional[Union[str, List[str]]] = None, **kwargs: Any) -> "Figure": diff --git a/python/pyspark/sql/plot/plotly.py b/python/pyspark/sql/plot/plotly.py index 959562b43552a..c7691f144ffaf 100644 --- a/python/pyspark/sql/plot/plotly.py +++ b/python/pyspark/sql/plot/plotly.py @@ -48,13 +48,34 @@ def plot_pyspark(data: "DataFrame", kind: str, **kwargs: Any) -> "Figure": def plot_pie(data: "DataFrame", **kwargs: Any) -> "Figure": - # TODO(SPARK-49530): Support pie subplots with plotly backend from plotly import express pdf = PySparkPlotAccessor.plot_data_map["pie"](data) x = kwargs.pop("x", None) y = kwargs.pop("y", None) - fig = express.pie(pdf, values=y, names=x, **kwargs) + subplots = kwargs.pop("subplots", False) + if y is None and not subplots: + raise PySparkValueError(errorClass="UNSUPPORTED_PIE_PLOT_PARAM", messageParameters={}) + + numeric_ys = process_column_param(y, data) + + if subplots: + # One pie chart per numeric column + from plotly.subplots import make_subplots + + fig = make_subplots( + rows=1, + cols=len(numeric_ys), + # To accommodate domain-based trace - pie chart + specs=[[{"type": "domain"}] * len(numeric_ys)], + ) + for i, y_col in enumerate(numeric_ys): + subplot_fig = express.pie(pdf, values=y_col, names=x, **kwargs) + fig.add_trace( + subplot_fig.data[0], row=1, col=i + 1 + ) # A single pie chart has only one trace + else: + fig = express.pie(pdf, values=numeric_ys[0], names=x, **kwargs) return fig diff --git a/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py index fd264c3488823..3dafd71c1a329 100644 --- a/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py +++ b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py @@ -301,6 +301,7 @@ def test_area_plot(self): self._check_fig_data(fig["data"][2], **expected_fig_data) def test_pie_plot(self): + # single column as 'y' fig = self.sdf3.plot(kind="pie", x="date", y="sales") expected_x = [ datetime(2018, 1, 31, 0, 0), @@ -308,13 +309,39 @@ def test_pie_plot(self): datetime(2018, 3, 31, 0, 0), datetime(2018, 4, 30, 0, 0), ] - expected_fig_data = { + expected_fig_data_sales = { "name": "", "labels": expected_x, "values": [3, 2, 3, 9], "type": "pie", } - self._check_fig_data(fig["data"][0], **expected_fig_data) + self._check_fig_data(fig["data"][0], **expected_fig_data_sales) + + # all numeric columns as 'y' + expected_fig_data_signups = { + "name": "", + "labels": expected_x, + "values": [5, 5, 6, 12], + "type": "pie", + } + expected_fig_data_visits = { + "name": "", + "labels": expected_x, + "values": [20, 42, 28, 62], + "type": "pie", + } + fig = self.sdf3.plot(kind="pie", x="date", subplots=True) + self._check_fig_data(fig["data"][0], **expected_fig_data_sales) + self._check_fig_data(fig["data"][1], **expected_fig_data_signups) + self._check_fig_data(fig["data"][2], **expected_fig_data_visits) + + # not specify subplots + with self.assertRaises(PySparkValueError) as pe: + self.sdf3.plot(kind="pie", x="date") + + self.check_error( + exception=pe.exception, errorClass="UNSUPPORTED_PIE_PLOT_PARAM", messageParameters={} + ) # y is not a numerical column with self.assertRaises(PySparkTypeError) as pe: @@ -322,8 +349,12 @@ def test_pie_plot(self): self.check_error( exception=pe.exception, - errorClass="PLOT_NOT_NUMERIC_COLUMN_ARGUMENT", - messageParameters={"arg_name": "y", "arg_type": "StringType"}, + errorClass="PLOT_INVALID_TYPE_COLUMN", + messageParameters={ + "col_name": "category", + "valid_types": "NumericType", + "col_type": "StringType", + }, ) def test_box_plot(self): From debae7173e87f55eb30d9ea50eb69fb571157cf3 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 24 Dec 2024 11:11:48 +0900 Subject: [PATCH 237/438] [SPARK-50651][SQL][DOCS] Add note about octal representation for characters ### What changes were proposed in this pull request? This PR modifies `docs/sql-ref-literals.md` to mention about octal representation for characters. Currently there is no mention of it anywhere. This change includes a trivial typo too which is in the same file. ### Why are the changes needed? It's a public feature. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Built the doc, then confirmed the result. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49273 from sarutak/octal-representation. Lead-authored-by: Kousuke Saruta Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- docs/sql-ref-literals.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-literals.md b/docs/sql-ref-literals.md index 141f985b0beac..7a10676cce237 100644 --- a/docs/sql-ref-literals.md +++ b/docs/sql-ref-literals.md @@ -46,6 +46,7 @@ A string literal is used to specify a character string value. One character from the character set. Use `\` to escape special characters (e.g., `'` or `\`). To represent unicode characters, use 16-bit or 32-bit unicode escape of the form `\uxxxx` or `\Uxxxxxxxx`, where xxxx and xxxxxxxx are 16-bit and 32-bit code points in hexadecimal respectively (e.g., `\u3042` for `あ` and `\U0001F44D` for `👍`). + An ASCII character can also be represented as an octal number preceded by `\` like `\101`, which represents `A`. * **r** @@ -78,14 +79,14 @@ SELECT "SPARK SQL" AS col; +---------+ | col| +---------+ -|Spark SQL| +|SPARK SQL| +---------+ SELECT 'it\'s $10.' AS col; +---------+ | col| +---------+ -|It's $10.| +|it's $10.| +---------+ SELECT r"'\n' represents newline character." AS col; From 50d49ee2df5c01c3042afbcde8ae1718dc82360f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 24 Dec 2024 12:22:06 +0800 Subject: [PATCH 238/438] [MINOR][INFRA] Skip step `List Python packages` when `PYTHON_TO_TEST` is not specified ### What changes were proposed in this pull request? Skip step `List Python packages` when `PYTHON_TO_TEST` is not specified ### Why are the changes needed? in 3.5 daily build, the `PYTHON_TO_TEST` is empty https://github.com/apache/spark/actions/runs/12465764579/job/34792186424 ![image](https://github.com/user-attachments/assets/d7da529c-8273-4065-9bef-a29a93ab53aa) ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? will manually check ### Was this patch authored or co-authored using generative AI tooling? no Closes #49279 from zhengruifeng/infra_skip_list. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 758deca93c82e..9900fbf6d82c4 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -584,6 +584,7 @@ jobs: distribution: zulu java-version: ${{ matrix.java }} - name: List Python packages (${{ env.PYTHON_TO_TEST }}) + if: ${{ env.PYTHON_TO_TEST != '' }} env: ${{ fromJSON(inputs.envs) }} shell: 'script -q -e -c "bash {0}"' run: | From 202b42e57fded2723a1bf18a981d03987b8d0172 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Tue, 24 Dec 2024 14:49:35 +0800 Subject: [PATCH 239/438] [SPARK-50649] Fix inconsistencies with casting between different collations ### What changes were proposed in this pull request? Fixing the inconsistent behavior of casts between different collations. Currently, we are allowed to do casts between them in dataframe API but not in the SQL API. I propose allowing casts in SQL as well (we are already allowing them for complex types anyways). Also, this means changing the behavior or `CAST(x AS STRING)` which was previously not altering the collation of `x`, and will now change it to the default collation. ### Why are the changes needed? To make collation casts between the dataframe and SQL api consistent. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added new unit tests for the dataframe API which we didn't have before and also updated the existing tests for the SQL API to match the new behavior. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49269 from stefankandic/changeCastBehavior. Authored-by: Stefan Kandic Signed-off-by: Wenchen Fan --- .../analysis/CollationTypeCoercion.scala | 43 ++++---------- .../sql/catalyst/parser/AstBuilder.scala | 28 --------- .../org/apache/spark/sql/CollationSuite.scala | 57 +++++++++---------- .../CollationTypePrecedenceSuite.scala | 40 +++++-------- 4 files changed, 55 insertions(+), 113 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index 980b627729ecf..168eadbd65cd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -38,9 +38,6 @@ object CollationTypeCoercion { } def apply(expression: Expression): Expression = expression match { - case cast: Cast if shouldRemoveCast(cast) => - cast.child - case ifExpr: If => ifExpr.withNewChildren( ifExpr.predicate +: collateToSingleType(Seq(ifExpr.trueValue, ifExpr.falseValue)) @@ -146,22 +143,13 @@ object CollationTypeCoercion { } /** - * If childType is collated and target is UTF8_BINARY, the collation of the output - * should be that of the childType. + * Returns true if the given data type has any StringType in it. */ - private def shouldRemoveCast(cast: Cast): Boolean = { - val isChildTypeCollatedString = cast.child.dataType match { - case st: StringType => !st.isUTF8BinaryCollation - case _ => false - } - val targetType = cast.dataType - - isUserDefined(cast) && isChildTypeCollatedString && targetType == StringType + private def hasStringType(dt: DataType): Boolean = dt.existsRecursively { + case _: StringType => true + case _ => false } - private def isUserDefined(cast: Cast): Boolean = - cast.getTagValue(Cast.USER_SPECIFIED_CAST).isDefined - /** * Changes the data type of the expression to the given `newType`. */ @@ -391,18 +379,18 @@ object CollationTypeCoercion { case collate: Collate => Some(addContextToStringType(collate.dataType, Explicit)) - case expr @ (_: NamedExpression | _: SubqueryExpression | _: VariableReference) => - Some(addContextToStringType(expr.dataType, Implicit)) - case cast: Cast => - if (isUserDefined(cast) && isComplexType(cast.dataType)) { - // since we can't use collate clause with complex types - // user defined casts should be treated as implicit - Some(addContextToStringType(cast.dataType, Implicit)) + val castStrength = if (hasStringType(cast.child.dataType)) { + Implicit } else { - Some(addContextToStringType(cast.dataType, Default)) + Default } + Some(addContextToStringType(cast.dataType, castStrength)) + + case expr @ (_: NamedExpression | _: SubqueryExpression | _: VariableReference) => + Some(addContextToStringType(expr.dataType, Implicit)) + case lit: Literal => Some(addContextToStringType(lit.dataType, Default)) @@ -473,13 +461,6 @@ object CollationTypeCoercion { else right } } - - private def isComplexType(dataType: DataType): Boolean = { - dataType match { - case _: ArrayType | _: MapType | _: StructType => true - case _ => false - } - } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f95c0d6556bad..f37879ecd9356 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2763,20 +2763,6 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) { val rawDataType = typedVisit[DataType](ctx.dataType()) - ctx.dataType() match { - case context: PrimitiveDataTypeContext => - val typeCtx = context.`type`() - if (typeCtx.start.getType == STRING) { - typeCtx.children.asScala.toSeq match { - case Seq(_, cctx: CollateClauseContext) => - throw QueryParsingErrors.dataTypeUnsupportedError( - rawDataType.typeName, - ctx.dataType().asInstanceOf[PrimitiveDataTypeContext]) - case _ => - } - } - case _ => - } val dataType = CharVarcharUtils.replaceCharVarcharWithStringForCast(rawDataType) ctx.name.getType match { case SqlBaseParser.CAST => @@ -2796,20 +2782,6 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitCastByColon(ctx: CastByColonContext): Expression = withOrigin(ctx) { val rawDataType = typedVisit[DataType](ctx.dataType()) - ctx.dataType() match { - case context: PrimitiveDataTypeContext => - val typeCtx = context.`type`() - if (typeCtx.start.getType == STRING) { - typeCtx.children.asScala.toSeq match { - case Seq(_, cctx: CollateClauseContext) => - throw QueryParsingErrors.dataTypeUnsupportedError( - rawDataType.typeName, - ctx.dataType().asInstanceOf[PrimitiveDataTypeContext]) - case _ => - } - } - case _ => - } val dataType = CharVarcharUtils.replaceCharVarcharWithStringForCast(rawDataType) val cast = Cast(expression(ctx.primaryExpression), dataType) cast.setTagValue(Cast.USER_SPECIFIED_CAST, ()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index a8fe36c9ba394..1571433a37e16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -22,7 +22,6 @@ import scala.jdk.CollectionConverters.MapHasAsJava import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.connector.{DatasourceV2SQLBase, FakeV2ProviderWithCustomSchema} import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable} @@ -34,6 +33,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.{SqlApiConf, SQLConf} import org.apache.spark.sql.types.{ArrayType, IntegerType, MapType, Metadata, MetadataBuilder, StringType, StructField, StructType} @@ -1106,46 +1106,45 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } - test("SPARK-47972: Cast expression limitation for collations") { - checkError( - exception = intercept[ParseException] - (sql("SELECT cast(1 as string collate unicode)")), - condition = "UNSUPPORTED_DATATYPE", - parameters = Map( - "typeName" -> toSQLType(StringType("UNICODE"))), - context = - ExpectedContext(fragment = s"cast(1 as string collate unicode)", start = 7, stop = 39) - ) + test("Cast expression for collations") { + checkAnswer( + sql(s"SELECT collation(cast('a' as string collate utf8_lcase))"), + Seq(Row(fullyQualifiedPrefix + "UTF8_LCASE"))) - checkError( - exception = intercept[ParseException] - (sql("SELECT 'A' :: string collate unicode")), - condition = "UNSUPPORTED_DATATYPE", - parameters = Map( - "typeName" -> toSQLType(StringType("UNICODE"))), - context = ExpectedContext(fragment = s"'A' :: string collate unicode", start = 7, stop = 35) - ) + checkAnswer( + sql(s"SELECT collation('a' :: string collate utf8_lcase)"), + Seq(Row(fullyQualifiedPrefix + "UTF8_LCASE"))) checkAnswer(sql(s"SELECT cast(1 as string)"), Seq(Row("1"))) checkAnswer(sql(s"SELECT cast('A' as string)"), Seq(Row("A"))) withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkError( - exception = intercept[ParseException] - (sql("SELECT cast(1 as string collate unicode)")), - condition = "UNSUPPORTED_DATATYPE", - parameters = Map( - "typeName" -> toSQLType(StringType("UNICODE"))), - context = - ExpectedContext(fragment = s"cast(1 as string collate unicode)", start = 7, stop = 39) - ) - + checkAnswer( + sql(s"SELECT collation(cast(1 as string collate unicode))"), + Seq(Row(fullyQualifiedPrefix + "UNICODE"))) checkAnswer(sql(s"SELECT cast(1 as string)"), Seq(Row("1"))) checkAnswer(sql(s"SELECT collation(cast(1 as string))"), Seq(Row(fullyQualifiedPrefix + "UNICODE"))) } } + test("cast using the dataframe api") { + val tableName = "cast_table" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (name STRING COLLATE UTF8_LCASE) USING PARQUET") + + var df = spark.read.table(tableName) + .withColumn("name", col("name").cast("STRING COLLATE UNICODE")) + + assert(df.schema.fields.head.dataType === StringType("UNICODE")) + + df = spark.read.table(tableName) + .withColumn("name", col("name").cast("STRING COLLATE UTF8_BINARY")) + + assert(df.schema.fields.head.dataType === StringType) + } + } + test("Create dataframe with non utf8 binary collation") { val schema = StructType(Seq(StructField("Name", StringType("UNICODE_CI")))) val data = Seq(Row("Alice"), Row("Bob"), Row("bob")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index 00fcc501fe744..7df54b372e8a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -437,46 +437,36 @@ class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { sql(s"CREATE TABLE $tableName (c1 STRING COLLATE $columnCollation) USING $dataSource") sql(s"INSERT INTO $tableName VALUES ('a')") - // only for non string inputs cast results in default collation checkAnswer( - sql(s"SELECT COLLATION(c1 || CAST(to_char(DATE'2016-04-08', 'y') AS STRING)) " + - s"FROM $tableName"), - Seq(Row(columnCollation))) - - checkAnswer( - sql(s"SELECT COLLATION(CAST(to_char(DATE'2016-04-08', 'y') AS STRING)) " + - s"FROM $tableName"), + sql(s"SELECT COLLATION(CAST(5 AS STRING)) FROM $tableName"), Seq(Row(UTF8_BINARY_COLLATION_NAME))) - // for string inputs collation is of the child expression checkAnswer( - sql(s"SELECT COLLATION(CAST('a' AS STRING)) FROM $tableName"), - Seq(Row(UTF8_BINARY_COLLATION_NAME))) + sql(s"SELECT c1 = cast(5 AS STRING) FROM $tableName"), + Seq(Row(false))) checkAnswer( sql(s"SELECT COLLATION(CAST(c1 AS STRING)) FROM $tableName"), - Seq(Row(columnCollation))) + Seq(Row(UTF8_BINARY_COLLATION_NAME))) checkAnswer( - sql(s"SELECT COLLATION(CAST(c1 collate UTF8_LCASE AS STRING)) FROM $tableName"), - Seq(Row(UTF8_LCASE_COLLATION_NAME))) + sql(s"SELECT c1 = cast(c1 as STRING COLLATE UNICODE) FROM $tableName"), + Seq(Row(true))) checkAnswer( - sql(s"SELECT COLLATION(c1 || CAST('a' AS STRING)) FROM $tableName"), - Seq(Row(columnCollation))) + sql(s"SELECT c1 = cast(5 as STRING COLLATE UNICODE) FROM $tableName"), + Seq(Row(false))) checkAnswer( - sql(s"SELECT COLLATION(c1 || CAST('a' collate UTF8_LCASE AS STRING)) FROM $tableName"), - Seq(Row(UTF8_LCASE_COLLATION_NAME))) + sql(s"SELECT COLLATION(CAST(c1 collate UTF8_LCASE AS STRING)) FROM $tableName"), + Seq(Row(UTF8_BINARY_COLLATION_NAME))) - checkAnswer( - sql(s"SELECT COLLATION(c1 || CAST(c1 AS STRING)) FROM $tableName"), - Seq(Row(columnCollation))) + assertImplicitMismatch( + sql(s"SELECT c1 = CAST(c1 AS STRING) FROM $tableName")) - checkAnswer( - sql(s"SELECT COLLATION(c1 || SUBSTRING(CAST(c1 AS STRING), 0, 1)) FROM $tableName"), - Seq(Row(columnCollation))) - } + assertImplicitMismatch( + sql(s"SELECT c1 = CAST(to_char(DATE'2016-04-08', 'y') AS STRING) FROM $tableName")) + } } test("str fns without params have default strength") { From 2c1c4d2614ae1ff902c244209f7ec3c79102d3e0 Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Tue, 24 Dec 2024 14:54:02 +0800 Subject: [PATCH 240/438] [SPARK-50644][SQL] Read variant struct in Parquet reader ### What changes were proposed in this pull request? It adds support for variant struct in Parquet reader. The concept of variant struct was introduced in https://github.com/apache/spark/pull/49235. It includes all the extracted fields from a variant column that the query requests. ### Why are the changes needed? By producing variant struct in Parquet reader, we can avoid reading/rebuilding the full variant and achieve more efficient variant processing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49263 from chenhao-db/spark_variant_struct_reader. Authored-by: Chenhao Li Signed-off-by: Wenchen Fan --- .../spark/types/variant/ShreddingUtils.java | 9 +- .../spark/types/variant/VariantSchema.java | 6 + .../parquet/ParquetColumnVector.java | 24 +- .../parquet/ParquetReadSupport.scala | 9 + .../parquet/ParquetRowConverter.scala | 26 +- .../parquet/ParquetSchemaConverter.scala | 4 + .../parquet/SparkShreddingUtils.scala | 597 +++++++++++++++++- .../spark/sql/VariantShreddingSuite.scala | 185 +++++- 8 files changed, 820 insertions(+), 40 deletions(-) diff --git a/common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java b/common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java index 59e16b77ab01d..6a04bf9a2b259 100644 --- a/common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java +++ b/common/variant/src/main/java/org/apache/spark/types/variant/ShreddingUtils.java @@ -49,9 +49,8 @@ public static Variant rebuild(ShreddedRow row, VariantSchema schema) { throw malformedVariant(); } byte[] metadata = row.getBinary(schema.topLevelMetadataIdx); - if (schema.variantIdx >= 0 && schema.typedIdx < 0) { - // The variant is unshredded. We are not required to do anything special, but we can have an - // optimization to avoid `rebuild`. + if (schema.isUnshredded()) { + // `rebuild` is unnecessary for unshredded variant. if (row.isNullAt(schema.variantIdx)) { throw malformedVariant(); } @@ -65,8 +64,8 @@ public static Variant rebuild(ShreddedRow row, VariantSchema schema) { // Rebuild a variant value from the shredded data according to the reconstruction algorithm in // https://github.com/apache/parquet-format/blob/master/VariantShredding.md. // Append the result to `builder`. - private static void rebuild(ShreddedRow row, byte[] metadata, VariantSchema schema, - VariantBuilder builder) { + public static void rebuild(ShreddedRow row, byte[] metadata, VariantSchema schema, + VariantBuilder builder) { int typedIdx = schema.typedIdx; int variantIdx = schema.variantIdx; if (typedIdx >= 0 && !row.isNullAt(typedIdx)) { diff --git a/common/variant/src/main/java/org/apache/spark/types/variant/VariantSchema.java b/common/variant/src/main/java/org/apache/spark/types/variant/VariantSchema.java index 551e46214859a..d1e6cc3a727fa 100644 --- a/common/variant/src/main/java/org/apache/spark/types/variant/VariantSchema.java +++ b/common/variant/src/main/java/org/apache/spark/types/variant/VariantSchema.java @@ -138,6 +138,12 @@ public VariantSchema(int typedIdx, int variantIdx, int topLevelMetadataIdx, int this.arraySchema = arraySchema; } + // Return whether the variant column is unshrededed. The user is not required to do anything + // special, but can have certain optimizations for unshrededed variant. + public boolean isUnshredded() { + return topLevelMetadataIdx >= 0 && variantIdx >= 0 && typedIdx < 0; + } + @Override public String toString() { return "VariantSchema{" + diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java index 0b9a25fc46a0f..7fb8be7caf286 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java @@ -35,7 +35,6 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.VariantType; import org.apache.spark.types.variant.VariantSchema; -import org.apache.spark.unsafe.types.VariantVal; /** * Contains necessary information representing a Parquet column, either of primitive or nested type. @@ -49,6 +48,9 @@ final class ParquetColumnVector { // contains only one child that reads the underlying file content. This `ParquetColumnVector` // should assemble Spark variant values from the file content. private VariantSchema variantSchema; + // Only meaningful if `variantSchema` is not null. See `SparkShreddingUtils.getFieldsToExtract` + // for its meaning. + private FieldToExtract[] fieldsToExtract; /** * Repetition & Definition levels @@ -117,6 +119,7 @@ final class ParquetColumnVector { fileContent, capacity, memoryMode, missingColumns, false, null); children.add(contentVector); variantSchema = SparkShreddingUtils.buildVariantSchema(fileContentCol.sparkType()); + fieldsToExtract = SparkShreddingUtils.getFieldsToExtract(column.sparkType(), variantSchema); repetitionLevels = contentVector.repetitionLevels; definitionLevels = contentVector.definitionLevels; } else if (isPrimitive) { @@ -188,20 +191,11 @@ void assemble() { if (variantSchema != null) { children.get(0).assemble(); WritableColumnVector fileContent = children.get(0).getValueVector(); - int numRows = fileContent.getElementsAppended(); - vector.reset(); - vector.reserve(numRows); - WritableColumnVector valueChild = vector.getChild(0); - WritableColumnVector metadataChild = vector.getChild(1); - for (int i = 0; i < numRows; ++i) { - if (fileContent.isNullAt(i)) { - vector.appendStruct(true); - } else { - vector.appendStruct(false); - VariantVal v = SparkShreddingUtils.rebuild(fileContent.getStruct(i), variantSchema); - valueChild.appendByteArray(v.getValue(), 0, v.getValue().length); - metadataChild.appendByteArray(v.getMetadata(), 0, v.getMetadata().length); - } + if (fieldsToExtract == null) { + SparkShreddingUtils.assembleVariantBatch(fileContent, vector, variantSchema); + } else { + SparkShreddingUtils.assembleVariantStructBatch(fileContent, vector, variantSchema, + fieldsToExtract); } return; } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 8dde02a4673f0..af0bf0d51f077 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -35,6 +35,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.datasources.VariantMetadata import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ @@ -221,6 +222,9 @@ object ParquetReadSupport extends Logging { clipParquetMapType( parquetType.asGroupType(), t.keyType, t.valueType, caseSensitive, useFieldId) + case t: StructType if VariantMetadata.isVariantStruct(t) => + clipVariantSchema(parquetType.asGroupType(), t) + case t: StructType => clipParquetGroup(parquetType.asGroupType(), t, caseSensitive, useFieldId) @@ -390,6 +394,11 @@ object ParquetReadSupport extends Logging { .named(parquetRecord.getName) } + private def clipVariantSchema(parquetType: GroupType, variantStruct: StructType): GroupType = { + // TODO(SHREDDING): clip `parquetType` to retain the necessary columns. + parquetType + } + /** * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[StructType]]. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 3ed7fe37ccd96..550c2af43a706 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.datasources.DataSourceUtils +import org.apache.spark.sql.execution.datasources.{DataSourceUtils, VariantMetadata} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{UTF8String, VariantVal} @@ -498,6 +498,9 @@ private[parquet] class ParquetRowConverter( case t: MapType => new ParquetMapConverter(parquetType.asGroupType(), t, updater) + case t: StructType if VariantMetadata.isVariantStruct(t) => + new ParquetVariantConverter(t, parquetType.asGroupType(), updater) + case t: StructType => val wrappedUpdater = { // SPARK-30338: avoid unnecessary InternalRow copying for nested structs: @@ -536,12 +539,7 @@ private[parquet] class ParquetRowConverter( case t: VariantType => if (SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_READING_SHREDDED)) { - // Infer a Spark type from `parquetType`. This piece of code is copied from - // `ParquetArrayConverter`. - val messageType = Types.buildMessage().addField(parquetType).named("foo") - val column = new ColumnIOFactory().getColumnIO(messageType) - val parquetSparkType = schemaConverter.convertField(column.getChild(0)).sparkType - new ParquetVariantConverter(parquetType.asGroupType(), parquetSparkType, updater) + new ParquetVariantConverter(t, parquetType.asGroupType(), updater) } else { new ParquetUnshreddedVariantConverter(parquetType.asGroupType(), updater) } @@ -909,13 +907,14 @@ private[parquet] class ParquetRowConverter( /** Parquet converter for Variant (shredded or unshredded) */ private final class ParquetVariantConverter( - parquetType: GroupType, - parquetSparkType: DataType, - updater: ParentContainerUpdater) + targetType: DataType, parquetType: GroupType, updater: ParentContainerUpdater) extends ParquetGroupConverter(updater) { private[this] var currentRow: Any = _ + private[this] val parquetSparkType = SparkShreddingUtils.parquetTypeToSparkType(parquetType) private[this] val variantSchema = SparkShreddingUtils.buildVariantSchema(parquetSparkType) + private[this] val fieldsToExtract = + SparkShreddingUtils.getFieldsToExtract(targetType, variantSchema) // A struct converter that reads the underlying file data. private[this] val fileConverter = new ParquetRowConverter( schemaConverter, @@ -932,7 +931,12 @@ private[parquet] class ParquetRowConverter( override def end(): Unit = { fileConverter.end() - val v = SparkShreddingUtils.rebuild(currentRow.asInstanceOf[InternalRow], variantSchema) + val row = currentRow.asInstanceOf[InternalRow] + val v = if (fieldsToExtract == null) { + SparkShreddingUtils.assembleVariant(row, variantSchema) + } else { + SparkShreddingUtils.assembleVariantStruct(row, variantSchema, fieldsToExtract) + } updater.set(v) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 7f1b49e737900..64c2a3126ca9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -28,6 +28,7 @@ import org.apache.parquet.schema.Type.Repetition._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.VariantMetadata import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -185,6 +186,9 @@ class ParquetToSparkSchemaConverter( } else { convertVariantField(groupColumn) } + case groupColumn: GroupColumnIO if targetType.exists(VariantMetadata.isVariantStruct) => + val col = convertGroupField(groupColumn) + col.copy(sparkType = targetType.get, variantFileType = Some(col)) case groupColumn: GroupColumnIO => convertGroupField(groupColumn, targetType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index f38e188ed042c..a83ca78455faa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -17,12 +17,23 @@ package org.apache.spark.sql.execution.datasources.parquet +import org.apache.parquet.io.ColumnIOFactory +import org.apache.parquet.schema.{Type => ParquetType, Types => ParquetTypes} + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.expressions.variant._ +import org.apache.spark.sql.catalyst.expressions.variant.VariantPathParser.PathSegment +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.execution.RowToColumnConverter +import org.apache.spark.sql.execution.datasources.VariantMetadata +import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types._ import org.apache.spark.types.variant._ +import org.apache.spark.types.variant.VariantUtil.Type import org.apache.spark.unsafe.types._ case class SparkShreddedRow(row: SpecializedGetters) extends ShreddingUtils.ShreddedRow { @@ -45,6 +56,369 @@ case class SparkShreddedRow(row: SpecializedGetters) extends ShreddingUtils.Shre override def numElements(): Int = row.asInstanceOf[ArrayData].numElements() } +// The search result of a `PathSegment` in a `VariantSchema`. +case class SchemaPathSegment( + rawPath: PathSegment, + // Whether this path segment is an object or array extraction. + isObject: Boolean, + // `schema.typedIdx`, if the path exists in the schema (for object extraction, the schema + // should contain an object `typed_value` containing the requested field; similar for array + // extraction). Negative otherwise. + typedIdx: Int, + // For object extraction, it is the index of the desired field in `schema.objectSchema`. If the + // requested field doesn't exist, both `extractionIdx/typedIdx` are set to negative. + // For array extraction, it is the array index. The information is already stored in `rawPath`, + // but accessing a raw int should be more efficient than `rawPath`, which is an `Either`. + extractionIdx: Int) + +// Represent a single field in a variant struct (see `VariantMetadata` for definition), that is, a +// single requested field that the scan should produce by extracting from the variant column. +case class FieldToExtract(path: Array[SchemaPathSegment], reader: ParquetVariantReader) + +// A helper class to cast from scalar `typed_value` into a scalar `dataType`. Need a custom +// expression because it has different error reporting code than `Cast`. +case class ScalarCastHelper( + child: Expression, + dataType: DataType, + castArgs: VariantCastArgs) extends UnaryExpression { + // The expression is only for the internal use of `ScalarReader`, which can guarantee the child + // is not nullable. + assert(!child.nullable) + + // If `cast` is null, it means the cast always fails because the type combination is not allowed. + private val cast = if (Cast.canAnsiCast(child.dataType, dataType)) { + Cast(child, dataType, castArgs.zoneStr, EvalMode.TRY) + } else { + null + } + // Cast the input to string. Only used for reporting an invalid cast. + private val castToString = Cast(child, StringType, castArgs.zoneStr, EvalMode.ANSI) + + override def nullable: Boolean = !castArgs.failOnError + override def withNewChildInternal(newChild: Expression): UnaryExpression = copy(child = newChild) + + // No need to define the interpreted version of `eval`: the codegen must succeed. + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + // Throw an error or do nothing, depending on `castArgs.failOnError`. + val invalidCastCode = if (castArgs.failOnError) { + val castToStringCode = castToString.genCode(ctx) + val typeObj = ctx.addReferenceObj("dataType", dataType) + val cls = classOf[ScalarCastHelper].getName + s""" + ${castToStringCode.code} + $cls.throwInvalidVariantCast(${castToStringCode.value}, $typeObj); + """ + } else { + "" + } + if (cast != null) { + val castCode = cast.genCode(ctx) + val code = code""" + ${castCode.code} + boolean ${ev.isNull} = ${castCode.isNull}; + ${CodeGenerator.javaType(dataType)} ${ev.value} = ${castCode.value}; + if (${ev.isNull}) { $invalidCastCode } + """ + ev.copy(code = code) + } else { + val code = code""" + boolean ${ev.isNull} = true; + ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + if (${ev.isNull}) { $invalidCastCode } + """ + ev.copy(code = code) + } + } +} + +object ScalarCastHelper { + // A helper function for codegen. The java compiler doesn't allow throwing a `Throwable` in a + // method without `throws` annotation. + def throwInvalidVariantCast(value: UTF8String, dataType: DataType): Any = + throw QueryExecutionErrors.invalidVariantCast(value.toString, dataType) +} + +// The base class to read Parquet variant values into a Spark type. +// For convenience, we also allow creating an instance of the base class itself. None of its +// functions can be used, but it can serve as a container of `targetType` and `castArgs`. +class ParquetVariantReader( + val schema: VariantSchema, val targetType: DataType, val castArgs: VariantCastArgs) { + // Read from a row containing a Parquet variant value (shredded or unshredded) and return a value + // of `targetType`. The row schema is described by `schema`. + // This function throws MALFORMED_VARIANT if the variant is missing. If the variant can be + // legally missing (the only possible situation is struct fields in object `typed_value`), the + // caller should check for it and avoid calling this function if the variant is missing. + def read(row: InternalRow, topLevelMetadata: Array[Byte]): Any = { + if (schema.typedIdx < 0 || row.isNullAt(schema.typedIdx)) { + if (schema.variantIdx < 0 || row.isNullAt(schema.variantIdx)) { + // Both `typed_value` and `value` are null, meaning the variant is missing. + throw QueryExecutionErrors.malformedVariant() + } + val v = new Variant(row.getBinary(schema.variantIdx), topLevelMetadata) + VariantGet.cast(v, targetType, castArgs) + } else { + readFromTyped(row, topLevelMetadata) + } + } + + // Subclasses should override it to produce the read result when `typed_value` is not null. + protected def readFromTyped(row: InternalRow, topLevelMetadata: Array[Byte]): Any = + throw QueryExecutionErrors.unreachableError() + + // A util function to rebuild the variant in binary format from a Parquet variant value. + protected final def rebuildVariant(row: InternalRow, topLevelMetadata: Array[Byte]): Variant = { + val builder = new VariantBuilder(false) + ShreddingUtils.rebuild(SparkShreddedRow(row), topLevelMetadata, schema, builder) + builder.result() + } + + // A util function to throw error or return null when an invalid cast happens. + protected final def invalidCast(row: InternalRow, topLevelMetadata: Array[Byte]): Any = { + if (castArgs.failOnError) { + throw QueryExecutionErrors.invalidVariantCast( + rebuildVariant(row, topLevelMetadata).toJson(castArgs.zoneId), targetType) + } else { + null + } + } +} + +object ParquetVariantReader { + // Create a reader for `targetType`. If `schema` is null, meaning that the extraction path doesn't + // exist in `typed_value`, it returns an instance of `ParquetVariantReader`. As described in the + // class comment, the reader is only a container of `targetType` and `castArgs` in this case. + def apply(schema: VariantSchema, targetType: DataType, castArgs: VariantCastArgs, + isTopLevelUnshredded: Boolean = false): ParquetVariantReader = targetType match { + case _ if schema == null => new ParquetVariantReader(schema, targetType, castArgs) + case s: StructType => new StructReader(schema, s, castArgs) + case a: ArrayType => new ArrayReader(schema, a, castArgs) + case m@MapType(_: StringType, _, _) => new MapReader(schema, m, castArgs) + case v: VariantType => new VariantReader(schema, v, castArgs, isTopLevelUnshredded) + case s: AtomicType => new ScalarReader(schema, s, castArgs) + case _ => + // Type check should have rejected map with non-string type. + throw QueryExecutionErrors.unreachableError(s"Invalid target type: `${targetType.sql}`") + } +} + +// Read Parquet variant values into a Spark struct type. It reads unshredded fields (fields that are +// not in the typed object) from the `value`, and reads the shredded fields from the object +// `typed_value`. +// `value` must not contain any shredded field according to the shredding spec, but this requirement +// is not enforced. If `value` does contain a shredded field, no error will occur, and the field in +// object `typed_value` will be the final result. +private[this] final class StructReader( + schema: VariantSchema, targetType: StructType, castArgs: VariantCastArgs) + extends ParquetVariantReader(schema, targetType, castArgs) { + // For each field in `targetType`, store the index of the field with the same name in object + // `typed_value`, or -1 if it doesn't exist in object `typed_value`. + private[this] val fieldInputIndices: Array[Int] = targetType.fields.map { f => + val inputIdx = if (schema.objectSchemaMap != null) schema.objectSchemaMap.get(f.name) else null + if (inputIdx != null) inputIdx.intValue() else -1 + } + // For each field in `targetType`, store the reader from the corresponding field in object + // `typed_value`, or null if it doesn't exist in object `typed_value`. + private[this] val fieldReaders: Array[ParquetVariantReader] = + targetType.fields.zip(fieldInputIndices).map { case (f, inputIdx) => + if (inputIdx >= 0) { + val fieldSchema = schema.objectSchema(inputIdx).schema + ParquetVariantReader(fieldSchema, f.dataType, castArgs) + } else { + null + } + } + // If all fields in `targetType` can be found in object `typed_value`, then the reader doesn't + // need to read from `value`. + private[this] val needUnshreddedObject: Boolean = fieldInputIndices.exists(_ < 0) + + override def readFromTyped(row: InternalRow, topLevelMetadata: Array[Byte]): Any = { + if (schema.objectSchema == null) return invalidCast(row, topLevelMetadata) + val obj = row.getStruct(schema.typedIdx, schema.objectSchema.length) + val result = new GenericInternalRow(fieldInputIndices.length) + var unshreddedObject: Variant = null + if (needUnshreddedObject && schema.variantIdx >= 0 && !row.isNullAt(schema.variantIdx)) { + unshreddedObject = new Variant(row.getBinary(schema.variantIdx), topLevelMetadata) + if (unshreddedObject.getType != Type.OBJECT) throw QueryExecutionErrors.malformedVariant() + } + val numFields = fieldInputIndices.length + var i = 0 + while (i < numFields) { + val inputIdx = fieldInputIndices(i) + if (inputIdx >= 0) { + // Shredded field must not be null. + if (obj.isNullAt(inputIdx)) throw QueryExecutionErrors.malformedVariant() + val fieldSchema = schema.objectSchema(inputIdx).schema + val fieldInput = obj.getStruct(inputIdx, fieldSchema.numFields) + // Only read from the shredded field if it is not missing. + if ((fieldSchema.typedIdx >= 0 && !fieldInput.isNullAt(fieldSchema.typedIdx)) || + (fieldSchema.variantIdx >= 0 && !fieldInput.isNullAt(fieldSchema.variantIdx))) { + result.update(i, fieldReaders(i).read(fieldInput, topLevelMetadata)) + } + } else if (unshreddedObject != null) { + val fieldName = targetType.fields(i).name + val fieldType = targetType.fields(i).dataType + val unshreddedField = unshreddedObject.getFieldByKey(fieldName) + if (unshreddedField != null) { + result.update(i, VariantGet.cast(unshreddedField, fieldType, castArgs)) + } + } + i += 1 + } + result + } +} + +// Read Parquet variant values into a Spark array type. +private[this] final class ArrayReader( + schema: VariantSchema, targetType: ArrayType, castArgs: VariantCastArgs) + extends ParquetVariantReader(schema, targetType, castArgs) { + private[this] val elementReader = if (schema.arraySchema != null) { + ParquetVariantReader(schema.arraySchema, targetType.elementType, castArgs) + } else { + null + } + + override def readFromTyped(row: InternalRow, topLevelMetadata: Array[Byte]): Any = { + if (schema.arraySchema == null) return invalidCast(row, topLevelMetadata) + val elementNumFields = schema.arraySchema.numFields + val arr = row.getArray(schema.typedIdx) + val size = arr.numElements() + val result = new Array[Any](size) + var i = 0 + while (i < size) { + // Shredded array element must not be null. + if (arr.isNullAt(i)) throw QueryExecutionErrors.malformedVariant() + result(i) = elementReader.read(arr.getStruct(i, elementNumFields), topLevelMetadata) + i += 1 + } + new GenericArrayData(result) + } +} + +// Read Parquet variant values into a Spark map type with string key type. The input must be object +// for a valid cast. The resulting map contains shredded fields from object `typed_value` and +// unshredded fields from object `value`. +// `value` must not contain any shredded field according to the shredding spec. Unlike +// `StructReader`, this requirement is enforced in `MapReader`. If `value` does contain a shredded +// field, throw a MALFORMED_VARIANT error. The purpose is to avoid duplicate map keys. +private[this] final class MapReader( + schema: VariantSchema, targetType: MapType, castArgs: VariantCastArgs) + extends ParquetVariantReader(schema, targetType, castArgs) { + // Readers that convert each shredded field into the map value type. + private[this] val valueReaders = if (schema.objectSchema != null) { + schema.objectSchema.map { f => + ParquetVariantReader(f.schema, targetType.valueType, castArgs) + } + } else { + null + } + // `UTF8String` representation of shredded field names. Do the `String -> UTF8String` once, so + // that `readFromTyped` doesn't need to do it repeatedly. + private[this] val shreddedFieldNames = if (schema.objectSchema != null) { + schema.objectSchema.map { f => UTF8String.fromString(f.fieldName) } + } else { + null + } + + override def readFromTyped(row: InternalRow, topLevelMetadata: Array[Byte]): Any = { + if (schema.objectSchema == null) return invalidCast(row, topLevelMetadata) + val obj = row.getStruct(schema.typedIdx, schema.objectSchema.length) + val numShreddedFields = valueReaders.length + var unshreddedObject: Variant = null + if (schema.variantIdx >= 0 && !row.isNullAt(schema.variantIdx)) { + unshreddedObject = new Variant(row.getBinary(schema.variantIdx), topLevelMetadata) + if (unshreddedObject.getType != Type.OBJECT) throw QueryExecutionErrors.malformedVariant() + } + val numUnshreddedFields = if (unshreddedObject != null) unshreddedObject.objectSize() else 0 + var keyArray = new Array[UTF8String](numShreddedFields + numUnshreddedFields) + var valueArray = new Array[Any](numShreddedFields + numUnshreddedFields) + var mapLength = 0 + var i = 0 + while (i < numShreddedFields) { + // Shredded field must not be null. + if (obj.isNullAt(i)) throw QueryExecutionErrors.malformedVariant() + val fieldSchema = schema.objectSchema(i).schema + val fieldInput = obj.getStruct(i, fieldSchema.numFields) + // Only add the shredded field to map if it is not missing. + if ((fieldSchema.typedIdx >= 0 && !fieldInput.isNullAt(fieldSchema.typedIdx)) || + (fieldSchema.variantIdx >= 0 && !fieldInput.isNullAt(fieldSchema.variantIdx))) { + keyArray(mapLength) = shreddedFieldNames(i) + valueArray(mapLength) = valueReaders(i).read(fieldInput, topLevelMetadata) + mapLength += 1 + } + i += 1 + } + i = 0 + while (i < numUnshreddedFields) { + val field = unshreddedObject.getFieldAtIndex(i) + if (schema.objectSchemaMap.containsKey(field.key)) { + throw QueryExecutionErrors.malformedVariant() + } + keyArray(mapLength) = UTF8String.fromString(field.key) + valueArray(mapLength) = VariantGet.cast(field.value, targetType.valueType, castArgs) + mapLength += 1 + i += 1 + } + // Need to shrink the arrays if there are missing shredded fields. + if (mapLength < keyArray.length) { + keyArray = keyArray.slice(0, mapLength) + valueArray = valueArray.slice(0, mapLength) + } + ArrayBasedMapData(keyArray, valueArray) + } +} + +// Read Parquet variant values into a Spark variant type (the binary format). +private[this] final class VariantReader( + schema: VariantSchema, targetType: DataType, castArgs: VariantCastArgs, + // An optional optimization: the user can set it to true if the Parquet variant column is + // unshredded and the extraction path is empty. We are not required to do anything special, bu + // we can avoid rebuilding variant for optimization purpose. + private[this] val isTopLevelUnshredded: Boolean) + extends ParquetVariantReader(schema, targetType, castArgs) { + override def read(row: InternalRow, topLevelMetadata: Array[Byte]): Any = { + if (isTopLevelUnshredded) { + if (row.isNullAt(schema.variantIdx)) throw QueryExecutionErrors.malformedVariant() + return new VariantVal(row.getBinary(schema.variantIdx), topLevelMetadata) + } + val v = rebuildVariant(row, topLevelMetadata) + new VariantVal(v.getValue, v.getMetadata) + } +} + +// Read Parquet variant values into a Spark scalar type. When `typed_value` is not null but not a +// scalar, all other target types should return an invalid cast, but only the string target type can +// still build a string from array/object `typed_value`. For scalar `typed_value`, it depends on +// `ScalarCastHelper` to perform the cast. +// According to the shredding spec, scalar `typed_value` and `value` must not be non-null at the +// same time. The requirement is not enforced in this reader. If they are both non-null, no error +// will occur, and the reader will read from `typed_value`. +private[this] final class ScalarReader( + schema: VariantSchema, targetType: DataType, castArgs: VariantCastArgs) + extends ParquetVariantReader(schema, targetType, castArgs) { + private[this] val castProject = if (schema.scalarSchema != null) { + val scalarType = SparkShreddingUtils.scalarSchemaToSparkType(schema.scalarSchema) + // Read the cast input from ordinal `schema.typedIdx` in the input row. The cast input is never + // null, because `readFromTyped` is only called when `typed_value` is not null. + val input = BoundReference(schema.typedIdx, scalarType, nullable = false) + MutableProjection.create(Seq(ScalarCastHelper(input, targetType, castArgs))) + } else { + null + } + + override def readFromTyped(row: InternalRow, topLevelMetadata: Array[Byte]): Any = { + if (castProject == null) { + return if (targetType.isInstanceOf[StringType]) { + UTF8String.fromString(rebuildVariant(row, topLevelMetadata).toJson(castArgs.zoneId)) + } else { + invalidCast(row, topLevelMetadata) + } + } + val result = castProject(row) + if (result.isNullAt(0)) null else result.get(0, targetType) + } +} + case object SparkShreddingUtils { val VariantValueFieldName = "value"; val TypedValueFieldName = "typed_value"; @@ -126,6 +500,11 @@ case object SparkShreddingUtils { var objectSchema: Array[VariantSchema.ObjectField] = null var arraySchema: VariantSchema = null + // The struct must not be empty or contain duplicate field names. The latter is enforced in the + // loop below (`if (typedIdx != -1)` and other similar checks). + if (schema.fields.isEmpty) { + throw QueryCompilationErrors.invalidVariantShreddingSchema(schema) + } schema.fields.zipWithIndex.foreach { case (f, i) => f.name match { case TypedValueFieldName => @@ -135,8 +514,11 @@ case object SparkShreddingUtils { typedIdx = i f.dataType match { case StructType(fields) => - objectSchema = - new Array[VariantSchema.ObjectField](fields.length) + // The struct must not be empty or contain duplicate field names. + if (fields.isEmpty || fields.map(_.name).distinct.length != fields.length) { + throw QueryCompilationErrors.invalidVariantShreddingSchema(schema) + } + objectSchema = new Array[VariantSchema.ObjectField](fields.length) fields.zipWithIndex.foreach { case (field, fieldIdx) => field.dataType match { case s: StructType => @@ -188,6 +570,32 @@ case object SparkShreddingUtils { scalarSchema, objectSchema, arraySchema) } + // Convert a scalar variant schema into a Spark scalar type. + def scalarSchemaToSparkType(scalar: VariantSchema.ScalarType): DataType = scalar match { + case _: VariantSchema.StringType => StringType + case it: VariantSchema.IntegralType => it.size match { + case VariantSchema.IntegralSize.BYTE => ByteType + case VariantSchema.IntegralSize.SHORT => ShortType + case VariantSchema.IntegralSize.INT => IntegerType + case VariantSchema.IntegralSize.LONG => LongType + } + case _: VariantSchema.FloatType => FloatType + case _: VariantSchema.DoubleType => DoubleType + case _: VariantSchema.BooleanType => BooleanType + case _: VariantSchema.BinaryType => BinaryType + case dt: VariantSchema.DecimalType => DecimalType(dt.precision, dt.scale) + case _: VariantSchema.DateType => DateType + case _: VariantSchema.TimestampType => TimestampType + case _: VariantSchema.TimestampNTZType => TimestampNTZType + } + + // Convert a Parquet type into a Spark data type. + def parquetTypeToSparkType(parquetType: ParquetType): DataType = { + val messageType = ParquetTypes.buildMessage().addField(parquetType).named("foo") + val column = new ColumnIOFactory().getColumnIO(messageType) + new ParquetToSparkSchemaConverter().convertField(column.getChild(0)).sparkType + } + class SparkShreddedResult(schema: VariantSchema) extends VariantShreddingWriter.ShreddedResult { // Result is stored as an InternalRow. val row = new GenericInternalRow(schema.numFields) @@ -243,8 +651,187 @@ case object SparkShreddingUtils { .row } - def rebuild(row: InternalRow, schema: VariantSchema): VariantVal = { + // Return a list of fields to extract. `targetType` must be either variant or variant struct. + // If it is variant, return null because the target is the full variant and there is no field to + // extract. If it is variant struct, return a list of fields matching the variant struct fields. + def getFieldsToExtract(targetType: DataType, inputSchema: VariantSchema): Array[FieldToExtract] = + targetType match { + case _: VariantType => null + case s: StructType if VariantMetadata.isVariantStruct(s) => + s.fields.map { f => + val metadata = VariantMetadata.fromMetadata(f.metadata) + val rawPath = metadata.parsedPath() + val schemaPath = new Array[SchemaPathSegment](rawPath.length) + var schema = inputSchema + // Search `rawPath` in `schema` to produce `schemaPath`. If a raw path segment cannot be + // found at a certain level of the file type, then `typedIdx` will be -1 starting from + // this position, and the final `schema` will be null. + for (i <- rawPath.indices) { + val isObject = rawPath(i).isLeft + var typedIdx = -1 + var extractionIdx = -1 + rawPath(i) match { + case scala.util.Left(key) if schema != null && schema.objectSchema != null => + val fieldIdx = schema.objectSchemaMap.get(key) + if (fieldIdx != null) { + typedIdx = schema.typedIdx + extractionIdx = fieldIdx + schema = schema.objectSchema(fieldIdx).schema + } else { + schema = null + } + case scala.util.Right(index) if schema != null && schema.arraySchema != null => + typedIdx = schema.typedIdx + extractionIdx = index + schema = schema.arraySchema + case _ => + schema = null + } + schemaPath(i) = SchemaPathSegment(rawPath(i), isObject, typedIdx, extractionIdx) + } + val reader = ParquetVariantReader(schema, f.dataType, VariantCastArgs( + metadata.failOnError, + Some(metadata.timeZoneId), + DateTimeUtils.getZoneId(metadata.timeZoneId)), + isTopLevelUnshredded = schemaPath.isEmpty && inputSchema.isUnshredded) + FieldToExtract(schemaPath, reader) + } + case _ => + throw QueryExecutionErrors.unreachableError(s"Invalid target type: `${targetType.sql}`") + } + + // Extract a single variant struct field from a Parquet variant value. It steps into `inputRow` + // according to the variant extraction path, and read the extracted value as the target type. + private def extractField( + inputRow: InternalRow, + topLevelMetadata: Array[Byte], + inputSchema: VariantSchema, + pathList: Array[SchemaPathSegment], + reader: ParquetVariantReader): Any = { + var pathIdx = 0 + val pathLen = pathList.length + var row = inputRow + var schema = inputSchema + while (pathIdx < pathLen) { + val path = pathList(pathIdx) + + if (path.typedIdx < 0) { + // The extraction doesn't exist in `typed_value`. Try to extract the remaining part of the + // path in `value`. + val variantIdx = schema.variantIdx + if (variantIdx < 0 || row.isNullAt(variantIdx)) return null + var v = new Variant(row.getBinary(variantIdx), topLevelMetadata) + while (pathIdx < pathLen) { + v = pathList(pathIdx).rawPath match { + case scala.util.Left(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) + case scala.util.Right(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) + case _ => null + } + if (v == null) return null + pathIdx += 1 + } + return VariantGet.cast(v, reader.targetType, reader.castArgs) + } + + if (row.isNullAt(path.typedIdx)) return null + if (path.isObject) { + val obj = row.getStruct(path.typedIdx, schema.objectSchema.length) + // Object field must not be null. + if (obj.isNullAt(path.extractionIdx)) throw QueryExecutionErrors.malformedVariant() + schema = schema.objectSchema(path.extractionIdx).schema + row = obj.getStruct(path.extractionIdx, schema.numFields) + // Return null if the field is missing. + if ((schema.typedIdx < 0 || row.isNullAt(schema.typedIdx)) && + (schema.variantIdx < 0 || row.isNullAt(schema.variantIdx))) { + return null + } + } else { + val arr = row.getArray(path.typedIdx) + // Return null if the extraction index is out of bound. + if (path.extractionIdx >= arr.numElements()) return null + // Array element must not be null. + if (arr.isNullAt(path.extractionIdx)) throw QueryExecutionErrors.malformedVariant() + schema = schema.arraySchema + row = arr.getStruct(path.extractionIdx, schema.numFields) + } + pathIdx += 1 + } + reader.read(row, topLevelMetadata) + } + + // Assemble a variant (binary format) from a Parquet variant value. + def assembleVariant(row: InternalRow, schema: VariantSchema): VariantVal = { val v = ShreddingUtils.rebuild(SparkShreddedRow(row), schema) new VariantVal(v.getValue, v.getMetadata) } + + // Assemble a variant struct, in which each field is extracted from the Parquet variant value. + def assembleVariantStruct( + inputRow: InternalRow, + schema: VariantSchema, + fields: Array[FieldToExtract]): InternalRow = { + if (inputRow.isNullAt(schema.topLevelMetadataIdx)) { + throw QueryExecutionErrors.malformedVariant() + } + val topLevelMetadata = inputRow.getBinary(schema.topLevelMetadataIdx) + val numFields = fields.length + val resultRow = new GenericInternalRow(numFields) + var fieldIdx = 0 + while (fieldIdx < numFields) { + resultRow.update(fieldIdx, extractField(inputRow, topLevelMetadata, schema, + fields(fieldIdx).path, fields(fieldIdx).reader)) + fieldIdx += 1 + } + resultRow + } + + // Assemble a batch of variant (binary format) from a batch of Parquet variant values. + def assembleVariantBatch( + input: WritableColumnVector, + output: WritableColumnVector, + schema: VariantSchema): Unit = { + val numRows = input.getElementsAppended + output.reset() + output.reserve(numRows) + val valueChild = output.getChild(0) + val metadataChild = output.getChild(1) + var i = 0 + while (i < numRows) { + if (input.isNullAt(i)) { + output.appendStruct(true) + } else { + output.appendStruct(false) + val v = SparkShreddingUtils.assembleVariant(input.getStruct(i), schema) + valueChild.appendByteArray(v.getValue, 0, v.getValue.length) + metadataChild.appendByteArray(v.getMetadata, 0, v.getMetadata.length) + } + i += 1 + } + } + + // Assemble a batch of variant struct from a batch of Parquet variant values. + def assembleVariantStructBatch( + input: WritableColumnVector, + output: WritableColumnVector, + schema: VariantSchema, + fields: Array[FieldToExtract]): Unit = { + val numRows = input.getElementsAppended + output.reset() + output.reserve(numRows) + val converter = new RowToColumnConverter(StructType(Array(StructField("", output.dataType())))) + val converterVectors = Array(output) + val converterRow = new GenericInternalRow(1) + output.reset() + output.reserve(input.getElementsAppended) + var i = 0 + while (i < numRows) { + if (input.isNullAt(i)) { + converterRow.update(0, null) + } else { + converterRow.update(0, assembleVariantStruct(input.getStruct(i), schema, fields)) + } + converter.convert(converterRow, converterVectors) + i += 1 + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala index 5d5c441052558..b6623bb57a716 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala @@ -22,13 +22,21 @@ import java.sql.{Date, Timestamp} import java.time.LocalDateTime import org.apache.spark.SparkThrowable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils import org.apache.spark.sql.execution.datasources.parquet.{ParquetTest, SparkShreddingUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.types.variant._ +import org.apache.spark.unsafe.types.{UTF8String, VariantVal} class VariantShreddingSuite extends QueryTest with SharedSparkSession with ParquetTest { + def parseJson(s: String): VariantVal = { + val v = VariantBuilder.parseJson(s, false) + new VariantVal(v.getValue, v.getMetadata) + } + // Make a variant value binary by parsing a JSON string. def value(s: String): Array[Byte] = VariantBuilder.parseJson(s, false).getValue @@ -53,9 +61,21 @@ class VariantShreddingSuite extends QueryTest with SharedSparkSession with Parqu def writeSchema(schema: DataType): StructType = StructType(Array(StructField("v", SparkShreddingUtils.variantShreddingSchema(schema)))) + def withPushConfigs(pushConfigs: Seq[Boolean] = Seq(true, false))(fn: => Unit): Unit = { + for (push <- pushConfigs) { + withSQLConf(SQLConf.PUSH_VARIANT_INTO_SCAN.key -> push.toString) { + fn + } + } + } + + def isPushEnabled: Boolean = SQLConf.get.getConf(SQLConf.PUSH_VARIANT_INTO_SCAN) + def testWithTempPath(name: String)(block: File => Unit): Unit = test(name) { - withTempPath { path => - block(path) + withPushConfigs() { + withTempPath { path => + block(path) + } } } @@ -63,6 +83,9 @@ class VariantShreddingSuite extends QueryTest with SharedSparkSession with Parqu spark.createDataFrame(spark.sparkContext.parallelize(rows.map(Row(_)), numSlices = 1), schema) .write.mode("overwrite").parquet(path.getAbsolutePath) + def writeRows(path: File, schema: String, rows: Row*): Unit = + writeRows(path, StructType.fromDDL(schema), rows: _*) + def read(path: File): DataFrame = spark.read.schema("v variant").parquet(path.getAbsolutePath) @@ -150,10 +173,13 @@ class VariantShreddingSuite extends QueryTest with SharedSparkSession with Parqu // Top-level variant must not be missing. writeRows(path, writeSchema(IntegerType), Row(metadata(Nil), null, null)) checkException(path, "v", "MALFORMED_VARIANT") + // Array-element variant must not be missing. writeRows(path, writeSchema(ArrayType(IntegerType)), Row(metadata(Nil), null, Array(Row(null, null)))) checkException(path, "v", "MALFORMED_VARIANT") + checkException(path, "variant_get(v, '$[0]')", "MALFORMED_VARIANT") + // Shredded field must not be null. // Construct the schema manually, because SparkShreddingUtils.variantShreddingSchema will make // `a` non-nullable, which would prevent us from writing the file. @@ -164,12 +190,163 @@ class VariantShreddingSuite extends QueryTest with SharedSparkSession with Parqu StructField("a", StructType(Seq( StructField("value", BinaryType), StructField("typed_value", BinaryType)))))))))))) - writeRows(path, schema, - Row(metadata(Seq("a")), null, Row(null))) + writeRows(path, schema, Row(metadata(Seq("a")), null, Row(null))) checkException(path, "v", "MALFORMED_VARIANT") + checkException(path, "variant_get(v, '$.a')", "MALFORMED_VARIANT") + // `value` must not contain any shredded field. writeRows(path, writeSchema(StructType.fromDDL("a int")), Row(metadata(Seq("a")), value("""{"a": 1}"""), Row(Row(null, null)))) checkException(path, "v", "MALFORMED_VARIANT") + checkException(path, "cast(v as map)", "MALFORMED_VARIANT") + if (isPushEnabled) { + checkExpr(path, "cast(v as struct)", Row(null)) + checkExpr(path, "variant_get(v, '$.a', 'int')", null) + } else { + checkException(path, "cast(v as struct)", "MALFORMED_VARIANT") + checkException(path, "variant_get(v, '$.a', 'int')", "MALFORMED_VARIANT") + } + + // Scalar reader reads from `typed_value` if both `value` and `typed_value` are not null. + // Cast from `value` succeeds, cast from `typed_value` fails. + writeRows(path, "v struct", + Row(metadata(Nil), value("1"), "invalid")) + checkException(path, "cast(v as int)", "INVALID_VARIANT_CAST") + checkExpr(path, "try_cast(v as int)", null) + + // Cast from `value` fails, cast from `typed_value` succeeds. + writeRows(path, "v struct", + Row(metadata(Nil), value("\"invalid\""), "1")) + checkExpr(path, "cast(v as int)", 1) + checkExpr(path, "try_cast(v as int)", 1) + } + + testWithTempPath("extract from shredded object") { path => + val keys1 = Seq("a", "b", "c", "d") + val keys2 = Seq("a", "b", "c", "e", "f") + writeRows(path, "v struct, b struct," + + "c struct>>", + // {"a":1,"b":"2","c":3.3,"d":4.4}, d is in the left over value. + Row(metadata(keys1), shreddedValue("""{"d": 4.4}""", keys1), + Row(Row(null, 1), Row(value("\"2\"")), Row(Decimal("3.3")))), + // {"a":5.4,"b":-6,"e":{"f":[true]}}, e is in the left over value. + Row(metadata(keys2), shreddedValue("""{"e": {"f": [true]}}""", keys2), + Row(Row(value("5.4"), null), Row(value("-6")), Row(null))), + // [{"a":1}], the unshredded array at the top-level is put into `value` as a whole. + Row(metadata(Seq("a")), value("""[{"a": 1}]"""), null)) + + checkAnswer(read(path).selectExpr("variant_get(v, '$.a', 'int')", + "variant_get(v, '$.b', 'long')", "variant_get(v, '$.c', 'double')", + "variant_get(v, '$.d', 'decimal(9, 4)')"), + Seq(Row(1, 2L, 3.3, BigDecimal("4.4")), Row(5, -6L, null, null), Row(null, null, null, null))) + checkExpr(path, "variant_get(v, '$.e.f[0]', 'boolean')", null, true, null) + checkExpr(path, "variant_get(v, '$[0].a', 'boolean')", null, null, true) + checkExpr(path, "try_cast(v as struct)", + Row(1.0F, null), Row(5.4F, parseJson("""{"f": [true]}""")), null) + + // String "2" cannot be cast into boolean. + checkException(path, "variant_get(v, '$.b', 'boolean')", "INVALID_VARIANT_CAST") + // Decimal cannot be cast into date. + checkException(path, "variant_get(v, '$.c', 'date')", "INVALID_VARIANT_CAST") + // The value of `c` doesn't fit into `decimal(1, 1)`. + checkException(path, "variant_get(v, '$.c', 'decimal(1, 1)')", "INVALID_VARIANT_CAST") + checkExpr(path, "try_variant_get(v, '$.b', 'boolean')", null, true, null) + // Scalar cannot be cast into struct. + checkException(path, "variant_get(v, '$.a', 'struct')", "INVALID_VARIANT_CAST") + checkExpr(path, "try_variant_get(v, '$.a', 'struct')", null, null, null) + + checkExpr(path, "try_cast(v as map)", + Map("a" -> 1.0, "b" -> 2.0, "c" -> 3.3, "d" -> 4.4), + Map("a" -> 5.4, "b" -> -6.0, "e" -> null), null) + checkExpr(path, "try_cast(v as array)", null, null, Seq("""{"a":1}""")) + + val strings = Seq("""{"a":1,"b":"2","c":3.3,"d":4.4}""", + """{"a":5.4,"b":-6,"e":{"f":[true]}}""", """[{"a":1}]""") + checkExpr(path, "cast(v as string)", strings: _*) + checkExpr(path, "v", + VariantExpressionEvalUtils.castToVariant( + InternalRow(1, UTF8String.fromString("2"), Decimal("3.3000000000"), Decimal("4.4")), + StructType.fromDDL("a int, b string, c decimal(20, 10), d decimal(2, 1)") + ), + parseJson(strings(1)), + parseJson(strings(2)) + ) + } + + testWithTempPath("extract from shredded array") { path => + val keys = Seq("a", "b") + writeRows(path, "v struct>>>>", + // [{"a":"2000-01-01"},{"a":"1000-01-01","b":[7]}], b is in the left over value. + Row(metadata(keys), null, Array( + Row(null, Row(Row(null, "2000-01-01"))), + Row(shreddedValue("""{"b": [7]}""", keys), Row(Row(null, "1000-01-01"))))), + // [null,{"a":null},{"a":"null"},{}] + Row(metadata(keys), null, Array( + Row(value("null"), null), + Row(null, Row(Row(value("null"), null))), + Row(null, Row(Row(null, "null"))), + Row(null, Row(Row(null, null)))))) + + val date1 = Date.valueOf("2000-01-01") + val date2 = Date.valueOf("1000-01-01") + checkExpr(path, "variant_get(v, '$[0].a', 'date')", date1, null) + // try_cast succeeds. + checkExpr(path, "try_variant_get(v, '$[1].a', 'date')", date2, null) + // The first array returns null because of out-of-bound index. + // The second array returns "null". + checkExpr(path, "variant_get(v, '$[2].a', 'string')", null, "null") + // Return null because of invalid cast. + checkExpr(path, "try_variant_get(v, '$[1].a', 'int')", null, null) + + checkExpr(path, "variant_get(v, '$[0].b[0]', 'int')", null, null) + checkExpr(path, "variant_get(v, '$[1].b[0]', 'int')", 7, null) + // Validate timestamp-related casts uses the session time zone correctly. + Seq("Etc/UTC", "America/Los_Angeles").foreach { tz => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + val expected = sql("select timestamp'1000-01-01', timestamp_ntz'1000-01-01'").head() + checkAnswer(read(path).selectExpr("variant_get(v, '$[1].a', 'timestamp')", + "variant_get(v, '$[1].a', 'timestamp_ntz')"), Seq(expected, Row(null, null))) + } + } + checkException(path, "variant_get(v, '$[0]', 'int')", "INVALID_VARIANT_CAST") + // An out-of-bound array access produces null. It never causes an invalid cast. + checkExpr(path, "variant_get(v, '$[4]', 'int')", null, null) + + checkExpr(path, "cast(v as array>>)", + Seq(Row("2000-01-01", null), Row("1000-01-01", Seq(7))), + Seq(null, Row(null, null), Row("null", null), Row(null, null))) + checkExpr(path, "cast(v as array>)", + Seq(Map("a" -> "2000-01-01"), Map("a" -> "1000-01-01", "b" -> "[7]")), + Seq(null, Map("a" -> null), Map("a" -> "null"), Map())) + checkExpr(path, "try_cast(v as array>)", + Seq(Map("a" -> date1), Map("a" -> date2, "b" -> null)), + Seq(null, Map("a" -> null), Map("a" -> null), Map())) + + val strings = Seq("""[{"a":"2000-01-01"},{"a":"1000-01-01","b":[7]}]""", + """[null,{"a":null},{"a":"null"},{}]""") + checkExpr(path, "cast(v as string)", strings: _*) + checkExpr(path, "v", strings.map(parseJson): _*) + } + + testWithTempPath("missing fields") { path => + writeRows(path, "v struct, b struct>>", + Row(metadata(Nil), Row(Row(null, null), Row(null))), + Row(metadata(Nil), Row(Row(value("null"), null), Row(null))), + Row(metadata(Nil), Row(Row(null, 1), Row(null))), + Row(metadata(Nil), Row(Row(null, null), Row(2))), + Row(metadata(Nil), Row(Row(value("null"), null), Row(2))), + Row(metadata(Nil), Row(Row(null, 3), Row(4)))) + + val strings = Seq("{}", """{"a":null}""", """{"a":1}""", """{"b":2}""", """{"a":null,"b":2}""", + """{"a":3,"b":4}""") + checkExpr(path, "cast(v as string)", strings: _*) + checkExpr(path, "v", strings.map(parseJson): _*) + + checkExpr(path, "variant_get(v, '$.a', 'string')", null, null, "1", null, null, "3") + checkExpr(path, "variant_get(v, '$.a')", null, parseJson("null"), parseJson("1"), null, + parseJson("null"), parseJson("3")) } } From f9e117e01672e65034ea96672e0f137d835ddcb0 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Wed, 25 Dec 2024 11:13:41 +0800 Subject: [PATCH 241/438] [SPARK-50659][SQL] Move Union-related errors to QueryCompilationErrors ### What changes were proposed in this pull request? Move Union-related `NUM_COLUMNS_MISMATCH` and `INCOMPATIBLE_COLUMN_TYPE` errors to `QueryCompilationErrors`. ### Why are the changes needed? To improve the code health and to reuse those in the single-pass Analyzer. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49284 from vladimirg-db/vladimirg-db/move-union-related-errors-to-query-compilation-errors. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/CheckAnalysis.scala | 32 +++++++-------- .../sql/errors/QueryCompilationErrors.scala | 40 +++++++++++++++++++ 2 files changed, 56 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 8f4431cb1ac11..b0d6a2a46baa0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -702,13 +702,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB operator.children.tail.zipWithIndex.foreach { case (child, ti) => // Check the number of columns if (child.output.length != ref.length) { - e.failAnalysis( - errorClass = "NUM_COLUMNS_MISMATCH", - messageParameters = Map( - "operator" -> toSQLStmt(operator.nodeName), - "firstNumColumns" -> ref.length.toString, - "invalidOrdinalNum" -> ordinalNumber(ti + 1), - "invalidNumColumns" -> child.output.length.toString)) + throw QueryCompilationErrors.numColumnsMismatch( + operator = operator.nodeName, + firstNumColumns = ref.length, + invalidOrdinalNum = ti + 1, + invalidNumColumns = child.output.length, + origin = operator.origin + ) } val dataTypesAreCompatibleFn = getDataTypesAreCompatibleFn(operator) @@ -716,15 +716,15 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB dataTypes(child).zip(ref).zipWithIndex.foreach { case ((dt1, dt2), ci) => // SPARK-18058: we shall not care about the nullability of columns if (!dataTypesAreCompatibleFn(dt1, dt2)) { - e.failAnalysis( - errorClass = "INCOMPATIBLE_COLUMN_TYPE", - messageParameters = Map( - "operator" -> toSQLStmt(operator.nodeName), - "columnOrdinalNumber" -> ordinalNumber(ci), - "tableOrdinalNumber" -> ordinalNumber(ti + 1), - "dataType1" -> toSQLType(dt1), - "dataType2" -> toSQLType(dt2), - "hint" -> extraHintForAnsiTypeCoercionPlan(operator))) + throw QueryCompilationErrors.incompatibleColumnTypeError( + operator = operator.nodeName, + columnOrdinalNumber = ci, + tableOrdinalNumber = ti + 1, + dataType1 = dt1, + dataType2 = dt2, + hint = extraHintForAnsiTypeCoercionPlan(operator), + origin = operator.origin + ) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index e55d4d1f95236..d38c7a01e1c44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4271,4 +4271,44 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat ) ) } + + def numColumnsMismatch( + operator: String, + firstNumColumns: Int, + invalidOrdinalNum: Int, + invalidNumColumns: Int, + origin: Origin): Throwable = { + new AnalysisException( + errorClass = "NUM_COLUMNS_MISMATCH", + messageParameters = Map( + "operator" -> toSQLStmt(operator), + "firstNumColumns" -> firstNumColumns.toString, + "invalidOrdinalNum" -> ordinalNumber(invalidOrdinalNum), + "invalidNumColumns" -> invalidNumColumns.toString + ), + origin = origin + ) + } + + def incompatibleColumnTypeError( + operator: String, + columnOrdinalNumber: Int, + tableOrdinalNumber: Int, + dataType1: DataType, + dataType2: DataType, + hint: String, + origin: Origin): Throwable = { + new AnalysisException( + errorClass = "INCOMPATIBLE_COLUMN_TYPE", + messageParameters = Map( + "operator" -> toSQLStmt(operator), + "columnOrdinalNumber" -> ordinalNumber(columnOrdinalNumber), + "tableOrdinalNumber" -> ordinalNumber(tableOrdinalNumber), + "dataType1" -> toSQLType(dataType1), + "dataType2" -> toSQLType(dataType2), + "hint" -> hint + ), + origin = origin + ) + } } From 495e24830ff2f7de5a295b7facedf81b9e0a2635 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 25 Dec 2024 12:03:24 +0800 Subject: [PATCH 242/438] [MINOR][SQL][DOCS] Fix spacing with SQL configuration documentation ### What changes were proposed in this pull request? This PR proposes to fix spacing with SQL configuration documentation. ### Why are the changes needed? For correct documentation. ### Does this PR introduce _any_ user-facing change? Trivial but yes. It affects spacing in user-facing documentation at https://spark.apache.org/docs/latest/configuration.html. ### How was this patch tested? Manually checked. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49280 from HyukjinKwon/minor-spaces. Authored-by: Hyukjin Kwon Signed-off-by: Ruifeng Zheng --- .../apache/spark/sql/internal/SQLConf.scala | 46 +++++++++---------- .../spark/sql/internal/StaticSQLConf.scala | 2 +- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index be883b2112d19..d5f18231a6c15 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -257,7 +257,7 @@ object SQLConf { "NameScope to control the visibility of names. In contrast to the current fixed-point " + "framework, subsequent in-tree traversals are disallowed. Most of the fixed-point " + "Analyzer code is reused in the form of specific node transformation functions " + - "(AliasResolution.resolve, FunctionResolution.resolveFunction, etc)." + + "(AliasResolution.resolve, FunctionResolution.resolveFunction, etc). " + "This feature is currently under development." ) .version("4.0.0") @@ -672,7 +672,7 @@ object SQLConf { val AUTO_BROADCASTJOIN_THRESHOLD = buildConf("spark.sql.autoBroadcastJoinThreshold") .doc("Configures the maximum size in bytes for a table that will be broadcast to all worker " + - "nodes when performing a join. By setting this value to -1 broadcasting can be disabled.") + "nodes when performing a join. By setting this value to -1 broadcasting can be disabled.") .version("1.1.0") .bytesConf(ByteUnit.BYTE) .createWithDefaultString("10MB") @@ -688,7 +688,7 @@ object SQLConf { val LIMIT_INITIAL_NUM_PARTITIONS = buildConf("spark.sql.limit.initialNumPartitions") .internal() .doc("Initial number of partitions to try when executing a take on a query. Higher values " + - "lead to more partitions read. Lower values might lead to longer execution times as more" + + "lead to more partitions read. Lower values might lead to longer execution times as more " + "jobs will be run") .version("3.4.0") .intConf @@ -1087,8 +1087,8 @@ object SQLConf { val FILE_COMPRESSION_FACTOR = buildConf("spark.sql.sources.fileCompressionFactor") .internal() .doc("When estimating the output data size of a table scan, multiply the file size with this " + - "factor as the estimated data size, in case the data is compressed in the file and lead to" + - " a heavily underestimated result.") + "factor as the estimated data size, in case the data is compressed in the file and lead to " + + "a heavily underestimated result.") .version("2.3.1") .doubleConf .checkValue(_ > 0, "the value of fileCompressionFactor must be greater than 0") @@ -1340,7 +1340,7 @@ object SQLConf { val ORC_COMPRESSION = buildConf("spark.sql.orc.compression.codec") .doc("Sets the compression codec used when writing ORC files. If either `compression` or " + "`orc.compress` is specified in the table-specific options/properties, the precedence " + - "would be `compression`, `orc.compress`, `spark.sql.orc.compression.codec`." + + "would be `compression`, `orc.compress`, `spark.sql.orc.compression.codec`. " + "Acceptable values include: none, uncompressed, snappy, zlib, lzo, zstd, lz4, brotli.") .version("2.3.0") .stringConf @@ -1511,7 +1511,7 @@ object SQLConf { "to produce the partition columns instead of table scans. It applies when all the columns " + "scanned are partition columns and the query has an aggregate operator that satisfies " + "distinct semantics. By default the optimization is disabled, and deprecated as of Spark " + - "3.0 since it may return incorrect results when the files are empty, see also SPARK-26709." + + "3.0 since it may return incorrect results when the files are empty, see also SPARK-26709. " + "It will be removed in the future releases. If you must use, use 'SparkSessionExtensions' " + "instead to inject it as a custom rule.") .version("2.1.1") @@ -1708,7 +1708,7 @@ object SQLConf { val V2_BUCKETING_SHUFFLE_ENABLED = buildConf("spark.sql.sources.v2.bucketing.shuffle.enabled") - .doc("During a storage-partitioned join, whether to allow to shuffle only one side." + + .doc("During a storage-partitioned join, whether to allow to shuffle only one side. " + "When only one side is KeyGroupedPartitioning, if the conditions are met, spark will " + "only shuffle the other side. This optimization will reduce the amount of data that " + s"needs to be shuffle. This config requires ${V2_BUCKETING_ENABLED.key} to be enabled") @@ -1718,9 +1718,9 @@ object SQLConf { val V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS = buildConf("spark.sql.sources.v2.bucketing.allowJoinKeysSubsetOfPartitionKeys.enabled") - .doc("Whether to allow storage-partition join in the case where join keys are" + + .doc("Whether to allow storage-partition join in the case where join keys are " + "a subset of the partition keys of the source tables. At planning time, " + - "Spark will group the partitions by only those keys that are in the join keys." + + "Spark will group the partitions by only those keys that are in the join keys. " + s"This is currently enabled only if ${REQUIRE_ALL_CLUSTER_KEYS_FOR_DISTRIBUTION.key} " + "is false." ) @@ -2058,7 +2058,7 @@ object SQLConf { val WHOLESTAGE_BROADCAST_CLEANED_SOURCE_THRESHOLD = buildConf("spark.sql.codegen.broadcastCleanedSourceThreshold") .internal() - .doc("A threshold (in string length) to determine if we should make the generated code a" + + .doc("A threshold (in string length) to determine if we should make the generated code a " + "broadcast variable in whole stage codegen. To disable this, set the threshold to < 0; " + "otherwise if the size is above the threshold, it'll use broadcast variable. Note that " + "maximum string length allowed in Java is Integer.MAX_VALUE, so anything above it would " + @@ -3378,7 +3378,7 @@ object SQLConf { buildConf("spark.sql.execution.pandas.structHandlingMode") .doc( "The conversion mode of struct type when creating pandas DataFrame. " + - "When \"legacy\"," + + "When \"legacy\", " + "1. when Arrow optimization is disabled, convert to Row object, " + "2. when Arrow optimization is enabled, convert to dict or raise an Exception " + "if there are duplicated nested field names. " + @@ -3466,7 +3466,7 @@ object SQLConf { buildConf("spark.sql.execution.pyspark.python") .internal() .doc("Python binary executable to use for PySpark in executors when running Python " + - "UDF, pandas UDF and pandas function APIs." + + "UDF, pandas UDF and pandas function APIs. " + "If not set, it falls back to 'spark.pyspark.python' by default.") .version("3.5.0") .stringConf @@ -3695,7 +3695,7 @@ object SQLConf { val ANSI_ENABLED = buildConf(SqlApiConfHelper.ANSI_ENABLED_KEY) .doc("When true, Spark SQL uses an ANSI compliant dialect instead of being Hive compliant. " + "For example, Spark will throw an exception at runtime instead of returning null results " + - "when the inputs to a SQL operator/function are invalid." + + "when the inputs to a SQL operator/function are invalid. " + "For full details of this dialect, you can find them in the section \"ANSI Compliance\" of " + "Spark's documentation. Some ANSI dialect features may be not from the ANSI SQL " + "standard directly, but their behaviors align with ANSI SQL's style") @@ -3786,7 +3786,7 @@ object SQLConf { .internal() .doc("When true, use the common expression ID for the alias when rewriting With " + "expressions. Otherwise, use the index of the common expression definition. When true " + - "this avoids duplicate alias names, but is helpful to set to false for testing to ensure" + + "this avoids duplicate alias names, but is helpful to set to false for testing to ensure " + "that alias names are consistent.") .version("4.0.0") .booleanConf @@ -4248,7 +4248,7 @@ object SQLConf { val LEGACY_ALLOW_UNTYPED_SCALA_UDF = buildConf("spark.sql.legacy.allowUntypedScalaUDF") .internal() - .doc("When set to true, user is allowed to use org.apache.spark.sql.functions." + + .doc("When set to true, user is allowed to use org.apache.spark.sql.functions. " + "udf(f: AnyRef, dataType: DataType). Otherwise, an exception will be thrown at runtime.") .version("3.0.0") .booleanConf @@ -4285,7 +4285,7 @@ object SQLConf { val MAX_TO_STRING_FIELDS = buildConf("spark.sql.debug.maxToStringFields") .doc("Maximum number of fields of sequence-like entries can be converted to strings " + - "in debug output. Any elements beyond the limit will be dropped and replaced by a" + + "in debug output. Any elements beyond the limit will be dropped and replaced by a " + """ "... N more fields" placeholder.""") .version("3.0.0") .intConf @@ -4421,7 +4421,7 @@ object SQLConf { val LEGACY_CTE_PRECEDENCE_POLICY = buildConf("spark.sql.legacy.ctePrecedencePolicy") .internal() .doc("When LEGACY, outer CTE definitions takes precedence over inner definitions. If set to " + - "EXCEPTION, AnalysisException is thrown while name conflict is detected in nested CTE." + + "EXCEPTION, AnalysisException is thrown while name conflict is detected in nested CTE. " + "The default is CORRECTED, inner CTE definitions take precedence. This config " + "will be removed in future versions and CORRECTED will be the only behavior.") .version("3.0.0") @@ -4849,7 +4849,7 @@ object SQLConf { .doc("When true, NULL-aware anti join execution will be planed into " + "BroadcastHashJoinExec with flag isNullAwareAntiJoin enabled, " + "optimized from O(M*N) calculation into O(M) calculation " + - "using Hash lookup instead of Looping lookup." + + "using Hash lookup instead of Looping lookup. " + "Only support for singleColumn NAAJ for now.") .version("3.1.0") .booleanConf @@ -5241,7 +5241,7 @@ object SQLConf { buildConf("spark.sql.legacy.raiseErrorWithoutErrorClass") .internal() .doc("When set to true, restores the legacy behavior of `raise_error` and `assert_true` to " + - "not return the `[USER_RAISED_EXCEPTION]` prefix." + + "not return the `[USER_RAISED_EXCEPTION]` prefix. " + "For example, `raise_error('error!')` returns `error!` instead of " + "`[USER_RAISED_EXCEPTION] Error!`.") .version("4.0.0") @@ -5299,7 +5299,7 @@ object SQLConf { .internal() .doc("When set to true, datetime formatter used for csv, json and xml " + "will support zone offsets that have seconds in it. e.g. LA timezone offset prior to 1883" + - "was -07:52:58. When this flag is not set we lose seconds information." ) + " was -07:52:58. When this flag is not set we lose seconds information." ) .version("4.0.0") .booleanConf .createWithDefault(true) @@ -5380,7 +5380,7 @@ object SQLConf { val LEGACY_BANG_EQUALS_NOT = buildConf("spark.sql.legacy.bangEqualsNot") .internal() .doc("When set to true, '!' is a lexical equivalent for 'NOT'. That is '!' can be used " + - "outside of the documented prefix usage in a logical expression." + + "outside of the documented prefix usage in a logical expression. " + "Examples are: `expr ! IN (1, 2)` and `expr ! BETWEEN 1 AND 2`, but also `IF ! EXISTS`." ) .version("4.0.0") @@ -5502,7 +5502,7 @@ object SQLConf { RemovedConfig("spark.sql.legacy.compareDateTimestampInTimestamp", "3.0.0", "true", "It was removed to prevent errors like SPARK-23549 for non-default value."), RemovedConfig("spark.sql.parquet.int64AsTimestampMillis", "3.0.0", "false", - "The config was deprecated since Spark 2.3." + + "The config was deprecated since Spark 2.3. " + s"Use '${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}' instead of it."), RemovedConfig("spark.sql.execution.pandas.respectSessionTimeZone", "3.0.0", "true", "The non-default behavior is considered as a bug, see SPARK-22395. " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 407baba8280c0..a14c584fdc6a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -280,7 +280,7 @@ object StaticSQLConf { buildStaticConf("spark.sql.streaming.ui.enabledCustomMetricList") .internal() .doc("Configures a list of custom metrics on Structured Streaming UI, which are enabled. " + - "The list contains the name of the custom metrics separated by comma. In aggregation" + + "The list contains the name of the custom metrics separated by comma. In aggregation " + "only sum used. The list of supported custom metrics is state store provider specific " + "and it can be found out for example from query progress log entry.") .version("3.1.0") From 062bc4c258442d3752f6209a300c8a986f4e117f Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Wed, 25 Dec 2024 14:37:03 +0800 Subject: [PATCH 243/438] [SPARK-50659][SQL] Refactor Union output computation out to reuse it in the single-pass Analyzer ### What changes were proposed in this pull request? Refactor `computeOutput` out to the `Union` companion object. ### Why are the changes needed? To reuse this piece of code in the single-pass Analyzer. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49283 from vladimirg-db/vladimirg-db/refactor-out-union-compute-output-to-use-in-single-pass-analyzer. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../plans/logical/basicLogicalOperators.scala | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 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 a6405f18ad444..b897026a0df46 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 @@ -445,6 +445,21 @@ object Union { def apply(left: LogicalPlan, right: LogicalPlan): Union = { Union (left :: right :: Nil) } + + // updating nullability to make all the children consistent + def mergeChildOutputs(childOutputs: Seq[Seq[Attribute]]): Seq[Attribute] = { + childOutputs.transpose.map { attrs => + val firstAttr = attrs.head + val nullable = attrs.exists(_.nullable) + val newDt = attrs.map(_.dataType).reduce(StructType.unionLikeMerge) + if (firstAttr.dataType == newDt) { + firstAttr.withNullability(nullable) + } else { + AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( + firstAttr.exprId, firstAttr.qualifier) + } + } + } } /** @@ -526,20 +541,7 @@ case class Union( private lazy val lazyOutput: Seq[Attribute] = computeOutput() - // updating nullability to make all the children consistent - private def computeOutput(): Seq[Attribute] = { - children.map(_.output).transpose.map { attrs => - val firstAttr = attrs.head - val nullable = attrs.exists(_.nullable) - val newDt = attrs.map(_.dataType).reduce(StructType.unionLikeMerge) - if (firstAttr.dataType == newDt) { - firstAttr.withNullability(nullable) - } else { - AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( - firstAttr.exprId, firstAttr.qualifier) - } - } - } + private def computeOutput(): Seq[Attribute] = Union.mergeChildOutputs(children.map(_.output)) /** * Maps the constraints containing a given (original) sequence of attributes to those with a From 241454253a73b7dfe641d85784d8855504ca59b7 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 25 Dec 2024 15:16:27 +0800 Subject: [PATCH 244/438] [SPARK-50590][INFRA][FOLLOW-UP] Further skip unnecessary image build by the job modules ### What changes were proposed in this pull request? Further skip unnecessary image build by the job modules ### Why are the changes needed? e.g. in a case where `sparkr` is not enabled, do not need to build the corresponding image ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? pr builder ### Was this patch authored or co-authored using generative AI tooling? no Closes #49281 from zhengruifeng/infra_further_skip. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 9900fbf6d82c4..13f4e66842ef1 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -435,7 +435,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-cache:${{ inputs.branch }} - name: Build and push (Documentation) - if: ${{ inputs.branch != 'branch-3.5' && hashFiles('dev/spark-test-image/docs/Dockerfile') != '' }} + if: ${{ inputs.branch != 'branch-3.5' && fromJson(needs.precondition.outputs.required).docs == 'true' && hashFiles('dev/spark-test-image/docs/Dockerfile') != '' }} id: docker_build_docs uses: docker/build-push-action@v6 with: @@ -446,7 +446,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-docs-cache:${{ inputs.branch }} - name: Build and push (Linter) - if: ${{ inputs.branch != 'branch-3.5' && hashFiles('dev/spark-test-image/lint/Dockerfile') != '' }} + if: ${{ inputs.branch != 'branch-3.5' && fromJson(needs.precondition.outputs.required).lint == 'true' && hashFiles('dev/spark-test-image/lint/Dockerfile') != '' }} id: docker_build_lint uses: docker/build-push-action@v6 with: @@ -457,7 +457,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-lint-cache:${{ inputs.branch }} - name: Build and push (SparkR) - if: ${{ inputs.branch != 'branch-3.5' && hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' }} + if: ${{ inputs.branch != 'branch-3.5' && fromJson(needs.precondition.outputs.required).sparkr == 'true' && hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' }} id: docker_build_sparkr uses: docker/build-push-action@v6 with: @@ -468,7 +468,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ inputs.branch }} - name: Build and push (PySpark with ${{ env.PYSPARK_IMAGE_TO_TEST }}) - if: ${{ inputs.branch != 'branch-3.5' && env.PYSPARK_IMAGE_TO_TEST != '' }} + if: ${{ inputs.branch != 'branch-3.5' && fromJson(needs.precondition.outputs.required).pyspark == 'true' && env.PYSPARK_IMAGE_TO_TEST != '' }} id: docker_build_pyspark env: ${{ fromJSON(inputs.envs) }} uses: docker/build-push-action@v6 From 6eb556fa99216fa2ed7a9dcd7d711e097abd4de7 Mon Sep 17 00:00:00 2001 From: Mihailo Milosevic Date: Wed, 25 Dec 2024 15:52:46 +0800 Subject: [PATCH 245/438] [SPARK-50409][SQL] Fix set statement to ignore `;` at the end of `SET;`, `SET -v;` and `SET key;` ### What changes were proposed in this pull request? This PR aims to fix the problem of trailing `;` in set statement. ### Why are the changes needed? We missed to add `;*` in the regex rules in SparkSqlParser.scala. Several queries were failing: ``` --------------------------------------- set; --------------------------------------- [INVALID_SET_SYNTAX] Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use backquotes, e.g., SET `key`=`value`. SQLSTATE: 42000 == SQL (line 1, position 1) == set; ^^^ --------------------------------------- set -v; --------------------------------------- [INVALID_SET_SYNTAX] Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use backquotes, e.g., SET `key`=`value`. SQLSTATE: 42000 == SQL (line 1, position 1) == set -v; ^^^^^^^ --------------------------------------- set spark.sql.ansi.enabled; --------------------------------------- [INVALID_SET_SYNTAX] Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use backquotes, e.g., SET `key`=`value`. SQLSTATE: 42000 == SQL (line 1, position 1) == set spark.sql.ansi.enabled; ^^^^^^^^^^^^^^^^^^^^^^^^^^ ``` All of these would get the `;` at the end and fail due to impossible match. ### Does this PR introduce _any_ user-facing change? Yes. This is a bug that we had for a long time. ### How was this patch tested? Added test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48951 from mihailom-db/fixreminder. Authored-by: Mihailo Milosevic Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/SparkSqlParser.scala | 10 +++++----- .../spark/sql/execution/SparkSqlParserSuite.scala | 9 +++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 9fbe400a555fc..e15250eb46b5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -63,7 +63,7 @@ class SparkSqlAstBuilder extends AstBuilder { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ private val configKeyValueDef = """([a-zA-Z_\d\\.:]+)\s*=([^;]*);*""".r - private val configKeyDef = """([a-zA-Z_\d\\.:]+)$""".r + private val configKeyDef = """([a-zA-Z_\d\\.:]+)\s*$""".r private val configValueDef = """([^;]*);*""".r private val strLiteralDef = """(".*?[^\\]"|'.*?[^\\]'|[^ \n\r\t"']+)""".r @@ -106,14 +106,14 @@ class SparkSqlAstBuilder extends AstBuilder { SetCommand(Some(keyStr -> None)) } } else { - remainder(ctx.SET.getSymbol).trim match { + remainder(ctx.SET.getSymbol).trim.replaceAll(";+$", "") match { case configKeyValueDef(key, value) => SetCommand(Some(key -> Option(value.trim))) case configKeyDef(key) => SetCommand(Some(key -> None)) - case s if s == "-v" => + case s if s.trim == "-v" => SetCommand(Some("-v" -> None)) - case s if s.isEmpty => + case s if s.trim.isEmpty => SetCommand(None) case _ => throw QueryParsingErrors.unexpectedFormatForSetConfigurationError(ctx) } @@ -146,7 +146,7 @@ class SparkSqlAstBuilder extends AstBuilder { */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { - remainder(ctx.RESET.getSymbol).trim match { + remainder(ctx.RESET.getSymbol).trim.replaceAll(";+$", "") match { case configKeyDef(key) => ResetCommand(Some(key)) case s if s.trim.isEmpty => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index d4a0f987d4993..acc3cdb01bf3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -93,6 +93,15 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { parameters = Map.empty) } + test("SET with semi-colons") { + assertEqual(s"SET;", SetCommand(None)) + assertEqual(s"SET ;", SetCommand(None)) + assertEqual(s"SET -v;", SetCommand(Some("-v" -> None))) + assertEqual(s"SET -v ;", SetCommand(Some("-v" -> None))) + assertEqual(s"SET spark.sql.ansi.enabled;", SetCommand(Some("spark.sql.ansi.enabled" -> None))) + assertEqual(s"SET spark.sql.ansi.enabled ;", SetCommand(Some("spark.sql.ansi.enabled" -> None))) + } + test("Report Error for invalid usage of SET command") { assertEqual("SET", SetCommand(None)) assertEqual("SET -v", SetCommand(Some("-v", None))) From 112276d93c0e7db9fdc894ebe6b3804aef045147 Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Wed, 25 Dec 2024 17:59:33 +0900 Subject: [PATCH 246/438] [SPARK-50360][SS][FOLLOWUP][TESTS] Changelog reader test improvements ### What changes were proposed in this pull request? In some test cases added by https://github.com/apache/spark/pull/48880, `changelogReader.closeIfNeeded()` is called before `assert(entries.size == expectedEntries.size)`, while `entries` are defined as `entries = changelogReader.toSeq`, which triggers the read. So the logic is ``` entries = changelogReader.toSeq changelogReader.closeIfNeeded() assert(entries.size == expectedEntries.size) ``` This is fine in most cases, but we find out that in certain environments, the `toSeq` is lazily evaluated; therefore, not until the `assert`, the actual read won't be triggered. To be extra safe, change the order of the above logic to be the following: ``` entries = changelogReader.toSeq assert(entries.size == expectedEntries.size) changelogReader.closeIfNeeded() ``` ### Why are the changes needed? Test case improvement ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Test only change ### Was this patch authored or co-authored using generative AI tooling? No Closes #49272 from WweiL/rocksdb-closeifneeded. Authored-by: Wei Liu Signed-off-by: Jungtaek Lim --- .../execution/streaming/state/RocksDBSuite.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 3ec81a7ef13e4..c26bbe78a2db9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -979,12 +979,12 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession null, StateStore.DEFAULT_COL_FAMILY_NAME) } - changelogReader.closeIfNeeded() - assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) } + + changelogReader.closeIfNeeded() } testWithChangelogCheckpointingEnabled("RocksDBFileManager: StateStoreChangelogReaderFactory " + @@ -1124,12 +1124,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession (RecordType.DELETE_RECORD, j.toString.getBytes, null, StateStore.DEFAULT_COL_FAMILY_NAME) } - changelogReader.closeIfNeeded() assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) } + + changelogReader.closeIfNeeded() } testWithChangelogCheckpointingEnabled( @@ -1161,12 +1162,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } ++ (2 to 4).map { j => (RecordType.DELETE_RECORD, j.toString.getBytes, null) } - changelogReader.closeIfNeeded() assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) } + + changelogReader.closeIfNeeded() } testWithChangelogCheckpointingEnabled("RocksDBFileManager: read and write v2 changelog with " + @@ -1206,12 +1208,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } ++ (2 to 4).map { j => (RecordType.DELETE_RECORD, j.toString.getBytes, null) } - changelogReader.closeIfNeeded() assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) } + + changelogReader.closeIfNeeded() } testWithColumnFamilies("RocksDBFileManager: create init dfs directory with " + From ef4be07fdad9c8078e22d4f3f068fee1b81cf967 Mon Sep 17 00:00:00 2001 From: Mikhail Nikoliukin Date: Wed, 25 Dec 2024 17:13:59 +0800 Subject: [PATCH 247/438] [SPARK-50220][PYTHON] Support listagg in PySpark ### What changes were proposed in this pull request? Added new function `listagg` to pyspark. Follow-up of https://github.com/apache/spark/pull/48748. ### Why are the changes needed? Allows to use native Python functions to write queries with `listagg`. E.g., `df.select(F.listagg(df.value, ",").alias("r"))`. ### Does this PR introduce _any_ user-facing change? Yes, new functions `listagg` and `listagg_distinct` (with aliases `string_agg` and `string_agg_distinct`) in pyspark. ### How was this patch tested? Unit tests ### Was this patch authored or co-authored using generative AI tooling? Generated-by: GitHub Copilot Closes #49231 from mikhailnik-db/SPARK-50220-listagg-for-pyspark. Authored-by: Mikhail Nikoliukin Signed-off-by: Ruifeng Zheng --- .../reference/pyspark.sql/functions.rst | 4 + .../pyspark/sql/connect/functions/builtin.py | 58 ++++ python/pyspark/sql/functions/__init__.py | 4 + python/pyspark/sql/functions/builtin.py | 308 ++++++++++++++++++ .../tests/connect/test_connect_function.py | 4 + python/pyspark/sql/tests/test_functions.py | 73 ++++- 6 files changed, 444 insertions(+), 7 deletions(-) diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst index 430e353dd701c..a1ba153110f10 100644 --- a/python/docs/source/reference/pyspark.sql/functions.rst +++ b/python/docs/source/reference/pyspark.sql/functions.rst @@ -451,6 +451,8 @@ Aggregate Functions kurtosis last last_value + listagg + listagg_distinct max max_by mean @@ -476,6 +478,8 @@ Aggregate Functions stddev stddev_pop stddev_samp + string_agg + string_agg_distinct sum sum_distinct try_avg diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index f52cdffb84b7c..f13eeab12dd35 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -1064,6 +1064,64 @@ def collect_set(col: "ColumnOrName") -> Column: collect_set.__doc__ = pysparkfuncs.collect_set.__doc__ +def listagg(col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None) -> Column: + if delimiter is None: + return _invoke_function_over_columns("listagg", col) + else: + return _invoke_function_over_columns("listagg", col, lit(delimiter)) + + +listagg.__doc__ = pysparkfuncs.listagg.__doc__ + + +def listagg_distinct( + col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None +) -> Column: + from pyspark.sql.connect.column import Column as ConnectColumn + + args = [col] + if delimiter is not None: + args += [lit(delimiter)] + + _exprs = [_to_col(c)._expr for c in args] + return ConnectColumn( + UnresolvedFunction("listagg", _exprs, is_distinct=True) # type: ignore[arg-type] + ) + + +listagg_distinct.__doc__ = pysparkfuncs.listagg_distinct.__doc__ + + +def string_agg( + col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None +) -> Column: + if delimiter is None: + return _invoke_function_over_columns("string_agg", col) + else: + return _invoke_function_over_columns("string_agg", col, lit(delimiter)) + + +string_agg.__doc__ = pysparkfuncs.string_agg.__doc__ + + +def string_agg_distinct( + col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None +) -> Column: + from pyspark.sql.connect.column import Column as ConnectColumn + + args = [col] + if delimiter is not None: + args += [lit(delimiter)] + + _exprs = [_to_col(c)._expr for c in args] + return ConnectColumn( + UnresolvedFunction("string_agg", _exprs, is_distinct=True) # type: ignore[arg-type] + ) + + +string_agg_distinct.__doc__ = pysparkfuncs.string_agg_distinct.__doc__ + + def corr(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: return _invoke_function_over_columns("corr", col1, col2) diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py index 98db2a7b091dd..fc0120bc681d8 100644 --- a/python/pyspark/sql/functions/__init__.py +++ b/python/pyspark/sql/functions/__init__.py @@ -364,6 +364,8 @@ "kurtosis", "last", "last_value", + "listagg", + "listagg_distinct", "max", "max_by", "mean", @@ -389,6 +391,8 @@ "stddev", "stddev_pop", "stddev_samp", + "string_agg", + "string_agg_distinct", "sum", "sum_distinct", "try_avg", diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 4b4c164055eaf..7b14598a0ef4e 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -1851,6 +1851,314 @@ def sum_distinct(col: "ColumnOrName") -> Column: return _invoke_function_over_columns("sum_distinct", col) +@_try_remote_functions +def listagg(col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None) -> Column: + """ + Aggregate function: returns the concatenation of non-null input values, + separated by the delimiter. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + target column to compute on. + delimiter : :class:`~pyspark.sql.Column`, literal string or bytes, optional + the delimiter to separate the values. The default value is None. + + Returns + ------- + :class:`~pyspark.sql.Column` + the column for computed results. + + Examples + -------- + Example 1: Using listagg function + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',)], ['strings']) + >>> df.select(sf.listagg('strings')).show() + +----------------------+ + |listagg(strings, NULL)| + +----------------------+ + | abc| + +----------------------+ + + Example 2: Using listagg function with a delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',)], ['strings']) + >>> df.select(sf.listagg('strings', ', ')).show() + +--------------------+ + |listagg(strings, , )| + +--------------------+ + | a, b, c| + +--------------------+ + + Example 3: Using listagg function with a binary column and delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(b'\x01',), (b'\x02',), (None,), (b'\x03',)], ['bytes']) + >>> df.select(sf.listagg('bytes', b'\x42')).show() + +---------------------+ + |listagg(bytes, X'42')| + +---------------------+ + | [01 42 02 42 03]| + +---------------------+ + + Example 4: Using listagg function on a column with all None values + + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql.types import StructType, StructField, StringType + >>> schema = StructType([StructField("strings", StringType(), True)]) + >>> df = spark.createDataFrame([(None,), (None,), (None,), (None,)], schema=schema) + >>> df.select(sf.listagg('strings')).show() + +----------------------+ + |listagg(strings, NULL)| + +----------------------+ + | NULL| + +----------------------+ + """ + if delimiter is None: + return _invoke_function_over_columns("listagg", col) + else: + return _invoke_function_over_columns("listagg", col, lit(delimiter)) + + +@_try_remote_functions +def listagg_distinct( + col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None +) -> Column: + """ + Aggregate function: returns the concatenation of distinct non-null input values, + separated by the delimiter. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + target column to compute on. + delimiter : :class:`~pyspark.sql.Column`, literal string or bytes, optional + the delimiter to separate the values. The default value is None. + + Returns + ------- + :class:`~pyspark.sql.Column` + the column for computed results. + + Examples + -------- + Example 1: Using listagg_distinct function + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',), ('b',)], ['strings']) + >>> df.select(sf.listagg_distinct('strings')).show() + +-------------------------------+ + |listagg(DISTINCT strings, NULL)| + +-------------------------------+ + | abc| + +-------------------------------+ + + Example 2: Using listagg_distinct function with a delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',), ('b',)], ['strings']) + >>> df.select(sf.listagg_distinct('strings', ', ')).show() + +-----------------------------+ + |listagg(DISTINCT strings, , )| + +-----------------------------+ + | a, b, c| + +-----------------------------+ + + Example 3: Using listagg_distinct function with a binary column and delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(b'\x01',), (b'\x02',), (None,), (b'\x03',), (b'\x02',)], + ... ['bytes']) + >>> df.select(sf.listagg_distinct('bytes', b'\x42')).show() + +------------------------------+ + |listagg(DISTINCT bytes, X'42')| + +------------------------------+ + | [01 42 02 42 03]| + +------------------------------+ + + Example 4: Using listagg_distinct function on a column with all None values + + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql.types import StructType, StructField, StringType + >>> schema = StructType([StructField("strings", StringType(), True)]) + >>> df = spark.createDataFrame([(None,), (None,), (None,), (None,)], schema=schema) + >>> df.select(sf.listagg_distinct('strings')).show() + +-------------------------------+ + |listagg(DISTINCT strings, NULL)| + +-------------------------------+ + | NULL| + +-------------------------------+ + """ + if delimiter is None: + return _invoke_function_over_columns("listagg_distinct", col) + else: + return _invoke_function_over_columns("listagg_distinct", col, lit(delimiter)) + + +@_try_remote_functions +def string_agg( + col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None +) -> Column: + """ + Aggregate function: returns the concatenation of non-null input values, + separated by the delimiter. + + An alias of :func:`listagg`. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + target column to compute on. + delimiter : :class:`~pyspark.sql.Column`, literal string or bytes, optional + the delimiter to separate the values. The default value is None. + + Returns + ------- + :class:`~pyspark.sql.Column` + the column for computed results. + + Examples + -------- + Example 1: Using string_agg function + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',)], ['strings']) + >>> df.select(sf.string_agg('strings')).show() + +-------------------------+ + |string_agg(strings, NULL)| + +-------------------------+ + | abc| + +-------------------------+ + + Example 2: Using string_agg function with a delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',)], ['strings']) + >>> df.select(sf.string_agg('strings', ', ')).show() + +-----------------------+ + |string_agg(strings, , )| + +-----------------------+ + | a, b, c| + +-----------------------+ + + Example 3: Using string_agg function with a binary column and delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(b'\x01',), (b'\x02',), (None,), (b'\x03',)], ['bytes']) + >>> df.select(sf.string_agg('bytes', b'\x42')).show() + +------------------------+ + |string_agg(bytes, X'42')| + +------------------------+ + | [01 42 02 42 03]| + +------------------------+ + + Example 4: Using string_agg function on a column with all None values + + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql.types import StructType, StructField, StringType + >>> schema = StructType([StructField("strings", StringType(), True)]) + >>> df = spark.createDataFrame([(None,), (None,), (None,), (None,)], schema=schema) + >>> df.select(sf.string_agg('strings')).show() + +-------------------------+ + |string_agg(strings, NULL)| + +-------------------------+ + | NULL| + +-------------------------+ + """ + if delimiter is None: + return _invoke_function_over_columns("string_agg", col) + else: + return _invoke_function_over_columns("string_agg", col, lit(delimiter)) + + +@_try_remote_functions +def string_agg_distinct( + col: "ColumnOrName", delimiter: Optional[Union[Column, str, bytes]] = None +) -> Column: + """ + Aggregate function: returns the concatenation of distinct non-null input values, + separated by the delimiter. + + An alias of :func:`listagg_distinct`. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + target column to compute on. + delimiter : :class:`~pyspark.sql.Column`, literal string or bytes, optional + the delimiter to separate the values. The default value is None. + + Returns + ------- + :class:`~pyspark.sql.Column` + the column for computed results. + + Examples + -------- + Example 1: Using string_agg_distinct function + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',), ('b',)], ['strings']) + >>> df.select(sf.string_agg_distinct('strings')).show() + +----------------------------------+ + |string_agg(DISTINCT strings, NULL)| + +----------------------------------+ + | abc| + +----------------------------------+ + + Example 2: Using string_agg_distinct function with a delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('a',), ('b',), (None,), ('c',), ('b',)], ['strings']) + >>> df.select(sf.string_agg_distinct('strings', ', ')).show() + +--------------------------------+ + |string_agg(DISTINCT strings, , )| + +--------------------------------+ + | a, b, c| + +--------------------------------+ + + Example 3: Using string_agg_distinct function with a binary column and delimiter + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(b'\x01',), (b'\x02',), (None,), (b'\x03',), (b'\x02',)], + ... ['bytes']) + >>> df.select(sf.string_agg_distinct('bytes', b'\x42')).show() + +---------------------------------+ + |string_agg(DISTINCT bytes, X'42')| + +---------------------------------+ + | [01 42 02 42 03]| + +---------------------------------+ + + Example 4: Using string_agg_distinct function on a column with all None values + + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql.types import StructType, StructField, StringType + >>> schema = StructType([StructField("strings", StringType(), True)]) + >>> df = spark.createDataFrame([(None,), (None,), (None,), (None,)], schema=schema) + >>> df.select(sf.string_agg_distinct('strings')).show() + +----------------------------------+ + |string_agg(DISTINCT strings, NULL)| + +----------------------------------+ + | NULL| + +----------------------------------+ + """ + if delimiter is None: + return _invoke_function_over_columns("string_agg_distinct", col) + else: + return _invoke_function_over_columns("string_agg_distinct", col, lit(delimiter)) + + @_try_remote_functions def product(col: "ColumnOrName") -> Column: """ diff --git a/python/pyspark/sql/tests/connect/test_connect_function.py b/python/pyspark/sql/tests/connect/test_connect_function.py index b7a02efcd5e2b..d1e2558305291 100644 --- a/python/pyspark/sql/tests/connect/test_connect_function.py +++ b/python/pyspark/sql/tests/connect/test_connect_function.py @@ -590,6 +590,10 @@ def test_aggregation_functions(self): (CF.avg, SF.avg), (CF.collect_list, SF.collect_list), (CF.collect_set, SF.collect_set), + (CF.listagg, SF.listagg), + (CF.listagg_distinct, SF.listagg_distinct), + (CF.string_agg, SF.string_agg), + (CF.string_agg_distinct, SF.string_agg_distinct), (CF.count, SF.count), (CF.first, SF.first), (CF.kurtosis, SF.kurtosis), diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 4607d5d3411fe..39db72b235bf9 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -30,6 +30,7 @@ from pyspark.sql.avro.functions import from_avro, to_avro from pyspark.sql.column import Column from pyspark.sql.functions.builtin import nullifzero, randstr, uniform, zeroifnull +from pyspark.sql.types import StructType, StructField, StringType from pyspark.testing.sqlutils import ReusedSQLTestCase, SQLTestUtils from pyspark.testing.utils import have_numpy, assertDataFrameEqual @@ -83,13 +84,7 @@ def test_function_parity(self): missing_in_py = jvm_fn_set.difference(py_fn_set) # Functions that we expect to be missing in python until they are added to pyspark - expected_missing_in_py = { - # TODO(SPARK-50220): listagg functions will soon be added and removed from this list - "listagg_distinct", - "listagg", - "string_agg", - "string_agg_distinct", - } + expected_missing_in_py = set() self.assertEqual( expected_missing_in_py, missing_in_py, "Missing functions in pyspark not as expected" @@ -1145,6 +1140,70 @@ def test_collect_functions(self): ["1", "2", "2", "2"], ) + def test_listagg_functions(self): + df = self.spark.createDataFrame( + [(1, "1"), (2, "2"), (None, None), (1, "2")], ["key", "value"] + ) + df_with_bytes = self.spark.createDataFrame( + [(b"\x01",), (b"\x02",), (None,), (b"\x03",), (b"\x02",)], ["bytes"] + ) + df_with_nulls = self.spark.createDataFrame( + [(None,), (None,), (None,), (None,), (None,)], + StructType([StructField("nulls", StringType(), True)]), + ) + # listagg and string_agg are aliases + for listagg_ref in [F.listagg, F.string_agg]: + self.assertEqual(df.select(listagg_ref(df.key).alias("r")).collect()[0].r, "121") + self.assertEqual(df.select(listagg_ref(df.value).alias("r")).collect()[0].r, "122") + self.assertEqual( + df.select(listagg_ref(df.value, ",").alias("r")).collect()[0].r, "1,2,2" + ) + self.assertEqual( + df_with_bytes.select(listagg_ref(df_with_bytes.bytes, b"\x42").alias("r")) + .collect()[0] + .r, + b"\x01\x42\x02\x42\x03\x42\x02", + ) + self.assertEqual( + df_with_nulls.select(listagg_ref(df_with_nulls.nulls).alias("r")).collect()[0].r, + None, + ) + + def test_listagg_distinct_functions(self): + df = self.spark.createDataFrame( + [(1, "1"), (2, "2"), (None, None), (1, "2")], ["key", "value"] + ) + df_with_bytes = self.spark.createDataFrame( + [(b"\x01",), (b"\x02",), (None,), (b"\x03",), (b"\x02",)], ["bytes"] + ) + df_with_nulls = self.spark.createDataFrame( + [(None,), (None,), (None,), (None,), (None,)], + StructType([StructField("nulls", StringType(), True)]), + ) + # listagg_distinct and string_agg_distinct are aliases + for listagg_distinct_ref in [F.listagg_distinct, F.string_agg_distinct]: + self.assertEqual( + df.select(listagg_distinct_ref(df.key).alias("r")).collect()[0].r, "12" + ) + self.assertEqual( + df.select(listagg_distinct_ref(df.value).alias("r")).collect()[0].r, "12" + ) + self.assertEqual( + df.select(listagg_distinct_ref(df.value, ",").alias("r")).collect()[0].r, "1,2" + ) + self.assertEqual( + df_with_bytes.select(listagg_distinct_ref(df_with_bytes.bytes, b"\x42").alias("r")) + .collect()[0] + .r, + b"\x01\x42\x02\x42\x03", + ) + self.assertEqual( + df_with_nulls.select(listagg_distinct_ref(df_with_nulls.nulls).alias("r")) + .collect()[0] + .r, + None, + ) + def test_datetime_functions(self): df = self.spark.range(1).selectExpr("'2017-01-22' as dateCol") parse_result = df.select(F.to_date(F.col("dateCol"))).first() From 9c9bdab64a4ca96fb648b19608d1b4def61c90ab Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 25 Dec 2024 12:11:45 -0800 Subject: [PATCH 248/438] [SPARK-50657][PYTHON] Upgrade the minimum version of `pyarrow` to 11.0.0 ### What changes were proposed in this pull request? Upgrade the minimum version of `pyarrow` to 11.0.0 ### Why are the changes needed? according to my test in https://github.com/apache/spark/pull/49267, pyspark with `pyarrow=10.0.0` has already been broken - pyspark-sql failed - pyspark-connect failed - pyspark-pandas failed see https://github.com/zhengruifeng/spark/actions/runs/12464102622/job/34787749014 ### Does this PR introduce _any_ user-facing change? doc changes ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #49282 from zhengruifeng/mini_arrow_11. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- dev/requirements.txt | 2 +- python/docs/source/getting_started/install.rst | 6 +++--- python/docs/source/migration_guide/pyspark_upgrade.rst | 2 +- python/packaging/classic/setup.py | 2 +- python/packaging/connect/setup.py | 2 +- python/pyspark/sql/pandas/utils.py | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/requirements.txt b/dev/requirements.txt index 04cab4cbfcc3c..33300cc28d3ca 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -3,7 +3,7 @@ py4j>=0.10.9.7 # PySpark dependencies (optional) numpy>=1.21 -pyarrow>=10.0.0 +pyarrow>=11.0.0 six==1.16.0 pandas>=2.0.0 scipy diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index 2b9f28135bb17..b35588a618acf 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -207,7 +207,7 @@ Installable with ``pip install "pyspark[connect]"``. Package Supported version Note ========================== ================= ========================== `pandas` >=2.0.0 Required for Spark Connect -`pyarrow` >=10.0.0 Required for Spark Connect +`pyarrow` >=11.0.0 Required for Spark Connect `grpcio` >=1.67.0 Required for Spark Connect `grpcio-status` >=1.67.0 Required for Spark Connect `googleapis-common-protos` >=1.65.0 Required for Spark Connect @@ -223,7 +223,7 @@ Installable with ``pip install "pyspark[sql]"``. Package Supported version Note ========= ================= ====================== `pandas` >=2.0.0 Required for Spark SQL -`pyarrow` >=10.0.0 Required for Spark SQL +`pyarrow` >=11.0.0 Required for Spark SQL ========= ================= ====================== Additional libraries that enhance functionality but are not included in the installation packages: @@ -240,7 +240,7 @@ Installable with ``pip install "pyspark[pandas_on_spark]"``. Package Supported version Note ========= ================= ================================ `pandas` >=2.0.0 Required for Pandas API on Spark -`pyarrow` >=10.0.0 Required for Pandas API on Spark +`pyarrow` >=11.0.0 Required for Pandas API on Spark ========= ================= ================================ Additional libraries that enhance functionality but are not included in the installation packages: diff --git a/python/docs/source/migration_guide/pyspark_upgrade.rst b/python/docs/source/migration_guide/pyspark_upgrade.rst index 5292530420025..55d067eb5fa2d 100644 --- a/python/docs/source/migration_guide/pyspark_upgrade.rst +++ b/python/docs/source/migration_guide/pyspark_upgrade.rst @@ -25,7 +25,7 @@ Upgrading from PySpark 3.5 to 4.0 * In Spark 4.0, Python 3.8 support was dropped in PySpark. * In Spark 4.0, the minimum supported version for Pandas has been raised from 1.0.5 to 2.0.0 in PySpark. * In Spark 4.0, the minimum supported version for Numpy has been raised from 1.15 to 1.21 in PySpark. -* In Spark 4.0, the minimum supported version for PyArrow has been raised from 4.0.0 to 10.0.0 in PySpark. +* In Spark 4.0, the minimum supported version for PyArrow has been raised from 4.0.0 to 11.0.0 in PySpark. * In Spark 4.0, ``Int64Index`` and ``Float64Index`` have been removed from pandas API on Spark, ``Index`` should be used directly. * In Spark 4.0, ``DataFrame.iteritems`` has been removed from pandas API on Spark, use ``DataFrame.items`` instead. * In Spark 4.0, ``Series.iteritems`` has been removed from pandas API on Spark, use ``Series.items`` instead. diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index 09f194278cdcc..f595b26450e39 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -152,7 +152,7 @@ def _supports_symlinks(): # python/packaging/connect/setup.py _minimum_pandas_version = "2.0.0" _minimum_numpy_version = "1.21" -_minimum_pyarrow_version = "10.0.0" +_minimum_pyarrow_version = "11.0.0" _minimum_grpc_version = "1.67.0" _minimum_googleapis_common_protos_version = "1.65.0" diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 5f67e5306b3ff..51d0a4c9e3601 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -132,7 +132,7 @@ # python/packaging/classic/setup.py _minimum_pandas_version = "2.0.0" _minimum_numpy_version = "1.21" - _minimum_pyarrow_version = "10.0.0" + _minimum_pyarrow_version = "11.0.0" _minimum_grpc_version = "1.59.3" _minimum_googleapis_common_protos_version = "1.56.4" diff --git a/python/pyspark/sql/pandas/utils.py b/python/pyspark/sql/pandas/utils.py index 5849ae0edd6d9..a351c13ff0a08 100644 --- a/python/pyspark/sql/pandas/utils.py +++ b/python/pyspark/sql/pandas/utils.py @@ -61,7 +61,7 @@ def require_minimum_pandas_version() -> None: def require_minimum_pyarrow_version() -> None: """Raise ImportError if minimum version of pyarrow is not installed""" # TODO(HyukjinKwon): Relocate and deduplicate the version specification. - minimum_pyarrow_version = "10.0.0" + minimum_pyarrow_version = "11.0.0" import os From 4ad7f3d1f3e5263924c791cb298848f3be704fa2 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 26 Dec 2024 09:27:19 +0800 Subject: [PATCH 249/438] [SPARK-50647][INFRA] Add a daily build for PySpark with old dependencies ### What changes were proposed in this pull request? Add a daily build for PySpark with old dependencies ### Why are the changes needed? to guard the installation described in https://apache.github.io/spark/api/python/getting_started/install.html The installation guide is outdated: - pyspark-sql/connect requires -- pyarrow>=11.0 -- numpy>=1.21 -- pandas>=2.0.0 - pyspark-pandas requires a even new versions of pandas/pyarrow/numpy -- pyarrow>=11.0 -- numpy>=1.22.4 -- pandas>=2.2.0 This PR excludes PS: we can either - make PS works in the old versions, and then add it in this workflow; - or upgrade the minimum requirements, and add a separate workflow for it; ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR build with ``` envs: default: '{"PYSPARK_IMAGE_TO_TEST": "python-minimum", "PYTHON_TO_TEST": "python3.9"}' jobs: default: '{"pyspark": "true"}' ``` https://github.com/zhengruifeng/spark/runs/34827211339 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49267 from zhengruifeng/infra_py_old. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../workflows/build_infra_images_cache.yml | 13 +++ .github/workflows/build_python_minimum.yml | 46 +++++++++++ .../python-minimum/Dockerfile | 81 +++++++++++++++++++ 3 files changed, 140 insertions(+) create mode 100644 .github/workflows/build_python_minimum.yml create mode 100644 dev/spark-test-image/python-minimum/Dockerfile diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index 930932420bda0..565bb8c7d6e68 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -109,6 +109,19 @@ jobs: - name: Image digest (SparkR) if: hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' run: echo ${{ steps.docker_build_sparkr.outputs.digest }} + - name: Build and push (PySpark with old dependencies) + if: hashFiles('dev/spark-test-image/python-minimum/Dockerfile') != '' + id: docker_build_pyspark_python_minimum + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/python-minimum/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-minimum-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-minimum-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-minimum-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark with old dependencies) + if: hashFiles('dev/spark-test-image/python-minimum/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_python_minimum.outputs.digest }} - name: Build and push (PySpark with PyPy 3.10) if: hashFiles('dev/spark-test-image/pypy-310/Dockerfile') != '' id: docker_build_pyspark_pypy_310 diff --git a/.github/workflows/build_python_minimum.yml b/.github/workflows/build_python_minimum.yml new file mode 100644 index 0000000000000..0efd2ad8265f7 --- /dev/null +++ b/.github/workflows/build_python_minimum.yml @@ -0,0 +1,46 @@ +# +# 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. +# + +name: "Build / Python-only (master, Python with old dependencies)" + +on: + schedule: + - cron: '0 9 * * *' + workflow_dispatch: + +jobs: + run-build: + permissions: + packages: write + name: Run + uses: ./.github/workflows/build_and_test.yml + if: github.repository == 'apache/spark' + with: + java: 17 + branch: master + hadoop: hadoop3 + envs: >- + { + "PYSPARK_IMAGE_TO_TEST": "python-minimum", + "PYTHON_TO_TEST": "python3.9" + } + jobs: >- + { + "pyspark": "true" + } diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile new file mode 100644 index 0000000000000..82e2508ec6e32 --- /dev/null +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -0,0 +1,81 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For PySpark with old dependencies" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE=20241223 + +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pkg-config \ + qpdf \ + tzdata \ + software-properties-common \ + wget \ + zlib1g-dev + + +# Should keep the installation consistent with https://apache.github.io/spark/api/python/getting_started/install.html + +# Install Python 3.9 +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt-get update && apt-get install -y \ + python3.9 \ + python3.9-distutils \ + && apt-get autoremove --purge -y \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +ARG BASIC_PIP_PKGS="numpy==1.21 pyarrow==11.0.0 pandas==2.0.0 six==1.16.0 scipy scikit-learn coverage unittest-xml-reporting" +# Python deps for Spark Connect +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" + +# Install Python 3.9 packages +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 +RUN python3.9 -m pip install --force $BASIC_PIP_PKGS $CONNECT_PIP_PKGS && \ + python3.9 -m pip cache purge From be2da526b4823216edbd0192ba5a2994718f13a9 Mon Sep 17 00:00:00 2001 From: Mihailo Milosevic Date: Thu, 26 Dec 2024 11:39:58 +0900 Subject: [PATCH 250/438] [SPARK-49632][SQL][FOLLOW-UP] Fix suggestion for `to_date` function ### What changes were proposed in this pull request? Change of suggested function on failure from `try_to_timestamp` to `try_to_date` for `to_date` function. ### Why are the changes needed? In original PR https://github.com/apache/spark/pull/49227 we removed ANSI suggestion and left only a suggestion of try function to use. In case of `to_date`, use of `try_to_date` is more appropriate. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Existing tests scope error message change. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49290 from mihailom-db/cannot-parse-timestamp-follow-up. Authored-by: Mihailo Milosevic Signed-off-by: Hyukjin Kwon --- .../resources/error/error-conditions.json | 2 +- .../expressions/datetimeExpressions.scala | 14 ++++++----- .../sql/errors/QueryExecutionErrors.scala | 6 +++-- .../function_to_date_with_format.explain | 2 +- ...ction_to_timestamp_ltz_with_format.explain | 2 +- ...ction_to_timestamp_ntz_with_format.explain | 2 +- .../function_to_timestamp_with_format.explain | 2 +- .../function_try_to_timestamp.explain | 2 +- .../function_unix_date.explain | 2 +- .../function_unix_micros.explain | 2 +- .../function_unix_millis.explain | 2 +- .../function_unix_seconds.explain | 2 +- .../resources/sql-tests/results/date.sql.out | 1 + .../sql-tests/results/datetime-legacy.sql.out | 24 +++++++++++++++++++ .../results/datetime-parsing-invalid.sql.out | 24 +++++++++++++++++++ .../sql-tests/results/timestamp.sql.out | 6 +++++ .../timestampNTZ/timestamp-ansi.sql.out | 6 +++++ .../native/stringCastAndExpressions.sql.out | 2 ++ .../QueryExecutionAnsiErrorsSuite.scala | 4 +++- 19 files changed, 88 insertions(+), 19 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index deb62866f072e..26b3de7f50890 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -377,7 +377,7 @@ }, "CANNOT_PARSE_TIMESTAMP" : { "message" : [ - ". Use `try_to_timestamp` to tolerate invalid input string and return NULL instead." + ". Use to tolerate invalid input string and return NULL instead." ], "sqlState" : "22007" }, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 55e6c7f1503fe..12fe456eace8d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1129,6 +1129,7 @@ case class GetTimestamp( left: Expression, right: Expression, override val dataType: DataType, + override val suggestedFuncOnFail: String = "try_to_timestamp", timeZoneId: Option[String] = None, failOnError: Boolean = SQLConf.get.ansiEnabled) extends ToTimestamp { @@ -1267,6 +1268,7 @@ object TryToTimestampExpressionBuilder extends ExpressionBuilder { abstract class ToTimestamp extends BinaryExpression with TimestampFormatterHelper with ExpectsInputTypes { + val suggestedFuncOnFail: String = "try_to_timestamp" def failOnError: Boolean // The result of the conversion to timestamp is microseconds divided by this factor. @@ -1321,9 +1323,9 @@ abstract class ToTimestamp } } catch { case e: DateTimeException if failOnError => - throw QueryExecutionErrors.ansiDateTimeParseError(e) + throw QueryExecutionErrors.ansiDateTimeParseError(e, suggestedFuncOnFail) case e: ParseException if failOnError => - throw QueryExecutionErrors.ansiDateTimeParseError(e) + throw QueryExecutionErrors.ansiDateTimeParseError(e, suggestedFuncOnFail) case e if isParseError(e) => null } } @@ -1334,7 +1336,7 @@ abstract class ToTimestamp override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = CodeGenerator.javaType(dataType) val parseErrorBranch: String = if (failOnError) { - "throw QueryExecutionErrors.ansiDateTimeParseError(e);" + s"throw QueryExecutionErrors.ansiDateTimeParseError(e, \"${suggestedFuncOnFail}\");" } else { s"${ev.isNull} = true;" } @@ -2100,8 +2102,8 @@ case class ParseToDate( extends RuntimeReplaceable with ImplicitCastInputTypes with TimeZoneAwareExpression { override lazy val replacement: Expression = format.map { f => - Cast(GetTimestamp(left, f, TimestampType, timeZoneId, ansiEnabled), DateType, timeZoneId, - EvalMode.fromBoolean(ansiEnabled)) + Cast(GetTimestamp(left, f, TimestampType, "try_to_date", timeZoneId, ansiEnabled), DateType, + timeZoneId, EvalMode.fromBoolean(ansiEnabled)) }.getOrElse(Cast(left, DateType, timeZoneId, EvalMode.fromBoolean(ansiEnabled))) // backwards compatibility @@ -2179,7 +2181,7 @@ case class ParseToTimestamp( extends RuntimeReplaceable with ImplicitCastInputTypes with TimeZoneAwareExpression { override lazy val replacement: Expression = format.map { f => - GetTimestamp(left, f, dataType, timeZoneId, failOnError = failOnError) + GetTimestamp(left, f, dataType, "try_to_timestamp", timeZoneId, failOnError = failOnError) }.getOrElse(Cast(left, dataType, timeZoneId, ansiEnabled = failOnError)) def this(left: Expression, format: Expression) = { 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 061eaf45cffb9..2ec85a38723cb 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 @@ -265,11 +265,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE summary = "") } - def ansiDateTimeParseError(e: Exception): SparkDateTimeException = { + def ansiDateTimeParseError(e: Exception, suggestedFunc: String): SparkDateTimeException = { new SparkDateTimeException( errorClass = "CANNOT_PARSE_TIMESTAMP", messageParameters = Map( - "message" -> e.getMessage), + "message" -> e.getMessage, + "func" -> toSQLId(suggestedFunc) + ), context = Array.empty, summary = "") } diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_date_with_format.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_date_with_format.explain index 3557274e9de8d..51270c147549e 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_date_with_format.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_date_with_format.explain @@ -1,2 +1,2 @@ -Project [cast(gettimestamp(s#0, yyyy-MM-dd, TimestampType, Some(America/Los_Angeles), false) as date) AS to_date(s, yyyy-MM-dd)#0] +Project [cast(gettimestamp(s#0, yyyy-MM-dd, TimestampType, try_to_date, Some(America/Los_Angeles), false) as date) AS to_date(s, yyyy-MM-dd)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ltz_with_format.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ltz_with_format.explain index e212c8d51a62f..e66fdba89e0ff 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ltz_with_format.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ltz_with_format.explain @@ -1,2 +1,2 @@ -Project [gettimestamp(g#0, g#0, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(g, g)#0] +Project [gettimestamp(g#0, g#0, TimestampType, try_to_timestamp, Some(America/Los_Angeles), false) AS to_timestamp_ltz(g, g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ntz_with_format.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ntz_with_format.explain index 10ca240877fe1..f133becf78237 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ntz_with_format.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ntz_with_format.explain @@ -1,2 +1,2 @@ -Project [gettimestamp(g#0, g#0, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(g, g)#0] +Project [gettimestamp(g#0, g#0, TimestampNTZType, try_to_timestamp, Some(America/Los_Angeles), false) AS to_timestamp_ntz(g, g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_with_format.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_with_format.explain index 54e1c0348a3a9..514b6705fa8e2 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_with_format.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_to_timestamp_with_format.explain @@ -1,2 +1,2 @@ -Project [gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS)#0] +Project [gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, try_to_timestamp, Some(America/Los_Angeles), false) AS to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_timestamp.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_timestamp.explain index 8074beab7db81..c4dd956e83427 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_timestamp.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_timestamp.explain @@ -1,2 +1,2 @@ -Project [gettimestamp(g#0, g#0, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(g, g)#0] +Project [gettimestamp(g#0, g#0, TimestampType, try_to_timestamp, Some(America/Los_Angeles), false) AS try_to_timestamp(g, g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_date.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_date.explain index a1934253d93bd..7ac1d31802baf 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_date.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_date.explain @@ -1,2 +1,2 @@ -Project [unix_date(cast(gettimestamp(s#0, yyyy-MM-dd, TimestampType, Some(America/Los_Angeles), false) as date)) AS unix_date(to_date(s, yyyy-MM-dd))#0] +Project [unix_date(cast(gettimestamp(s#0, yyyy-MM-dd, TimestampType, try_to_date, Some(America/Los_Angeles), false) as date)) AS unix_date(to_date(s, yyyy-MM-dd))#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_micros.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_micros.explain index fb5cdd36f9b70..e5337b0f6c499 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_micros.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_micros.explain @@ -1,2 +1,2 @@ -Project [unix_micros(gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, Some(America/Los_Angeles), false)) AS unix_micros(to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS))#0L] +Project [unix_micros(gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, try_to_timestamp, Some(America/Los_Angeles), false)) AS unix_micros(to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS))#0L] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_millis.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_millis.explain index 3382c9ed679c5..5c852467a3507 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_millis.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_millis.explain @@ -1,2 +1,2 @@ -Project [unix_millis(gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, Some(America/Los_Angeles), false)) AS unix_millis(to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS))#0L] +Project [unix_millis(gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, try_to_timestamp, Some(America/Los_Angeles), false)) AS unix_millis(to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS))#0L] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_seconds.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_seconds.explain index d21c368869732..03d4386edda71 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_seconds.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_unix_seconds.explain @@ -1,2 +1,2 @@ -Project [unix_seconds(gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, Some(America/Los_Angeles), false)) AS unix_seconds(to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS))#0L] +Project [unix_seconds(gettimestamp(s#0, yyyy-MM-dd HH:mm:ss.SSSS, TimestampType, try_to_timestamp, Some(America/Los_Angeles), false)) AS unix_seconds(to_timestamp(s, yyyy-MM-dd HH:mm:ss.SSSS))#0L] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/core/src/test/resources/sql-tests/results/date.sql.out b/sql/core/src/test/resources/sql-tests/results/date.sql.out index 8b892b1795a17..37cf05c6386d7 100644 --- a/sql/core/src/test/resources/sql-tests/results/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date.sql.out @@ -207,6 +207,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_date`", "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 9a40d4fd13169..feae373b52ca7 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -207,6 +207,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_date`", "message" : "Unparseable date: \"02-29\"" } } @@ -1584,6 +1585,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.\"" } } @@ -1599,6 +1601,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.0\"" } } @@ -1614,6 +1617,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.1\"" } } @@ -1629,6 +1633,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" } } @@ -1644,6 +1649,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.123UTC\"" } } @@ -1659,6 +1665,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" } } @@ -1674,6 +1681,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.12345CST\"" } } @@ -1689,6 +1697,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.123456PST\"" } } @@ -1704,6 +1713,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.1234567PST\"" } } @@ -1719,6 +1729,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"123456 2019-10-06 10:11:12.123456PST\"" } } @@ -1734,6 +1745,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"223456 2019-10-06 10:11:12.123456PST\"" } } @@ -1749,6 +1761,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" } } @@ -1764,6 +1777,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.123\"" } } @@ -1779,6 +1793,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12\"" } } @@ -1794,6 +1809,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" } } @@ -1809,6 +1825,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 10:11\"" } } @@ -1824,6 +1841,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06S10:11:12.12345\"" } } @@ -1839,6 +1857,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"12.12342019-10-06S10:11\"" } } @@ -1854,6 +1873,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" } } @@ -1869,6 +1889,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" } } @@ -1884,6 +1905,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"12.1234019-10-06S10:11\"" } } @@ -1955,6 +1977,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"02-29\"" } } @@ -2185,6 +2208,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Unparseable date: \"2019-10-06 A\"" } } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out index 9e3b0e2d1e0f5..cdc027846c34b 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -18,6 +18,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '1' could not be parsed at index 0" } } @@ -33,6 +34,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '-12' could not be parsed at index 0" } } @@ -48,6 +50,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '123' could not be parsed, unparsed text found at index 2" } } @@ -63,6 +66,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '1' could not be parsed at index 0" } } @@ -95,6 +99,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" } } @@ -110,6 +115,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '9' could not be parsed at index 0" } } @@ -125,6 +131,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" } } @@ -140,6 +147,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '9' could not be parsed at index 0" } } @@ -155,6 +163,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '99' could not be parsed at index 0" } } @@ -170,6 +179,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." } } @@ -185,6 +195,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31." } } @@ -200,6 +211,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year" } } @@ -215,6 +227,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." } } @@ -230,6 +243,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30" } } @@ -245,6 +259,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15" } } @@ -260,6 +275,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'" } } @@ -283,6 +299,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_date`", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -298,6 +315,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_date`", "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -313,6 +331,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -328,6 +347,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -343,6 +363,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -358,6 +379,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text 'Unparseable' could not be parsed at index 0" } } @@ -373,6 +395,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" } } @@ -388,6 +411,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text 'Unparseable' could not be parsed at index 0" } } diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out index 69025271601e9..06a8fbad8a29c 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out @@ -395,6 +395,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" } } @@ -466,6 +467,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" } } @@ -489,6 +491,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" } } @@ -560,6 +563,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" } } @@ -575,6 +579,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" } } @@ -654,6 +659,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out index 14835ae0c5926..933982f5cff6a 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out @@ -409,6 +409,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" } } @@ -480,6 +481,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" } } @@ -503,6 +505,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" } } @@ -574,6 +577,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" } } @@ -589,6 +593,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" } } @@ -668,6 +673,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Invalid date 'February 29' as '1970' is not a leap year" } } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index 0b043c70cc235..ea17c7f5289fc 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -376,6 +376,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text 'aa' could not be parsed at index 0" } } @@ -408,6 +409,7 @@ org.apache.spark.SparkDateTimeException "errorClass" : "CANNOT_PARSE_TIMESTAMP", "sqlState" : "22007", "messageParameters" : { + "func" : "`try_to_timestamp`", "message" : "Text 'aa' could not be parsed at index 0" } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala index 52ef662080fb0..fde5a32e722f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala @@ -239,7 +239,9 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest sql("select to_timestamp('abc', 'yyyy-MM-dd HH:mm:ss')").collect() }, condition = "CANNOT_PARSE_TIMESTAMP", - parameters = Map("message" -> "Text 'abc' could not be parsed at index 0") + parameters = Map( + "func" -> "`try_to_timestamp`", + "message" -> "Text 'abc' could not be parsed at index 0") ) } From a483dfd8601868ceec74d2763c04f0ed82abde76 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Thu, 26 Dec 2024 10:50:37 +0800 Subject: [PATCH 251/438] [SPARK-50650][SQL] Improve logging in single-pass Analyzer ### What changes were proposed in this pull request? 1. Log initial unresolved plans. This was we see the full plan, and track the downwards traversal. 2. Log expression tree changes in the same manner as operator tree changes. ### Why are the changes needed? To make single-pass Analyzer debugging easier. Examples: ![image](https://github.com/user-attachments/assets/507c3daf-0fe2-48a9-a66c-73e038550511) ![image](https://github.com/user-attachments/assets/8d8c9fbc-7371-4d29-a8c5-71aba57a65e5) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49271 from vladimirg-db/vladimirg-db/single-pass-analyzer/improve-plan-logger. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../resolver/ExpressionResolver.scala | 17 ++-- .../analysis/resolver/PlanLogger.scala | 88 ++++++++++++++----- .../catalyst/analysis/resolver/Resolver.scala | 18 +++- .../apache/spark/sql/internal/SQLConf.scala | 15 ++++ .../resolver/TracksResolvedNodesSuite.scala | 4 +- 5 files changed, 108 insertions(+), 34 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala index 0a9e2b9c5a872..1d072509626b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala @@ -64,11 +64,13 @@ import org.apache.spark.sql.types.MetadataBuilder * operators which are nested in expressions. * @param scopes [[NameScopeStack]] to resolve the expression tree in the correct scope. * @param functionResolution [[FunctionResolution]] to resolve function expressions. + * @param planLogger [[PlanLogger]] to log expression tree resolution events. */ class ExpressionResolver( resolver: Resolver, scopes: NameScopeStack, - functionResolution: FunctionResolution) + functionResolution: FunctionResolution, + planLogger: PlanLogger) extends TreeNodeResolver[Expression, Expression] with ProducesUnresolvedSubtree with ResolvesExpressionChildren @@ -118,7 +120,9 @@ class ExpressionResolver( * In this case `IN` is an expression and `SELECT 1` is a nested operator tree for which * the [[ExpressionResolver]] would invoke the [[Resolver]]. */ - override def resolve(unresolvedExpression: Expression): Expression = + override def resolve(unresolvedExpression: Expression): Expression = { + planLogger.logExpressionTreeResolutionEvent(unresolvedExpression, "Unresolved expression tree") + if (unresolvedExpression .getTagValue(ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY) .nonEmpty) { @@ -126,7 +130,7 @@ class ExpressionResolver( } else { throwIfNodeWasResolvedEarlier(unresolvedExpression) - val resolvedExpr = unresolvedExpression match { + val resolvedExpression = unresolvedExpression match { case unresolvedBinaryArithmetic: BinaryArithmetic => binaryArithmeticResolver.resolve(unresolvedBinaryArithmetic) case unresolvedExtractANSIIntervalDays: ExtractANSIIntervalDays => @@ -157,10 +161,13 @@ class ExpressionResolver( } } - markNodeAsResolved(resolvedExpr) + markNodeAsResolved(resolvedExpression) + + planLogger.logExpressionTreeResolution(unresolvedExpression, resolvedExpression) - resolvedExpr + resolvedExpression } + } private def resolveNamedExpression( unresolvedNamedExpression: Expression, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala index fcf1eab0c04a9..8870befede4f4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.catalyst.analysis.resolver import org.apache.spark.internal.{Logging, MDC, MessageWithContext} -import org.apache.spark.internal.LogKeys.QUERY_PLAN +import org.apache.spark.internal.LogKeys.{MESSAGE, QUERY_PLAN} +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.internal.SQLConf @@ -27,32 +28,71 @@ import org.apache.spark.sql.internal.SQLConf * [[PlanLogger]] is used by the [[Resolver]] to log intermediate resolution results. */ class PlanLogger extends Logging { - private val logLevel = SQLConf.get.planChangeLogLevel + private val planChangeLogLevel = SQLConf.get.planChangeLogLevel + private val expressionTreeChangeLogLevel = SQLConf.get.expressionTreeChangeLogLevel - /** - * Logs the transition from the `unresolvedPlan` to the `resolvedPlan`. - */ - def log(unresolvedPlan: LogicalPlan, resolvedPlan: LogicalPlan): Unit = { - logBasedOnLevel(() => createMessage(unresolvedPlan, resolvedPlan)) + def logPlanResolutionEvent(plan: LogicalPlan, event: String): Unit = { + log(() => log""" + |=== Plan resolution: ${MDC(MESSAGE, event)} === + |${MDC(QUERY_PLAN, plan.treeString)} + """.stripMargin, planChangeLogLevel) } - private def createMessage( - unresolvedPlan: LogicalPlan, - resolvedPlan: LogicalPlan): MessageWithContext = - log""" - |=== Unresolved/resolved operator subtree === + def logPlanResolution(unresolvedPlan: LogicalPlan, resolvedPlan: LogicalPlan): Unit = { + log( + () => + log""" + |=== Unresolved plan -> Resolved plan === |${MDC( - QUERY_PLAN, - sideBySide(unresolvedPlan.treeString, resolvedPlan.treeString).mkString("\n") - )} - """.stripMargin - - private def logBasedOnLevel(createMessage: () => MessageWithContext): Unit = logLevel match { - case "TRACE" => logTrace(createMessage().message) - case "DEBUG" => logDebug(createMessage().message) - case "INFO" => logInfo(createMessage()) - case "WARN" => logWarning(createMessage()) - case "ERROR" => logError(createMessage()) - case _ => logTrace(createMessage().message) + QUERY_PLAN, + sideBySide( + unresolvedPlan.withNewChildren(resolvedPlan.children).treeString, + resolvedPlan.treeString + ).mkString("\n") + )} + """.stripMargin, + planChangeLogLevel + ) } + + def logExpressionTreeResolutionEvent(expressionTree: Expression, event: String): Unit = { + log( + () => log""" + |=== Expression tree resolution: ${MDC(MESSAGE, event)} === + |${MDC(QUERY_PLAN, expressionTree.treeString)} + """.stripMargin, + expressionTreeChangeLogLevel + ) + } + + def logExpressionTreeResolution( + unresolvedExpressionTree: Expression, + resolvedExpressionTree: Expression): Unit = { + log( + () => + log""" + |=== Unresolved expression tree -> Resolved expression tree === + |${MDC( + QUERY_PLAN, + sideBySide( + unresolvedExpressionTree + .withNewChildren(resolvedExpressionTree.children) + .treeString, + resolvedExpressionTree.treeString + ).mkString("\n") + )} + """.stripMargin, + expressionTreeChangeLogLevel + ) + } + + private def log(createMessage: () => MessageWithContext, logLevel: String): Unit = + logLevel match { + case "TRACE" => logTrace(createMessage().message) + case "DEBUG" => logDebug(createMessage().message) + case "INFO" => logInfo(createMessage()) + case "WARN" => logWarning(createMessage()) + case "ERROR" => logError(createMessage()) + case _ => logTrace(createMessage().message) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala index b0e6828a97a08..37b875abaade6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala @@ -70,11 +70,12 @@ class Resolver( with TracksResolvedNodes[LogicalPlan] with DelegatesResolutionToExtensions { private val scopes = new NameScopeStack + private val planLogger = new PlanLogger private val relationResolution = Resolver.createRelationResolution(catalogManager) private val functionResolution = new FunctionResolution(catalogManager, relationResolution) - private val expressionResolver = new ExpressionResolver(this, scopes, functionResolution) + private val expressionResolver = + new ExpressionResolver(this, scopes, functionResolution, planLogger) private val limitExpressionResolver = new LimitExpressionResolver(expressionResolver) - private val planLogger = new PlanLogger /** * [[relationMetadataProvider]] is used to resolve metadata for relations. It's initialized with @@ -101,6 +102,8 @@ class Resolver( def lookupMetadataAndResolve( unresolvedPlan: LogicalPlan, analyzerBridgeState: Option[AnalyzerBridgeState] = None): LogicalPlan = { + planLogger.logPlanResolutionEvent(unresolvedPlan, "Lookup metadata and resolve") + relationMetadataProvider = analyzerBridgeState match { case Some(analyzerBridgeState) => new BridgedRelationMetadataProvider( @@ -134,6 +137,8 @@ class Resolver( * producing a fully resolved plan or a descriptive error message. */ override def resolve(unresolvedPlan: LogicalPlan): LogicalPlan = { + planLogger.logPlanResolutionEvent(unresolvedPlan, "Unresolved plan") + throwIfNodeWasResolvedEarlier(unresolvedPlan) val resolvedPlan = @@ -167,7 +172,9 @@ class Resolver( } markNodeAsResolved(resolvedPlan) - planLogger.log(unresolvedPlan, resolvedPlan) + + planLogger.logPlanResolution(unresolvedPlan, resolvedPlan) + resolvedPlan } @@ -260,7 +267,10 @@ class Resolver( private def resolveRelation(unresolvedRelation: UnresolvedRelation): LogicalPlan = { relationMetadataProvider.getRelationWithResolvedMetadata(unresolvedRelation) match { case Some(relationWithResolvedMetadata) => - planLogger.log(unresolvedRelation, relationWithResolvedMetadata) + planLogger.logPlanResolutionEvent( + relationWithResolvedMetadata, + "Relation metadata retrieved" + ) withPosition(unresolvedRelation) { resolve(relationWithResolvedMetadata) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d5f18231a6c15..08f77d58979f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -400,6 +400,19 @@ object SQLConf { .booleanConf .createWithDefault(Utils.isTesting) + val EXPRESSION_TREE_CHANGE_LOG_LEVEL = buildConf("spark.sql.expressionTreeChangeLog.level") + .internal() + .doc("Configures the log level for logging the change from the unresolved expression tree to " + + "the resolved expression tree in the single-pass bottom-up Resolver. The value can be " + + "'trace', 'debug', 'info', 'warn', or 'error'. The default log level is 'trace'.") + .version("4.0.0") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValue(logLevel => Set("TRACE", "DEBUG", "INFO", "WARN", "ERROR").contains(logLevel), + "Invalid value for 'spark.sql.expressionTreeChangeLog.level'. Valid values are " + + "'trace', 'debug', 'info', 'warn' and 'error'.") + .createWithDefault("trace") + val LIGHTWEIGHT_PLAN_CHANGE_VALIDATION = buildConf("spark.sql.lightweightPlanChangeValidation") .internal() .doc(s"Similar to ${PLAN_CHANGE_VALIDATION.key}, this validates plan changes and runs after " + @@ -5578,6 +5591,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def planChangeBatches: Option[String] = getConf(PLAN_CHANGE_LOG_BATCHES) + def expressionTreeChangeLogLevel: String = getConf(EXPRESSION_TREE_CHANGE_LOG_LEVEL) + def dynamicPartitionPruningEnabled: Boolean = getConf(DYNAMIC_PARTITION_PRUNING_ENABLED) def dynamicPartitionPruningUseStats: Boolean = getConf(DYNAMIC_PARTITION_PRUNING_USE_STATS) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala index 28ccebc89bc52..b7bf73f326fa8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/analysis/resolver/TracksResolvedNodesSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionResolution import org.apache.spark.sql.catalyst.analysis.resolver.{ ExpressionResolver, NameScopeStack, + PlanLogger, Resolver } import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, ExprId} @@ -127,7 +128,8 @@ class TracksResolvedNodesSuite extends QueryTest with SharedSparkSession { new FunctionResolution( spark.sessionState.catalogManager, Resolver.createRelationResolution(spark.sessionState.catalogManager) - ) + ), + new PlanLogger ) } } From 2475b35a2ac795a3f31a3ed0098111e3b55d286e Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Thu, 26 Dec 2024 10:51:59 +0800 Subject: [PATCH 252/438] [SPARK-50665][SQL] Substitute LocalRelation with ComparableLocalRelation in NormalizePlan ### What changes were proposed in this pull request? Substitute `LocalRelation` with `ComparableLocalRelation` in `NormalizePlan`. `ComparableLocalRelation` has `Seq[Seq[Expression]]` instead of `Seq[InternalRow]`. The conversion happens through `Literal`s. ### Why are the changes needed? `LocalRelation`'s data field is incomparable if it contains maps, because `ArrayBasedMapData` doesn't define `equals`: https://github.com/apache/spark/pull/13847 ### Does this PR introduce _any_ user-facing change? No. This is to compare logical plans in the single-pass Analyzer. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? copilot.nvim. Closes #49287 from vladimirg-db/vladimirg-db/normalize-local-relation. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../sql/catalyst/plans/NormalizePlan.scala | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala index 1cc876588550b..ee68e433fbeab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.GetViewColumnByNameAndOrdinal import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.read.streaming.SparkDataStream object NormalizePlan extends PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = @@ -104,6 +105,8 @@ object NormalizePlan extends PredicateHelper { case Project(projectList, child) => Project(normalizeProjectList(projectList), child) case c: KeepAnalyzedQuery => c.storeAnalyzedQuery() + case localRelation: LocalRelation => + ComparableLocalRelation.fromLocalRelation(localRelation) } } @@ -134,3 +137,33 @@ object NormalizePlan extends PredicateHelper { case _ => condition // Don't reorder. } } + +/** + * A substitute for the [[LocalRelation]] that has comparable `data` field. [[LocalRelation]]'s + * `data` is incomparable for maps, because [[ArrayBasedMapData]] doesn't define [[equals]]. + */ +case class ComparableLocalRelation( + override val output: Seq[Attribute], + data: Seq[Seq[Expression]], + override val isStreaming: Boolean, + stream: Option[SparkDataStream]) extends LeafNode + +object ComparableLocalRelation { + def fromLocalRelation(localRelation: LocalRelation): ComparableLocalRelation = { + val dataTypes = localRelation.output.map(_.dataType) + ComparableLocalRelation( + output = localRelation.output, + data = localRelation.data.map { row => + if (row != null) { + row.toSeq(dataTypes).zip(dataTypes).map { + case (value, dataType) => Literal(value, dataType) + } + } else { + Seq.empty + } + }, + isStreaming = localRelation.isStreaming, + stream = localRelation.stream + ) + } +} From 5c075c3e577f05f4d2806a360524c121280e820f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 26 Dec 2024 12:40:28 +0900 Subject: [PATCH 253/438] [SPARK-50667][PYTHON][TESTS] Make `jinja2` optional in PySpark Tests ### What changes were proposed in this pull request? Make `jinja2` optional in PySpark Tests ### Why are the changes needed? `jinja2` is an optional dependency of `pandas` https://pypi.org/pypi/pandas/2.2.0/json ``` 'jinja2>=3.1.2; extra == "output-formatting"' ``` It is not a mandatory requirement of pyspark, so PySpark tests should succeed even it is not installed ### Does this PR introduce _any_ user-facing change? no, test-only ### How was this patch tested? manually test after uninstalling it ### Was this patch authored or co-authored using generative AI tooling? no Closes #49288 from zhengruifeng/optional_jinja2. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/pandas/frame.py | 2 +- python/pyspark/pandas/tests/io/test_dataframe_conversion.py | 2 ++ python/pyspark/testing/utils.py | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index f315d59a4fe94..35b96543b9ebb 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -2632,7 +2632,7 @@ def to_latex( ... 'mask': ['red', 'purple'], ... 'weapon': ['sai', 'bo staff']}, ... columns=['name', 'mask', 'weapon']) - >>> print(df.to_latex(index=False)) # doctest: +NORMALIZE_WHITESPACE + >>> print(df.to_latex(index=False)) # doctest: +SKIP \begin{tabular}{lll} \toprule name & mask & weapon \\ diff --git a/python/pyspark/pandas/tests/io/test_dataframe_conversion.py b/python/pyspark/pandas/tests/io/test_dataframe_conversion.py index d4b03a855d382..7cb997153729e 100644 --- a/python/pyspark/pandas/tests/io/test_dataframe_conversion.py +++ b/python/pyspark/pandas/tests/io/test_dataframe_conversion.py @@ -26,6 +26,7 @@ from pyspark import pandas as ps from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils from pyspark.testing.sqlutils import SQLTestUtils +from pyspark.testing.utils import have_jinja2, jinja2_requirement_message class DataFrameConversionMixin: @@ -199,6 +200,7 @@ def test_to_clipboard(self): psdf.to_clipboard(sep=";", index=False), pdf.to_clipboard(sep=";", index=False) ) + @unittest.skipIf(not have_jinja2, jinja2_requirement_message) def test_to_latex(self): pdf = self.pdf psdf = self.psdf diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index c38cd928d584b..a89add74ca8f8 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -94,6 +94,9 @@ def have_package(name: str) -> bool: have_flameprof = have_package("flameprof") flameprof_requirement_message = None if have_flameprof else "No module named 'flameprof'" +have_jinja2 = have_package("jinja2") +jinja2_requirement_message = None if have_jinja2 else "No module named 'jinja2'" + pandas_requirement_message = None try: from pyspark.sql.pandas.utils import require_minimum_pandas_version From 38c6ef456ed67ed55e186625dcde017d85d6431f Mon Sep 17 00:00:00 2001 From: Jovan Markovic Date: Thu, 26 Dec 2024 13:13:06 +0800 Subject: [PATCH 254/438] [SPARK-50529][SQL] Change char/varchar behavior under the `spark.sql.preserveCharVarcharTypeInfo` config ### What changes were proposed in this pull request? This PR changes char/varchar behaviour under the `PRESERVE_CHAR_VARCHAR_TYPE_INFO` configuration flag, (exposed as `spark.sql.preserveCharVarcharTypeInfo`). ### Why are the changes needed? This PR enables the improvement of char/varchar types in a backwards compatible way. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests in: - `RowEncoderSuite` - `LiteralExpressionSuite` - `CharVarcharTestSuite` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49128 from jovanm-db/char_varchar_conf. Authored-by: Jovan Markovic Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/Encoders.scala | 14 +++++ .../catalyst/encoders/AgnosticEncoder.scala | 2 + .../sql/catalyst/encoders/RowEncoder.scala | 12 ++--- .../catalyst/util/SparkCharVarcharUtils.scala | 3 +- .../spark/sql/internal/SqlApiConf.scala | 2 + .../sql/catalyst/CatalystTypeConverters.scala | 29 +++++++++++ .../catalyst/DeserializerBuildHelper.scala | 34 ++++++++++++- .../sql/catalyst/SerializerBuildHelper.scala | 24 ++++++++- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 ++- .../catalyst/encoders/ExpressionEncoder.scala | 6 ++- .../sql/catalyst/expressions/literals.scala | 12 +++-- .../sql/catalyst/util/CharVarcharUtils.scala | 12 ++++- .../apache/spark/sql/internal/SQLConf.scala | 10 ++++ .../sql/catalyst/analysis/AnalysisSuite.scala | 18 +++++++ .../catalyst/encoders/RowEncoderSuite.scala | 37 ++++++++++++++ .../expressions/LiteralExpressionSuite.scala | 51 ++++++++++++++----- .../expressions/ToPrettyStringSuite.scala | 4 ++ .../command/AnalyzeColumnCommand.scala | 1 + .../spark/sql/CharVarcharTestSuite.scala | 21 ++++++++ 19 files changed, 265 insertions(+), 33 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala index 9976b34f7a01f..4957d76af9a29 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -81,6 +81,20 @@ object Encoders { */ def DOUBLE: Encoder[java.lang.Double] = BoxedDoubleEncoder + /** + * An encoder for nullable char type. + * + * @since 4.0.0 + */ + def CHAR(length: Int): Encoder[java.lang.String] = CharEncoder(length) + + /** + * An encoder for nullable varchar type. + * + * @since 4.0.0 + */ + def VARCHAR(length: Int): Encoder[java.lang.String] = VarcharEncoder(length) + /** * An encoder for nullable string type. * diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala index 9ae7de97abf58..d998502ac1b25 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala @@ -231,6 +231,8 @@ object AgnosticEncoders { // Nullable leaf encoders case object NullEncoder extends LeafEncoder[java.lang.Void](NullType) case object StringEncoder extends LeafEncoder[String](StringType) + case class CharEncoder(length: Int) extends LeafEncoder[String](CharType(length)) + case class VarcharEncoder(length: Int) extends LeafEncoder[String](VarcharType(length)) case object BinaryEncoder extends LeafEncoder[Array[Byte]](BinaryType) case object ScalaBigIntEncoder extends LeafEncoder[BigInt](DecimalType.BigIntDecimal) case object JavaBigIntEncoder extends LeafEncoder[JBigInt](DecimalType.BigIntDecimal) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 3f384235ff32d..718d99043abf2 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.reflect.classTag import org.apache.spark.sql.{AnalysisException, Row} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, NullEncoder, RowEncoder => AgnosticRowEncoder, StringEncoder, TimestampEncoder, UDTEncoder, VariantEncoder, YearMonthIntervalEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, NullEncoder, RowEncoder => AgnosticRowEncoder, StringEncoder, TimestampEncoder, UDTEncoder, VarcharEncoder, VariantEncoder, YearMonthIntervalEncoder} import org.apache.spark.sql.errors.{DataTypeErrorsBase, ExecutionErrors} import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types._ @@ -80,11 +80,11 @@ object RowEncoder extends DataTypeErrorsBase { case DoubleType => BoxedDoubleEncoder case dt: DecimalType => JavaDecimalEncoder(dt, lenientSerialization = true) case BinaryType => BinaryEncoder - case CharType(_) | VarcharType(_) => - throw new AnalysisException( - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_ENCODER", - messageParameters = Map("dataType" -> toSQLType(dataType))) - case _: StringType => StringEncoder + case CharType(length) if SqlApiConf.get.preserveCharVarcharTypeInfo => + CharEncoder(length) + case VarcharType(length) if SqlApiConf.get.preserveCharVarcharTypeInfo => + VarcharEncoder(length) + case s: StringType if s.constraint == NoConstraint => StringEncoder case TimestampType if SqlApiConf.get.datetimeJava8ApiEnabled => InstantEncoder(lenient) case TimestampType => TimestampEncoder(lenient) case TimestampNTZType => LocalDateTimeEncoder diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala index 2a26c079e8d4d..51b2c40f9bf2e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala @@ -54,8 +54,7 @@ trait SparkCharVarcharUtils { StructType(fields.map { field => field.copy(dataType = replaceCharVarcharWithString(field.dataType)) }) - case _: CharType => StringType - case _: VarcharType => StringType + case CharType(_) | VarcharType(_) if !SqlApiConf.get.preserveCharVarcharTypeInfo => StringType case _ => dt } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index d5668cc721750..76cd436b39b58 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -40,6 +40,7 @@ private[sql] trait SqlApiConf { def timestampType: AtomicType def allowNegativeScaleOfDecimalEnabled: Boolean def charVarcharAsString: Boolean + def preserveCharVarcharTypeInfo: Boolean def datetimeJava8ApiEnabled: Boolean def sessionLocalTimeZone: String def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value @@ -80,6 +81,7 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf { override def timestampType: AtomicType = TimestampType override def allowNegativeScaleOfDecimalEnabled: Boolean = false override def charVarcharAsString: Boolean = false + override def preserveCharVarcharTypeInfo: Boolean = false override def datetimeJava8ApiEnabled: Boolean = false override def sessionLocalTimeZone: String = TimeZone.getDefault.getID override def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value = LegacyBehaviorPolicy.CORRECTED diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 2b2a186f76d9d..fab65251ed51b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -66,6 +66,8 @@ object CatalystTypeConverters { case arrayType: ArrayType => ArrayConverter(arrayType.elementType) case mapType: MapType => MapConverter(mapType.keyType, mapType.valueType) case structType: StructType => StructConverter(structType) + case CharType(length) => new CharConverter(length) + case VarcharType(length) => new VarcharConverter(length) case _: StringType => StringConverter case DateType if SQLConf.get.datetimeJava8ApiEnabled => LocalDateConverter case DateType => DateConverter @@ -296,6 +298,33 @@ object CatalystTypeConverters { toScala(row.getStruct(column, structType.size)) } + private class CharConverter(length: Int) extends CatalystTypeConverter[Any, String, UTF8String] { + override def toCatalystImpl(scalaValue: Any): UTF8String = + CharVarcharCodegenUtils.charTypeWriteSideCheck( + StringConverter.toCatalystImpl(scalaValue), length) + override def toScala(catalystValue: UTF8String): String = if (catalystValue == null) { + null + } else { + CharVarcharCodegenUtils.charTypeWriteSideCheck(catalystValue, length).toString + } + override def toScalaImpl(row: InternalRow, column: Int): String = + CharVarcharCodegenUtils.charTypeWriteSideCheck(row.getUTF8String(column), length).toString + } + + private class VarcharConverter(length: Int) + extends CatalystTypeConverter[Any, String, UTF8String] { + override def toCatalystImpl(scalaValue: Any): UTF8String = + CharVarcharCodegenUtils.varcharTypeWriteSideCheck( + StringConverter.toCatalystImpl(scalaValue), length) + override def toScala(catalystValue: UTF8String): String = if (catalystValue == null) { + null + } else { + CharVarcharCodegenUtils.varcharTypeWriteSideCheck(catalystValue, length).toString + } + override def toScalaImpl(row: InternalRow, column: Int): String = + CharVarcharCodegenUtils.varcharTypeWriteSideCheck(row.getUTF8String(column), length).toString + } + private object StringConverter extends CatalystTypeConverter[Any, String, UTF8String] { override def toCatalystImpl(scalaValue: Any): UTF8String = scalaValue match { case str: String => UTF8String.fromString(str) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index 4752434015375..55613b2b20134 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.{expressions => exprs} import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders, Codec, JavaSerializationCodec, KryoSerializationCodec} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedLeafEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, YearMonthIntervalEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedLeafEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, VarcharEncoder, YearMonthIntervalEncoder} import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{externalDataTypeFor, isNativeEncoder} import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, IsNull, Literal, MapKeys, MapValues, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, DecodeUsingSerializer, InitializeJavaBean, Invoke, NewInstance, StaticInvoke, UnresolvedCatalystToExternalMap, UnresolvedMapObjects, WrapOption} -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, IntervalUtils} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, CharVarcharCodegenUtils, DateTimeUtils, IntervalUtils} import org.apache.spark.sql.types._ object DeserializerBuildHelper { @@ -80,6 +80,32 @@ object DeserializerBuildHelper { returnNullable = false) } + def createDeserializerForChar( + path: Expression, + returnNullable: Boolean, + length: Int): Expression = { + val expr = StaticInvoke( + classOf[CharVarcharCodegenUtils], + StringType, + "charTypeWriteSideCheck", + path :: Literal(length) :: Nil, + returnNullable = returnNullable) + createDeserializerForString(expr, returnNullable) + } + + def createDeserializerForVarchar( + path: Expression, + returnNullable: Boolean, + length: Int): Expression = { + val expr = StaticInvoke( + classOf[CharVarcharCodegenUtils], + StringType, + "varcharTypeWriteSideCheck", + path :: Literal(length) :: Nil, + returnNullable = returnNullable) + createDeserializerForString(expr, returnNullable) + } + def createDeserializerForString(path: Expression, returnNullable: Boolean): Expression = { Invoke(path, "toString", ObjectType(classOf[java.lang.String]), returnNullable = returnNullable) @@ -258,6 +284,10 @@ object DeserializerBuildHelper { "withName", createDeserializerForString(path, returnNullable = false) :: Nil, returnNullable = false) + case CharEncoder(length) => + createDeserializerForChar(path, returnNullable = false, length) + case VarcharEncoder(length) => + createDeserializerForVarchar(path, returnNullable = false, length) case StringEncoder => createDeserializerForString(path, returnNullable = false) case _: ScalaDecimalEncoder => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala index daebe15c298f6..089d463ecacbb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala @@ -22,11 +22,11 @@ import scala.language.existentials import org.apache.spark.sql.catalyst.{expressions => exprs} import org.apache.spark.sql.catalyst.DeserializerBuildHelper.expressionWithNullSafety import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders, Codec, JavaSerializationCodec, KryoSerializationCodec} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLeafEncoder, BoxedLongEncoder, BoxedShortEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveLeafEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, YearMonthIntervalEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLeafEncoder, BoxedLongEncoder, BoxedShortEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveLeafEncoder, ProductEncoder, ScalaBigIntEncoder, ScalaDecimalEncoder, ScalaEnumEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, VarcharEncoder, YearMonthIntervalEncoder} import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{externalDataTypeFor, isNativeEncoder, lenientExternalDataTypeFor} import org.apache.spark.sql.catalyst.expressions.{BoundReference, CheckOverflow, CreateNamedStruct, Expression, IsNull, KnownNotNull, Literal, UnsafeArrayData} import org.apache.spark.sql.catalyst.expressions.objects._ -import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, GenericArrayData, IntervalUtils} +import org.apache.spark.sql.catalyst.util.{ArrayData, CharVarcharCodegenUtils, DateTimeUtils, GenericArrayData, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -63,6 +63,24 @@ object SerializerBuildHelper { Invoke(inputObject, "doubleValue", DoubleType) } + def createSerializerForChar(inputObject: Expression, length: Int): Expression = { + StaticInvoke( + classOf[CharVarcharCodegenUtils], + CharType(length), + "charTypeWriteSideCheck", + createSerializerForString(inputObject) :: Literal(length) :: Nil, + returnNullable = false) + } + + def createSerializerForVarchar(inputObject: Expression, length: Int): Expression = { + StaticInvoke( + classOf[CharVarcharCodegenUtils], + VarcharType(length), + "varcharTypeWriteSideCheck", + createSerializerForString(inputObject) :: Literal(length) :: Nil, + returnNullable = false) + } + def createSerializerForString(inputObject: Expression): Expression = { StaticInvoke( classOf[UTF8String], @@ -298,6 +316,8 @@ object SerializerBuildHelper { case BoxedDoubleEncoder => createSerializerForDouble(input) case JavaEnumEncoder(_) => createSerializerForJavaEnum(input) case ScalaEnumEncoder(_, _) => createSerializerForScalaEnum(input) + case CharEncoder(length) => createSerializerForChar(input, length) + case VarcharEncoder(length) => createSerializerForVarchar(input, length) case StringEncoder => createSerializerForString(input) case ScalaDecimalEncoder(dt) => createSerializerForBigDecimal(input, dt) case JavaDecimalEncoder(dt, false) => createSerializerForBigDecimal(input, dt) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index b0d6a2a46baa0..6cd394fd79e95 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -283,9 +283,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB plan.foreachUp { case p if p.analyzed => // Skip already analyzed sub-plans - case leaf: LeafNode if leaf.output.map(_.dataType).exists(CharVarcharUtils.hasCharVarchar) => + case leaf: LeafNode if !SQLConf.get.preserveCharVarcharTypeInfo && + leaf.output.map(_.dataType).exists(CharVarcharUtils.hasCharVarchar) => throw SparkException.internalError( - "Logical plan should not have output of char/varchar type: " + leaf) + s"Logical plan should not have output of char/varchar type when " + + s"${SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key} is false: " + leaf) case u: UnresolvedNamespace => u.schemaNotFound(u.multipartIdentifier) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index f2f86a90d5172..5f0b42fec0fa8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -87,7 +87,8 @@ object ExpressionEncoder { } constructProjection(row).get(0, anyObjectType).asInstanceOf[T] } catch { - case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => + case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" || + e.getCondition == "EXCEED_LIMIT_LENGTH" => throw e case e: Exception => throw QueryExecutionErrors.expressionDecodingError(e, expressions) @@ -115,7 +116,8 @@ object ExpressionEncoder { inputRow(0) = t extractProjection(inputRow) } catch { - case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => + case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" || + e.getCondition == "EXCEED_LIMIT_LENGTH" => throw e case e: Exception => throw QueryExecutionErrors.expressionEncodingError(e, expressions) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index b874cb53cb31f..f3bed39bcb9f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -166,6 +166,8 @@ object Literal { case _: DayTimeIntervalType if v.isInstanceOf[Duration] => Literal(CatalystTypeConverters.createToCatalystConverter(dataType)(v), dataType) case _: ObjectType => Literal(v, dataType) + case CharType(_) | VarcharType(_) if SQLConf.get.preserveCharVarcharTypeInfo => + Literal(CatalystTypeConverters.createToCatalystConverter(dataType)(v), dataType) case _ => Literal(CatalystTypeConverters.convertToCatalyst(v), dataType) } } @@ -196,9 +198,13 @@ object Literal { case TimestampNTZType => create(0L, TimestampNTZType) case it: DayTimeIntervalType => create(0L, it) case it: YearMonthIntervalType => create(0, it) - case CharType(_) | VarcharType(_) => - throw QueryExecutionErrors.noDefaultForDataTypeError(dataType) - case st: StringType => Literal(UTF8String.fromString(""), st) + case CharType(length) => + create(CharVarcharCodegenUtils.charTypeWriteSideCheck(UTF8String.fromString(""), length), + dataType) + case VarcharType(length) => + create(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(UTF8String.fromString(""), length), + dataType) + case st: StringType if st.constraint == NoConstraint => Literal(UTF8String.fromString(""), st) case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8)) case CalendarIntervalType => Literal(new CalendarInterval(0, 0, 0)) case arr: ArrayType => create(Array(), arr) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala index 628fdcebd3084..3db0f54f1a8f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala @@ -164,7 +164,11 @@ object CharVarcharUtils extends Logging with SparkCharVarcharUtils { case CharType(length) if charFuncName.isDefined => StaticInvoke( classOf[CharVarcharCodegenUtils], - StringType, + if (SQLConf.get.preserveCharVarcharTypeInfo) { + CharType(length) + } else { + StringType + }, charFuncName.get, expr :: Literal(length) :: Nil, returnNullable = false) @@ -172,7 +176,11 @@ object CharVarcharUtils extends Logging with SparkCharVarcharUtils { case VarcharType(length) if varcharFuncName.isDefined => StaticInvoke( classOf[CharVarcharCodegenUtils], - StringType, + if (SQLConf.get.preserveCharVarcharTypeInfo) { + VarcharType(length) + } else { + StringType + }, varcharFuncName.get, expr :: Literal(length) :: Nil, returnNullable = false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 08f77d58979f0..5e630577638a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4938,6 +4938,14 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PRESERVE_CHAR_VARCHAR_TYPE_INFO = buildConf("spark.sql.preserveCharVarcharTypeInfo") + .doc("When true, Spark does not replace CHAR/VARCHAR types the STRING type, which is the " + + "default behavior of Spark 3.0 and earlier versions. This means the length checks for " + + "CHAR/VARCHAR types is enforced and CHAR type is also properly padded.") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + val READ_SIDE_CHAR_PADDING = buildConf("spark.sql.readSideCharPadding") .doc("When true, Spark applies string padding when reading CHAR type columns/fields, " + "in addition to the write-side padding. This config is true by default to better enforce " + @@ -6343,6 +6351,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def charVarcharAsString: Boolean = getConf(SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING) + def preserveCharVarcharTypeInfo: Boolean = getConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO) + def readSideCharPadding: Boolean = getConf(SQLConf.READ_SIDE_CHAR_PADDING) def cliPrintHeader: Boolean = getConf(SQLConf.CLI_PRINT_HEADER) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index ae27985a3ba64..2ffe6de974c74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -81,6 +81,24 @@ class AnalysisSuite extends AnalysisTest with Matchers { } } + test(s"do not fail if a leaf node has char/varchar type output and " + + s"${SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key} is true") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + val schema1 = new StructType().add("c", CharType(5)) + val schema2 = new StructType().add("c", VarcharType(5)) + val schema3 = new StructType().add("c", ArrayType(CharType(5))) + Seq(schema1, schema2, schema3).foreach { schema => + val table = new InMemoryTable("t", schema, Array.empty, Map.empty[String, String].asJava) + DataSourceV2Relation( + table, + DataTypeUtils.toAttributes(schema), + None, + None, + CaseInsensitiveStringMap.empty()).analyze + } + } + } + test("union project *") { val plan = (1 to 120) .map(_ => testRelation) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 79c6d07d6d218..645b80ffaacb8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -488,4 +488,41 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { val data = Row(mutable.ArraySeq.make(Array(Row("key", "value".getBytes)))) val row = encoder.createSerializer()(data) } + + test("do not allow serializing too long strings into char/varchar") { + Seq(CharType(5), VarcharType(5)).foreach { typ => + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + val schema = new StructType().add("c", typ) + val encoder = ExpressionEncoder(schema).resolveAndBind() + val value = "abcdef" + checkError( + exception = intercept[SparkRuntimeException]({ + val row = toRow(encoder, Row(value)) + }), + condition = "EXCEED_LIMIT_LENGTH", + parameters = Map("limit" -> "5") + ) + } + } + } + + test("do not allow deserializing too long strings into char/varchar") { + Seq(CharType(5), VarcharType(5)).foreach { typ => + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + val fromSchema = new StructType().add("c", StringType) + val fromEncoder = ExpressionEncoder(fromSchema).resolveAndBind() + val toSchema = new StructType().add("c", typ) + val toEncoder = ExpressionEncoder(toSchema).resolveAndBind() + val value = "abcdef" + val row = toRow(fromEncoder, Row(value)) + checkError( + exception = intercept[SparkRuntimeException]({ + val value = fromRow(toEncoder, row) + }), + condition = "EXCEED_LIMIT_LENGTH", + parameters = Map("limit" -> "5") + ) + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index b351d69d3a0bb..5da5c6ac412cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -25,13 +25,12 @@ import java.util.TimeZone import scala.collection.mutable import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.{SparkFunSuite, SparkRuntimeException} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection} import org.apache.spark.sql.catalyst.encoders.ExamplePointUDT import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLType import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DayTimeIntervalType._ @@ -91,16 +90,8 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { // ExamplePointUDT.sqlType is ArrayType(DoubleType, false). checkEvaluation(Literal.default(new ExamplePointUDT), Array()) - // DateType without default value` - List(CharType(1), VarcharType(1)).foreach(errType => { - checkError( - exception = intercept[SparkException] { - Literal.default(errType) - }, - condition = "INTERNAL_ERROR", - parameters = Map("message" -> s"No default value for type: ${toSQLType(errType)}.") - ) - }) + checkEvaluation(Literal.default(CharType(5)), " ") + checkEvaluation(Literal.default(VarcharType(5)), "") } test("boolean literals") { @@ -160,6 +151,42 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.create("\u0000"), "\u0000") } + test("char literals") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + val typ = CharType(5) + checkEvaluation(Literal.create("", typ), " ") + checkEvaluation(Literal.create("test", typ), "test ") + checkEvaluation(Literal.create("test ", typ), "test ") + checkEvaluation(Literal.create("\u0000", typ), "\u0000 ") + + checkError( + exception = intercept[SparkRuntimeException]({ + Literal.create("123456", typ) + }), + condition = "EXCEED_LIMIT_LENGTH", + parameters = Map("limit" -> "5") + ) + } + } + + test("varchar literals") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + val typ = VarcharType(5) + checkEvaluation(Literal.create("", typ), "") + checkEvaluation(Literal.create("test", typ), "test") + checkEvaluation(Literal.create("test ", typ), "test ") + checkEvaluation(Literal.create("\u0000", typ), "\u0000") + + checkError( + exception = intercept[SparkRuntimeException]({ + Literal.create("123456", typ) + }), + condition = "EXCEED_LIMIT_LENGTH", + parameters = Map("limit" -> "5") + ) + } + } + test("sum two literals") { checkEvaluation(Add(Literal(1), Literal(1)), 2) checkEvaluation(Add(Literal.create(1), Literal.create(1)), 2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala index 783fba3bfc0df..2a5f76cab3619 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ToPrettyStringSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC_OPT +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{UTF8String, VariantVal} @@ -89,6 +90,9 @@ class ToPrettyStringSuite extends SparkFunSuite with ExpressionEvalHelper { test("Char as pretty strings") { checkEvaluation(ToPrettyString(Literal.create('a', CharType(5))), "a") + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + checkEvaluation(ToPrettyString(Literal.create('a', CharType(5))), "a ") + } } test("Byte as pretty strings") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 23555c98135f6..1268b14a32fb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -140,6 +140,7 @@ case class AnalyzeColumnCommand( case DoubleType | FloatType => true case BooleanType => true case _: DatetimeType => true + case CharType(_) | VarcharType(_) => false case BinaryType | _: StringType => true case _ => false } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index d3b11274fe1c8..a5cbeb552dcba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -86,6 +86,27 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { } } + test("preserve char/varchar type info") { + Seq(CharType(5), VarcharType(5)).foreach { typ => + for { + char_varchar_as_string <- Seq(false, true) + preserve_char_varchar <- Seq(false, true) + } { + withSQLConf(SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key -> char_varchar_as_string.toString, + SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> preserve_char_varchar.toString) { + withTable("t") { + val name = typ.typeName + sql(s"CREATE TABLE t(i STRING, c $name) USING $format") + val schema = spark.table("t").schema + assert(schema.fields(0).dataType == StringType) + val expectedType = if (preserve_char_varchar) typ else StringType + assert(schema.fields(1).dataType == expectedType) + } + } + } + } + } + test("char type values should be padded or trimmed: partitioned columns") { // via dynamic partitioned columns withTable("t") { From ac91a7d3eac92d57963e6cefd0d35705e5afd296 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 26 Dec 2024 13:53:26 +0800 Subject: [PATCH 255/438] [SPARK-50608][SQL][DOCS] Fix malformed configuration page caused by unclosed tags ### What changes were proposed in this pull request? Fix the malformed configuration page caused by unclosed tags ### Why are the changes needed? docfix ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? #### Before ![image](https://github.com/user-attachments/assets/d5fe7630-a26f-48cb-8bc3-9a93155eeef1) #### After ![image](https://github.com/user-attachments/assets/57bcbfdf-b5b5-43ba-ab60-89dc062f6016) ### Was this patch authored or co-authored using generative AI tooling? no Closes #49223 from yaooqinn/SPARK-50608. Authored-by: Kent Yao Signed-off-by: Kent Yao --- sql/gen-sql-config-docs.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/gen-sql-config-docs.py b/sql/gen-sql-config-docs.py index b69a903b44f90..4db22ff3b8e46 100644 --- a/sql/gen-sql-config-docs.py +++ b/sql/gen-sql-config-docs.py @@ -103,6 +103,14 @@ def generate_sql_configs_table_html(sql_configs, path): ) ) + if config.name == "spark.sql.files.ignoreInvalidPartitionPaths": + description = config.description.replace("<", "<").replace(">", ">") + elif config.name == "spark.sql.hive.quoteHiveStructFieldName": + description = config.description.replace( + "<", "<").replace(">", ">").replace("`", "`") + else: + description = config.description + f.write(dedent( """ @@ -115,7 +123,7 @@ def generate_sql_configs_table_html(sql_configs, path): .format( name=config.name, default=default, - description=markdown.markdown(config.description), + description=markdown.markdown(description), version=config.version ) )) From 7a4114cdf9948480d86ec7c870f7a2fa4939cc67 Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Thu, 26 Dec 2024 15:24:23 +0800 Subject: [PATCH 256/438] [SPARK-50644][FOLLOWUP][SQL] Fix scalar cast in the shredded reader ### What changes were proposed in this pull request? We mostly use the `Cast` expression to implement the cast, but we need some custom logic for certain type combinations. We already have special handling for `long/decimal -> timestamp` in `VariantGet.cast`, so we should do the same thing in `ScalarCastHelper` to ensure consistency. `ScalarCastHelper` also needs special handling for `decimal -> string` to strip any trailing zeros. ### Why are the changes needed? To ensure that cast on shredded variant has the same semantics as cast on unshredded variant. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. They would fail without the change. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49293 from chenhao-db/fix_shredded_cast. Authored-by: Chenhao Li Signed-off-by: Wenchen Fan --- .../variant/variantExpressions.scala | 36 ++++++++++++------- .../parquet/SparkShreddingUtils.scala | 26 +++++++++++++- .../spark/sql/VariantShreddingSuite.scala | 31 ++++++++++++++++ 3 files changed, 80 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala index c19df82e6576b..ba910b8c7e5fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala @@ -427,24 +427,15 @@ case object VariantGet { messageParameters = Map("id" -> v.getTypeInfo.toString) ) } - // We mostly use the `Cast` expression to implement the cast. However, `Cast` silently - // ignores the overflow in the long/decimal -> timestamp cast, and we want to enforce - // strict overflow checks. input.dataType match { case LongType if dataType == TimestampType => - try Math.multiplyExact(input.value.asInstanceOf[Long], MICROS_PER_SECOND) + try castLongToTimestamp(input.value.asInstanceOf[Long]) catch { case _: ArithmeticException => invalidCast() } case _: DecimalType if dataType == TimestampType => - try { - input.value - .asInstanceOf[Decimal] - .toJavaBigDecimal - .multiply(new java.math.BigDecimal(MICROS_PER_SECOND)) - .toBigInteger - .longValueExact() - } catch { + try castDecimalToTimestamp(input.value.asInstanceOf[Decimal]) + catch { case _: ArithmeticException => invalidCast() } case _ => @@ -497,6 +488,27 @@ case object VariantGet { } } } + + // We mostly use the `Cast` expression to implement the cast, but we need some custom logic for + // certain type combinations. + // + // `castLongToTimestamp/castDecimalToTimestamp`: `Cast` silently ignores the overflow in the + // long/decimal -> timestamp cast, and we want to enforce strict overflow checks. They both throw + // an `ArithmeticException` when overflow happens. + def castLongToTimestamp(input: Long): Long = + Math.multiplyExact(input, MICROS_PER_SECOND) + + def castDecimalToTimestamp(input: Decimal): Long = { + val multiplier = new java.math.BigDecimal(MICROS_PER_SECOND) + input.toJavaBigDecimal.multiply(multiplier).toBigInteger.longValueExact() + } + + // Cast decimal to string, but strip any trailing zeros. We don't have to call it if the decimal + // is returned by `Variant.getDecimal`, which already strips any trailing zeros. But we need it + // if the decimal is produced by Spark internally, e.g., on a shredded decimal produced by the + // Spark Parquet reader. + def castDecimalToString(input: Decimal): UTF8String = + UTF8String.fromString(input.toJavaBigDecimal.stripTrailingZeros.toPlainString) } abstract class ParseJsonExpressionBuilderBase(failOnError: Boolean) extends ExpressionBuilder { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index a83ca78455faa..34c167aea363a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -111,7 +111,31 @@ case class ScalarCastHelper( } else { "" } - if (cast != null) { + val customCast = (child.dataType, dataType) match { + case (_: LongType, _: TimestampType) => "castLongToTimestamp" + case (_: DecimalType, _: TimestampType) => "castDecimalToTimestamp" + case (_: DecimalType, _: StringType) => "castDecimalToString" + case _ => null + } + if (customCast != null) { + val childCode = child.genCode(ctx) + // We can avoid the try-catch block for decimal -> string, but the performance benefit is + // little. We can also be more specific in the exception type, like catching + // `ArithmeticException` instead of `Exception`, but it is unnecessary. The `try_cast` codegen + // also catches `Exception` instead of specific exceptions. + val code = code""" + ${childCode.code} + boolean ${ev.isNull} = false; + ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + try { + ${ev.value} = ${classOf[VariantGet].getName}.$customCast(${childCode.value}); + } catch (Exception e) { + ${ev.isNull} = true; + $invalidCastCode + } + """ + ev.copy(code = code) + } else if (cast != null) { val castCode = cast.genCode(ctx) val code = code""" ${castCode.code} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala index b6623bb57a716..3443028ba45b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala @@ -24,6 +24,8 @@ import java.time.LocalDateTime import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils +import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetTest, SparkShreddingUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -349,4 +351,33 @@ class VariantShreddingSuite extends QueryTest with SharedSparkSession with Parqu checkExpr(path, "variant_get(v, '$.a')", null, parseJson("null"), parseJson("1"), null, parseJson("null"), parseJson("3")) } + + testWithTempPath("custom casts") { path => + writeRows(path, writeSchema(LongType), + Row(metadata(Nil), null, Long.MaxValue / MICROS_PER_SECOND + 1), + Row(metadata(Nil), null, Long.MaxValue / MICROS_PER_SECOND)) + + // long -> timestamp + checkException(path, "cast(v as timestamp)", "INVALID_VARIANT_CAST") + checkExpr(path, "try_cast(v as timestamp)", + null, toJavaTimestamp(Long.MaxValue / MICROS_PER_SECOND * MICROS_PER_SECOND)) + + writeRows(path, writeSchema(DecimalType(38, 19)), + Row(metadata(Nil), null, Decimal("1E18")), + Row(metadata(Nil), null, Decimal("100")), + Row(metadata(Nil), null, Decimal("10")), + Row(metadata(Nil), null, Decimal("1")), + Row(metadata(Nil), null, Decimal("0")), + Row(metadata(Nil), null, Decimal("0.1")), + Row(metadata(Nil), null, Decimal("0.01")), + Row(metadata(Nil), null, Decimal("1E-18"))) + + checkException(path, "cast(v as timestamp)", "INVALID_VARIANT_CAST") + // decimal -> timestamp + checkExpr(path, "try_cast(v as timestamp)", + (null +: Seq(100000000, 10000000, 1000000, 0, 100000, 10000, 0).map(toJavaTimestamp(_))): _*) + // decimal -> string + checkExpr(path, "cast(v as string)", + "1000000000000000000", "100", "10", "1", "0", "0.1", "0.01", "0.000000000000000001") + } } From 92948e73713f6f6629e1610ed0975fa8e619f1a8 Mon Sep 17 00:00:00 2001 From: Dejan Krakovic Date: Thu, 26 Dec 2024 17:05:24 +0800 Subject: [PATCH 257/438] [SPARK-50675][SQL] Table and view level collations support MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This change introduces table and view level collations support in Spark SQL, allowing CREATE TABLE, ALTER TABLE and CREATE VIEW commands to specify DEFAULT COLLATION to be used. For CREATE commands, this refers to all the underlying columns added as part of the table/view creation. For ALTER TABLE command, this refers to only newly created columns in the future, whereas existing ones are not affected, i.e. their collation remains the same. The PR has been modelled after the original changes made by stefankandic in https://github.com/apache/spark/pull/48090, with this PR covering table and view-level collations, whereas a follow up PR will be made covering schema-level collations. This PR is adding/extending the corresponding DDL commands for specifying table/view level collation, whereas a follow up PR will be created separately to leverage the table/view collation in order to determine default collations for input queries of DML commands. ### Why are the changes needed? From our internal users feedback, many people would like to be able to specify collation for their objects, instead of each individual columns. This change adds support for table and view level collations, whereas subsequent changes will add support for other objects such as schema-level collations. ### Does this PR introduce _any_ user-facing change? The change follows the agreed additions in syntax for collation support. The following syntax is now supported (**bold** parts denote additions): { { [CREATE OR] REPLACE TABLE | CREATE [EXTERNAL] TABLE [ IF NOT EXISTS ] } table_name [ table_specification ] [ USING data_source ] [ table_clauses ] [ AS query ] } table_specification ( { column_identifier column_type [ column_properties ] ] } [, ...] [ , table_constraint ] [...] ) table_clauses { OPTIONS clause | PARTITIONED BY clause | CLUSTER BY clause | clustered_by_clause | LOCATION path [ WITH ( CREDENTIAL credential_name ) ] | COMMENT table_comment | TBLPROPERTIES clause | **DEFAULT COLLATION table_collation_name |** WITH { ROW FILTER clause } } [...] CREATE [ OR REPLACE ] [ TEMPORARY ] VIEW [ IF NOT EXISTS ] view_name [ column_list ] [ schema_binding | COMMENT view_comment | TBLPROPERTIES clause | **DEFAULT COLLATION collation_name** ] [...] AS query ALTER TABLE table_name { ADD COLUMN clause | ALTER COLUMN clause | DROP COLUMN clause | RENAME COLUMN clause | **DEFAULT COLLATION clause | …** } ### How was this patch tested? Tests for the new syntax/functionality were added as part of the change. Also, some of the existing tests were extended/amended to cover the new DEFAULT COLLATION for table/view objects. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49084 from dejankrak-db/object-level-collations. Authored-by: Dejan Krakovic Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 7 +++ .../sql/connector/catalog/TableCatalog.java | 5 ++ .../catalyst/analysis/ResolveTableSpec.scala | 1 + .../sql/catalyst/catalog/interface.scala | 2 + .../sql/catalyst/parser/AstBuilder.scala | 54 ++++++++++++++---- .../plans/logical/v2AlterTableCommands.scala | 12 ++++ .../catalyst/plans/logical/v2Commands.scala | 6 +- .../sql/connector/catalog/CatalogV2Util.scala | 5 +- .../spark/sql/connector/catalog/V1Table.scala | 1 + ...eateTablePartitioningValidationSuite.scala | 2 +- .../sql/catalyst/parser/DDLParserSuite.scala | 55 +++++++++++++++++-- .../connect/planner/SparkConnectPlanner.scala | 1 + .../scala/org/apache/spark/sql/Dataset.scala | 1 + .../analysis/ResolveSessionCatalog.scala | 14 +++-- .../spark/sql/execution/SparkSqlParser.scala | 6 +- .../spark/sql/execution/command/views.scala | 5 +- .../datasources/v2/CacheTableExec.scala | 1 + .../datasources/v2/ShowCreateTableExec.scala | 7 +++ .../datasources/v2/V2SessionCatalog.scala | 6 +- .../spark/sql/internal/CatalogImpl.scala | 1 + .../sql/internal/DataFrameWriterImpl.scala | 3 + .../sql/internal/DataFrameWriterV2Impl.scala | 2 + .../sql/streaming/DataStreamWriter.scala | 1 + .../sql/connector/DataSourceV2SQLSuite.scala | 10 +++- .../V2CommandsCaseSensitivitySuite.scala | 12 ++-- .../AlterTableSetTblPropertiesSuiteBase.scala | 6 +- ...lterTableUnsetTblPropertiesSuiteBase.scala | 6 +- .../command/DDLCommandTestUtils.scala | 6 ++ .../execution/command/DDLParserSuite.scala | 6 ++ .../command/DescribeTableSuiteBase.scala | 25 +++++++++ .../command/v1/DescribeTableSuite.scala | 2 + .../sql/hive/client/HiveClientImpl.scala | 5 ++ .../sql/hive/client/HiveClientSuite.scala | 19 +++++++ 33 files changed, 257 insertions(+), 38 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index e743aa2a744f7..a5d217486bf20 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -236,6 +236,7 @@ statement | ALTER TABLE identifierReference RECOVER PARTITIONS #recoverPartitions | ALTER TABLE identifierReference (clusterBySpec | CLUSTER BY NONE) #alterClusterBy + | ALTER TABLE identifierReference collationSpec #alterTableCollation | DROP TABLE (IF EXISTS)? identifierReference PURGE? #dropTable | DROP VIEW (IF EXISTS)? identifierReference #dropView | CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)? @@ -243,6 +244,7 @@ statement identifierCommentList? (commentSpec | schemaBinding | + collationSpec | (PARTITIONED ON identifierList) | (TBLPROPERTIES propertyList))* AS query #createView @@ -528,6 +530,7 @@ createTableClauses createFileFormat | locationSpec | commentSpec | + collationSpec | (TBLPROPERTIES tableProps=propertyList))* ; @@ -1232,6 +1235,10 @@ colPosition : position=FIRST | position=AFTER afterCol=errorCapturingIdentifier ; +collationSpec + : DEFAULT COLLATION collationName=identifier + ; + collateClause : COLLATE collationName=multipartIdentifier ; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index ba3470f85338c..77dbaa7687b41 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -67,6 +67,11 @@ public interface TableCatalog extends CatalogPlugin { */ String PROP_COMMENT = "comment"; + /** + * A reserved property to specify the collation of the table. + */ + String PROP_COLLATION = "collation"; + /** * A reserved property to specify the provider of the table. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala index cc9979ad4c5e5..05158fbee3de6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala @@ -92,6 +92,7 @@ object ResolveTableSpec extends Rule[LogicalPlan] { options = newOptions.toMap, location = u.location, comment = u.comment, + collation = u.collation, serde = u.serde, external = u.external) withNewSpec(newTableSpec) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index dcd1d3137da3f..32a90833e2e7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -350,6 +350,7 @@ case class CatalogTable( stats: Option[CatalogStatistics] = None, viewText: Option[String] = None, comment: Option[String] = None, + collation: Option[String] = None, unsupportedFeatures: Seq[String] = Seq.empty, tracksPartitionsInCatalog: Boolean = false, schemaPreservesCase: Boolean = true, @@ -546,6 +547,7 @@ case class CatalogTable( provider.foreach(map.put("Provider", _)) bucketSpec.foreach(map ++= _.toLinkedHashMap) comment.foreach(map.put("Comment", _)) + collation.foreach(map.put("Collation", _)) if (tableType == CatalogTableType.VIEW) { viewText.foreach(map.put("View Text", _)) viewOriginalText.foreach(map.put("View Original Text", _)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f37879ecd9356..aa32cc9100515 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin} import org.apache.spark.sql.catalyst.trees.TreePattern.PARAMETER import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils, SparkParserUtils} +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, CollationFactory, DateTimeUtils, IntervalUtils, SparkParserUtils} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTimestamp, stringToTimestampWithoutTimeZone} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog, TableWritePrivilege} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition @@ -3869,6 +3869,16 @@ class AstBuilder extends DataTypeAstBuilder ctx.asScala.headOption.map(visitCommentSpec) } + protected def visitCollationSpecList( + ctx: java.util.List[CollationSpecContext]): Option[String] = { + ctx.asScala.headOption.map(visitCollationSpec) + } + + override def visitCollationSpec(ctx: CollationSpecContext): String = withOrigin(ctx) { + val collationName = ctx.identifier.getText + CollationFactory.fetchCollation(collationName).collationName + } + /** * Create a [[BucketSpec]]. */ @@ -4000,6 +4010,7 @@ class AstBuilder extends DataTypeAstBuilder * - options * - location * - comment + * - collation * - serde * - clusterBySpec * @@ -4008,8 +4019,8 @@ class AstBuilder extends DataTypeAstBuilder * types like `i INT`, which should be appended to the existing table schema. */ type TableClauses = ( - Seq[Transform], Seq[ColumnDefinition], Option[BucketSpec], Map[String, String], - OptionList, Option[String], Option[String], Option[SerdeInfo], Option[ClusterBySpec]) + Seq[Transform], Seq[ColumnDefinition], Option[BucketSpec], Map[String, String], OptionList, + Option[String], Option[String], Option[String], Option[SerdeInfo], Option[ClusterBySpec]) /** * Validate a create table statement and return the [[TableIdentifier]]. @@ -4296,6 +4307,10 @@ class AstBuilder extends DataTypeAstBuilder throw QueryParsingErrors.cannotCleanReservedTablePropertyError( PROP_EXTERNAL, ctx, "please use CREATE EXTERNAL TABLE") case (PROP_EXTERNAL, _) => false + case (PROP_COLLATION, _) if !legacyOn => + throw QueryParsingErrors.cannotCleanReservedTablePropertyError( + PROP_COLLATION, ctx, "please use the DEFAULT COLLATION clause to specify it") + case (PROP_COLLATION, _) => false // It's safe to set whatever table comment, so we don't make it a reserved table property. case (PROP_COMMENT, _) => true case (k, _) => @@ -4475,6 +4490,7 @@ class AstBuilder extends DataTypeAstBuilder checkDuplicateClauses(ctx.createFileFormat, "STORED AS/BY", ctx) checkDuplicateClauses(ctx.rowFormat, "ROW FORMAT", ctx) checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx) + checkDuplicateClauses(ctx.collationSpec(), "DEFAULT COLLATION", ctx) checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) checkDuplicateClauses(ctx.clusterBySpec(), "CLUSTER BY", ctx) checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) @@ -4493,6 +4509,7 @@ class AstBuilder extends DataTypeAstBuilder val location = visitLocationSpecList(ctx.locationSpec()) val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) val comment = visitCommentSpecList(ctx.commentSpec()) + val collation = visitCollationSpecList(ctx.collationSpec()) val serdeInfo = getSerdeInfo(ctx.rowFormat.asScala.toSeq, ctx.createFileFormat.asScala.toSeq, ctx) val clusterBySpec = ctx.clusterBySpec().asScala.headOption.map(visitClusterBySpec) @@ -4507,7 +4524,7 @@ class AstBuilder extends DataTypeAstBuilder } (partTransforms, partCols, bucketSpec, cleanedProperties, cleanedOptions, newLocation, comment, - serdeInfo, clusterBySpec) + collation, serdeInfo, clusterBySpec) } protected def getSerdeInfo( @@ -4567,6 +4584,7 @@ class AstBuilder extends DataTypeAstBuilder * ] * [LOCATION path] * [COMMENT table_comment] + * [DEFAULT COLLATION collation_name] * [TBLPROPERTIES (property_name=property_value, ...)] * * partition_fields: @@ -4580,8 +4598,8 @@ class AstBuilder extends DataTypeAstBuilder val columns = Option(ctx.colDefinitionList()).map(visitColDefinitionList).getOrElse(Nil) val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) - val (partTransforms, partCols, bucketSpec, properties, options, location, - comment, serdeInfo, clusterBySpec) = visitCreateTableClauses(ctx.createTableClauses()) + val (partTransforms, partCols, bucketSpec, properties, options, location, comment, + collation, serdeInfo, clusterBySpec) = visitCreateTableClauses(ctx.createTableClauses()) if (provider.isDefined && serdeInfo.isDefined) { invalidStatement(s"CREATE TABLE ... USING ... ${serdeInfo.get.describe}", ctx) @@ -4599,7 +4617,7 @@ class AstBuilder extends DataTypeAstBuilder clusterBySpec.map(_.asTransform) val tableSpec = UnresolvedTableSpec(properties, provider, options, location, comment, - serdeInfo, external) + collation, serdeInfo, external) Option(ctx.query).map(plan) match { case Some(_) if columns.nonEmpty => @@ -4648,6 +4666,7 @@ class AstBuilder extends DataTypeAstBuilder * ] * [LOCATION path] * [COMMENT table_comment] + * [DEFAULT COLLATION collation_name] * [TBLPROPERTIES (property_name=property_value, ...)] * * partition_fields: @@ -4657,8 +4676,8 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitReplaceTable(ctx: ReplaceTableContext): LogicalPlan = withOrigin(ctx) { val orCreate = ctx.replaceTableHeader().CREATE() != null - val (partTransforms, partCols, bucketSpec, properties, options, location, comment, serdeInfo, - clusterBySpec) = visitCreateTableClauses(ctx.createTableClauses()) + val (partTransforms, partCols, bucketSpec, properties, options, location, comment, collation, + serdeInfo, clusterBySpec) = visitCreateTableClauses(ctx.createTableClauses()) val columns = Option(ctx.colDefinitionList()).map(visitColDefinitionList).getOrElse(Nil) val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) @@ -4672,7 +4691,7 @@ class AstBuilder extends DataTypeAstBuilder clusterBySpec.map(_.asTransform) val tableSpec = UnresolvedTableSpec(properties, provider, options, location, comment, - serdeInfo, external = false) + collation, serdeInfo, external = false) Option(ctx.query).map(plan) match { case Some(_) if columns.nonEmpty => @@ -5078,6 +5097,21 @@ class AstBuilder extends DataTypeAstBuilder } } + /** + * Parse a [[AlterTableCollation]] command. + * + * For example: + * {{{ + * ALTER TABLE table1 DEFAULT COLLATION name + * }}} + */ + override def visitAlterTableCollation(ctx: AlterTableCollationContext): LogicalPlan = + withOrigin(ctx) { + val table = createUnresolvedTable( + ctx.identifierReference, "ALTER TABLE ... DEFAULT COLLATION") + AlterTableCollation(table, visitCollationSpec(ctx.collationSpec())) + } + /** * Parse [[SetViewProperties]] or [[SetTableProperties]] commands. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index 2f5d4b9c86e25..dbd2c0ba8e420 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -261,3 +261,15 @@ case class AlterTableClusterBy( protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = copy(table = newChild) } + +/** + * The logical plan of the ALTER TABLE ... DEFAULT COLLATION name command. + */ +case class AlterTableCollation( + table: LogicalPlan, collation: String) extends AlterTableCommand { + override def changes: Seq[TableChange] = { + Seq(TableChange.setProperty(TableCatalog.PROP_COLLATION, collation)) + } + + protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = copy(table = newChild) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 857522728eaff..85b5e8379d3d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -1338,6 +1338,7 @@ case class CreateView( child: LogicalPlan, userSpecifiedColumns: Seq[(String, Option[String])], comment: Option[String], + collation: Option[String], properties: Map[String, String], originalText: Option[String], query: LogicalPlan, @@ -1486,6 +1487,7 @@ trait TableSpecBase { def provider: Option[String] def location: Option[String] def comment: Option[String] + def collation: Option[String] def serde: Option[SerdeInfo] def external: Boolean } @@ -1496,6 +1498,7 @@ case class UnresolvedTableSpec( optionExpression: OptionList, location: Option[String], comment: Option[String], + collation: Option[String], serde: Option[SerdeInfo], external: Boolean) extends UnaryExpression with Unevaluable with TableSpecBase { @@ -1541,10 +1544,11 @@ case class TableSpec( options: Map[String, String], location: Option[String], comment: Option[String], + collation: Option[String], serde: Option[SerdeInfo], external: Boolean) extends TableSpecBase { def withNewLocation(newLocation: Option[String]): TableSpec = { - TableSpec(properties, provider, options, newLocation, comment, serde, external) + TableSpec(properties, provider, options, newLocation, comment, collation, serde, external) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index e1f114a6170a4..97cc263c56c5f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -53,6 +53,7 @@ private[sql] object CatalogV2Util { */ val TABLE_RESERVED_PROPERTIES = Seq(TableCatalog.PROP_COMMENT, + TableCatalog.PROP_COLLATION, TableCatalog.PROP_LOCATION, TableCatalog.PROP_PROVIDER, TableCatalog.PROP_OWNER, @@ -459,7 +460,7 @@ private[sql] object CatalogV2Util { def convertTableProperties(t: TableSpec): Map[String, String] = { val props = convertTableProperties( t.properties, t.options, t.serde, t.location, t.comment, - t.provider, t.external) + t.collation, t.provider, t.external) withDefaultOwnership(props) } @@ -469,6 +470,7 @@ private[sql] object CatalogV2Util { serdeInfo: Option[SerdeInfo], location: Option[String], comment: Option[String], + collation: Option[String], provider: Option[String], external: Boolean = false): Map[String, String] = { properties ++ @@ -478,6 +480,7 @@ private[sql] object CatalogV2Util { (if (external) Some(TableCatalog.PROP_EXTERNAL -> "true") else None) ++ provider.map(TableCatalog.PROP_PROVIDER -> _) ++ comment.map(TableCatalog.PROP_COMMENT -> _) ++ + collation.map(TableCatalog.PROP_COLLATION -> _) ++ location.map(TableCatalog.PROP_LOCATION -> _) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala index 4a5a607e8a8ae..570ab1338dbf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala @@ -85,6 +85,7 @@ private[sql] object V1Table { TableCatalog.OPTION_PREFIX + key -> value } ++ v1Table.provider.map(TableCatalog.PROP_PROVIDER -> _) ++ v1Table.comment.map(TableCatalog.PROP_COMMENT -> _) ++ + v1Table.collation.map(TableCatalog.PROP_COLLATION -> _) ++ v1Table.storage.locationUri.map { loc => TableCatalog.PROP_LOCATION -> CatalogUtils.URIToString(loc) } ++ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala index 6b034d3dbee09..133670d5fcced 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.ArrayImplicits._ class CreateTablePartitioningValidationSuite extends AnalysisTest { val tableSpec = - UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, false) + UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, None, false) test("CreateTableAsSelect: fail missing top-level column") { val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 5e871208698af..0ec2c80282fc2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2655,7 +2655,7 @@ class DDLParserSuite extends AnalysisTest { val createTableResult = CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, false), false) + OptionList(Seq.empty), None, None, None, None, false), false) // Parse the CREATE TABLE statement twice, swapping the order of the NOT NULL and DEFAULT // options, to make sure that the parser accepts any ordering of these options. comparePlans(parsePlan( @@ -2668,7 +2668,7 @@ class DDLParserSuite extends AnalysisTest { "b STRING NOT NULL DEFAULT 'abc') USING parquet"), ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, false), false)) + OptionList(Seq.empty), None, None, None, None, false), false)) // These ALTER TABLE statements should parse successfully. comparePlans( parsePlan("ALTER TABLE t1 ADD COLUMN x int NOT NULL DEFAULT 42"), @@ -2828,12 +2828,12 @@ class DDLParserSuite extends AnalysisTest { "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithGenerationExpr, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, false), false)) + OptionList(Seq.empty), None, None, None, None, false), false)) comparePlans(parsePlan( "REPLACE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithGenerationExpr, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, false), false)) + OptionList(Seq.empty), None, None, None, None, false), false)) // Two generation expressions checkError( exception = parseException("CREATE TABLE my_tab(a INT, " + @@ -2903,6 +2903,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, + None, false ), false @@ -2925,6 +2926,7 @@ class DDLParserSuite extends AnalysisTest { None, None, None, + None, false ), false @@ -3198,4 +3200,49 @@ class DDLParserSuite extends AnalysisTest { condition = "INTERNAL_ERROR", parameters = Map("message" -> "INSERT OVERWRITE DIRECTORY is not supported.")) } + + test("create table with bad collation name") { + checkError( + exception = internalException("CREATE TABLE t DEFAULT COLLATION XD"), + condition = "COLLATION_INVALID_NAME", + parameters = Map("proposals" -> "id, xh, af", "collationName" -> "XD") + ) + } + + private val testSuppCollations = + Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI", "UNICODE_CI_RTRIM", "sr", "sr_CI_AI") + + test("create table with default collation") { + testSuppCollations.foreach { collation => + comparePlans(parsePlan( + s"CREATE TABLE t (c STRING) USING parquet DEFAULT COLLATION ${collation.toLowerCase()}"), + CreateTable(UnresolvedIdentifier(Seq("t")), + Seq(ColumnDefinition("c", StringType)), + Seq.empty[Transform], + UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), + None, None, Some(collation), None, false), false)) + } + } + + test("replace table with default collation") { + testSuppCollations.foreach { collation => + comparePlans(parsePlan( + s"REPLACE TABLE t (c STRING) USING parquet DEFAULT COLLATION ${collation.toLowerCase()}"), + ReplaceTable(UnresolvedIdentifier(Seq("t")), + Seq(ColumnDefinition("c", StringType)), + Seq.empty[Transform], + UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), + None, None, Some(collation), None, false), false)) + } + } + + test("alter table collation") { + testSuppCollations.foreach { collation => + comparePlans(parsePlan( + s"ALTER TABLE t DEFAULT COLLATION ${collation.toLowerCase()}"), + AlterTableCollation(UnresolvedTable(Seq("t"), + "ALTER TABLE ... DEFAULT COLLATION"), collation) + ) + } + } } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 82dfcf7a3694e..bfb5f8f3fab75 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -2719,6 +2719,7 @@ class SparkConnectPlanner( name = tableIdentifier, userSpecifiedColumns = Nil, comment = None, + collation = None, properties = Map.empty, originalText = None, plan = transformRelation(createView.getInput), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index c8c1bacfb9ded..b9ae0e5b91318 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1618,6 +1618,7 @@ class Dataset[T] private[sql]( name = TableIdentifier(identifier.last), userSpecifiedColumns = Nil, comment = None, + collation = None, properties = Map.empty, originalText = None, plan = logicalPlan, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 5f1ab089cf3e9..fa28a2cb9ead6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -421,11 +421,12 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) AlterViewSchemaBindingCommand(ident, viewSchemaMode) case CreateView(ResolvedIdentifierInSessionCatalog(ident), userSpecifiedColumns, comment, - properties, originalText, child, allowExisting, replace, viewSchemaMode) => + collation, properties, originalText, child, allowExisting, replace, viewSchemaMode) => CreateViewCommand( name = ident, userSpecifiedColumns = userSpecifiedColumns, comment = comment, + collation = collation, properties = properties, originalText = originalText, plan = child, @@ -434,7 +435,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) viewType = PersistedView, viewSchemaMode = viewSchemaMode) - case CreateView(ResolvedIdentifier(catalog, _), _, _, _, _, _, _, _, _) => + case CreateView(ResolvedIdentifier(catalog, _), _, _, _, _, _, _, _, _, _) => throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "views") case ShowViews(ns: ResolvedNamespace, pattern, output) => @@ -508,8 +509,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) storageFormat: CatalogStorageFormat, provider: String): CreateTableV1 = { val tableDesc = buildCatalogTable( - ident, tableSchema, partitioning, tableSpec.properties, provider, - tableSpec.location, tableSpec.comment, storageFormat, tableSpec.external) + ident, tableSchema, partitioning, tableSpec.properties, provider, tableSpec.location, + tableSpec.comment, tableSpec.collation, storageFormat, tableSpec.external) val mode = if (ignoreIfExists) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableV1(tableDesc, mode, query) } @@ -585,6 +586,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) provider: String, location: Option[String], comment: Option[String], + collation: Option[String], storageFormat: CatalogStorageFormat, external: Boolean): CatalogTable = { val tableType = if (external || location.isDefined) { @@ -605,7 +607,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) properties = properties ++ maybeClusterBySpec.map( clusterBySpec => ClusterBySpec.toProperty(schema, clusterBySpec, conf.resolver)), - comment = comment) + comment = comment, + collation = collation + ) } object ResolvedViewIdentifier { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index e15250eb46b5c..8d5ddb2d85c4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -377,7 +377,7 @@ class SparkSqlAstBuilder extends AstBuilder { invalidStatement("CREATE TEMPORARY TABLE IF NOT EXISTS", ctx) } - val (_, _, _, _, options, location, _, _, _) = + val (_, _, _, _, options, location, _, _, _, _) = visitCreateTableClauses(ctx.createTableClauses()) val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText).getOrElse( throw QueryParsingErrors.createTempTableNotSpecifyProviderError(ctx)) @@ -520,6 +520,7 @@ class SparkSqlAstBuilder extends AstBuilder { * * create_view_clauses (order insensitive): * [COMMENT view_comment] + * [DEFAULT COLLATION collation_name] * [TBLPROPERTIES (property_name = property_value, ...)] * }}} */ @@ -529,6 +530,7 @@ class SparkSqlAstBuilder extends AstBuilder { } checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx) + checkDuplicateClauses(ctx.collationSpec(), "DEFAULT COLLATION", ctx) checkDuplicateClauses(ctx.schemaBinding(), "WITH SCHEMA", ctx) checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED ON", ctx) checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) @@ -584,6 +586,7 @@ class SparkSqlAstBuilder extends AstBuilder { withIdentClause(ctx.identifierReference(), UnresolvedIdentifier(_)), userSpecifiedColumns, visitCommentSpecList(ctx.commentSpec()), + visitCollationSpecList(ctx.collationSpec()), properties, Some(originalText), qPlan, @@ -609,6 +612,7 @@ class SparkSqlAstBuilder extends AstBuilder { tableIdentifier, userSpecifiedColumns, visitCommentSpecList(ctx.commentSpec()), + visitCollationSpecList(ctx.collationSpec()), properties, Option(source(ctx.query)), otherPlans.head, 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 a98d9886a2730..d5a72fd6c441a 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 @@ -47,6 +47,7 @@ import org.apache.spark.util.ArrayImplicits._ * @param userSpecifiedColumns the output column names and optional comments specified by users, * can be Nil if not specified. * @param comment the comment of this view. + * @param collation the collation of this view. * @param properties the properties of this view. * @param originalText the original SQL text of this view, can be None if this view is created via * Dataset API. @@ -64,6 +65,7 @@ case class CreateViewCommand( name: TableIdentifier, userSpecifiedColumns: Seq[(String, Option[String])], comment: Option[String], + collation: Option[String], properties: Map[String, String], originalText: Option[String], plan: LogicalPlan, @@ -220,7 +222,8 @@ case class CreateViewCommand( properties = newProperties, viewOriginalText = originalText, viewText = originalText, - comment = comment + comment = comment, + collation = collation ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala index 56c44a1256815..86fa0c8523f1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala @@ -89,6 +89,7 @@ case class CacheTableAsSelectExec( name = TableIdentifier(tempViewName), userSpecifiedColumns = Nil, comment = None, + collation = None, properties = Map.empty, originalText = Some(originalText), plan = query, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala index 37339a34af3db..4195560c5cc1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala @@ -57,6 +57,7 @@ case class ShowCreateTableExec( showTableOptions(builder, tableOptions) showTablePartitioning(table, builder) showTableComment(table, builder) + showTableCollation(table, builder) showTableLocation(table, builder) showTableProperties(table, builder, tableOptions) } @@ -155,6 +156,12 @@ case class ShowCreateTableExec( .foreach(builder.append) } + private def showTableCollation(table: Table, builder: StringBuilder): Unit = { + Option(table.properties.get(TableCatalog.PROP_COLLATION)) + .map("COLLATION '" + escapeSingleQuotedString(_) + "'\n") + .foreach(builder.append) + } + private def concatByMultiLines(iter: Iterable[String]): String = { iter.mkString("(\n ", ",\n ", ")\n") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 22c13fd98ced1..e9927cdcc7a33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -239,7 +239,8 @@ class V2SessionCatalog(catalog: SessionCatalog) maybeClusterBySpec.map( clusterBySpec => ClusterBySpec.toProperty(newSchema, clusterBySpec, conf.resolver)), tracksPartitionsInCatalog = conf.manageFilesourcePartitions, - comment = Option(properties.get(TableCatalog.PROP_COMMENT))) + comment = Option(properties.get(TableCatalog.PROP_COMMENT)), + collation = Option(properties.get(TableCatalog.PROP_COLLATION))) try { catalog.createTable(tableDesc, ignoreIfExists = false) @@ -290,6 +291,7 @@ class V2SessionCatalog(catalog: SessionCatalog) val schema = CatalogV2Util.applySchemaChanges( catalogTable.schema, changes, catalogTable.provider, "ALTER TABLE") val comment = properties.get(TableCatalog.PROP_COMMENT) + val collation = properties.get(TableCatalog.PROP_COLLATION) val owner = properties.getOrElse(TableCatalog.PROP_OWNER, catalogTable.owner) val location = properties.get(TableCatalog.PROP_LOCATION).map(CatalogUtils.stringToURI) val storage = if (location.isDefined) { @@ -303,7 +305,7 @@ class V2SessionCatalog(catalog: SessionCatalog) catalog.alterTable( catalogTable.copy( properties = finalProperties, schema = schema, owner = owner, comment = comment, - storage = storage)) + collation = collation, storage = storage)) } catch { case _: NoSuchTableException => throw QueryCompilationErrors.noSuchTableError(ident) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 64689e75e2e5e..5fd88b417ac44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -685,6 +685,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { optionExpression = newOptions, location = location, comment = { if (description.isEmpty) None else Some(description) }, + collation = None, serde = None, external = tableType == CatalogTableType.EXTERNAL) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala index 16f9fcf77d622..0069062e63078 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala @@ -209,6 +209,7 @@ final class DataFrameWriterImpl[T] private[sql](ds: Dataset[T]) extends DataFram optionExpression = OptionList(Seq.empty), location = extraOptions.get("path"), comment = extraOptions.get(TableCatalog.PROP_COMMENT), + collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false) runCommand(df.sparkSession) { @@ -469,6 +470,7 @@ final class DataFrameWriterImpl[T] private[sql](ds: Dataset[T]) extends DataFram optionExpression = OptionList(Seq.empty), location = extraOptions.get("path"), comment = extraOptions.get(TableCatalog.PROP_COMMENT), + collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false) ReplaceTableAsSelect( @@ -489,6 +491,7 @@ final class DataFrameWriterImpl[T] private[sql](ds: Dataset[T]) extends DataFram optionExpression = OptionList(Seq.empty), location = extraOptions.get("path"), comment = extraOptions.get(TableCatalog.PROP_COMMENT), + collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterV2Impl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterV2Impl.scala index 0a19e6c47afa9..86ea55bc59b7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterV2Impl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterV2Impl.scala @@ -150,6 +150,7 @@ final class DataFrameWriterV2Impl[T] private[sql](table: String, ds: Dataset[T]) optionExpression = OptionList(Seq.empty), location = None, comment = None, + collation = None, serde = None, external = false) runCommand( @@ -215,6 +216,7 @@ final class DataFrameWriterV2Impl[T] private[sql](table: String, ds: Dataset[T]) optionExpression = OptionList(Seq.empty), location = None, comment = None, + collation = None, serde = None, external = false) runCommand(ReplaceTableAsSelect( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index b0233d2c51b75..d41933c6a135c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -175,6 +175,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) extends api.DataStr extraOptions.get("path"), None, None, + None, external = false) val cmd = CreateTable( UnresolvedIdentifier(originalMultipartIdentifier), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 6a659fa6e3ee9..87d0a1ff4e7bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1260,8 +1260,12 @@ class DataSourceV2SQLSuiteV1Filter PROP_OWNER -> "it will be set to the current user", PROP_EXTERNAL -> "please use CREATE EXTERNAL TABLE" ) + val excludedProperties = Set(TableCatalog.PROP_COMMENT, TableCatalog.PROP_COLLATION) + val tableLegacyProperties = CatalogV2Util.TABLE_RESERVED_PROPERTIES + .filterNot(excludedProperties.contains) + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + tableLegacyProperties.foreach { key => Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => Seq("CREATE", "REPLACE").foreach { action => val sqlText = s"$action TABLE testcat.reservedTest (key int) " + @@ -1314,7 +1318,7 @@ class DataSourceV2SQLSuiteV1Filter } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + tableLegacyProperties.foreach { key => Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => withTable("testcat.reservedTest") { Seq("CREATE", "REPLACE").foreach { action => @@ -3389,6 +3393,7 @@ class DataSourceV2SQLSuiteV1Filter |TBLPROPERTIES ('prop1' = '1', 'prop2' = '2') |PARTITIONED BY (a) |LOCATION '/tmp' + |DEFAULT COLLATION sr_CI_AI """.stripMargin) val table = spark.sessionState.catalogManager.v2SessionCatalog.asTableCatalog @@ -3396,6 +3401,7 @@ class DataSourceV2SQLSuiteV1Filter val properties = table.properties assert(properties.get(TableCatalog.PROP_PROVIDER) == "parquet") assert(properties.get(TableCatalog.PROP_COMMENT) == "This is a comment") + assert(properties.get(TableCatalog.PROP_COLLATION) == "sr_CI_AI") assert(properties.get(TableCatalog.PROP_LOCATION) == "file:/tmp") assert(properties.containsKey(TableCatalog.PROP_OWNER)) assert(properties.get(TableCatalog.PROP_EXTERNAL) == "true") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index 5091c72ef96ac..67fca09802139 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -53,7 +53,8 @@ class V2CommandsCaseSensitivitySuite withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { Seq("ID", "iD").foreach { ref => val tableSpec = - UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, false) + UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), + None, None, None, None, false) val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.identity(ref) :: Nil, @@ -77,7 +78,8 @@ class V2CommandsCaseSensitivitySuite withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => val tableSpec = - UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, false) + UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), + None, None, None, None, false) val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.bucket(4, ref) :: Nil, @@ -102,7 +104,8 @@ class V2CommandsCaseSensitivitySuite withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { Seq("ID", "iD").foreach { ref => val tableSpec = - UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, false) + UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), + None, None, None, None, false) val plan = ReplaceTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.identity(ref) :: Nil, @@ -126,7 +129,8 @@ class V2CommandsCaseSensitivitySuite withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => val tableSpec = - UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, false) + UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), + None, None, None, None, false) val plan = ReplaceTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.bucket(4, ref) :: Nil, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala index 52a90497fdd37..9ec63acb1d3a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.internal.SQLConf @@ -89,7 +89,7 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU PROP_EXTERNAL -> "please use CREATE EXTERNAL TABLE" ) withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + tableLegacyProperties.foreach { key => withNamespaceAndTable("ns", "tbl") { t => val sqlText = s"ALTER TABLE $t SET TBLPROPERTIES ('$key'='bar')" checkError( @@ -109,7 +109,7 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + tableLegacyProperties.foreach { key => Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => withNamespaceAndTable("ns", "tbl") { t => sql(s"CREATE TABLE $t (key int) USING parquet $clause ('$key'='bar')") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala index 0013919fca08f..0e9e9d9c60815 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.internal.SQLConf @@ -109,7 +109,7 @@ trait AlterTableUnsetTblPropertiesSuiteBase extends QueryTest with DDLCommandTes PROP_EXTERNAL -> "please use CREATE EXTERNAL TABLE" ) withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + tableLegacyProperties.foreach { key => withNamespaceAndTable("ns", "tbl") { t => val sqlText = s"ALTER TABLE $t UNSET TBLPROPERTIES ('$key')" checkError( @@ -129,7 +129,7 @@ trait AlterTableUnsetTblPropertiesSuiteBase extends QueryTest with DDLCommandTes } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + tableLegacyProperties.foreach { key => Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => withNamespaceAndTable("ns", "tbl") { t => sql(s"CREATE TABLE $t (key int) USING parquet $clause ('$key'='bar')") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandTestUtils.scala index 39f2abd35c2b5..39624a33d8614 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandTestUtils.scala @@ -26,6 +26,7 @@ import org.scalatest.Tag import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.test.SQLTestUtils @@ -172,6 +173,11 @@ trait DDLCommandTestUtils extends SQLTestUtils { FileUtils.copyDirectory(new File(part0Loc), new File(part1Loc)) part1Loc } + + def tableLegacyProperties: Seq[String] = { + val excludedProperties = Set(TableCatalog.PROP_COMMENT, TableCatalog.PROP_COLLATION) + CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(excludedProperties.contains) + } } object DDLCommandTestUtils { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 8b868c0e17230..d38708ab3745c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -498,6 +498,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { UnresolvedIdentifier(Seq("view1")), Seq.empty[(String, Option[String])], None, + None, Map.empty[String, String], Some("SELECT * FROM tab1"), parser.parsePlan("SELECT * FROM tab1"), @@ -513,6 +514,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { Seq("a").asTableIdentifier, Seq.empty[(String, Option[String])], None, + None, Map.empty[String, String], Some("SELECT * FROM tab1"), parser.parsePlan("SELECT * FROM tab1"), @@ -539,6 +541,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |(col1, col3 COMMENT 'hello') |TBLPROPERTIES('prop1Key'="prop1Val") |COMMENT 'BLABLA' + |DEFAULT COLLATION uNiCodE |AS SELECT * FROM tab1 """.stripMargin val parsed1 = parser.parsePlan(v1) @@ -546,6 +549,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { UnresolvedIdentifier(Seq("view1")), Seq("col1" -> None, "col3" -> Some("hello")), Some("BLABLA"), + Some("UNICODE"), Map("prop1Key" -> "prop1Val"), Some("SELECT * FROM tab1"), parser.parsePlan("SELECT * FROM tab1"), @@ -559,6 +563,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { |CREATE OR REPLACE GLOBAL TEMPORARY VIEW a |(col1, col3 COMMENT 'hello') |COMMENT 'BLABLA' + |DEFAULT COLLATION uNiCoDe |AS SELECT * FROM tab1 """.stripMargin val parsed2 = parser.parsePlan(v2) @@ -566,6 +571,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { Seq("a").asTableIdentifier, Seq("col1" -> None, "col3" -> Some("hello")), Some("BLABLA"), + Some("UNICODE"), Map(), Some("SELECT * FROM tab1"), parser.parsePlan("SELECT * FROM tab1"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala index c4e9ff93ef85d..f8d2e9dd3a3cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala @@ -293,4 +293,29 @@ trait DescribeTableSuiteBase extends QueryTest with DDLCommandTestUtils { Row("col1", "string", null))) } } + + Seq(true, false).foreach { hasCollations => + test(s"DESCRIBE TABLE EXTENDED with collation specified = $hasCollations") { + + withNamespaceAndTable("ns", "tbl") { tbl => + val getCollationDescription = () => sql(s"DESCRIBE TABLE EXTENDED $tbl") + .where("col_name = 'Collation'") + + val defaultCollation = if (hasCollations) "DEFAULT COLLATION uNiCoDe" else "" + + sql(s"CREATE TABLE $tbl (id string) $defaultUsing $defaultCollation") + val descriptionDf = getCollationDescription() + + if (hasCollations) { + checkAnswer(descriptionDf, Seq(Row("Collation", "UNICODE", ""))) + } else { + assert(descriptionDf.isEmpty) + } + + sql(s"ALTER TABLE $tbl DEFAULT COLLATION UniCode_cI_rTrIm") + val newDescription = getCollationDescription() + checkAnswer(newDescription, Seq(Row("Collation", "UNICODE_CI_RTRIM", ""))) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index eaf016ac2fa9f..164ac2bff9f63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -218,6 +218,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { " PARTITIONED BY (id)" + " TBLPROPERTIES ('bar'='baz')" + " COMMENT 'this is a test table'" + + " DEFAULT COLLATION unicode" + " LOCATION 'file:/tmp/testcat/table_name'") val descriptionDf = spark.sql(s"DESCRIBE TABLE EXTENDED $tbl") assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( @@ -241,6 +242,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { Row("Type", "EXTERNAL", ""), Row("Provider", getProvider(), ""), Row("Comment", "this is a test table", ""), + Row("Collation", "UNICODE", ""), Row("Table Properties", "[bar=baz]", ""), Row("Location", "file:/tmp/testcat/table_name", ""), Row("Partition Provider", "Catalog", ""))) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index fd4d3220f367d..acc588fb719c2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -518,6 +518,8 @@ private[hive] class HiveClientImpl( val excludedTableProperties = HiveStatisticsProperties ++ Set( // The property value of "comment" is moved to the dedicated field "comment" "comment", + // The property value of "collation" is moved to the dedicated field "collation" + "collation", // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added // in the function toHiveTable. "EXTERNAL" @@ -527,6 +529,7 @@ private[hive] class HiveClientImpl( case (key, _) => excludedTableProperties.contains(key) } val comment = properties.get("comment") + val collation = properties.get("collation") CatalogTable( identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), @@ -569,6 +572,7 @@ private[hive] class HiveClientImpl( properties = filteredProperties, stats = readHiveStats(properties), comment = comment, + collation = collation, // In older versions of Spark(before 2.2.0), we expand the view original text and // store that into `viewExpandedText`, that should be used in view resolution. // We get `viewExpandedText` as viewText, and also get `viewOriginalText` in order to @@ -1212,6 +1216,7 @@ private[hive] object HiveClientImpl extends Logging { table.storage.properties.foreach { case (k, v) => hiveTable.setSerdeParam(k, v) } table.properties.foreach { case (k, v) => hiveTable.setProperty(k, v) } table.comment.foreach { c => hiveTable.setProperty("comment", c) } + table.collation.foreach { c => hiveTable.setProperty("collation", c) } // Hive will expand the view text, so it needs 2 fields: viewOriginalText and viewExpandedText. // Since we don't expand the view text, but only add table properties, we map the `viewText` to // the both fields in hive table. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala index 5c65eb8b12bac..27dc80fbfc173 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, NoSuchDatabaseException, PartitionsAlreadyExistException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.test.TestHiveVersion import org.apache.spark.sql.types.{IntegerType, StructType} @@ -68,11 +69,13 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) { } def table(database: String, tableName: String, + collation: Option[String] = None, tableType: CatalogTableType = CatalogTableType.MANAGED): CatalogTable = { CatalogTable( identifier = TableIdentifier(tableName, Some(database)), tableType = tableType, schema = new StructType().add("key", "int"), + collation = collation, storage = CatalogStorageFormat( locationUri = None, inputFormat = Some(classOf[TextInputFormat].getName), @@ -204,6 +207,22 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) { ignoreIfExists = false) } + test("create/alter table with collations") { + client.createTable(table("default", tableName = "collation_table", + collation = Some("UNICODE")), ignoreIfExists = false) + + val readBack = client.getTable("default", "collation_table") + assert(!readBack.properties.contains(TableCatalog.PROP_COLLATION)) + assert(readBack.collation === Some("UNICODE")) + + client.alterTable("default", "collation_table", + readBack.copy(collation = Some("UNICODE_CI"))) + val alteredTbl = client.getTable("default", "collation_table") + assert(alteredTbl.collation === Some("UNICODE_CI")) + + client.dropTable("default", "collation_table", ignoreIfNotExists = true, purge = true) + } + test("loadTable") { client.loadTable( emptyDir, From d3022e99ce08c68f2a59b279f0cdbe4948365757 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 26 Dec 2024 21:21:47 +0800 Subject: [PATCH 258/438] [SPARK-50672][PYTHON][TESTS] Make `openpyxl` optional in PySpark Tests ### What changes were proposed in this pull request? Make `openpyxl` optional in PySpark Tests ### Why are the changes needed? `openpyxl` is an optional dependency of pandas, also optional to pyspark the test should not fail without it. ### Does this PR introduce _any_ user-facing change? no, test only ### How was this patch tested? manually check ### Was this patch authored or co-authored using generative AI tooling? no Closes #49294 from zhengruifeng/optional_openpyxl. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../pyspark/pandas/tests/io/test_dataframe_conversion.py | 8 +++++++- python/pyspark/testing/utils.py | 3 +++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/python/pyspark/pandas/tests/io/test_dataframe_conversion.py b/python/pyspark/pandas/tests/io/test_dataframe_conversion.py index 7cb997153729e..7a4c635ee2941 100644 --- a/python/pyspark/pandas/tests/io/test_dataframe_conversion.py +++ b/python/pyspark/pandas/tests/io/test_dataframe_conversion.py @@ -26,7 +26,12 @@ from pyspark import pandas as ps from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils from pyspark.testing.sqlutils import SQLTestUtils -from pyspark.testing.utils import have_jinja2, jinja2_requirement_message +from pyspark.testing.utils import ( + have_openpyxl, + openpyxl_requirement_message, + have_jinja2, + jinja2_requirement_message, +) class DataFrameConversionMixin: @@ -87,6 +92,7 @@ def get_excel_dfs(pandas_on_spark_location, pandas_location): "expected": pd.read_excel(pandas_location, index_col=0), } + @unittest.skipIf(not have_openpyxl, openpyxl_requirement_message) def test_to_excel(self): with self.temp_dir() as dirpath: pandas_location = dirpath + "/" + "output1.xlsx" diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index a89add74ca8f8..d5f097065dc5c 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -97,6 +97,9 @@ def have_package(name: str) -> bool: have_jinja2 = have_package("jinja2") jinja2_requirement_message = None if have_jinja2 else "No module named 'jinja2'" +have_openpyxl = have_package("openpyxl") +openpyxl_requirement_message = None if have_openpyxl else "No module named 'openpyxl'" + pandas_requirement_message = None try: from pyspark.sql.pandas.utils import require_minimum_pandas_version From aac494e74c60acfd9abd65386a931fdbbf75c433 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Thu, 26 Dec 2024 13:31:02 -0800 Subject: [PATCH 259/438] [SPARK-50134][SPARK-50130][SQL][CONNECT] Support DataFrame API for SCALAR and EXISTS subqueries in Spark Connect ### What changes were proposed in this pull request? Supports DataFrame API for SCALAR and EXISTS subqueries in Spark Connect. The proto plan will be, using `with_relations`: ``` with_relations [id 10] root: plan [id 9] using subquery expressions holding plan ids reference: refs#1: [id 8] plan for the subquery 1 refs#2: [id 5] plan for the subquery 2 ``` ### Why are the changes needed? DataFrame APIs for SCALAR and EXISTS subqueries are missing in Spark Connect. ### Does this PR introduce _any_ user-facing change? Yes, SCALAR and EXISTS subqueries will be available in Spark Connect. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49182 from ueshin/issues/SPARK-50130/scalar_exists_connect. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../scala/org/apache/spark/sql/Dataset.scala | 68 +-- .../spark/sql/RelationalGroupedDataset.scala | 2 +- .../org/apache/spark/sql/SparkSession.scala | 95 +++- .../spark/sql/TableValuedFunction.scala | 2 +- .../sql/internal/columnNodeSupport.scala | 29 ++ .../spark/sql/DataFrameSubquerySuite.scala | 440 ++++++++++++++++++ .../ColumnNodeToProtoConverterSuite.scala | 1 + .../org/apache/spark/sql/test/QueryTest.scala | 155 ++++++ .../org/apache/spark/sql/test/SQLHelper.scala | 17 + python/pyspark/sql/connect/dataframe.py | 21 +- python/pyspark/sql/connect/expressions.py | 100 +++- python/pyspark/sql/connect/plan.py | 234 ++++++---- .../sql/connect/proto/expressions_pb2.py | 156 ++++--- .../sql/connect/proto/expressions_pb2.pyi | 53 +++ python/pyspark/sql/dataframe.py | 8 +- .../sql/tests/connect/test_parity_subquery.py | 57 ++- python/pyspark/sql/tests/test_subquery.py | 121 ++++- python/pyspark/testing/utils.py | 28 +- .../spark/sql/internal/columnNodes.scala | 69 ++- .../sql/catalyst/analysis/unresolved.scala | 15 + .../sql/catalyst/expressions/subquery.scala | 17 + .../sql/catalyst/trees/TreePatterns.scala | 1 + .../protobuf/spark/connect/expressions.proto | 15 + .../connect/planner/SparkConnectPlanner.scala | 62 ++- .../sql/internal/columnNodeSupport.scala | 2 + .../spark/sql/DataFrameSubquerySuite.scala | 98 ++++ ...ColumnNodeToExpressionConverterSuite.scala | 1 + 27 files changed, 1593 insertions(+), 274 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index ffaa8a70cc7c4..75df538678a3d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions.{struct, to_json} -import org.apache.spark.sql.internal.{ColumnNodeToProtoConverter, DataFrameWriterImpl, DataFrameWriterV2Impl, MergeIntoWriterImpl, ToScalaUDF, UDFAdaptors, UnresolvedAttribute, UnresolvedRegex} +import org.apache.spark.sql.internal.{ColumnNodeToProtoConverter, DataFrameWriterImpl, DataFrameWriterV2Impl, MergeIntoWriterImpl, SubqueryExpressionNode, SubqueryType, ToScalaUDF, UDFAdaptors, UnresolvedAttribute, UnresolvedRegex} import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types.{Metadata, StructType} import org.apache.spark.storage.StorageLevel @@ -288,9 +288,10 @@ class Dataset[T] private[sql] ( /** @inheritdoc */ def stat: DataFrameStatFunctions = new DataFrameStatFunctions(toDF()) - private def buildJoin(right: Dataset[_])(f: proto.Join.Builder => Unit): DataFrame = { + private def buildJoin(right: Dataset[_], cols: Seq[Column] = Seq.empty)( + f: proto.Join.Builder => Unit): DataFrame = { checkSameSparkSession(right) - sparkSession.newDataFrame { builder => + sparkSession.newDataFrame(cols) { builder => val joinBuilder = builder.getJoinBuilder joinBuilder.setLeft(plan.getRoot).setRight(right.plan.getRoot) f(joinBuilder) @@ -334,7 +335,7 @@ class Dataset[T] private[sql] ( /** @inheritdoc */ def join(right: Dataset[_], joinExprs: Column, joinType: String): DataFrame = { - buildJoin(right) { builder => + buildJoin(right, Seq(joinExprs)) { builder => builder .setJoinType(toJoinType(joinType)) .setJoinCondition(joinExprs.expr) @@ -394,7 +395,7 @@ class Dataset[T] private[sql] ( case _ => throw new IllegalArgumentException(s"Unsupported lateral join type $joinType") } - sparkSession.newDataFrame { builder => + sparkSession.newDataFrame(joinExprs.toSeq) { builder => val lateralJoinBuilder = builder.getLateralJoinBuilder lateralJoinBuilder.setLeft(plan.getRoot).setRight(right.plan.getRoot) joinExprs.foreach(c => lateralJoinBuilder.setJoinCondition(c.expr)) @@ -426,7 +427,7 @@ class Dataset[T] private[sql] ( val sortExprs = sortCols.map { c => ColumnNodeToProtoConverter(c.sortOrder).getSortOrder } - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, sortCols) { builder => builder.getSortBuilder .setInput(plan.getRoot) .setIsGlobal(global) @@ -502,7 +503,7 @@ class Dataset[T] private[sql] ( * methods and typed select methods is the encoder used to build the return dataset. */ private def selectUntyped(encoder: AgnosticEncoder[_], cols: Seq[Column]): Dataset[_] = { - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(encoder, cols) { builder => builder.getProjectBuilder .setInput(plan.getRoot) .addAllExpressions(cols.map(_.typedExpr(this.encoder)).asJava) @@ -510,29 +511,32 @@ class Dataset[T] private[sql] ( } /** @inheritdoc */ - def filter(condition: Column): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { - builder => + def filter(condition: Column): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, Seq(condition)) { builder => builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) + } } private def buildUnpivot( ids: Array[Column], valuesOption: Option[Array[Column]], variableColumnName: String, - valueColumnName: String): DataFrame = sparkSession.newDataFrame { builder => - val unpivot = builder.getUnpivotBuilder - .setInput(plan.getRoot) - .addAllIds(ids.toImmutableArraySeq.map(_.expr).asJava) - .setVariableColumnName(variableColumnName) - .setValueColumnName(valueColumnName) - valuesOption.foreach { values => - unpivot.getValuesBuilder - .addAllValues(values.toImmutableArraySeq.map(_.expr).asJava) + valueColumnName: String): DataFrame = { + sparkSession.newDataFrame(ids.toSeq ++ valuesOption.toSeq.flatten) { builder => + val unpivot = builder.getUnpivotBuilder + .setInput(plan.getRoot) + .addAllIds(ids.toImmutableArraySeq.map(_.expr).asJava) + .setVariableColumnName(variableColumnName) + .setValueColumnName(valueColumnName) + valuesOption.foreach { values => + unpivot.getValuesBuilder + .addAllValues(values.toImmutableArraySeq.map(_.expr).asJava) + } } } private def buildTranspose(indices: Seq[Column]): DataFrame = - sparkSession.newDataFrame { builder => + sparkSession.newDataFrame(indices) { builder => val transpose = builder.getTransposeBuilder.setInput(plan.getRoot) indices.foreach { indexColumn => transpose.addIndexColumns(indexColumn.expr) @@ -624,18 +628,15 @@ class Dataset[T] private[sql] ( def transpose(): DataFrame = buildTranspose(Seq.empty) - // TODO(SPARK-50134): Support scalar Subquery API in Spark Connect - // scalastyle:off not.implemented.error.usage /** @inheritdoc */ def scalar(): Column = { - ??? + Column(SubqueryExpressionNode(plan.getRoot, SubqueryType.SCALAR)) } /** @inheritdoc */ def exists(): Column = { - ??? + Column(SubqueryExpressionNode(plan.getRoot, SubqueryType.EXISTS)) } - // scalastyle:on not.implemented.error.usage /** @inheritdoc */ def limit(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => @@ -782,7 +783,7 @@ class Dataset[T] private[sql] ( val aliases = values.zip(names).map { case (value, name) => value.name(name).expr.getAlias } - sparkSession.newDataFrame { builder => + sparkSession.newDataFrame(values) { builder => builder.getWithColumnsBuilder .setInput(plan.getRoot) .addAllAliases(aliases.asJava) @@ -842,10 +843,12 @@ class Dataset[T] private[sql] ( @scala.annotation.varargs def drop(col: Column, cols: Column*): DataFrame = buildDrop(col +: cols) - private def buildDrop(cols: Seq[Column]): DataFrame = sparkSession.newDataFrame { builder => - builder.getDropBuilder - .setInput(plan.getRoot) - .addAllColumns(cols.map(_.expr).asJava) + private def buildDrop(cols: Seq[Column]): DataFrame = { + sparkSession.newDataFrame(cols) { builder => + builder.getDropBuilder + .setInput(plan.getRoot) + .addAllColumns(cols.map(_.expr).asJava) + } } private def buildDropByNames(cols: Seq[String]): DataFrame = sparkSession.newDataFrame { @@ -1015,12 +1018,13 @@ class Dataset[T] private[sql] ( private def buildRepartitionByExpression( numPartitions: Option[Int], - partitionExprs: Seq[Column]): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { - builder => + partitionExprs: Seq[Column]): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, partitionExprs) { builder => val repartitionBuilder = builder.getRepartitionByExpressionBuilder .setInput(plan.getRoot) .addAllPartitionExprs(partitionExprs.map(_.expr).asJava) numPartitions.foreach(repartitionBuilder.setNumPartitions) + } } /** @inheritdoc */ @@ -1152,7 +1156,7 @@ class Dataset[T] private[sql] ( /** @inheritdoc */ @scala.annotation.varargs def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, expr +: exprs) { builder => builder.getCollectMetricsBuilder .setInput(plan.getRoot) .setName(name) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 5bded40b0d132..0944c88a67906 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -45,7 +45,7 @@ class RelationalGroupedDataset private[sql] ( import df.sparkSession.RichColumn protected def toDF(aggExprs: Seq[Column]): DataFrame = { - df.sparkSession.newDataFrame { builder => + df.sparkSession.newDataFrame(groupingExprs ++ aggExprs) { builder => val aggBuilder = builder.getAggregateBuilder .setInput(df.plan.getRoot) groupingExprs.foreach(c => aggBuilder.addGroupingExpressions(c.expr)) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index b6bba8251913f..89519034d07cc 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -46,7 +46,7 @@ import org.apache.spark.sql.connect.client.{ClassFinder, CloseableIterator, Spar import org.apache.spark.sql.connect.client.SparkConnectClient.Configuration import org.apache.spark.sql.connect.client.arrow.ArrowSerializer import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.internal.{CatalogImpl, ConnectRuntimeConfig, SessionCleaner, SessionState, SharedState, SqlApiConf} +import org.apache.spark.sql.internal.{CatalogImpl, ConnectRuntimeConfig, SessionCleaner, SessionState, SharedState, SqlApiConf, SubqueryExpressionNode} import org.apache.spark.sql.internal.ColumnNodeToProtoConverter.{toExpr, toTypedExpr} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.streaming.DataStreamReader @@ -324,20 +324,111 @@ class SparkSession private[sql] ( } } + /** + * Create a DataFrame including the proto plan built by the given function. + * + * @param f + * The function to build the proto plan. + * @return + * The DataFrame created from the proto plan. + */ @Since("4.0.0") @DeveloperApi def newDataFrame(f: proto.Relation.Builder => Unit): DataFrame = { newDataset(UnboundRowEncoder)(f) } + /** + * Create a DataFrame including the proto plan built by the given function. + * + * Use this method when columns are used to create a new DataFrame. When there are columns + * referring to other Dataset or DataFrame, the plan will be wrapped with a `WithRelation`. + * + * {{{ + * with_relations [id 10] + * root: plan [id 9] using columns referring to other Dataset or DataFrame, holding plan ids + * reference: + * refs#1: [id 8] plan for the reference 1 + * refs#2: [id 5] plan for the reference 2 + * }}} + * + * @param cols + * The columns to be used in the DataFrame. + * @param f + * The function to build the proto plan. + * @return + * The DataFrame created from the proto plan. + */ + @Since("4.0.0") + @DeveloperApi + def newDataFrame(cols: Seq[Column])(f: proto.Relation.Builder => Unit): DataFrame = { + newDataset(UnboundRowEncoder, cols)(f) + } + + /** + * Create a Dataset including the proto plan built by the given function. + * + * @param encoder + * The encoder for the Dataset. + * @param f + * The function to build the proto plan. + * @return + * The Dataset created from the proto plan. + */ @Since("4.0.0") @DeveloperApi def newDataset[T](encoder: AgnosticEncoder[T])( f: proto.Relation.Builder => Unit): Dataset[T] = { + newDataset[T](encoder, Seq.empty)(f) + } + + /** + * Create a Dataset including the proto plan built by the given function. + * + * Use this method when columns are used to create a new Dataset. When there are columns + * referring to other Dataset or DataFrame, the plan will be wrapped with a `WithRelation`. + * + * {{{ + * with_relations [id 10] + * root: plan [id 9] using columns referring to other Dataset or DataFrame, holding plan ids + * reference: + * refs#1: [id 8] plan for the reference 1 + * refs#2: [id 5] plan for the reference 2 + * }}} + * + * @param encoder + * The encoder for the Dataset. + * @param cols + * The columns to be used in the DataFrame. + * @param f + * The function to build the proto plan. + * @return + * The Dataset created from the proto plan. + */ + @Since("4.0.0") + @DeveloperApi + def newDataset[T](encoder: AgnosticEncoder[T], cols: Seq[Column])( + f: proto.Relation.Builder => Unit): Dataset[T] = { + val references = cols.flatMap(_.node.collect { case n: SubqueryExpressionNode => + n.relation + }) + val builder = proto.Relation.newBuilder() f(builder) builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) - val plan = proto.Plan.newBuilder().setRoot(builder).build() + + val rootBuilder = if (references.length == 0) { + builder + } else { + val rootBuilder = proto.Relation.newBuilder() + rootBuilder.getWithRelationsBuilder + .setRoot(builder) + .addAllReferences(references.asJava) + rootBuilder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) + rootBuilder + } + + val plan = proto.Plan.newBuilder().setRoot(rootBuilder).build() new Dataset[T](this, plan, encoder) } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala index 4f2687b537862..2a5afd1d58717 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/TableValuedFunction.scala @@ -47,7 +47,7 @@ class TableValuedFunction(sparkSession: SparkSession) extends api.TableValuedFun } private def fn(name: String, args: Seq[Column]): Dataset[Row] = { - sparkSession.newDataFrame { builder => + sparkSession.newDataFrame(args) { builder => builder.getUnresolvedTableValuedFunctionBuilder .setFunctionName(name) .addAllArguments(args.map(toExpr).asJava) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala index 0e8889e19de22..8d57a8d3efd44 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala @@ -167,6 +167,15 @@ object ColumnNodeToProtoConverter extends (ColumnNode => proto.Expression) { case LazyExpression(child, _) => builder.getLazyExpressionBuilder.setChild(apply(child, e)) + case SubqueryExpressionNode(relation, subqueryType, _) => + val b = builder.getSubqueryExpressionBuilder + b.setSubqueryType(subqueryType match { + case SubqueryType.SCALAR => proto.SubqueryExpression.SubqueryType.SUBQUERY_TYPE_SCALAR + case SubqueryType.EXISTS => proto.SubqueryExpression.SubqueryType.SUBQUERY_TYPE_EXISTS + }) + assert(relation.hasCommon && relation.getCommon.hasPlanId) + b.setPlanId(relation.getCommon.getPlanId) + case ProtoColumnNode(e, _) => return e @@ -217,4 +226,24 @@ case class ProtoColumnNode( override val origin: Origin = CurrentOrigin.get) extends ColumnNode { override def sql: String = expr.toString + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty +} + +sealed trait SubqueryType + +object SubqueryType { + case object SCALAR extends SubqueryType + case object EXISTS extends SubqueryType +} + +case class SubqueryExpressionNode( + relation: proto.Relation, + subqueryType: SubqueryType, + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { + override def sql: String = subqueryType match { + case SubqueryType.SCALAR => s"($relation)" + case _ => s"$subqueryType ($relation)" + } + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index 91f60b1fefb91..fc37444f77197 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -17,12 +17,306 @@ package org.apache.spark.sql +import org.apache.spark.{SparkException, SparkRuntimeException} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { import testImplicits._ + val row = identity[(java.lang.Integer, java.lang.Double)](_) + + lazy val l = Seq( + row((1, 2.0)), + row((1, 2.0)), + row((2, 1.0)), + row((2, 1.0)), + row((3, 3.0)), + row((null, null)), + row((null, 5.0)), + row((6, null))).toDF("a", "b") + + lazy val r = Seq( + row((2, 3.0)), + row((2, 3.0)), + row((3, 2.0)), + row((4, 1.0)), + row((null, null)), + row((null, 5.0)), + row((6, null))).toDF("c", "d") + + override def beforeAll(): Unit = { + super.beforeAll() + l.createOrReplaceTempView("l") + r.createOrReplaceTempView("r") + } + + test("noop outer()") { + checkAnswer(spark.range(1).select($"id".outer()), Row(0)) + checkError( + intercept[AnalysisException](spark.range(1).select($"outer_col".outer()).collect()), + "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`outer_col`", "proposal" -> "`id`")) + } + + test("simple uncorrelated scalar subquery") { + checkAnswer( + spark.range(1).select(spark.range(1).select(lit(1)).scalar().as("b")), + sql("select (select 1 as b) as b")) + + checkAnswer( + spark + .range(1) + .select( + spark.range(1).select(spark.range(1).select(lit(1)).scalar() + 1).scalar() + lit(1)), + sql("select (select (select 1) + 1) + 1")) + + // string type + checkAnswer( + spark.range(1).select(spark.range(1).select(lit("s")).scalar().as("b")), + sql("select (select 's' as s) as b")) + } + + test("uncorrelated scalar subquery should return null if there is 0 rows") { + checkAnswer( + spark.range(1).select(spark.range(1).select(lit("s")).limit(0).scalar().as("b")), + sql("select (select 's' as s limit 0) as b")) + } + + test("uncorrelated scalar subquery on a DataFrame generated query") { + withTempView("subqueryData") { + val df = Seq((1, "one"), (2, "two"), (3, "three")).toDF("key", "value") + df.createOrReplaceTempView("subqueryData") + + checkAnswer( + spark + .range(1) + .select( + spark + .table("subqueryData") + .select($"key") + .where($"key" > 2) + .orderBy($"key") + .limit(1) + .scalar() + lit(1)), + sql("select (select key from subqueryData where key > 2 order by key limit 1) + 1")) + + checkAnswer( + spark.range(1).select(-spark.table("subqueryData").select(max($"key")).scalar()), + sql("select -(select max(key) from subqueryData)")) + + checkAnswer( + spark.range(1).select(spark.table("subqueryData").select($"value").limit(0).scalar()), + sql("select (select value from subqueryData limit 0)")) + + checkAnswer( + spark + .range(1) + .select( + spark + .table("subqueryData") + .where($"key" === spark.table("subqueryData").select(max($"key")).scalar() - lit(1)) + .select(min($"value")) + .scalar()), + sql( + "select (select min(value) from subqueryData" + + " where key = (select max(key) from subqueryData) - 1)")) + } + } + + test("correlated scalar subquery in SELECT with outer() function") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") + // We can use the `.outer()` function to wrap either the outer column, or the entire condition, + // or the SQL string of the condition. + Seq($"t1.a" === $"t2.a".outer(), ($"t1.a" === $"t2.a").outer(), expr("t1.a = t2.a").outer()) + .foreach { cond => + checkAnswer( + df1.select($"a", df2.where(cond).select(sum($"b")).scalar().as("sum_b")), + sql("select a, (select sum(b) from l t1 where t1.a = t2.a) sum_b from l t2")) + } + } + + test("correlated scalar subquery in WHERE with outer() function") { + // We can use the `.outer()` function to wrap either the outer column, or the entire condition, + // or the SQL string of the condition. + Seq($"a".outer() === $"c", ($"a" === $"c").outer(), expr("a = c").outer()).foreach { cond => + checkAnswer( + spark.table("l").where($"b" < spark.table("r").where(cond).select(max($"d")).scalar()), + sql("select * from l where b < (select max(d) from r where a = c)")) + } + } + + test("EXISTS predicate subquery with outer() function") { + // We can use the `.outer()` function to wrap either the outer column, or the entire condition, + // or the SQL string of the condition. + Seq($"a".outer() === $"c", ($"a" === $"c").outer(), expr("a = c").outer()).foreach { cond => + checkAnswer( + spark.table("l").where(spark.table("r").where(cond).exists()), + sql("select * from l where exists (select * from r where l.a = r.c)")) + + checkAnswer( + spark.table("l").where(spark.table("r").where(cond).exists() && $"a" <= lit(2)), + sql("select * from l where exists (select * from r where l.a = r.c) and l.a <= 2")) + } + } + + test("SPARK-15677: Queries against local relations with scalar subquery in Select list") { + withTempView("t1", "t2") { + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") + + checkAnswer( + spark.table("t1").select(spark.range(1).select(lit(1).as("col")).scalar()), + sql("SELECT (select 1 as col) from t1")) + + checkAnswer( + spark.table("t1").select(spark.table("t2").select(max($"c1")).scalar()), + sql("SELECT (select max(c1) from t2) from t1")) + + checkAnswer( + spark.table("t1").select(lit(1) + spark.range(1).select(lit(1).as("col")).scalar()), + sql("SELECT 1 + (select 1 as col) from t1")) + + checkAnswer( + spark.table("t1").select($"c1", spark.table("t2").select(max($"c1")).scalar() + $"c2"), + sql("SELECT c1, (select max(c1) from t2) + c2 from t1")) + + checkAnswer( + spark + .table("t1") + .select( + $"c1", + spark.table("t2").where($"t1.c2".outer() === $"t2.c2").select(max($"c1")).scalar()), + sql("SELECT c1, (select max(c1) from t2 where t1.c2 = t2.c2) from t1")) + } + } + + test("NOT EXISTS predicate subquery") { + checkAnswer( + spark.table("l").where(!spark.table("r").where($"a".outer() === $"c").exists()), + sql("select * from l where not exists (select * from r where l.a = r.c)")) + + checkAnswer( + spark + .table("l") + .where(!spark.table("r").where($"a".outer() === $"c" && $"b".outer() < $"d").exists()), + sql("select * from l where not exists (select * from r where l.a = r.c and l.b < r.d)")) + } + + test("EXISTS predicate subquery within OR") { + checkAnswer( + spark + .table("l") + .where(spark.table("r").where($"a".outer() === $"c").exists() || + spark.table("r").where($"a".outer() === $"c").exists()), + sql( + "select * from l where exists (select * from r where l.a = r.c)" + + " or exists (select * from r where l.a = r.c)")) + + checkAnswer( + spark + .table("l") + .where(!spark.table("r").where($"a".outer() === $"c" && $"b".outer() < $"d").exists() || + !spark.table("r").where($"a".outer() === $"c").exists()), + sql( + "select * from l where not exists (select * from r where l.a = r.c and l.b < r.d)" + + " or not exists (select * from r where l.a = r.c)")) + } + + test("correlated scalar subquery in select (null safe equal)") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") + checkAnswer( + df1.select( + $"a", + df2.where($"t2.a" <=> $"t1.a".outer()).select(sum($"b")).scalar().as("sum_b")), + sql("select a, (select sum(b) from l t2 where t2.a <=> t1.a) sum_b from l t1")) + } + + test("correlated scalar subquery in aggregate") { + checkAnswer( + spark + .table("l") + .groupBy( + $"a", + spark.table("r").where($"a".outer() === $"c").select(sum($"d")).scalar().as("sum_d")) + .agg(Map.empty[String, String]), + sql("select a, (select sum(d) from r where a = c) sum_d from l l1 group by 1, 2")) + } + + test("SPARK-34269: correlated subquery with view in aggregate's grouping expression") { + withTable("tr") { + withView("vr") { + r.write.saveAsTable("tr") + sql("create view vr as select * from tr") + checkAnswer( + spark + .table("l") + .groupBy( + $"a", + spark + .table("vr") + .where($"a".outer() === $"c") + .select(sum($"d")) + .scalar() + .as("sum_d")) + .agg(Map.empty[String, String]), + sql("select a, (select sum(d) from vr where a = c) sum_d from l l1 group by 1, 2")) + } + } + } + + test("non-aggregated correlated scalar subquery") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") + val exception1 = intercept[SparkRuntimeException] { + df1 + .select($"a", df2.where($"t1.a" === $"t2.a".outer()).select($"b").scalar().as("sum_b")) + .collect() + } + checkError(exception1, condition = "SCALAR_SUBQUERY_TOO_MANY_ROWS") + } + + test("non-equal correlated scalar subquery") { + val df1 = spark.table("l").as("t1") + val df2 = spark.table("l").as("t2") + checkAnswer( + df1.select( + $"a", + df2.where($"t2.a" < $"t1.a".outer()).select(sum($"b")).scalar().as("sum_b")), + sql("select a, (select sum(b) from l t2 where t2.a < t1.a) sum_b from l t1")) + } + + test("disjunctive correlated scalar subquery") { + checkAnswer( + spark + .table("l") + .where( + spark + .table("r") + .where(($"a".outer() === $"c" && $"d" === 2.0) || + ($"a".outer() === $"c" && $"d" === 1.0)) + .select(count(lit(1))) + .scalar() > 0) + .select($"a"), + sql(""" + |select a + |from l + |where (select count(*) + | from r + | where (a = c and d = 2.0) or (a = c and d = 1.0)) > 0 + """.stripMargin)) + } + + test("correlated scalar subquery with missing outer reference") { + checkAnswer( + spark + .table("l") + .select($"a", spark.table("r").where($"c" === $"a").select(sum($"d")).scalar()), + sql("select a, (select sum(d) from r where c = a) from l")) + } + private def table1() = { sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") spark.table("t1") @@ -182,6 +476,60 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { } } + test("scalar subquery inside lateral join") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + // uncorrelated + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c2".outer(), t2.select(min($"c2")).scalar())) + .toDF("c1", "c2", "c3", "c4"), + sql("SELECT * FROM t1, LATERAL (SELECT c2, (SELECT MIN(c2) FROM t2))") + .toDF("c1", "c2", "c3", "c4")) + + // correlated + checkAnswer( + t1.lateralJoin( + spark + .range(1) + .select($"c1".outer().as("a")) + .select(t2.where($"c1" === $"a".outer()).select(sum($"c2")).scalar())), + sql(""" + |SELECT * FROM t1, LATERAL ( + | SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = a) FROM (SELECT c1 AS a) + |) + |""".stripMargin)) + } + } + + test("lateral join inside subquery") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + // uncorrelated + checkAnswer( + t1.where( + $"c1" === t2 + .lateralJoin(spark.range(1).select($"c1".outer().as("a"))) + .select(min($"a")) + .scalar()), + sql("SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a))")) + // correlated + checkAnswer( + t1.where( + $"c1" === t2 + .lateralJoin(spark.range(1).select($"c1".outer().as("a"))) + .where($"c1" === $"t1.c1".outer()) + .select(min($"a")) + .scalar()), + sql( + "SELECT * FROM t1 " + + "WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a) WHERE c1 = t1.c1)")) + } + } + test("lateral join with table-valued functions") { withView("t1", "t3") { val t1 = table1() @@ -219,4 +567,96 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { sql("SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3")) } } + + test("subquery with generator / table-valued functions") { + withView("t1") { + val t1 = table1() + + checkAnswer( + spark.range(1).select(explode(t1.select(collect_list("c2")).scalar())), + sql("SELECT EXPLODE((SELECT COLLECT_LIST(c2) FROM t1))")) + checkAnswer( + spark.tvf.explode(t1.select(collect_list("c2")).scalar()), + sql("SELECT * FROM EXPLODE((SELECT COLLECT_LIST(c2) FROM t1))")) + } + } + + test("subquery in join condition") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.join(t2, $"t1.c1" === t1.select(max("c1")).scalar()).toDF("c1", "c2", "c3", "c4"), + sql("SELECT * FROM t1 JOIN t2 ON t1.c1 = (SELECT MAX(c1) FROM t1)") + .toDF("c1", "c2", "c3", "c4")) + } + } + + test("subquery in unpivot") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkError( + intercept[AnalysisException] { + t1.unpivot(Array(t2.exists()), "c1", "c2").collect() + }, + "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_IN_EXISTS_SUBQUERY", + parameters = Map("treeNode" -> "(?s)'Unpivot.*"), + matchPVals = true) + checkError( + intercept[AnalysisException] { + t1.unpivot(Array($"c1"), Array(t2.exists()), "c1", "c2").collect() + }, + "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_IN_EXISTS_SUBQUERY", + parameters = Map("treeNode" -> "(?s)Expand.*"), + matchPVals = true) + } + } + + test("subquery in transpose") { + withView("t1") { + val t1 = table1() + + checkError( + intercept[AnalysisException] { + t1.transpose(t1.select(max("c1")).scalar()).collect() + }, + "TRANSPOSE_INVALID_INDEX_COLUMN", + parameters = Map("reason" -> "Index column must be an atomic attribute")) + } + } + + test("subquery in withColumns") { + withView("t1") { + val t1 = table1() + + // TODO(SPARK-50601): Fix the SparkConnectPlanner to support this case + checkError( + intercept[SparkException] { + t1.withColumn("scalar", spark.range(1).select($"c1".outer() + $"c2".outer()).scalar()) + .collect() + }, + "INTERNAL_ERROR", + parameters = Map("message" -> "Found the unresolved operator: .*"), + matchPVals = true) + } + } + + test("subquery in drop") { + withView("t1") { + val t1 = table1() + + checkAnswer(t1.drop(spark.range(1).select(lit("c1")).scalar()), t1) + } + } + + test("subquery in repartition") { + withView("t1") { + val t1 = table1() + + checkAnswer(t1.repartition(spark.range(1).select(lit(1)).scalar()), t1) + } + } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala index 2efd396735191..4cb03420c4d07 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala @@ -431,4 +431,5 @@ class ColumnNodeToProtoConverterSuite extends ConnectFunSuite { private[internal] case class Nope(override val origin: Origin = CurrentOrigin.get) extends ColumnNode { override def sql: String = "nope" + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/QueryTest.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/QueryTest.scala index 8837c76b76aeb..f22644074324c 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/QueryTest.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/QueryTest.scala @@ -19,8 +19,11 @@ package org.apache.spark.sql.test import java.util.TimeZone +import scala.jdk.CollectionConverters._ + import org.scalatest.Assertions +import org.apache.spark.{QueryContextType, SparkThrowable} import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.util.SparkStringUtils.sideBySide import org.apache.spark.util.ArrayImplicits._ @@ -53,6 +56,158 @@ abstract class QueryTest extends ConnectFunSuite with SQLHelper { checkAnswer(df, expectedAnswer.toImmutableArraySeq) } + case class ExpectedContext( + contextType: QueryContextType, + objectType: String, + objectName: String, + startIndex: Int, + stopIndex: Int, + fragment: String, + callSitePattern: String) + + object ExpectedContext { + def apply(fragment: String, start: Int, stop: Int): ExpectedContext = { + ExpectedContext("", "", start, stop, fragment) + } + + def apply( + objectType: String, + objectName: String, + startIndex: Int, + stopIndex: Int, + fragment: String): ExpectedContext = { + new ExpectedContext( + QueryContextType.SQL, + objectType, + objectName, + startIndex, + stopIndex, + fragment, + "") + } + + def apply(fragment: String, callSitePattern: String): ExpectedContext = { + new ExpectedContext(QueryContextType.DataFrame, "", "", -1, -1, fragment, callSitePattern) + } + } + + /** + * Checks an exception with an error condition against expected results. + * @param exception + * The exception to check + * @param condition + * The expected error condition identifying the error + * @param sqlState + * Optional the expected SQLSTATE, not verified if not supplied + * @param parameters + * A map of parameter names and values. The names are as defined in the error-classes file. + * @param matchPVals + * Optionally treat the parameters value as regular expression pattern. false if not supplied. + */ + protected def checkError( + exception: SparkThrowable, + condition: String, + sqlState: Option[String] = None, + parameters: Map[String, String] = Map.empty, + matchPVals: Boolean = false, + queryContext: Array[ExpectedContext] = Array.empty): Unit = { + assert(exception.getCondition === condition) + sqlState.foreach(state => assert(exception.getSqlState === state)) + val expectedParameters = exception.getMessageParameters.asScala + if (matchPVals) { + assert(expectedParameters.size === parameters.size) + expectedParameters.foreach(exp => { + val parm = parameters.getOrElse( + exp._1, + throw new IllegalArgumentException("Missing parameter" + exp._1)) + if (!exp._2.matches(parm)) { + throw new IllegalArgumentException( + "For parameter '" + exp._1 + "' value '" + exp._2 + + "' does not match: " + parm) + } + }) + } else { + assert(expectedParameters === parameters) + } + val actualQueryContext = exception.getQueryContext() + assert( + actualQueryContext.length === queryContext.length, + "Invalid length of the query context") + actualQueryContext.zip(queryContext).foreach { case (actual, expected) => + assert( + actual.contextType() === expected.contextType, + "Invalid contextType of a query context Actual:" + actual.toString) + if (actual.contextType() == QueryContextType.SQL) { + assert( + actual.objectType() === expected.objectType, + "Invalid objectType of a query context Actual:" + actual.toString) + assert( + actual.objectName() === expected.objectName, + "Invalid objectName of a query context. Actual:" + actual.toString) + assert( + actual.startIndex() === expected.startIndex, + "Invalid startIndex of a query context. Actual:" + actual.toString) + assert( + actual.stopIndex() === expected.stopIndex, + "Invalid stopIndex of a query context. Actual:" + actual.toString) + assert( + actual.fragment() === expected.fragment, + "Invalid fragment of a query context. Actual:" + actual.toString) + } else if (actual.contextType() == QueryContextType.DataFrame) { + assert( + actual.fragment() === expected.fragment, + "Invalid code fragment of a query context. Actual:" + actual.toString) + if (expected.callSitePattern.nonEmpty) { + assert( + actual.callSite().matches(expected.callSitePattern), + "Invalid callSite of a query context. Actual:" + actual.toString) + } + } + } + } + + protected def checkError( + exception: SparkThrowable, + condition: String, + sqlState: String, + parameters: Map[String, String]): Unit = + checkError(exception, condition, Some(sqlState), parameters) + + protected def checkError( + exception: SparkThrowable, + condition: String, + sqlState: String, + parameters: Map[String, String], + context: ExpectedContext): Unit = + checkError(exception, condition, Some(sqlState), parameters, false, Array(context)) + + protected def checkError( + exception: SparkThrowable, + condition: String, + parameters: Map[String, String], + context: ExpectedContext): Unit = + checkError(exception, condition, None, parameters, false, Array(context)) + + protected def checkError( + exception: SparkThrowable, + condition: String, + sqlState: String, + context: ExpectedContext): Unit = + checkError(exception, condition, Some(sqlState), Map.empty, false, Array(context)) + + protected def checkError( + exception: SparkThrowable, + condition: String, + sqlState: Option[String], + parameters: Map[String, String], + context: ExpectedContext): Unit = + checkError(exception, condition, sqlState, parameters, false, Array(context)) + + protected def getCurrentClassCallSitePattern: String = { + val cs = Thread.currentThread().getStackTrace()(2) + s"${cs.getClassName}\\..*\\(${cs.getFileName}:\\d+\\)" + } + /** * Evaluates a dataset to make sure that the result of calling collect matches the given * expected answer. diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala index 007d4f0648e44..d9828ae92267b 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala @@ -22,6 +22,7 @@ import java.util.UUID import org.scalatest.Assertions.fail import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession, SQLImplicits} +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.util.{SparkErrorUtils, SparkFileUtils} trait SQLHelper { @@ -110,6 +111,22 @@ trait SQLHelper { finally SparkFileUtils.deleteRecursively(path) } + /** + * Drops temporary view `viewNames` after calling `f`. + */ + protected def withTempView(viewNames: String*)(f: => Unit): Unit = { + SparkErrorUtils.tryWithSafeFinally(f) { + viewNames.foreach { viewName => + try spark.catalog.dropTempView(viewName) + catch { + // If the test failed part way, we don't want to mask the failure by failing to remove + // temp views that never got created. + case _: NoSuchTableException => + } + } + } + } + /** * Drops table `tableName` after calling `f`. */ diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index ee1886b8ef290..185ddc88cd08b 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -79,6 +79,7 @@ from pyspark.sql.column import Column from pyspark.sql.connect.expressions import ( ColumnReference, + SubqueryExpression, UnresolvedRegex, UnresolvedStar, ) @@ -1801,18 +1802,14 @@ def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> ParentDataF ) def scalar(self) -> Column: - # TODO(SPARK-50134): Implement this method - raise PySparkNotImplementedError( - errorClass="NOT_IMPLEMENTED", - messageParameters={"feature": "scalar()"}, - ) + from pyspark.sql.connect.column import Column as ConnectColumn + + return ConnectColumn(SubqueryExpression(self._plan, subquery_type="scalar")) def exists(self) -> Column: - # TODO(SPARK-50134): Implement this method - raise PySparkNotImplementedError( - errorClass="NOT_IMPLEMENTED", - messageParameters={"feature": "exists()"}, - ) + from pyspark.sql.connect.column import Column as ConnectColumn + + return ConnectColumn(SubqueryExpression(self._plan, subquery_type="exists")) @property def schema(self) -> StructType: @@ -2278,10 +2275,6 @@ def _test() -> None: del pyspark.sql.dataframe.DataFrame.toJSON.__doc__ del pyspark.sql.dataframe.DataFrame.rdd.__doc__ - # TODO(SPARK-50134): Support subquery in connect - del pyspark.sql.dataframe.DataFrame.scalar.__doc__ - del pyspark.sql.dataframe.DataFrame.exists.__doc__ - globs["spark"] = ( PySparkSession.builder.appName("sql.connect.dataframe tests") .remote(os.environ.get("SPARK_CONNECT_TESTING_REMOTE", "local[4]")) diff --git a/python/pyspark/sql/connect/expressions.py b/python/pyspark/sql/connect/expressions.py index 5d7b348f6d383..413a69181683b 100644 --- a/python/pyspark/sql/connect/expressions.py +++ b/python/pyspark/sql/connect/expressions.py @@ -82,6 +82,7 @@ if TYPE_CHECKING: from pyspark.sql.connect.client import SparkConnectClient from pyspark.sql.connect.window import WindowSpec + from pyspark.sql.connect.plan import LogicalPlan class Expression: @@ -128,6 +129,15 @@ def _create_proto_expression(self) -> proto.Expression: plan.common.origin.CopyFrom(self.origin) return plan + @property + def children(self) -> Sequence["Expression"]: + return [] + + def foreach(self, f: Callable[["Expression"], None]) -> None: + f(self) + for c in self.children: + c.foreach(f) + class CaseWhen(Expression): def __init__( @@ -162,6 +172,16 @@ def to_plan(self, session: "SparkConnectClient") -> "proto.Expression": return unresolved_function.to_plan(session) + @property + def children(self) -> Sequence["Expression"]: + children = [] + for branch in self._branches: + children.append(branch[0]) + children.append(branch[1]) + if self._else_value is not None: + children.append(self._else_value) + return children + def __repr__(self) -> str: _cases = "".join([f" WHEN {c} THEN {v}" for c, v in self._branches]) _else = f" ELSE {self._else_value}" if self._else_value is not None else "" @@ -196,6 +216,10 @@ def to_plan(self, session: "SparkConnectClient") -> "proto.Expression": exp.alias.expr.CopyFrom(self._child.to_plan(session)) return exp + @property + def children(self) -> Sequence["Expression"]: + return [self._child] + def __repr__(self) -> str: return f"{self._child} AS {','.join(self._alias)}" @@ -622,6 +646,10 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: return sort + @property + def children(self) -> Sequence["Expression"]: + return [self._child] + class UnresolvedFunction(Expression): def __init__( @@ -649,6 +677,10 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: fun.unresolved_function.is_distinct = self._is_distinct return fun + @property + def children(self) -> Sequence["Expression"]: + return self._args + def __repr__(self) -> str: # Default print handling: if self._is_distinct: @@ -730,12 +762,12 @@ def __init__( function_name: str, function: Union[PythonUDF, JavaUDF], deterministic: bool = False, - arguments: Sequence[Expression] = [], + arguments: Optional[Sequence[Expression]] = None, ): super().__init__() self._function_name = function_name self._deterministic = deterministic - self._arguments = arguments + self._arguments: Sequence[Expression] = arguments or [] self._function = function def to_plan(self, session: "SparkConnectClient") -> "proto.Expression": @@ -770,6 +802,10 @@ def to_plan_judf( expr.java_udf.CopyFrom(cast(proto.JavaUDF, self._function.to_plan(session))) return expr + @property + def children(self) -> Sequence["Expression"]: + return self._arguments + def __repr__(self) -> str: return f"{self._function_name}({', '.join([str(arg) for arg in self._arguments])})" @@ -799,6 +835,10 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: expr.update_fields.value_expression.CopyFrom(self._valueExpr.to_plan(session)) return expr + @property + def children(self) -> Sequence["Expression"]: + return [self._structExpr, self._valueExpr] + def __repr__(self) -> str: return f"update_field({self._structExpr}, {self._fieldName}, {self._valueExpr})" @@ -823,6 +863,10 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: expr.update_fields.field_name = self._fieldName return expr + @property + def children(self) -> Sequence["Expression"]: + return [self._structExpr] + def __repr__(self) -> str: return f"drop_field({self._structExpr}, {self._fieldName})" @@ -847,6 +891,10 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: expr.unresolved_extract_value.extraction.CopyFrom(self._extraction.to_plan(session)) return expr + @property + def children(self) -> Sequence["Expression"]: + return [self._child, self._extraction] + def __repr__(self) -> str: return f"{self._child}['{self._extraction}']" @@ -906,6 +954,10 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: return fun + @property + def children(self) -> Sequence["Expression"]: + return [self._expr] + def __repr__(self) -> str: # We cannot guarantee the string representations be exactly the same, e.g. # str(sf.col("a").cast("long")): @@ -989,6 +1041,10 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: ) return expr + @property + def children(self) -> Sequence["Expression"]: + return [self._function] + self._arguments + def __repr__(self) -> str: return ( f"LambdaFunction({str(self._function)}, " @@ -1098,6 +1154,12 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: return expr + @property + def children(self) -> Sequence["Expression"]: + return ( + [self._windowFunction] + self._windowSpec._partitionSpec + self._windowSpec._orderSpec + ) + def __repr__(self) -> str: return f"WindowExpression({str(self._windowFunction)}, ({str(self._windowSpec)}))" @@ -1128,6 +1190,10 @@ def to_plan(self, session: "SparkConnectClient") -> "proto.Expression": expr.call_function.arguments.extend([arg.to_plan(session) for arg in self._args]) return expr + @property + def children(self) -> Sequence["Expression"]: + return self._args + def __repr__(self) -> str: if len(self._args) > 0: return f"CallFunction('{self._name}', {', '.join([str(arg) for arg in self._args])})" @@ -1151,6 +1217,10 @@ def to_plan(self, session: "SparkConnectClient") -> "proto.Expression": expr.named_argument_expression.value.CopyFrom(self._value.to_plan(session)) return expr + @property + def children(self) -> Sequence["Expression"]: + return [self._value] + def __repr__(self) -> str: return f"{self._key} => {self._value}" @@ -1166,5 +1236,31 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: expr.lazy_expression.child.CopyFrom(self._expr.to_plan(session)) return expr + @property + def children(self) -> Sequence["Expression"]: + return [self._expr] + def __repr__(self) -> str: return f"lazy({self._expr})" + + +class SubqueryExpression(Expression): + def __init__(self, plan: "LogicalPlan", subquery_type: str) -> None: + assert isinstance(subquery_type, str) + assert subquery_type in ("scalar", "exists") + + super().__init__() + self._plan = plan + self._subquery_type = subquery_type + + def to_plan(self, session: "SparkConnectClient") -> proto.Expression: + expr = self._create_proto_expression() + expr.subquery_expression.plan_id = self._plan._plan_id + if self._subquery_type == "scalar": + expr.subquery_expression.subquery_type = proto.SubqueryExpression.SUBQUERY_TYPE_SCALAR + elif self._subquery_type == "exists": + expr.subquery_expression.subquery_type = proto.SubqueryExpression.SUBQUERY_TYPE_EXISTS + return expr + + def __repr__(self) -> str: + return f"SubqueryExpression({self._plan}, {self._subquery_type})" diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index c411baf17ce94..02b60381ab939 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -52,7 +52,7 @@ from pyspark.sql.connect.logging import logger from pyspark.sql.connect.proto import base_pb2 as spark_dot_connect_dot_base__pb2 from pyspark.sql.connect.conversion import storage_level_to_proto -from pyspark.sql.connect.expressions import Expression +from pyspark.sql.connect.expressions import Expression, SubqueryExpression from pyspark.sql.connect.types import pyspark_types_to_proto_types, UnparsedDataType from pyspark.errors import ( AnalysisException, @@ -73,9 +73,30 @@ class LogicalPlan: INDENT = 2 - def __init__(self, child: Optional["LogicalPlan"]) -> None: + def __init__( + self, child: Optional["LogicalPlan"], references: Optional[Sequence["LogicalPlan"]] = None + ) -> None: + """ + + Parameters + ---------- + child : :class:`LogicalPlan`, optional. + The child logical plan. + references : list of :class:`LogicalPlan`, optional. + The list of logical plans that are referenced as subqueries in this logical plan. + """ self._child = child - self._plan_id = LogicalPlan._fresh_plan_id() + self._root_plan_id = LogicalPlan._fresh_plan_id() + + self._references: Sequence["LogicalPlan"] = references or [] + self._plan_id_with_rel: Optional[int] = None + if len(self._references) > 0: + assert all(isinstance(r, LogicalPlan) for r in self._references) + self._plan_id_with_rel = LogicalPlan._fresh_plan_id() + + @property + def _plan_id(self) -> int: + return self._plan_id_with_rel or self._root_plan_id @staticmethod def _fresh_plan_id() -> int: @@ -89,7 +110,7 @@ def _fresh_plan_id() -> int: def _create_proto_relation(self) -> proto.Relation: plan = proto.Relation() - plan.common.plan_id = self._plan_id + plan.common.plan_id = self._root_plan_id return plan def plan(self, session: "SparkConnectClient") -> proto.Relation: # type: ignore[empty-body] @@ -136,6 +157,42 @@ def observations(self) -> Dict[str, "Observation"]: else: return self._child.observations + @staticmethod + def _collect_references( + cols_or_exprs: Sequence[Union[Column, Expression]] + ) -> Sequence["LogicalPlan"]: + references: List[LogicalPlan] = [] + + def append_reference(e: Expression) -> None: + if isinstance(e, SubqueryExpression): + references.append(e._plan) + + for col_or_expr in cols_or_exprs: + if isinstance(col_or_expr, Column): + col_or_expr._expr.foreach(append_reference) + else: + col_or_expr.foreach(append_reference) + return references + + def _with_relations( + self, root: proto.Relation, session: "SparkConnectClient" + ) -> proto.Relation: + if len(self._references) == 0: + return root + else: + # When there are references to other DataFrame, e.g., subqueries, build new plan like: + # with_relations [id 10] + # root: plan [id 9] + # reference: + # refs#1: [id 8] + # refs#2: [id 5] + plan = proto.Relation() + assert isinstance(self._plan_id_with_rel, int) + plan.common.plan_id = self._plan_id_with_rel + plan.with_relations.root.CopyFrom(root) + plan.with_relations.references.extend([ref.plan(session) for ref in self._references]) + return plan + def _parameters_to_print(self, parameters: Mapping[str, Any]) -> Mapping[str, Any]: """ Extracts the parameters that are able to be printed. It looks up the signature @@ -192,6 +249,7 @@ def _parameters_to_print(self, parameters: Mapping[str, Any]) -> Mapping[str, An getattr(a, "__forward_arg__", "").endswith("LogicalPlan") for a in getattr(tpe.annotation, "__args__", ()) ) + if ( not is_logical_plan and not is_forwardref_logical_plan @@ -473,8 +531,8 @@ def __init__( child: Optional["LogicalPlan"], columns: List[Column], ) -> None: - super().__init__(child) assert all(isinstance(c, Column) for c in columns) + super().__init__(child, self._collect_references(columns)) self._columns = columns def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -482,7 +540,8 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan = self._create_proto_relation() plan.project.input.CopyFrom(self._child.plan(session)) plan.project.expressions.extend([c.to_plan(session) for c in self._columns]) - return plan + + return self._with_relations(plan, session) class WithColumns(LogicalPlan): @@ -495,8 +554,6 @@ def __init__( columns: Sequence[Column], metadata: Optional[Sequence[str]] = None, ) -> None: - super().__init__(child) - assert isinstance(columnNames, list) assert len(columnNames) > 0 assert all(isinstance(c, str) for c in columnNames) @@ -513,6 +570,8 @@ def __init__( # validate json string assert m == "" or json.loads(m) is not None + super().__init__(child, self._collect_references(columns)) + self._columnNames = columnNames self._columns = columns self._metadata = metadata @@ -530,7 +589,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: alias.metadata = self._metadata[i] plan.with_columns.aliases.append(alias) - return plan + return self._with_relations(plan, session) class WithWatermark(LogicalPlan): @@ -608,16 +667,14 @@ def __init__( name: str, parameters: Sequence[Column], ) -> None: - super().__init__(child) - assert isinstance(name, str) - self._name = name - assert parameters is not None and isinstance(parameters, List) for param in parameters: assert isinstance(param, Column) + super().__init__(child, self._collect_references(parameters)) + self._name = name self._parameters = parameters def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -626,12 +683,12 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.hint.input.CopyFrom(self._child.plan(session)) plan.hint.name = self._name plan.hint.parameters.extend([param.to_plan(session) for param in self._parameters]) - return plan + return self._with_relations(plan, session) class Filter(LogicalPlan): def __init__(self, child: Optional["LogicalPlan"], filter: Column) -> None: - super().__init__(child) + super().__init__(child, self._collect_references([filter])) self.filter = filter def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -639,7 +696,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan = self._create_proto_relation() plan.filter.input.CopyFrom(self._child.plan(session)) plan.filter.condition.CopyFrom(self.filter.to_plan(session)) - return plan + return self._with_relations(plan, session) class Limit(LogicalPlan): @@ -712,11 +769,10 @@ def __init__( columns: List[Column], is_global: bool, ) -> None: - super().__init__(child) - assert all(isinstance(c, Column) for c in columns) assert isinstance(is_global, bool) + super().__init__(child, self._collect_references(columns)) self.columns = columns self.is_global = is_global @@ -726,7 +782,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.sort.input.CopyFrom(self._child.plan(session)) plan.sort.order.extend([c.to_plan(session).sort_order for c in self.columns]) plan.sort.is_global = self.is_global - return plan + return self._with_relations(plan, session) class Drop(LogicalPlan): @@ -735,9 +791,12 @@ def __init__( child: Optional["LogicalPlan"], columns: List[Union[Column, str]], ) -> None: - super().__init__(child) if len(columns) > 0: assert all(isinstance(c, (Column, str)) for c in columns) + + super().__init__( + child, self._collect_references([c for c in columns if isinstance(c, Column)]) + ) self._columns = columns def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -749,7 +808,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.drop.columns.append(c.to_plan(session)) else: plan.drop.column_names.append(c) - return plan + return self._with_relations(plan, session) class Sample(LogicalPlan): @@ -792,8 +851,6 @@ def __init__( pivot_values: Optional[Sequence[Column]], grouping_sets: Optional[Sequence[Sequence[Column]]], ) -> None: - super().__init__(child) - assert isinstance(group_type, str) and group_type in [ "groupby", "rollup", @@ -801,15 +858,12 @@ def __init__( "pivot", "grouping_sets", ] - self._group_type = group_type assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) - self._grouping_cols = grouping_cols assert isinstance(aggregate_cols, list) and all( isinstance(c, Column) for c in aggregate_cols ) - self._aggregate_cols = aggregate_cols if group_type == "pivot": assert pivot_col is not None and isinstance(pivot_col, Column) @@ -821,6 +875,19 @@ def __init__( assert pivot_values is None assert grouping_sets is None + super().__init__( + child, + self._collect_references( + grouping_cols + + aggregate_cols + + ([pivot_col] if pivot_col is not None else []) + + (pivot_values if pivot_values is not None else []) + + ([g for gs in grouping_sets for g in gs] if grouping_sets is not None else []) + ), + ) + self._group_type = group_type + self._grouping_cols = grouping_cols + self._aggregate_cols = aggregate_cols self._pivot_col = pivot_col self._pivot_values = pivot_values self._grouping_sets = grouping_sets @@ -859,7 +926,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: grouping_set=[c.to_plan(session) for c in grouping_set] ) ) - return plan + return self._with_relations(plan, session) class Join(LogicalPlan): @@ -870,7 +937,16 @@ def __init__( on: Optional[Union[str, List[str], Column, List[Column]]], how: Optional[str], ) -> None: - super().__init__(left) + super().__init__( + left, + self._collect_references( + [] + if on is None or isinstance(on, str) + else [on] + if isinstance(on, Column) + else [c for c in on if isinstance(c, Column)] + ), + ) self.left = cast(LogicalPlan, left) self.right = right self.on = on @@ -942,7 +1018,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: merge_column = functools.reduce(lambda c1, c2: c1 & c2, self.on) plan.join.join_condition.CopyFrom(cast(Column, merge_column).to_plan(session)) plan.join.join_type = self.how - return plan + return self._with_relations(plan, session) @property def observations(self) -> Dict[str, "Observation"]: @@ -982,7 +1058,20 @@ def __init__( allow_exact_matches: bool, direction: str, ) -> None: - super().__init__(left) + super().__init__( + left, + self._collect_references( + [left_as_of, right_as_of] + + ( + [] + if on is None or isinstance(on, str) + else [on] + if isinstance(on, Column) + else [c for c in on if isinstance(c, Column)] + ) + + ([tolerance] if tolerance is not None else []) + ), + ) self.left = left self.right = right self.left_as_of = left_as_of @@ -1022,7 +1111,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.as_of_join.allow_exact_matches = self.allow_exact_matches plan.as_of_join.direction = self.direction - return plan + return self._with_relations(plan, session) @property def observations(self) -> Dict[str, "Observation"]: @@ -1064,7 +1153,7 @@ def __init__( on: Optional[Column], how: Optional[str], ) -> None: - super().__init__(left) + super().__init__(left, self._collect_references([on] if on is not None else [])) self.left = cast(LogicalPlan, left) self.right = right self.on = on @@ -1097,7 +1186,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: if self.on is not None: plan.lateral_join.join_condition.CopyFrom(self.on.to_plan(session)) plan.lateral_join.join_type = self.how - return plan + return self._with_relations(plan, session) @property def observations(self) -> Dict[str, "Observation"]: @@ -1225,9 +1314,9 @@ def __init__( num_partitions: Optional[int], columns: List[Column], ) -> None: - super().__init__(child) - self.num_partitions = num_partitions assert all(isinstance(c, Column) for c in columns) + super().__init__(child, self._collect_references(columns)) + self.num_partitions = num_partitions self.columns = columns def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -1240,7 +1329,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.repartition_by_expression.input.CopyFrom(self._child.plan(session)) if self.num_partitions is not None: plan.repartition_by_expression.num_partitions = self.num_partitions - return plan + return self._with_relations(plan, session) class SubqueryAlias(LogicalPlan): @@ -1286,8 +1375,6 @@ def __init__( named_args: Optional[Dict[str, Column]] = None, views: Optional[Sequence[SubqueryAlias]] = None, ) -> None: - super().__init__(None) - if args is not None: assert isinstance(args, List) assert all(isinstance(arg, Column) for arg in args) @@ -1301,10 +1388,8 @@ def __init__( if views is not None: assert isinstance(views, List) assert all(isinstance(v, SubqueryAlias) for v in views) - if len(views) > 0: - # reserved plan id for WithRelations - self._plan_id_with_rel = LogicalPlan._fresh_plan_id() + super().__init__(None, views) self._query = query self._args = args self._named_args = named_args @@ -1320,20 +1405,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: for k, arg in self._named_args.items(): plan.sql.named_arguments[k].CopyFrom(arg.to_plan(session)) - if self._views is not None and len(self._views) > 0: - # build new plan like - # with_relations [id 10] - # root: sql [id 9] - # reference: - # view#1: [id 8] - # view#2: [id 5] - sql_plan = plan - plan = proto.Relation() - plan.common.plan_id = self._plan_id_with_rel - plan.with_relations.root.CopyFrom(sql_plan) - plan.with_relations.references.extend([v.plan(session) for v in self._views]) - - return plan + return self._with_relations(plan, session) def command(self, session: "SparkConnectClient") -> proto.Command: cmd = proto.Command() @@ -1407,7 +1479,7 @@ def __init__( variable_column_name: str, value_column_name: str, ) -> None: - super().__init__(child) + super().__init__(child, self._collect_references(ids + (values or []))) self.ids = ids self.values = values self.variable_column_name = variable_column_name @@ -1422,7 +1494,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.unpivot.values.values.extend([v.to_plan(session) for v in self.values]) plan.unpivot.variable_column_name = self.variable_column_name plan.unpivot.value_column_name = self.value_column_name - return plan + return self._with_relations(plan, session) class Transpose(LogicalPlan): @@ -1433,7 +1505,7 @@ def __init__( child: Optional["LogicalPlan"], index_columns: Sequence[Column], ) -> None: - super().__init__(child) + super().__init__(child, self._collect_references(index_columns)) self.index_columns = index_columns def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -1443,12 +1515,12 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: if self.index_columns is not None and len(self.index_columns) > 0: for index_column in self.index_columns: plan.transpose.index_columns.append(index_column.to_plan(session)) - return plan + return self._with_relations(plan, session) class UnresolvedTableValuedFunction(LogicalPlan): def __init__(self, name: str, args: Sequence[Column]): - super().__init__(None) + super().__init__(None, self._collect_references(args)) self._name = name self._args = args @@ -1457,7 +1529,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.unresolved_table_valued_function.function_name = self._name for arg in self._args: plan.unresolved_table_valued_function.arguments.append(arg.to_plan(session)) - return plan + return self._with_relations(plan, session) class CollectMetrics(LogicalPlan): @@ -1469,9 +1541,9 @@ def __init__( observation: Union[str, "Observation"], exprs: List[Column], ) -> None: - super().__init__(child) - self._observation = observation assert all(isinstance(e, Column) for e in exprs) + super().__init__(child, self._collect_references(exprs)) + self._observation = observation self._exprs = exprs def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -1484,7 +1556,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: else str(self._observation._name) ) plan.collect_metrics.metrics.extend([e.to_plan(session) for e in self._exprs]) - return plan + return self._with_relations(plan, session) @property def observations(self) -> Dict[str, "Observation"]: @@ -1569,13 +1641,13 @@ def __init__( cols: Optional[List[str]], replacements: Sequence[Tuple[Column, Column]], ) -> None: - super().__init__(child) - self.cols = cols - assert replacements is not None and isinstance(replacements, List) for k, v in replacements: assert k is not None and isinstance(k, Column) assert v is not None and isinstance(v, Column) + + super().__init__(child, self._collect_references([e for t in replacements for e in t])) + self.cols = cols self.replacements = replacements def plan(self, session: "SparkConnectClient") -> proto.Relation: @@ -1590,7 +1662,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: replacement.old_value.CopyFrom(old_value.to_plan(session).literal) replacement.new_value.CopyFrom(new_value.to_plan(session).literal) plan.replace.replacements.append(replacement) - return plan + return self._with_relations(plan, session) class StatSummary(LogicalPlan): @@ -1700,8 +1772,6 @@ def __init__( fractions: Sequence[Tuple[Column, float]], seed: int, ) -> None: - super().__init__(child) - assert col is not None and isinstance(col, (Column, str)) assert fractions is not None and isinstance(fractions, List) @@ -1711,6 +1781,12 @@ def __init__( assert seed is None or isinstance(seed, int) + super().__init__( + child, + self._collect_references( + [col] if isinstance(col, Column) else [] + [c for c, _ in fractions] + ), + ) self._col = col self._fractions = fractions self._seed = seed @@ -1727,7 +1803,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: fraction.fraction = float(v) plan.sample_by.fractions.append(fraction) plan.sample_by.seed = self._seed - return plan + return self._with_relations(plan, session) class StatCorr(LogicalPlan): @@ -2375,7 +2451,7 @@ def __init__( ): assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) - super().__init__(child) + super().__init__(child, self._collect_references(grouping_cols)) self._grouping_cols = grouping_cols self._function = function._build_common_inline_user_defined_function(*cols) @@ -2387,7 +2463,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: [c.to_plan(session) for c in self._grouping_cols] ) plan.group_map.func.CopyFrom(self._function.to_plan_udf(session)) - return plan + return self._with_relations(plan, session) class CoGroupMap(LogicalPlan): @@ -2408,7 +2484,7 @@ def __init__( isinstance(c, Column) for c in other_grouping_cols ) - super().__init__(input) + super().__init__(input, self._collect_references(input_grouping_cols + other_grouping_cols)) self._input_grouping_cols = input_grouping_cols self._other_grouping_cols = other_grouping_cols self._other = cast(LogicalPlan, other) @@ -2428,7 +2504,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: [c.to_plan(session) for c in self._other_grouping_cols] ) plan.co_group_map.func.CopyFrom(self._function.to_plan_udf(session)) - return plan + return self._with_relations(plan, session) class ApplyInPandasWithState(LogicalPlan): @@ -2447,7 +2523,7 @@ def __init__( ): assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) - super().__init__(child) + super().__init__(child, self._collect_references(grouping_cols)) self._grouping_cols = grouping_cols self._function = function._build_common_inline_user_defined_function(*cols) self._output_schema = output_schema @@ -2467,7 +2543,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.apply_in_pandas_with_state.state_schema = self._state_schema plan.apply_in_pandas_with_state.output_mode = self._output_mode plan.apply_in_pandas_with_state.timeout_conf = self._timeout_conf - return plan + return self._with_relations(plan, session) class PythonUDTF: @@ -2531,7 +2607,7 @@ def __init__( deterministic: bool, arguments: Sequence[Expression], ) -> None: - super().__init__(None) + super().__init__(None, self._collect_references(arguments)) self._function_name = function_name self._deterministic = deterministic self._arguments = arguments @@ -2548,7 +2624,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.common_inline_user_defined_table_function.python_udtf.CopyFrom( self._function.to_plan(session) ) - return plan + return self._with_relations(plan, session) def udtf_plan( self, session: "SparkConnectClient" diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index 87070fd5ad3c5..093997a0d0c56 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -40,7 +40,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\x8b\x31\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hildB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xe1\x31\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12T\n\x13subquery_expression\x18\x16 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild"\xe1\x01\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType"]\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -54,79 +54,83 @@ "DESCRIPTOR" ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_EXPRESSION"]._serialized_start = 133 - _globals["_EXPRESSION"]._serialized_end = 6416 - _globals["_EXPRESSION_WINDOW"]._serialized_start = 1974 - _globals["_EXPRESSION_WINDOW"]._serialized_end = 2757 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2264 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_end = 2757 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_start = 2531 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_end = 2676 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_start = 2678 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_end = 2757 - _globals["_EXPRESSION_SORTORDER"]._serialized_start = 2760 - _globals["_EXPRESSION_SORTORDER"]._serialized_end = 3185 - _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_start = 2990 - _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_end = 3098 - _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_start = 3100 - _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_end = 3185 - _globals["_EXPRESSION_CAST"]._serialized_start = 3188 - _globals["_EXPRESSION_CAST"]._serialized_end = 3503 - _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3389 - _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3487 - _globals["_EXPRESSION_LITERAL"]._serialized_start = 3506 - _globals["_EXPRESSION_LITERAL"]._serialized_end = 5069 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4341 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4458 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4460 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4558 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4561 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4691 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4694 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 4921 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 4924 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5053 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5072 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5258 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5261 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5465 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5467 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5517 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5519 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5643 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5645 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5731 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5734 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 5866 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 5869 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6056 - _globals["_EXPRESSION_ALIAS"]._serialized_start = 6058 - _globals["_EXPRESSION_ALIAS"]._serialized_end = 6178 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6181 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6339 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6341 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6403 - _globals["_EXPRESSIONCOMMON"]._serialized_start = 6418 - _globals["_EXPRESSIONCOMMON"]._serialized_end = 6483 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6486 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6850 - _globals["_PYTHONUDF"]._serialized_start = 6853 - _globals["_PYTHONUDF"]._serialized_end = 7057 - _globals["_SCALARSCALAUDF"]._serialized_start = 7060 - _globals["_SCALARSCALAUDF"]._serialized_end = 7274 - _globals["_JAVAUDF"]._serialized_start = 7277 - _globals["_JAVAUDF"]._serialized_end = 7426 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7428 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7527 - _globals["_CALLFUNCTION"]._serialized_start = 7529 - _globals["_CALLFUNCTION"]._serialized_end = 7637 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7639 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7731 - _globals["_MERGEACTION"]._serialized_start = 7734 - _globals["_MERGEACTION"]._serialized_end = 8246 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 7956 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8062 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8065 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8232 - _globals["_LAZYEXPRESSION"]._serialized_start = 8248 - _globals["_LAZYEXPRESSION"]._serialized_end = 8313 + _globals["_EXPRESSION"]._serialized_end = 6502 + _globals["_EXPRESSION_WINDOW"]._serialized_start = 2060 + _globals["_EXPRESSION_WINDOW"]._serialized_end = 2843 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2350 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_end = 2843 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_start = 2617 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_end = 2762 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_start = 2764 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_end = 2843 + _globals["_EXPRESSION_SORTORDER"]._serialized_start = 2846 + _globals["_EXPRESSION_SORTORDER"]._serialized_end = 3271 + _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_start = 3076 + _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_end = 3184 + _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_start = 3186 + _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_end = 3271 + _globals["_EXPRESSION_CAST"]._serialized_start = 3274 + _globals["_EXPRESSION_CAST"]._serialized_end = 3589 + _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3475 + _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3573 + _globals["_EXPRESSION_LITERAL"]._serialized_start = 3592 + _globals["_EXPRESSION_LITERAL"]._serialized_end = 5155 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4427 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4544 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4546 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4644 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4647 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4777 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4780 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 5007 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 5010 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5139 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5158 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5344 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5347 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5551 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5553 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5603 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5605 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5729 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5731 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5817 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5820 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 5952 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 5955 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6142 + _globals["_EXPRESSION_ALIAS"]._serialized_start = 6144 + _globals["_EXPRESSION_ALIAS"]._serialized_end = 6264 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6267 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6425 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6427 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6489 + _globals["_EXPRESSIONCOMMON"]._serialized_start = 6504 + _globals["_EXPRESSIONCOMMON"]._serialized_end = 6569 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6572 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6936 + _globals["_PYTHONUDF"]._serialized_start = 6939 + _globals["_PYTHONUDF"]._serialized_end = 7143 + _globals["_SCALARSCALAUDF"]._serialized_start = 7146 + _globals["_SCALARSCALAUDF"]._serialized_end = 7360 + _globals["_JAVAUDF"]._serialized_start = 7363 + _globals["_JAVAUDF"]._serialized_end = 7512 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7514 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7613 + _globals["_CALLFUNCTION"]._serialized_start = 7615 + _globals["_CALLFUNCTION"]._serialized_end = 7723 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7725 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7817 + _globals["_MERGEACTION"]._serialized_start = 7820 + _globals["_MERGEACTION"]._serialized_end = 8332 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8042 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8148 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8151 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8318 + _globals["_LAZYEXPRESSION"]._serialized_start = 8334 + _globals["_LAZYEXPRESSION"]._serialized_end = 8399 + _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8402 + _globals["_SUBQUERYEXPRESSION"]._serialized_end = 8627 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 8534 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 8627 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index df4106cfc5f79..0a6f3caee8b54 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -1185,6 +1185,7 @@ class Expression(google.protobuf.message.Message): MERGE_ACTION_FIELD_NUMBER: builtins.int TYPED_AGGREGATE_EXPRESSION_FIELD_NUMBER: builtins.int LAZY_EXPRESSION_FIELD_NUMBER: builtins.int + SUBQUERY_EXPRESSION_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def common(self) -> global___ExpressionCommon: ... @@ -1231,6 +1232,8 @@ class Expression(google.protobuf.message.Message): @property def lazy_expression(self) -> global___LazyExpression: ... @property + def subquery_expression(self) -> global___SubqueryExpression: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary relations they can add them here. During the planning the correct resolution is done. @@ -1260,6 +1263,7 @@ class Expression(google.protobuf.message.Message): merge_action: global___MergeAction | None = ..., typed_aggregate_expression: global___TypedAggregateExpression | None = ..., lazy_expression: global___LazyExpression | None = ..., + subquery_expression: global___SubqueryExpression | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -1293,6 +1297,8 @@ class Expression(google.protobuf.message.Message): b"named_argument_expression", "sort_order", b"sort_order", + "subquery_expression", + b"subquery_expression", "typed_aggregate_expression", b"typed_aggregate_expression", "unresolved_attribute", @@ -1344,6 +1350,8 @@ class Expression(google.protobuf.message.Message): b"named_argument_expression", "sort_order", b"sort_order", + "subquery_expression", + b"subquery_expression", "typed_aggregate_expression", b"typed_aggregate_expression", "unresolved_attribute", @@ -1388,6 +1396,7 @@ class Expression(google.protobuf.message.Message): "merge_action", "typed_aggregate_expression", "lazy_expression", + "subquery_expression", "extension", ] | None @@ -1829,3 +1838,47 @@ class LazyExpression(google.protobuf.message.Message): def ClearField(self, field_name: typing_extensions.Literal["child", b"child"]) -> None: ... global___LazyExpression = LazyExpression + +class SubqueryExpression(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _SubqueryType: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _SubqueryTypeEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + SubqueryExpression._SubqueryType.ValueType + ], + builtins.type, + ): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + SUBQUERY_TYPE_UNKNOWN: SubqueryExpression._SubqueryType.ValueType # 0 + SUBQUERY_TYPE_SCALAR: SubqueryExpression._SubqueryType.ValueType # 1 + SUBQUERY_TYPE_EXISTS: SubqueryExpression._SubqueryType.ValueType # 2 + + class SubqueryType(_SubqueryType, metaclass=_SubqueryTypeEnumTypeWrapper): ... + SUBQUERY_TYPE_UNKNOWN: SubqueryExpression.SubqueryType.ValueType # 0 + SUBQUERY_TYPE_SCALAR: SubqueryExpression.SubqueryType.ValueType # 1 + SUBQUERY_TYPE_EXISTS: SubqueryExpression.SubqueryType.ValueType # 2 + + PLAN_ID_FIELD_NUMBER: builtins.int + SUBQUERY_TYPE_FIELD_NUMBER: builtins.int + plan_id: builtins.int + """(Required) The id of corresponding connect plan.""" + subquery_type: global___SubqueryExpression.SubqueryType.ValueType + """(Required) The type of the subquery.""" + def __init__( + self, + *, + plan_id: builtins.int = ..., + subquery_type: global___SubqueryExpression.SubqueryType.ValueType = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "plan_id", b"plan_id", "subquery_type", b"subquery_type" + ], + ) -> None: ... + +global___SubqueryExpression = SubqueryExpression diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f88ca5348ff2d..660f577f56f8b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -6612,7 +6612,7 @@ def scalar(self) -> Column: >>> from pyspark.sql import functions as sf >>> employees.where( ... sf.col("salary") > employees.select(sf.avg("salary")).scalar() - ... ).select("name", "salary", "department_id").show() + ... ).select("name", "salary", "department_id").orderBy("name").show() +-----+------+-------------+ | name|salary|department_id| +-----+------+-------------+ @@ -6630,7 +6630,7 @@ def scalar(self) -> Column: ... > employees.alias("e2").where( ... sf.col("e2.department_id") == sf.col("e1.department_id").outer() ... ).select(sf.avg("salary")).scalar() - ... ).select("name", "salary", "department_id").show() + ... ).select("name", "salary", "department_id").orderBy("name").show() +-----+------+-------------+ | name|salary|department_id| +-----+------+-------------+ @@ -6651,15 +6651,15 @@ def scalar(self) -> Column: ... ).select(sf.sum("salary")).scalar().alias("avg_salary"), ... 1 ... ).alias("salary_proportion_in_department") - ... ).show() + ... ).orderBy("name").show() +-------+------+-------------+-------------------------------+ | name|salary|department_id|salary_proportion_in_department| +-------+------+-------------+-------------------------------+ | Alice| 45000| 101| 30.6| | Bob| 54000| 101| 36.7| |Charlie| 29000| 102| 32.2| - | Eve| 48000| 101| 32.7| | David| 61000| 102| 67.8| + | Eve| 48000| 101| 32.7| +-------+------+-------------+-------------------------------+ """ ... diff --git a/python/pyspark/sql/tests/connect/test_parity_subquery.py b/python/pyspark/sql/tests/connect/test_parity_subquery.py index cffb6fc39059d..dae60a354d20a 100644 --- a/python/pyspark/sql/tests/connect/test_parity_subquery.py +++ b/python/pyspark/sql/tests/connect/test_parity_subquery.py @@ -17,42 +17,37 @@ import unittest +from pyspark.sql import functions as sf from pyspark.sql.tests.test_subquery import SubqueryTestsMixin +from pyspark.testing import assertDataFrameEqual from pyspark.testing.connectutils import ReusedConnectTestCase class SubqueryParityTests(SubqueryTestsMixin, ReusedConnectTestCase): - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_simple_uncorrelated_scalar_subquery(self): - super().test_simple_uncorrelated_scalar_subquery() - - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_uncorrelated_scalar_subquery_with_view(self): - super().test_uncorrelated_scalar_subquery_with_view() - - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_scalar_subquery_against_local_relations(self): - super().test_scalar_subquery_against_local_relations() - - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_correlated_scalar_subquery(self): - super().test_correlated_scalar_subquery() - - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_exists_subquery(self): - super().test_exists_subquery() - - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_scalar_subquery_with_outer_reference_errors(self): - super().test_scalar_subquery_with_outer_reference_errors() - - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_scalar_subquery_inside_lateral_join(self): - super().test_scalar_subquery_inside_lateral_join() - - @unittest.skip("TODO(SPARK-50134): Support subquery in connect") - def test_lateral_join_inside_subquery(self): - super().test_lateral_join_inside_subquery() + def test_scalar_subquery_with_missing_outer_reference(self): + with self.tempView("l", "r"): + self.df1.createOrReplaceTempView("l") + self.df2.createOrReplaceTempView("r") + + assertDataFrameEqual( + self.spark.table("l").select( + "a", + ( + self.spark.table("r") + .where(sf.col("c") == sf.col("a")) + .select(sf.sum("d")) + .scalar() + ), + ), + self.spark.sql("""SELECT a, (SELECT sum(d) FROM r WHERE c = a) FROM l"""), + ) + + def test_subquery_in_unpivot(self): + self.check_subquery_in_unpivot(None, None) + + @unittest.skip("SPARK-50601: Fix the SparkConnectPlanner to support this case") + def test_subquery_in_with_columns(self): + super().test_subquery_in_with_columns() if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 91789f74d9da6..0f431589b4616 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -459,30 +459,29 @@ def test_exists_subquery(self): ), ) - def test_scalar_subquery_with_outer_reference_errors(self): + def test_scalar_subquery_with_missing_outer_reference(self): with self.tempView("l", "r"): self.df1.createOrReplaceTempView("l") self.df2.createOrReplaceTempView("r") - with self.subTest("missing `outer()`"): - with self.assertRaises(AnalysisException) as pe: - self.spark.table("l").select( - "a", - ( - self.spark.table("r") - .where(sf.col("c") == sf.col("a")) - .select(sf.sum("d")) - .scalar() - ), - ).collect() + with self.assertRaises(AnalysisException) as pe: + self.spark.table("l").select( + "a", + ( + self.spark.table("r") + .where(sf.col("c") == sf.col("a")) + .select(sf.sum("d")) + .scalar() + ), + ).collect() - self.check_error( - exception=pe.exception, - errorClass="UNRESOLVED_COLUMN.WITH_SUGGESTION", - messageParameters={"objectName": "`a`", "proposal": "`c`, `d`"}, - query_context_type=QueryContextType.DataFrame, - fragment="col", - ) + self.check_error( + exception=pe.exception, + errorClass="UNRESOLVED_COLUMN.WITH_SUGGESTION", + messageParameters={"objectName": "`a`", "proposal": "`c`, `d`"}, + query_context_type=QueryContextType.DataFrame, + fragment="col", + ) def table1(self): t1 = self.spark.sql("VALUES (0, 1), (1, 2) AS t1(c1, c2)") @@ -833,6 +832,90 @@ def test_lateral_join_with_table_valued_functions_and_join_conditions(self): ), ) + def test_subquery_with_generator_and_tvf(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + self.spark.range(1).select(sf.explode(t1.select(sf.collect_list("c2")).scalar())), + self.spark.sql("""SELECT EXPLODE((SELECT COLLECT_LIST(c2) FROM t1))"""), + ) + assertDataFrameEqual( + self.spark.tvf.explode(t1.select(sf.collect_list("c2")).scalar()), + self.spark.sql("""SELECT * FROM EXPLODE((SELECT COLLECT_LIST(c2) FROM t1))"""), + ) + + def test_subquery_in_join_condition(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.join(t2, sf.col("t1.c1") == t1.select(sf.max("c1")).scalar()), + self.spark.sql("""SELECT * FROM t1 JOIN t2 ON t1.c1 = (SELECT MAX(c1) FROM t1)"""), + ) + + def test_subquery_in_unpivot(self): + self.check_subquery_in_unpivot(QueryContextType.DataFrame, "exists") + + def check_subquery_in_unpivot(self, query_context_type, fragment): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + with self.assertRaises(AnalysisException) as pe: + t1.unpivot("c1", t2.exists(), "c1", "c2").collect() + + self.check_error( + exception=pe.exception, + errorClass=( + "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_IN_EXISTS_SUBQUERY" + ), + messageParameters={"treeNode": "Expand.*"}, + query_context_type=query_context_type, + fragment=fragment, + matchPVals=True, + ) + + def test_subquery_in_transpose(self): + with self.tempView("t1"): + t1 = self.table1() + + with self.assertRaises(AnalysisException) as pe: + t1.transpose(t1.select(sf.max("c1")).scalar()).collect() + + self.check_error( + exception=pe.exception, + errorClass="TRANSPOSE_INVALID_INDEX_COLUMN", + messageParameters={"reason": "Index column must be an atomic attribute"}, + ) + + def test_subquery_in_with_columns(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + t1.withColumn( + "scalar", + self.spark.range(1) + .select(sf.col("c1").outer() + sf.col("c2").outer()) + .scalar(), + ), + t1.withColumn("scalar", sf.col("c1") + sf.col("c2")), + ) + + def test_subquery_in_drop(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual(t1.drop(self.spark.range(1).select(sf.lit("c1")).scalar()), t1) + + def test_subquery_in_repartition(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual(t1.repartition(self.spark.range(1).select(sf.lit(1)).scalar()), t1) + class SubqueryTests(SubqueryTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index d5f097065dc5c..233b432766b75 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -344,6 +344,7 @@ def check_error( messageParameters: Optional[Dict[str, str]] = None, query_context_type: Optional[QueryContextType] = None, fragment: Optional[str] = None, + matchPVals: bool = False, ): query_context = exception.getQueryContext() assert bool(query_context) == (query_context_type is not None), ( @@ -367,9 +368,30 @@ def check_error( # Test message parameters expected = messageParameters actual = exception.getMessageParameters() - self.assertEqual( - expected, actual, f"Expected message parameters was '{expected}', got '{actual}'" - ) + if matchPVals: + self.assertEqual( + len(expected), + len(actual), + "Expected message parameters count does not match actual message parameters count" + f": {len(expected)}, {len(actual)}.", + ) + for key, value in expected.items(): + self.assertIn( + key, + actual, + f"Expected message parameter key '{key}' was not found " + "in actual message parameters.", + ) + self.assertRegex( + actual[key], + value, + f"Expected message parameter value '{value}' does not match actual message " + f"parameter value '{actual[key]}'.", + ), + else: + self.assertEqual( + expected, actual, f"Expected message parameters was '{expected}', got '{actual}'" + ) # Test query context if query_context: diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala index f745c152170ee..ef4bdb8d5bdff 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala @@ -70,6 +70,19 @@ private[sql] trait ColumnNode extends ColumnNodeLike { trait ColumnNodeLike { private[internal] def normalize(): ColumnNodeLike = this private[internal] def sql: String + private[internal] def children: Seq[ColumnNodeLike] + + private[sql] def foreach(f: ColumnNodeLike => Unit): Unit = { + f(this) + children.foreach(_.foreach(f)) + } + + private[sql] def collect[A](pf: PartialFunction[ColumnNodeLike, A]): Seq[A] = { + val ret = new collection.mutable.ArrayBuffer[A]() + val lifted = pf.lift + foreach(node => lifted(node).foreach(ret.+=)) + ret.toSeq + } } private[internal] object ColumnNode { @@ -118,6 +131,8 @@ private[sql] case class Literal( case v: Short => toSQLValue(v) case _ => value.toString } + + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } /** @@ -141,6 +156,8 @@ private[sql] case class UnresolvedAttribute( copy(planId = None, origin = NO_ORIGIN) override def sql: String = nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } private[sql] object UnresolvedAttribute { @@ -183,6 +200,7 @@ private[sql] case class UnresolvedStar( override private[internal] def normalize(): UnresolvedStar = copy(planId = None, origin = NO_ORIGIN) override def sql: String = unparsedTarget.map(_ + ".*").getOrElse("*") + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } /** @@ -208,6 +226,8 @@ private[sql] case class UnresolvedFunction( copy(arguments = ColumnNode.normalize(arguments), origin = NO_ORIGIN) override def sql: String = functionName + argumentsToSql(arguments) + + override private[internal] def children: Seq[ColumnNodeLike] = arguments } /** @@ -222,6 +242,7 @@ private[sql] case class SqlExpression( extends ColumnNode { override private[internal] def normalize(): SqlExpression = copy(origin = NO_ORIGIN) override def sql: String = expression + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } /** @@ -250,6 +271,8 @@ private[sql] case class Alias( } s"${child.sql} AS $alias" } + + override private[internal] def children: Seq[ColumnNodeLike] = Seq(child) } /** @@ -275,10 +298,14 @@ private[sql] case class Cast( override def sql: String = { s"${optionToSql(evalMode)}CAST(${child.sql} AS ${dataType.sql})" } + + override private[internal] def children: Seq[ColumnNodeLike] = Seq(child) ++ evalMode } private[sql] object Cast { - sealed abstract class EvalMode(override val sql: String = "") extends ColumnNodeLike + sealed abstract class EvalMode(override val sql: String = "") extends ColumnNodeLike { + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty + } object Legacy extends EvalMode object Ansi extends EvalMode object Try extends EvalMode("TRY_") @@ -300,6 +327,7 @@ private[sql] case class UnresolvedRegex( override private[internal] def normalize(): UnresolvedRegex = copy(planId = None, origin = NO_ORIGIN) override def sql: String = regex + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } /** @@ -322,13 +350,19 @@ private[sql] case class SortOrder( copy(child = child.normalize(), origin = NO_ORIGIN) override def sql: String = s"${child.sql} ${sortDirection.sql} ${nullOrdering.sql}" + + override def children: Seq[ColumnNodeLike] = Seq(child, sortDirection, nullOrdering) } private[sql] object SortOrder { - sealed abstract class SortDirection(override val sql: String) extends ColumnNodeLike + sealed abstract class SortDirection(override val sql: String) extends ColumnNodeLike { + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty + } object Ascending extends SortDirection("ASC") object Descending extends SortDirection("DESC") - sealed abstract class NullOrdering(override val sql: String) extends ColumnNodeLike + sealed abstract class NullOrdering(override val sql: String) extends ColumnNodeLike { + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty + } object NullsFirst extends NullOrdering("NULLS FIRST") object NullsLast extends NullOrdering("NULLS LAST") } @@ -352,6 +386,8 @@ private[sql] case class Window( origin = NO_ORIGIN) override def sql: String = s"${windowFunction.sql} OVER (${windowSpec.sql})" + + override private[internal] def children: Seq[ColumnNodeLike] = Seq(windowFunction, windowSpec) } private[sql] case class WindowSpec( @@ -370,6 +406,9 @@ private[sql] case class WindowSpec( optionToSql(frame)) parts.filter(_.nonEmpty).mkString(" ") } + override private[internal] def children: Seq[ColumnNodeLike] = { + partitionColumns ++ sortColumns ++ frame + } } private[sql] case class WindowFrame( @@ -381,15 +420,19 @@ private[sql] case class WindowFrame( copy(lower = lower.normalize(), upper = upper.normalize()) override private[internal] def sql: String = s"${frameType.sql} BETWEEN ${lower.sql} AND ${upper.sql}" + override private[internal] def children: Seq[ColumnNodeLike] = Seq(frameType, lower, upper) } private[sql] object WindowFrame { - sealed abstract class FrameType(override val sql: String) extends ColumnNodeLike + sealed abstract class FrameType(override val sql: String) extends ColumnNodeLike { + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty + } object Row extends FrameType("ROWS") object Range extends FrameType("RANGE") sealed abstract class FrameBoundary extends ColumnNodeLike { override private[internal] def normalize(): FrameBoundary = this + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } object CurrentRow extends FrameBoundary { override private[internal] def sql = "CURRENT ROW" @@ -403,6 +446,7 @@ private[sql] object WindowFrame { case class Value(value: ColumnNode) extends FrameBoundary { override private[internal] def normalize(): Value = copy(value.normalize()) override private[internal] def sql: String = value.sql + override private[internal] def children: Seq[ColumnNodeLike] = Seq(value) } def value(i: Int): Value = Value(Literal(i, Some(IntegerType))) def value(l: Long): Value = Value(Literal(l, Some(LongType))) @@ -434,6 +478,8 @@ private[sql] case class LambdaFunction( } argumentsSql + " -> " + function.sql } + + override private[internal] def children: Seq[ColumnNodeLike] = function +: arguments } object LambdaFunction { @@ -455,6 +501,8 @@ private[sql] case class UnresolvedNamedLambdaVariable( copy(origin = NO_ORIGIN) override def sql: String = name + + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } object UnresolvedNamedLambdaVariable { @@ -495,6 +543,8 @@ private[sql] case class UnresolvedExtractValue( copy(child = child.normalize(), extraction = extraction.normalize(), origin = NO_ORIGIN) override def sql: String = s"${child.sql}[${extraction.sql}]" + + override private[internal] def children: Seq[ColumnNodeLike] = Seq(child, extraction) } /** @@ -521,6 +571,9 @@ private[sql] case class UpdateFields( case Some(value) => s"update_field(${structExpression.sql}, $fieldName, ${value.sql})" case None => s"drop_field(${structExpression.sql}, $fieldName)" } + override private[internal] def children: Seq[ColumnNodeLike] = { + structExpression +: valueExpression.toSeq + } } /** @@ -549,6 +602,11 @@ private[sql] case class CaseWhenOtherwise( branches.map(cv => s" WHEN ${cv._1.sql} THEN ${cv._2.sql}").mkString + otherwise.map(o => s" ELSE ${o.sql}").getOrElse("") + " END" + + override private[internal] def children: Seq[ColumnNodeLike] = { + val branchChildren = branches.flatMap { case (condition, value) => Seq(condition, value) } + branchChildren ++ otherwise + } } /** @@ -570,6 +628,8 @@ private[sql] case class InvokeInlineUserDefinedFunction( override def sql: String = function.name + argumentsToSql(arguments) + + override private[internal] def children: Seq[ColumnNodeLike] = arguments } private[sql] trait UserDefinedFunctionLike { @@ -589,4 +649,5 @@ private[sql] case class LazyExpression( override private[internal] def normalize(): ColumnNode = copy(child = child.normalize(), origin = NO_ORIGIN) override def sql: String = "lazy" + argumentsToSql(Seq(child)) + override private[internal] def children: Seq[ColumnNodeLike] = Seq(child) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 61b68b743a5c4..87a5e94d9f630 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -1056,3 +1056,18 @@ case class LazyExpression(child: Expression) extends UnaryExpression with Uneval } final override val nodePatterns: Seq[TreePattern] = Seq(LAZY_EXPRESSION) } + +trait UnresolvedPlanId extends LeafExpression with Unevaluable { + override def nullable: Boolean = throw new UnresolvedException("nullable") + override def dataType: DataType = throw new UnresolvedException("dataType") + override lazy val resolved = false + + def planId: Long + def withPlan(plan: LogicalPlan): Expression + + final override val nodePatterns: Seq[TreePattern] = + Seq(UNRESOLVED_PLAN_ID) ++ nodePatternsInternal() + + // Subclasses can override this function to provide more TreePatterns. + def nodePatternsInternal(): Seq[TreePattern] = Seq() +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 0c8253659dd56..c0a2bf25fbe67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer +import org.apache.spark.sql.catalyst.analysis.UnresolvedPlanId import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -439,6 +440,14 @@ object ScalarSubquery { } } +case class UnresolvedScalarSubqueryPlanId(planId: Long) + extends UnresolvedPlanId { + + override def withPlan(plan: LogicalPlan): Expression = { + ScalarSubquery(plan) + } +} + /** * A subquery that can return multiple rows and columns. This should be rewritten as a join * with the outer query during the optimization phase. @@ -592,3 +601,11 @@ case class Exists( final override def nodePatternsInternal(): Seq[TreePattern] = Seq(EXISTS_SUBQUERY) } + +case class UnresolvedExistsPlanId(planId: Long) + extends UnresolvedPlanId { + + override def withPlan(plan: LogicalPlan): Expression = { + Exists(plan) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 95b5832392ec3..1dfb0336ecf08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -155,6 +155,7 @@ object TreePattern extends Enumeration { val UNRESOLVED_FUNCTION: Value = Value val UNRESOLVED_HINT: Value = Value val UNRESOLVED_WINDOW_EXPRESSION: Value = Value + val UNRESOLVED_PLAN_ID: Value = Value // Unresolved Plan patterns (Alphabetically ordered) val UNRESOLVED_FUNC: Value = Value diff --git a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto index 811dd032aa419..a01b5229a7b74 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -53,6 +53,7 @@ message Expression { MergeAction merge_action = 19; TypedAggregateExpression typed_aggregate_expression = 20; LazyExpression lazy_expression = 21; + SubqueryExpression subquery_expression = 22; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // relations they can add them here. During the planning the correct resolution is done. @@ -457,3 +458,17 @@ message LazyExpression { // (Required) The expression to be marked as lazy. Expression child = 1; } + +message SubqueryExpression { + // (Required) The id of corresponding connect plan. + int64 plan_id = 1; + + // (Required) The type of the subquery. + SubqueryType subquery_type = 2; + + enum SubqueryType { + SUBQUERY_TYPE_UNKNOWN = 0; + SUBQUERY_TYPE_SCALAR = 1; + SUBQUERY_TYPE_EXISTS = 2; + } +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index bfb5f8f3fab75..5ace916ba3e9e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -45,7 +45,7 @@ import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTableValuedFunction, UnresolvedTranspose} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ @@ -55,7 +55,7 @@ import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, L import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.{AppendColumns, Assignment, CoGroup, CollectMetrics, CommandResult, Deduplicate, DeduplicateWithinWatermark, DeleteAction, DeserializeToObject, Except, FlatMapGroupsWithState, InsertAction, InsertStarAction, Intersect, JoinWith, LocalRelation, LogicalGroupState, LogicalPlan, MapGroups, MapPartitions, MergeAction, Project, Sample, SerializeFromObject, Sort, SubqueryAlias, TypedFilter, Union, Unpivot, UnresolvedHint, UpdateAction, UpdateStarAction} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes -import org.apache.spark.sql.catalyst.trees.CurrentOrigin +import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreePattern} import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.classic.ClassicConversions._ @@ -161,9 +161,8 @@ class SparkConnectPlanner( case proto.Relation.RelTypeCase.DROP => transformDrop(rel.getDrop) case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate) case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql) - case proto.Relation.RelTypeCase.WITH_RELATIONS - if isValidSQLWithRefs(rel.getWithRelations) => - transformSqlWithRefs(rel.getWithRelations) + case proto.Relation.RelTypeCase.WITH_RELATIONS => + transformWithRelations(rel.getWithRelations) case proto.Relation.RelTypeCase.LOCAL_RELATION => transformLocalRelation(rel.getLocalRelation) case proto.Relation.RelTypeCase.SAMPLE => transformSample(rel.getSample) @@ -1559,6 +1558,8 @@ class SparkConnectPlanner( transformTypedAggregateExpression(exp.getTypedAggregateExpression, baseRelationOpt) case proto.Expression.ExprTypeCase.LAZY_EXPRESSION => transformLazyExpression(exp.getLazyExpression) + case proto.Expression.ExprTypeCase.SUBQUERY_EXPRESSION => + transformSubqueryExpression(exp.getSubqueryExpression) case _ => throw InvalidPlanInput( s"Expression with ID: ${exp.getExprTypeCase.getNumber} is not supported") @@ -3724,7 +3725,56 @@ class SparkConnectPlanner( LazyExpression(transformExpression(getLazyExpression.getChild)) } - private def assertPlan(assertion: Boolean, message: String = ""): Unit = { + private def transformSubqueryExpression( + getSubqueryExpression: proto.SubqueryExpression): Expression = { + val planId = getSubqueryExpression.getPlanId + getSubqueryExpression.getSubqueryType match { + case proto.SubqueryExpression.SubqueryType.SUBQUERY_TYPE_SCALAR => + UnresolvedScalarSubqueryPlanId(planId) + case proto.SubqueryExpression.SubqueryType.SUBQUERY_TYPE_EXISTS => + UnresolvedExistsPlanId(planId) + case other => throw InvalidPlanInput(s"Unknown SubqueryType $other") + } + } + + private def transformWithRelations(getWithRelations: proto.WithRelations): LogicalPlan = { + if (isValidSQLWithRefs(getWithRelations)) { + transformSqlWithRefs(getWithRelations) + } else { + // Wrap the plan to keep the original planId. + val plan = Project(Seq(UnresolvedStar(None)), transformRelation(getWithRelations.getRoot)) + + val relations = getWithRelations.getReferencesList.asScala.map { ref => + if (ref.hasCommon && ref.getCommon.hasPlanId) { + val planId = ref.getCommon.getPlanId + val plan = transformRelation(ref) + planId -> plan + } else { + throw InvalidPlanInput("Invalid WithRelation reference") + } + }.toMap + + val missingPlanIds = mutable.Set.empty[Long] + val withRelations = plan + .transformAllExpressionsWithPruning(_.containsPattern(TreePattern.UNRESOLVED_PLAN_ID)) { + case u: UnresolvedPlanId => + if (relations.contains(u.planId)) { + u.withPlan(relations(u.planId)) + } else { + missingPlanIds += u.planId + u + } + } + assertPlan( + missingPlanIds.isEmpty, + "Missing relation in WithRelations: " + + s"${missingPlanIds.mkString("(", ", ", ")")} not in " + + s"${relations.keys.mkString("(", ", ", ")")}") + withRelations + } + } + + private def assertPlan(assertion: Boolean, message: => String = ""): Unit = { if (!assertion) throw InvalidPlanInput(message) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala index 8b4726114890f..8f37f5c32de34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala @@ -273,6 +273,8 @@ private[sql] case class ExpressionColumnNode private( } override def sql: String = expression.sql + + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } private[sql] object ExpressionColumnNode { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index cd425162fb016..f94cf89276ec4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -664,4 +664,102 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { ) } } + + test("subquery with generator / table-valued functions") { + withView("t1") { + val t1 = table1() + + checkAnswer( + spark.range(1).select(explode(t1.select(collect_list("c2")).scalar())), + sql("SELECT EXPLODE((SELECT COLLECT_LIST(c2) FROM t1))") + ) + checkAnswer( + spark.tvf.explode(t1.select(collect_list("c2")).scalar()), + sql("SELECT * FROM EXPLODE((SELECT COLLECT_LIST(c2) FROM t1))") + ) + } + } + + test("subquery in join condition") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.join(t2, $"t1.c1" === t1.select(max("c1")).scalar()), + sql("SELECT * FROM t1 JOIN t2 ON t1.c1 = (SELECT MAX(c1) FROM t1)") + ) + } + } + + test("subquery in unpivot") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkError( + intercept[AnalysisException] { + t1.unpivot(Array(t2.exists()), "c1", "c2").collect() + }, + "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_IN_EXISTS_SUBQUERY", + parameters = Map("treeNode" -> "(?s)'Unpivot.*"), + matchPVals = true, + queryContext = Array(ExpectedContext( + fragment = "exists", + callSitePattern = getCurrentClassCallSitePattern)) + ) + checkError( + intercept[AnalysisException] { + t1.unpivot(Array($"c1"), Array(t2.exists()), "c1", "c2").collect() + }, + "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_IN_EXISTS_SUBQUERY", + parameters = Map("treeNode" -> "(?s)Expand.*"), + matchPVals = true, + queryContext = Array(ExpectedContext( + fragment = "exists", + callSitePattern = getCurrentClassCallSitePattern)) + ) + } + } + + test("subquery in transpose") { + withView("t1") { + val t1 = table1() + + checkError( + intercept[AnalysisException] { + t1.transpose(t1.select(max("c1")).scalar()).collect() + }, + "TRANSPOSE_INVALID_INDEX_COLUMN", + parameters = Map("reason" -> "Index column must be an atomic attribute") + ) + } + } + + test("subquery in withColumns") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.withColumn("scalar", spark.range(1).select($"c1".outer() + $"c2".outer()).scalar()), + t1.withColumn("scalar", $"c1" + $"c2") + ) + } + } + + test("subquery in drop") { + withView("t1") { + val t1 = table1() + + checkAnswer(t1.drop(spark.range(1).select(lit("c1")).scalar()), t1) + } + } + + test("subquery in repartition") { + withView("t1") { + val t1 = table1() + + checkAnswer(t1.repartition(spark.range(1).select(lit(1)).scalar()), t1) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToExpressionConverterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToExpressionConverterSuite.scala index 76fcdfc380950..d72e86450de22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToExpressionConverterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToExpressionConverterSuite.scala @@ -405,4 +405,5 @@ private[internal] case class Nope(override val origin: Origin = CurrentOrigin.ge extends ColumnNode { override private[internal] def normalize(): Nope = this override def sql: String = "nope" + override private[internal] def children: Seq[ColumnNodeLike] = Seq.empty } From c92021091502b15b6020e6e4cc9b148009450ba5 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Fri, 27 Dec 2024 08:40:53 +0900 Subject: [PATCH 260/438] [SPARK-50578][PYTHON][SS] Add support for new version of state metadata for TransformWithStateInPandas ### What changes were proposed in this pull request? Enable TransformWithStateInPandas operator to write new versions of state metadata and state schema. This will enable state metadata source and state data source reader. And will also support future schema evolution changes. To achieve this purpose, in this PR, we add a new implementation of driver side Python runner. This is because spark will need to get the state schema on the driver during planning inside `IncrementalExecution`. We will also need to start another state server in the new driver side Python runner to handle API calls in init(). ### Why are the changes needed? This is to match with the new versions of state metadata and state schema version implemented in Scala side of TransformWithState. ### Does this PR introduce _any_ user-facing change? No. But now users will be able to get results from state metadata source reader and state data source reader using the same API as Scala. E.g. for state metadata source reader, we can now read out state metadata as follows: ``` metadata_df = spark.read.format("state-metadata").load(checkpoint_path) ``` And we can read out state rows by using state data source as follows: ``` list_state_df = spark.read.format("statestore")\ .option("path", checkpoint_path)\ .option("stateVarName", "listState")\ .load() ``` ### How was this patch tested? Add unit tests in `python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py`. Test state metadata and state schema files are written correctly by using state metadata source reader and state data source reader. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49156 from jingz-db/python-metadata. Lead-authored-by: jingz-db Co-authored-by: Jing Zhan <135738831+jingz-db@users.noreply.github.com> Signed-off-by: Jungtaek Lim --- .../api/python/StreamingPythonRunner.scala | 16 +- python/pyspark/sql/pandas/group_ops.py | 24 + .../sql/streaming/proto/StateMessage_pb2.py | 4 +- .../sql/streaming/proto/StateMessage_pb2.pyi | 22 +- .../stateful_processor_api_client.py | 14 +- .../sql/streaming/stateful_processor_util.py | 1 + .../transform_with_state_driver_worker.py | 102 +++++ .../test_pandas_transform_with_state.py | 423 +++++++++++++++++- .../execution/streaming/StateMessage.proto | 11 +- .../v2/state/StateDataSource.scala | 11 +- .../TransformWithStateInPandasExec.scala | 65 ++- ...ansformWithStateInPandasPythonRunner.scala | 134 ++++-- ...ransformWithStateInPandasStateServer.scala | 11 +- .../streaming/IncrementalExecution.scala | 2 +- .../streaming/TransformWithStateExec.scala | 103 +---- .../TransformWithStateVariableUtils.scala | 110 ++++- 16 files changed, 887 insertions(+), 166 deletions(-) create mode 100644 python/pyspark/sql/streaming/transform_with_state_driver_worker.py diff --git a/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala index 0ff2b79ab6623..ce933337afc35 100644 --- a/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala @@ -48,10 +48,10 @@ private[spark] class StreamingPythonRunner( protected val bufferSize: Int = conf.get(BUFFER_SIZE) protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT) - private val envVars: java.util.Map[String, String] = func.envVars - private val pythonExec: String = func.pythonExec - private var pythonWorker: Option[PythonWorker] = None - private var pythonWorkerFactory: Option[PythonWorkerFactory] = None + protected val envVars: java.util.Map[String, String] = func.envVars + protected val pythonExec: String = func.pythonExec + protected var pythonWorker: Option[PythonWorker] = None + protected var pythonWorkerFactory: Option[PythonWorkerFactory] = None protected val pythonVer: String = func.pythonVer /** @@ -68,7 +68,9 @@ private[spark] class StreamingPythonRunner( envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString) envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString) - envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl) + if (!connectUrl.isEmpty) { + envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl) + } val workerFactory = new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap, false) @@ -83,7 +85,9 @@ private[spark] class StreamingPythonRunner( PythonWorkerUtils.writePythonVersion(pythonVer, dataOut) // Send sessionId - PythonRDD.writeUTF(sessionId, dataOut) + if (!sessionId.isEmpty) { + PythonRDD.writeUTF(sessionId, dataOut) + } // Send the user function to python process PythonWorkerUtils.writePythonFunction(func, dataOut) diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 688ad4b05732e..bd12b41b34361 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -504,6 +504,24 @@ def transformWithStateInPandas( if isinstance(outputStructType, str): outputStructType = cast(StructType, _parse_datatype_string(outputStructType)) + def handle_pre_init( + statefulProcessorApiClient: StatefulProcessorApiClient, + ) -> Iterator["PandasDataFrameLike"]: + # Driver handle is different from the handle used on executors; + # On JVM side, we will use `DriverStatefulProcessorHandleImpl` for driver handle which + # will only be used for handling init() and get the state schema on the driver. + driver_handle = StatefulProcessorHandle(statefulProcessorApiClient) + statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.PRE_INIT) + statefulProcessor.init(driver_handle) + + # This method is used for the driver-side stateful processor after we have collected + # all the necessary schemas. This instance of the DriverStatefulProcessorHandleImpl + # won't be used again on JVM. + statefulProcessor.close() + + # return a dummy results, no return value is needed for pre init + return iter([]) + def handle_data_rows( statefulProcessorApiClient: StatefulProcessorApiClient, key: Any, @@ -560,6 +578,9 @@ def transformWithStateUDF( key: Any, inputRows: Iterator["PandasDataFrameLike"], ) -> Iterator["PandasDataFrameLike"]: + if mode == TransformWithStateInPandasFuncMode.PRE_INIT: + return handle_pre_init(statefulProcessorApiClient) + handle = StatefulProcessorHandle(statefulProcessorApiClient) if statefulProcessorApiClient.handle_state == StatefulProcessorHandleState.CREATED: @@ -606,6 +627,9 @@ def transformWithStateWithInitStateUDF( - `initialStates` is None, while `inputRows` is not empty. This is not first batch. `initialStates` is initialized to the positional value as None. """ + if mode == TransformWithStateInPandasFuncMode.PRE_INIT: + return handle_pre_init(statefulProcessorApiClient) + handle = StatefulProcessorHandle(statefulProcessorApiClient) if statefulProcessorApiClient.handle_state == StatefulProcessorHandleState.CREATED: diff --git a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py index bcd8e0fc68f5c..20af541f307cd 100644 --- a/python/pyspark/sql/streaming/proto/StateMessage_pb2.py +++ b/python/pyspark/sql/streaming/proto/StateMessage_pb2.py @@ -40,7 +40,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n;org/apache/spark/sql/execution/streaming/StateMessage.proto\x12.org.apache.spark.sql.execution.streaming.state"\x84\x05\n\x0cStateRequest\x12\x18\n\x07version\x18\x01 \x01(\x05R\x07version\x12}\n\x15statefulProcessorCall\x18\x02 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.StatefulProcessorCallH\x00R\x15statefulProcessorCall\x12z\n\x14stateVariableRequest\x18\x03 \x01(\x0b\x32\x44.org.apache.spark.sql.execution.streaming.state.StateVariableRequestH\x00R\x14stateVariableRequest\x12\x8c\x01\n\x1aimplicitGroupingKeyRequest\x18\x04 \x01(\x0b\x32J.org.apache.spark.sql.execution.streaming.state.ImplicitGroupingKeyRequestH\x00R\x1aimplicitGroupingKeyRequest\x12\x62\n\x0ctimerRequest\x18\x05 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.TimerRequestH\x00R\x0ctimerRequest\x12\x62\n\x0cutilsRequest\x18\x06 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.UtilsRequestH\x00R\x0cutilsRequestB\x08\n\x06method"i\n\rStateResponse\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x0cR\x05value"x\n\x1cStateResponseWithLongTypeVal\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x03R\x05value"z\n\x1eStateResponseWithStringTypeVal\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\tR\x05value"\xa0\x05\n\x15StatefulProcessorCall\x12h\n\x0esetHandleState\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetHandleStateH\x00R\x0esetHandleState\x12h\n\rgetValueState\x18\x02 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\rgetValueState\x12\x66\n\x0cgetListState\x18\x03 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0cgetListState\x12\x64\n\x0bgetMapState\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0bgetMapState\x12o\n\x0etimerStateCall\x18\x05 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.TimerStateCallCommandH\x00R\x0etimerStateCall\x12j\n\x0e\x64\x65leteIfExists\x18\x06 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0e\x64\x65leteIfExistsB\x08\n\x06method"\xd5\x02\n\x14StateVariableRequest\x12h\n\x0evalueStateCall\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.ValueStateCallH\x00R\x0evalueStateCall\x12\x65\n\rlistStateCall\x18\x02 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.ListStateCallH\x00R\rlistStateCall\x12\x62\n\x0cmapStateCall\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.MapStateCallH\x00R\x0cmapStateCallB\x08\n\x06method"\x83\x02\n\x1aImplicitGroupingKeyRequest\x12h\n\x0esetImplicitKey\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetImplicitKeyH\x00R\x0esetImplicitKey\x12q\n\x11removeImplicitKey\x18\x02 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.RemoveImplicitKeyH\x00R\x11removeImplicitKeyB\x08\n\x06method"\x81\x02\n\x0cTimerRequest\x12q\n\x11timerValueRequest\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.TimerValueRequestH\x00R\x11timerValueRequest\x12t\n\x12\x65xpiryTimerRequest\x18\x02 \x01(\x0b\x32\x42.org.apache.spark.sql.execution.streaming.state.ExpiryTimerRequestH\x00R\x12\x65xpiryTimerRequestB\x08\n\x06method"\xf6\x01\n\x11TimerValueRequest\x12s\n\x12getProcessingTimer\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.GetProcessingTimeH\x00R\x12getProcessingTimer\x12\x62\n\x0cgetWatermark\x18\x02 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.GetWatermarkH\x00R\x0cgetWatermarkB\x08\n\x06method"B\n\x12\x45xpiryTimerRequest\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs"\x13\n\x11GetProcessingTime"\x0e\n\x0cGetWatermark"\x8b\x01\n\x0cUtilsRequest\x12q\n\x11parseStringSchema\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.ParseStringSchemaH\x00R\x11parseStringSchemaB\x08\n\x06method"+\n\x11ParseStringSchema\x12\x16\n\x06schema\x18\x01 \x01(\tR\x06schema"\xc7\x01\n\x10StateCallCommand\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12\x16\n\x06schema\x18\x02 \x01(\tR\x06schema\x12\x30\n\x13mapStateValueSchema\x18\x03 \x01(\tR\x13mapStateValueSchema\x12K\n\x03ttl\x18\x04 \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.TTLConfigR\x03ttl"\xa7\x02\n\x15TimerStateCallCommand\x12[\n\x08register\x18\x01 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.RegisterTimerH\x00R\x08register\x12U\n\x06\x64\x65lete\x18\x02 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.DeleteTimerH\x00R\x06\x64\x65lete\x12P\n\x04list\x18\x03 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.ListTimersH\x00R\x04listB\x08\n\x06method"\x92\x03\n\x0eValueStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12G\n\x03get\x18\x03 \x01(\x0b\x32\x33.org.apache.spark.sql.execution.streaming.state.GetH\x00R\x03get\x12n\n\x10valueStateUpdate\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.ValueStateUpdateH\x00R\x10valueStateUpdate\x12M\n\x05\x63lear\x18\x05 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xdf\x04\n\rListStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12\x62\n\x0clistStateGet\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStateGetH\x00R\x0clistStateGet\x12\x62\n\x0clistStatePut\x18\x04 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStatePutH\x00R\x0clistStatePut\x12_\n\x0b\x61ppendValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.AppendValueH\x00R\x0b\x61ppendValue\x12\\\n\nappendList\x18\x06 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.AppendListH\x00R\nappendList\x12M\n\x05\x63lear\x18\x07 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xc2\x06\n\x0cMapStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12V\n\x08getValue\x18\x03 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.GetValueH\x00R\x08getValue\x12_\n\x0b\x63ontainsKey\x18\x04 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.ContainsKeyH\x00R\x0b\x63ontainsKey\x12_\n\x0bupdateValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.UpdateValueH\x00R\x0bupdateValue\x12V\n\x08iterator\x18\x06 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.IteratorH\x00R\x08iterator\x12J\n\x04keys\x18\x07 \x01(\x0b\x32\x34.org.apache.spark.sql.execution.streaming.state.KeysH\x00R\x04keys\x12P\n\x06values\x18\x08 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ValuesH\x00R\x06values\x12Y\n\tremoveKey\x18\t \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.RemoveKeyH\x00R\tremoveKey\x12M\n\x05\x63lear\x18\n \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method""\n\x0eSetImplicitKey\x12\x10\n\x03key\x18\x01 \x01(\x0cR\x03key"\x13\n\x11RemoveImplicitKey"\x08\n\x06\x45xists"\x05\n\x03Get"=\n\rRegisterTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs";\n\x0b\x44\x65leteTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs",\n\nListTimers\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x10ValueStateUpdate\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x07\n\x05\x43lear".\n\x0cListStateGet\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"\x0e\n\x0cListStatePut"#\n\x0b\x41ppendValue\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x0c\n\nAppendList"$\n\x08GetValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"\'\n\x0b\x43ontainsKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"=\n\x0bUpdateValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey\x12\x14\n\x05value\x18\x02 \x01(\x0cR\x05value"*\n\x08Iterator\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"&\n\x04Keys\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x06Values\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"%\n\tRemoveKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"c\n\x0eSetHandleState\x12Q\n\x05state\x18\x01 \x01(\x0e\x32;.org.apache.spark.sql.execution.streaming.state.HandleStateR\x05state"+\n\tTTLConfig\x12\x1e\n\ndurationMs\x18\x01 \x01(\x05R\ndurationMs*`\n\x0bHandleState\x12\x0b\n\x07\x43REATED\x10\x00\x12\x0f\n\x0bINITIALIZED\x10\x01\x12\x12\n\x0e\x44\x41TA_PROCESSED\x10\x02\x12\x13\n\x0fTIMER_PROCESSED\x10\x03\x12\n\n\x06\x43LOSED\x10\x04\x62\x06proto3' + b'\n;org/apache/spark/sql/execution/streaming/StateMessage.proto\x12.org.apache.spark.sql.execution.streaming.state"\x84\x05\n\x0cStateRequest\x12\x18\n\x07version\x18\x01 \x01(\x05R\x07version\x12}\n\x15statefulProcessorCall\x18\x02 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.StatefulProcessorCallH\x00R\x15statefulProcessorCall\x12z\n\x14stateVariableRequest\x18\x03 \x01(\x0b\x32\x44.org.apache.spark.sql.execution.streaming.state.StateVariableRequestH\x00R\x14stateVariableRequest\x12\x8c\x01\n\x1aimplicitGroupingKeyRequest\x18\x04 \x01(\x0b\x32J.org.apache.spark.sql.execution.streaming.state.ImplicitGroupingKeyRequestH\x00R\x1aimplicitGroupingKeyRequest\x12\x62\n\x0ctimerRequest\x18\x05 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.TimerRequestH\x00R\x0ctimerRequest\x12\x62\n\x0cutilsRequest\x18\x06 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.UtilsRequestH\x00R\x0cutilsRequestB\x08\n\x06method"i\n\rStateResponse\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x0cR\x05value"x\n\x1cStateResponseWithLongTypeVal\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\x03R\x05value"z\n\x1eStateResponseWithStringTypeVal\x12\x1e\n\nstatusCode\x18\x01 \x01(\x05R\nstatusCode\x12"\n\x0c\x65rrorMessage\x18\x02 \x01(\tR\x0c\x65rrorMessage\x12\x14\n\x05value\x18\x03 \x01(\tR\x05value"\xa0\x05\n\x15StatefulProcessorCall\x12h\n\x0esetHandleState\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetHandleStateH\x00R\x0esetHandleState\x12h\n\rgetValueState\x18\x02 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\rgetValueState\x12\x66\n\x0cgetListState\x18\x03 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0cgetListState\x12\x64\n\x0bgetMapState\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0bgetMapState\x12o\n\x0etimerStateCall\x18\x05 \x01(\x0b\x32\x45.org.apache.spark.sql.execution.streaming.state.TimerStateCallCommandH\x00R\x0etimerStateCall\x12j\n\x0e\x64\x65leteIfExists\x18\x06 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.StateCallCommandH\x00R\x0e\x64\x65leteIfExistsB\x08\n\x06method"\xd5\x02\n\x14StateVariableRequest\x12h\n\x0evalueStateCall\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.ValueStateCallH\x00R\x0evalueStateCall\x12\x65\n\rlistStateCall\x18\x02 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.ListStateCallH\x00R\rlistStateCall\x12\x62\n\x0cmapStateCall\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.MapStateCallH\x00R\x0cmapStateCallB\x08\n\x06method"\x83\x02\n\x1aImplicitGroupingKeyRequest\x12h\n\x0esetImplicitKey\x18\x01 \x01(\x0b\x32>.org.apache.spark.sql.execution.streaming.state.SetImplicitKeyH\x00R\x0esetImplicitKey\x12q\n\x11removeImplicitKey\x18\x02 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.RemoveImplicitKeyH\x00R\x11removeImplicitKeyB\x08\n\x06method"\x81\x02\n\x0cTimerRequest\x12q\n\x11timerValueRequest\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.TimerValueRequestH\x00R\x11timerValueRequest\x12t\n\x12\x65xpiryTimerRequest\x18\x02 \x01(\x0b\x32\x42.org.apache.spark.sql.execution.streaming.state.ExpiryTimerRequestH\x00R\x12\x65xpiryTimerRequestB\x08\n\x06method"\xf6\x01\n\x11TimerValueRequest\x12s\n\x12getProcessingTimer\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.GetProcessingTimeH\x00R\x12getProcessingTimer\x12\x62\n\x0cgetWatermark\x18\x02 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.GetWatermarkH\x00R\x0cgetWatermarkB\x08\n\x06method"B\n\x12\x45xpiryTimerRequest\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs"\x13\n\x11GetProcessingTime"\x0e\n\x0cGetWatermark"\x8b\x01\n\x0cUtilsRequest\x12q\n\x11parseStringSchema\x18\x01 \x01(\x0b\x32\x41.org.apache.spark.sql.execution.streaming.state.ParseStringSchemaH\x00R\x11parseStringSchemaB\x08\n\x06method"+\n\x11ParseStringSchema\x12\x16\n\x06schema\x18\x01 \x01(\tR\x06schema"\xc7\x01\n\x10StateCallCommand\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12\x16\n\x06schema\x18\x02 \x01(\tR\x06schema\x12\x30\n\x13mapStateValueSchema\x18\x03 \x01(\tR\x13mapStateValueSchema\x12K\n\x03ttl\x18\x04 \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.TTLConfigR\x03ttl"\xa7\x02\n\x15TimerStateCallCommand\x12[\n\x08register\x18\x01 \x01(\x0b\x32=.org.apache.spark.sql.execution.streaming.state.RegisterTimerH\x00R\x08register\x12U\n\x06\x64\x65lete\x18\x02 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.DeleteTimerH\x00R\x06\x64\x65lete\x12P\n\x04list\x18\x03 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.ListTimersH\x00R\x04listB\x08\n\x06method"\x92\x03\n\x0eValueStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12G\n\x03get\x18\x03 \x01(\x0b\x32\x33.org.apache.spark.sql.execution.streaming.state.GetH\x00R\x03get\x12n\n\x10valueStateUpdate\x18\x04 \x01(\x0b\x32@.org.apache.spark.sql.execution.streaming.state.ValueStateUpdateH\x00R\x10valueStateUpdate\x12M\n\x05\x63lear\x18\x05 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xdf\x04\n\rListStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12\x62\n\x0clistStateGet\x18\x03 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStateGetH\x00R\x0clistStateGet\x12\x62\n\x0clistStatePut\x18\x04 \x01(\x0b\x32<.org.apache.spark.sql.execution.streaming.state.ListStatePutH\x00R\x0clistStatePut\x12_\n\x0b\x61ppendValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.AppendValueH\x00R\x0b\x61ppendValue\x12\\\n\nappendList\x18\x06 \x01(\x0b\x32:.org.apache.spark.sql.execution.streaming.state.AppendListH\x00R\nappendList\x12M\n\x05\x63lear\x18\x07 \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method"\xc2\x06\n\x0cMapStateCall\x12\x1c\n\tstateName\x18\x01 \x01(\tR\tstateName\x12P\n\x06\x65xists\x18\x02 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ExistsH\x00R\x06\x65xists\x12V\n\x08getValue\x18\x03 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.GetValueH\x00R\x08getValue\x12_\n\x0b\x63ontainsKey\x18\x04 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.ContainsKeyH\x00R\x0b\x63ontainsKey\x12_\n\x0bupdateValue\x18\x05 \x01(\x0b\x32;.org.apache.spark.sql.execution.streaming.state.UpdateValueH\x00R\x0bupdateValue\x12V\n\x08iterator\x18\x06 \x01(\x0b\x32\x38.org.apache.spark.sql.execution.streaming.state.IteratorH\x00R\x08iterator\x12J\n\x04keys\x18\x07 \x01(\x0b\x32\x34.org.apache.spark.sql.execution.streaming.state.KeysH\x00R\x04keys\x12P\n\x06values\x18\x08 \x01(\x0b\x32\x36.org.apache.spark.sql.execution.streaming.state.ValuesH\x00R\x06values\x12Y\n\tremoveKey\x18\t \x01(\x0b\x32\x39.org.apache.spark.sql.execution.streaming.state.RemoveKeyH\x00R\tremoveKey\x12M\n\x05\x63lear\x18\n \x01(\x0b\x32\x35.org.apache.spark.sql.execution.streaming.state.ClearH\x00R\x05\x63learB\x08\n\x06method""\n\x0eSetImplicitKey\x12\x10\n\x03key\x18\x01 \x01(\x0cR\x03key"\x13\n\x11RemoveImplicitKey"\x08\n\x06\x45xists"\x05\n\x03Get"=\n\rRegisterTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs";\n\x0b\x44\x65leteTimer\x12,\n\x11\x65xpiryTimestampMs\x18\x01 \x01(\x03R\x11\x65xpiryTimestampMs",\n\nListTimers\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x10ValueStateUpdate\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x07\n\x05\x43lear".\n\x0cListStateGet\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"\x0e\n\x0cListStatePut"#\n\x0b\x41ppendValue\x12\x14\n\x05value\x18\x01 \x01(\x0cR\x05value"\x0c\n\nAppendList"$\n\x08GetValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"\'\n\x0b\x43ontainsKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"=\n\x0bUpdateValue\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey\x12\x14\n\x05value\x18\x02 \x01(\x0cR\x05value"*\n\x08Iterator\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"&\n\x04Keys\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"(\n\x06Values\x12\x1e\n\niteratorId\x18\x01 \x01(\tR\niteratorId"%\n\tRemoveKey\x12\x18\n\x07userKey\x18\x01 \x01(\x0cR\x07userKey"c\n\x0eSetHandleState\x12Q\n\x05state\x18\x01 \x01(\x0e\x32;.org.apache.spark.sql.execution.streaming.state.HandleStateR\x05state"+\n\tTTLConfig\x12\x1e\n\ndurationMs\x18\x01 \x01(\x05R\ndurationMs*n\n\x0bHandleState\x12\x0c\n\x08PRE_INIT\x10\x00\x12\x0b\n\x07\x43REATED\x10\x01\x12\x0f\n\x0bINITIALIZED\x10\x02\x12\x12\n\x0e\x44\x41TA_PROCESSED\x10\x03\x12\x13\n\x0fTIMER_PROCESSED\x10\x04\x12\n\n\x06\x43LOSED\x10\x05\x62\x06proto3' ) _globals = globals() @@ -51,7 +51,7 @@ if not _descriptor._USE_C_DESCRIPTORS: DESCRIPTOR._loaded_options = None _globals["_HANDLESTATE"]._serialized_start = 6408 - _globals["_HANDLESTATE"]._serialized_end = 6504 + _globals["_HANDLESTATE"]._serialized_end = 6518 _globals["_STATEREQUEST"]._serialized_start = 112 _globals["_STATEREQUEST"]._serialized_end = 756 _globals["_STATERESPONSE"]._serialized_start = 758 diff --git a/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi b/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi index 03ede5d25b2b7..ac4b03b820349 100644 --- a/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi +++ b/python/pyspark/sql/streaming/proto/StateMessage_pb2.pyi @@ -56,19 +56,21 @@ class _HandleStateEnumTypeWrapper( builtins.type, ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor - CREATED: _HandleState.ValueType # 0 - INITIALIZED: _HandleState.ValueType # 1 - DATA_PROCESSED: _HandleState.ValueType # 2 - TIMER_PROCESSED: _HandleState.ValueType # 3 - CLOSED: _HandleState.ValueType # 4 + PRE_INIT: _HandleState.ValueType # 0 + CREATED: _HandleState.ValueType # 1 + INITIALIZED: _HandleState.ValueType # 2 + DATA_PROCESSED: _HandleState.ValueType # 3 + TIMER_PROCESSED: _HandleState.ValueType # 4 + CLOSED: _HandleState.ValueType # 5 class HandleState(_HandleState, metaclass=_HandleStateEnumTypeWrapper): ... -CREATED: HandleState.ValueType # 0 -INITIALIZED: HandleState.ValueType # 1 -DATA_PROCESSED: HandleState.ValueType # 2 -TIMER_PROCESSED: HandleState.ValueType # 3 -CLOSED: HandleState.ValueType # 4 +PRE_INIT: HandleState.ValueType # 0 +CREATED: HandleState.ValueType # 1 +INITIALIZED: HandleState.ValueType # 2 +DATA_PROCESSED: HandleState.ValueType # 3 +TIMER_PROCESSED: HandleState.ValueType # 4 +CLOSED: HandleState.ValueType # 5 global___HandleState = HandleState class StateRequest(google.protobuf.message.Message): diff --git a/python/pyspark/sql/streaming/stateful_processor_api_client.py b/python/pyspark/sql/streaming/stateful_processor_api_client.py index 79bb63d81d79f..c25b2a2392a69 100644 --- a/python/pyspark/sql/streaming/stateful_processor_api_client.py +++ b/python/pyspark/sql/streaming/stateful_processor_api_client.py @@ -40,6 +40,7 @@ class StatefulProcessorHandleState(Enum): + PRE_INIT = 0 CREATED = 1 INITIALIZED = 2 DATA_PROCESSED = 3 @@ -48,14 +49,19 @@ class StatefulProcessorHandleState(Enum): class StatefulProcessorApiClient: - def __init__(self, state_server_port: int, key_schema: StructType) -> None: + def __init__( + self, state_server_port: int, key_schema: StructType, is_driver: bool = False + ) -> None: self.key_schema = key_schema self._client_socket = socket.socket() self._client_socket.connect(("localhost", state_server_port)) self.sockfile = self._client_socket.makefile( "rwb", int(os.environ.get("SPARK_BUFFER_SIZE", 65536)) ) - self.handle_state = StatefulProcessorHandleState.CREATED + if is_driver: + self.handle_state = StatefulProcessorHandleState.PRE_INIT + else: + self.handle_state = StatefulProcessorHandleState.CREATED self.utf8_deserializer = UTF8Deserializer() self.pickleSer = CPickleSerializer() self.serializer = ArrowStreamSerializer() @@ -70,7 +76,9 @@ def __init__(self, state_server_port: int, key_schema: StructType) -> None: def set_handle_state(self, state: StatefulProcessorHandleState) -> None: import pyspark.sql.streaming.proto.StateMessage_pb2 as stateMessage - if state == StatefulProcessorHandleState.CREATED: + if state == StatefulProcessorHandleState.PRE_INIT: + proto_state = stateMessage.PRE_INIT + elif state == StatefulProcessorHandleState.CREATED: proto_state = stateMessage.CREATED elif state == StatefulProcessorHandleState.INITIALIZED: proto_state = stateMessage.INITIALIZED diff --git a/python/pyspark/sql/streaming/stateful_processor_util.py b/python/pyspark/sql/streaming/stateful_processor_util.py index 6130a9581bc24..d69c1a943862c 100644 --- a/python/pyspark/sql/streaming/stateful_processor_util.py +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -25,3 +25,4 @@ class TransformWithStateInPandasFuncMode(Enum): PROCESS_DATA = 1 PROCESS_TIMER = 2 COMPLETE = 3 + PRE_INIT = 4 diff --git a/python/pyspark/sql/streaming/transform_with_state_driver_worker.py b/python/pyspark/sql/streaming/transform_with_state_driver_worker.py new file mode 100644 index 0000000000000..99d386f07b5b6 --- /dev/null +++ b/python/pyspark/sql/streaming/transform_with_state_driver_worker.py @@ -0,0 +1,102 @@ +# +# 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. +# + +import os +import json +from typing import Any, Iterator, TYPE_CHECKING + +from pyspark.util import local_connect_and_auth +from pyspark.serializers import ( + write_int, + read_int, + UTF8Deserializer, + CPickleSerializer, +) +from pyspark import worker +from pyspark.util import handle_worker_exception +from typing import IO +from pyspark.worker_util import check_python_version +from pyspark.sql.streaming.stateful_processor_api_client import StatefulProcessorApiClient +from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode +from pyspark.sql.types import StructType + +if TYPE_CHECKING: + from pyspark.sql.pandas._typing import ( + DataFrameLike as PandasDataFrameLike, + ) + +pickle_ser = CPickleSerializer() +utf8_deserializer = UTF8Deserializer() + + +def main(infile: IO, outfile: IO) -> None: + check_python_version(infile) + + log_name = "Streaming TransformWithStateInPandas Python worker" + print(f"Starting {log_name}.\n") + + def process( + processor: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, + key: Any, + input: Iterator["PandasDataFrameLike"], + ) -> None: + print(f"{log_name} Starting execution of UDF: {func}.\n") + func(processor, mode, key, input) + print(f"{log_name} Completed execution of UDF: {func}.\n") + + try: + func, return_type = worker.read_command(pickle_ser, infile) + print( + f"{log_name} finish init stage of Python runner. Received UDF from JVM: {func}, " + f"received return type of UDF: {return_type}.\n" + ) + # send signal for getting args + write_int(0, outfile) + outfile.flush() + + # This driver runner will only be used on the first batch of a query, + # and the following code block should be only run once for each query run + state_server_port = read_int(infile) + key_schema = StructType.fromJson(json.loads(utf8_deserializer.loads(infile))) + print( + f"{log_name} received parameters for UDF. State server port: {state_server_port}, " + f"key schema: {key_schema}.\n" + ) + + stateful_processor_api_client = StatefulProcessorApiClient(state_server_port, key_schema) + process( + stateful_processor_api_client, + TransformWithStateInPandasFuncMode.PRE_INIT, + None, + iter([]), + ) + write_int(0, outfile) + outfile.flush() + except Exception as e: + handle_worker_exception(e, outfile) + outfile.flush() + + +if __name__ == "__main__": + # Read information about how to connect back to the JVM from the environment. + java_port = int(os.environ["PYTHON_WORKER_FACTORY_PORT"]) + auth_secret = os.environ["PYTHON_WORKER_FACTORY_SECRET"] + (sock_file, sock) = local_connect_and_auth(java_port, auth_secret) + write_int(os.getpid(), sock_file) + sock_file.flush() + main(sock_file, sock_file) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 5405a0ae6765c..183e93a871bd8 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -15,6 +15,7 @@ # limitations under the License. # +import json import os import time import tempfile @@ -26,7 +27,7 @@ from pyspark import SparkConf from pyspark.errors import PySparkRuntimeError -from pyspark.sql.functions import split +from pyspark.sql.functions import array_sort, col, explode, split from pyspark.sql.types import StringType, StructType, StructField, Row, IntegerType, TimestampType from pyspark.testing import assertDataFrameEqual from pyspark.testing.sqlutils import ( @@ -99,9 +100,17 @@ def build_test_df_with_3_cols(self, input_path): return df_final def _test_transform_with_state_in_pandas_basic( - self, stateful_processor, check_results, single_batch=False, timeMode="None" + self, + stateful_processor, + check_results, + single_batch=False, + timeMode="None", + checkpoint_path=None, + initial_state=None, ): input_path = tempfile.mkdtemp() + if checkpoint_path is None: + checkpoint_path = tempfile.mkdtemp() self._prepare_test_resource1(input_path) if not single_batch: time.sleep(2) @@ -127,8 +136,10 @@ def _test_transform_with_state_in_pandas_basic( outputStructType=output_schema, outputMode="Update", timeMode=timeMode, + initialState=initial_state, ) .writeStream.queryName("this_query") + .option("checkpointLocation", checkpoint_path) .foreachBatch(check_results) .outputMode("update") .start() @@ -358,7 +369,7 @@ def check_results(batch_df, batch_id): for q in self.spark.streams.active: q.stop() if batch_id == 0 or batch_id == 1: - time.sleep(6) + time.sleep(4) input_dir = tempfile.TemporaryDirectory() input_path = input_dir.name @@ -591,17 +602,24 @@ def check_results(batch_df, batch_id): ) def _test_transform_with_state_init_state_in_pandas( - self, stateful_processor, check_results, time_mode="None" + self, + stateful_processor, + check_results, + time_mode="None", + checkpoint_path=None, + initial_state=None, ): input_path = tempfile.mkdtemp() + if checkpoint_path is None: + checkpoint_path = tempfile.mkdtemp() self._prepare_test_resource1(input_path) time.sleep(2) self._prepare_input_data(input_path + "/text-test2.txt", [0, 3], [67, 12]) - df = self._build_test_df(input_path) - for q in self.spark.streams.active: q.stop() + + df = self._build_test_df(input_path) self.assertTrue(df.isStreaming) output_schema = StructType( @@ -611,8 +629,9 @@ def _test_transform_with_state_init_state_in_pandas( ] ) - data = [("0", 789), ("3", 987)] - initial_state = self.spark.createDataFrame(data, "id string, initVal int").groupBy("id") + if initial_state is None: + data = [("0", 789), ("3", 987)] + initial_state = self.spark.createDataFrame(data, "id string, initVal int").groupBy("id") q = ( df.groupBy("id") @@ -624,6 +643,7 @@ def _test_transform_with_state_init_state_in_pandas( initialState=initial_state, ) .writeStream.queryName("this_query") + .option("checkpointLocation", checkpoint_path) .foreachBatch(check_results) .outputMode("update") .start() @@ -852,13 +872,6 @@ def check_results(batch_df, batch_id): StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" ) - # run the same test suites again but with single shuffle partition - def test_transform_with_state_with_timers_single_partition(self): - with self.sql_conf({"spark.sql.shuffle.partitions": "1"}): - self.test_transform_with_state_init_state_with_timers() - self.test_transform_with_state_in_pandas_event_time() - self.test_transform_with_state_in_pandas_proc_timer() - def test_transform_with_state_in_pandas_batch_query(self): data = [("0", 123), ("0", 46), ("1", 146), ("1", 346)] df = self.spark.createDataFrame(data, "id string, temperature int") @@ -907,6 +920,374 @@ def test_transform_with_state_in_pandas_batch_query_initial_state(self): Row(id="1", value=str(146 + 346)), } + # This test covers mapState with TTL, an empty state variable + # and additional test against initial state python runner + def test_transform_with_map_state_metadata(self): + checkpoint_path = tempfile.mktemp() + + def check_results(batch_df, batch_id): + if batch_id == 0: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } + else: + # check for state metadata source + metadata_df = self.spark.read.format("state-metadata").load(checkpoint_path) + assert set( + metadata_df.select( + "operatorId", + "operatorName", + "stateStoreName", + "numPartitions", + "minBatchId", + "maxBatchId", + ).collect() + ) == { + Row( + operatorId=0, + operatorName="transformWithStateInPandasExec", + stateStoreName="default", + numPartitions=5, + minBatchId=0, + maxBatchId=0, + ) + } + operator_properties_json_obj = json.loads( + metadata_df.select("operatorProperties").collect()[0][0] + ) + assert operator_properties_json_obj["timeMode"] == "ProcessingTime" + assert operator_properties_json_obj["outputMode"] == "Update" + + state_var_list = operator_properties_json_obj["stateVariables"] + assert len(state_var_list) == 3 + for state_var in state_var_list: + if state_var["stateName"] == "mapState": + assert state_var["stateVariableType"] == "MapState" + assert state_var["ttlEnabled"] + elif state_var["stateName"] == "listState": + assert state_var["stateVariableType"] == "ListState" + assert not state_var["ttlEnabled"] + else: + assert state_var["stateName"] == "$procTimers_keyToTimestamp" + assert state_var["stateVariableType"] == "TimerState" + + # check for state data source + map_state_df = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "mapState") + .load() + ) + assert map_state_df.selectExpr( + "key.id AS groupingKey", + "user_map_key.name AS mapKey", + "user_map_value.value.count AS mapValue", + ).sort("groupingKey").collect() == [ + Row(groupingKey="0", mapKey="key2", mapValue=2), + Row(groupingKey="1", mapKey="key2", mapValue=2), + ] + + # check for map state with flatten option + map_state_df_non_flatten = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "mapState") + .option("flattenCollectionTypes", False) + .load() + ) + assert map_state_df_non_flatten.select( + "key.id", explode(col("map_value")).alias("map_key", "map_value") + ).selectExpr( + "id AS groupingKey", + "map_key.name AS mapKey", + "map_value.value.count AS mapValue", + ).sort( + "groupingKey" + ).collect() == [ + Row(groupingKey="0", mapKey="key2", mapValue=2), + Row(groupingKey="1", mapKey="key2", mapValue=2), + ] + + ttl_df = map_state_df.selectExpr( + "user_map_value.ttlExpirationMs AS TTLVal" + ).collect() + # check if there are two rows containing TTL value in map state dataframe + assert len(ttl_df) == 2 + # check if two rows are of the same TTL value + assert len(set(ttl_df)) == 1 + + list_state_df = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "listState") + .load() + ) + assert list_state_df.isEmpty() + + for q in self.spark.streams.active: + q.stop() + + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=None, + ) + + # run the same test suite again but with no-op initial state + # TWS with initial state is using a different python runner + init_data = [("0", 789), ("3", 987)] + initial_state = self.spark.createDataFrame(init_data, "id string, temperature int").groupBy( + "id" + ) + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=initial_state, + ) + + # This test covers multiple list state variables and flatten option + def test_transform_with_list_state_metadata(self): + checkpoint_path = tempfile.mktemp() + + def check_results(batch_df, batch_id): + if batch_id == 0: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } + else: + # check for state metadata source + metadata_df = self.spark.read.format("state-metadata").load(checkpoint_path) + operator_properties_json_obj = json.loads( + metadata_df.select("operatorProperties").collect()[0][0] + ) + state_var_list = operator_properties_json_obj["stateVariables"] + assert len(state_var_list) == 3 + for state_var in state_var_list: + if state_var["stateName"] in ["listState1", "listState2"]: + state_var["stateVariableType"] == "ListState" + else: + assert state_var["stateName"] == "$procTimers_keyToTimestamp" + assert state_var["stateVariableType"] == "TimerState" + + # check for state data source and flatten option + list_state_1_df = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "listState1") + .option("flattenCollectionTypes", True) + .load() + ) + assert list_state_1_df.selectExpr( + "key.id AS groupingKey", + "list_element.temperature AS listElement", + ).sort("groupingKey", "listElement").collect() == [ + Row(groupingKey="0", listElement=20), + Row(groupingKey="0", listElement=20), + Row(groupingKey="0", listElement=111), + Row(groupingKey="0", listElement=120), + Row(groupingKey="0", listElement=120), + Row(groupingKey="1", listElement=20), + Row(groupingKey="1", listElement=20), + Row(groupingKey="1", listElement=111), + Row(groupingKey="1", listElement=120), + Row(groupingKey="1", listElement=120), + ] + + list_state_2_df = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "listState2") + .option("flattenCollectionTypes", False) + .load() + ) + assert list_state_2_df.selectExpr( + "key.id AS groupingKey", "list_value.temperature AS valueList" + ).sort("groupingKey").withColumn( + "valueSortedList", array_sort(col("valueList")) + ).select( + "groupingKey", "valueSortedList" + ).collect() == [ + Row(groupingKey="0", valueSortedList=[20, 20, 120, 120, 222]), + Row(groupingKey="1", valueSortedList=[20, 20, 120, 120, 222]), + ] + + for q in self.spark.streams.active: + q.stop() + + self._test_transform_with_state_in_pandas_basic( + ListStateProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=None, + ) + + # This test covers value state variable and read change feed, + # snapshotStartBatchId related options + def test_transform_with_value_state_metadata(self): + checkpoint_path = tempfile.mktemp() + + def check_results(batch_df, batch_id): + if batch_id == 0: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } + else: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="3"), + Row(id="1", countAsString="2"), + } + + # check for state metadata source + metadata_df = self.spark.read.format("state-metadata").load(checkpoint_path) + operator_properties_json_obj = json.loads( + metadata_df.select("operatorProperties").collect()[0][0] + ) + state_var_list = operator_properties_json_obj["stateVariables"] + + assert len(state_var_list) == 3 + for state_var in state_var_list: + if state_var["stateName"] in ["numViolations", "tempState"]: + state_var["stateVariableType"] == "ValueState" + else: + assert state_var["stateName"] == "$procTimers_keyToTimestamp" + assert state_var["stateVariableType"] == "TimerState" + + # check for state data source and readChangeFeed + value_state_df = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "numViolations") + .option("readChangeFeed", True) + .option("changeStartBatchId", 0) + .load() + ).selectExpr( + "change_type", "key.id AS groupingKey", "value.value AS value", "partition_id" + ) + + assert value_state_df.select("change_type", "groupingKey", "value").sort( + "groupingKey" + ).collect() == [ + Row(change_type="update", groupingKey="0", value=1), + Row(change_type="update", groupingKey="1", value=2), + ] + + partition_id_list = [ + row["partition_id"] for row in value_state_df.select("partition_id").collect() + ] + + for partition_id in partition_id_list: + # check for state data source and snapshotStartBatchId options + state_snapshot_df = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "numViolations") + .option("snapshotPartitionId", partition_id) + .option("snapshotStartBatchId", 0) + .load() + ) + + assert ( + value_state_df.select("partition_id", "groupingKey", "value") + .filter(value_state_df["partition_id"] == partition_id) + .sort("groupingKey") + .collect() + == state_snapshot_df.selectExpr( + "partition_id", "key.id AS groupingKey", "value.value AS value" + ) + .sort("groupingKey") + .collect() + ) + + for q in self.spark.streams.active: + q.stop() + + with self.sql_conf( + {"spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled": "true"} + ): + self._test_transform_with_state_in_pandas_basic( + SimpleStatefulProcessor(), + check_results, + False, + "processingTime", + checkpoint_path=checkpoint_path, + ) + + def test_transform_with_state_restart_with_multiple_rows_init_state(self): + def check_results(batch_df, _): + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } + + def check_results_for_new_query(batch_df, batch_id): + if batch_id == 0: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", value=str(123 + 46)), + Row(id="1", value=str(146 + 346)), + } + else: + assert set(batch_df.sort("id").collect()) == { + Row(id="0", value=str(123 + 46 + 67)), + Row(id="3", value=str(12)), + } + # verify values in initial state is appended into list state for all keys + df = ( + self.spark.read.format("statestore") + .option("path", new_checkpoint_path) + .option("stateVarName", "list_state") + .load() + ).selectExpr("key.id AS id", "list_element.value AS value") + + def dataframe_to_value_list(output_df): + return [ + row["value"] for row in output_df.sort("value").select("value").collect() + ] + + assert dataframe_to_value_list(df.filter(df.id == "0")) == [20, 20, 111, 120, 120] + assert dataframe_to_value_list(df.filter(df.id == "1")) == [20, 20, 111, 120, 120] + + # run a tws query and read state data source dataframe from its checkpoint + checkpoint_path = tempfile.mkdtemp() + self._test_transform_with_state_in_pandas_basic( + ListStateProcessor(), check_results, True, checkpoint_path=checkpoint_path + ) + list_state_df = ( + self.spark.read.format("statestore") + .option("path", checkpoint_path) + .option("stateVarName", "listState1") + .load() + ).selectExpr("key.id AS id", "list_element.temperature AS initVal") + init_df = list_state_df.groupBy("id") + + # run a new tws query and pass state data source dataframe as initial state + # multiple rows exist in the initial state with the same grouping key + new_checkpoint_path = tempfile.mkdtemp() + self._test_transform_with_state_init_state_in_pandas( + StatefulProcessorWithListStateInitialState(), + check_results_for_new_query, + checkpoint_path=new_checkpoint_path, + initial_state=init_df, + ) + + # run the same test suites again but with single shuffle partition + def test_transform_with_state_with_timers_single_partition(self): + with self.sql_conf({"spark.sql.shuffle.partitions": "1"}): + self.test_transform_with_state_init_state_with_timers() + self.test_transform_with_state_in_pandas_event_time() + self.test_transform_with_state_in_pandas_proc_timer() + self.test_transform_with_state_restart_with_multiple_rows_init_state() + class SimpleStatefulProcessorWithInitialState(StatefulProcessor): # this dict is the same as input initial state dataframe @@ -963,6 +1344,17 @@ def handleInitialState(self, key, initialState, timer_values) -> None: self.handle.registerTimer(timer_values.get_current_processing_time_in_ms() - 1) +class StatefulProcessorWithListStateInitialState(SimpleStatefulProcessorWithInitialState): + def init(self, handle: StatefulProcessorHandle) -> None: + super().init(handle) + list_ele_schema = StructType([StructField("value", IntegerType(), True)]) + self.list_state = handle.getListState("list_state", list_ele_schema) + + def handleInitialState(self, key, initialState, timer_values) -> None: + for val in initialState["initVal"].tolist(): + self.list_state.append_value((val,)) + + # A stateful processor that output the max event time it has seen. Register timer for # current watermark. Clear max state if timer expires. class EventTimeStatefulProcessor(StatefulProcessor): @@ -1297,6 +1689,7 @@ def init(self, handle: StatefulProcessorHandle) -> None: key_schema = StructType([StructField("name", StringType(), True)]) value_schema = StructType([StructField("count", IntegerType(), True)]) self.map_state = handle.getMapState("mapState", key_schema, value_schema, 30000) + self.list_state = handle.getListState("listState", key_schema) class TransformWithStateInPandasTests(TransformWithStateInPandasTestsMixin, ReusedSQLTestCase): diff --git a/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto b/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto index e69727a260a90..1374bd100a2fe 100644 --- a/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto +++ b/sql/core/src/main/protobuf/org/apache/spark/sql/execution/streaming/StateMessage.proto @@ -236,11 +236,12 @@ message RemoveKey { } enum HandleState { - CREATED = 0; - INITIALIZED = 1; - DATA_PROCESSED = 2; - TIMER_PROCESSED = 3; - CLOSED = 4; + PRE_INIT = 0; + CREATED = 1; + INITIALIZED = 2; + DATA_PROCESSED = 3; + TIMER_PROCESSED = 4; + CLOSED = 5; } message SetHandleState { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala index 2a9abfa5d6a50..4eebc19acee89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala @@ -53,6 +53,10 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging private lazy val serializedHadoopConf = new SerializableConfiguration(hadoopConf) + // Seq of operator names who uses state schema v3 and TWS related options. + // This Seq was used in checks before reading state schema files. + private val twsShortNameSeq = Seq("transformWithStateExec", "transformWithStateInPandasExec") + override def shortName(): String = "statestore" override def getTable( @@ -132,12 +136,11 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging private def runStateVarChecks( sourceOptions: StateSourceOptions, stateStoreMetadata: Array[StateMetadataTableEntry]): Unit = { - val twsShortName = "transformWithStateExec" if (sourceOptions.stateVarName.isDefined || sourceOptions.readRegisteredTimers) { // Perform checks for transformWithState operator in case state variable name is provided require(stateStoreMetadata.size == 1) val opMetadata = stateStoreMetadata.head - if (opMetadata.operatorName != twsShortName) { + if (!twsShortNameSeq.contains(opMetadata.operatorName)) { // if we are trying to query state source with state variable name, then the operator // should be transformWithState val errorMsg = "Providing state variable names is only supported with the " + @@ -178,7 +181,7 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging } else { // if the operator is transformWithState, then a state variable argument is mandatory if (stateStoreMetadata.size == 1 && - stateStoreMetadata.head.operatorName == twsShortName) { + twsShortNameSeq.contains(stateStoreMetadata.head.operatorName)) { throw StateDataSourceErrors.requiredOptionUnspecified("stateVarName") } } @@ -212,7 +215,7 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging // Read the schema file path from operator metadata version v2 onwards // for the transformWithState operator val oldSchemaFilePath = if (storeMetadata.length > 0 && storeMetadata.head.version == 2 - && storeMetadata.head.operatorName.contains("transformWithStateExec")) { + && twsShortNameSeq.exists(storeMetadata.head.operatorName.contains)) { val storeMetadataEntry = storeMetadata.head val operatorProperties = TransformWithStateOperatorProperties.fromJson( storeMetadataEntry.operatorPropertiesJson) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala index f8e9f11f4d738..9b51822679a91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala @@ -22,7 +22,7 @@ import scala.concurrent.duration.NANOSECONDS import org.apache.hadoop.conf.Configuration -import org.apache.spark.JobArtifactSet +import org.apache.spark.{JobArtifactSet, SparkException} import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -35,9 +35,9 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.PandasGroupUtils.{executePython, groupAndProject, resolveArgOffsets} -import org.apache.spark.sql.execution.streaming.{StatefulOperatorCustomMetric, StatefulOperatorCustomSumMetric, StatefulOperatorPartitioning, StatefulOperatorStateInfo, StatefulProcessorHandleImpl, StateStoreWriter, WatermarkSupport} +import org.apache.spark.sql.execution.streaming.{DriverStatefulProcessorHandleImpl, StatefulOperatorCustomMetric, StatefulOperatorCustomSumMetric, StatefulOperatorPartitioning, StatefulOperatorStateInfo, StatefulProcessorHandleImpl, StateStoreWriter, TransformWithStateMetadataUtils, TransformWithStateVariableInfo, WatermarkSupport} import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper -import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, RocksDBStateStoreProvider, StateSchemaValidationResult, StateStore, StateStoreConf, StateStoreId, StateStoreOps, StateStoreProvider, StateStoreProviderId} +import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, OperatorStateMetadata, RocksDBStateStoreProvider, StateSchemaValidationResult, StateStore, StateStoreColFamilySchema, StateStoreConf, StateStoreId, StateStoreOps, StateStoreProvider, StateStoreProviderId} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, TimeMode} import org.apache.spark.sql.types.{BinaryType, StructField, StructType} @@ -79,10 +79,12 @@ case class TransformWithStateInPandasExec( initialState: SparkPlan, initialStateGroupingAttrs: Seq[Attribute], initialStateSchema: StructType) - extends BinaryExecNode with StateStoreWriter with WatermarkSupport { + extends BinaryExecNode + with StateStoreWriter + with WatermarkSupport + with TransformWithStateMetadataUtils { override def shortName: String = "transformWithStateInPandasExec" - private val pythonUDF = functionExpr.asInstanceOf[PythonUDF] private val pythonFunction = pythonUDF.func private val chainedFunc = @@ -91,6 +93,7 @@ case class TransformWithStateInPandasExec( private val sessionLocalTimeZone = conf.sessionLocalTimeZone private val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf) private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) + private val (dedupAttributes, argOffsets) = resolveArgOffsets(child.output, groupingAttributes) private val groupingKeyStructFields = groupingAttributes .map(a => StructField(a.name, a.dataType, a.nullable)) @@ -109,6 +112,22 @@ case class TransformWithStateInPandasExec( // Each state variable has its own schema, this is a dummy one. protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType) + override def operatorStateMetadataVersion: Int = 2 + + override def getColFamilySchemas(): Map[String, StateStoreColFamilySchema] = { + driverProcessorHandle.getColumnFamilySchemas + } + + override def getStateVariableInfos(): Map[String, TransformWithStateVariableInfo] = { + driverProcessorHandle.getStateVariableInfos + } + + /** Metadata of this stateful operator and its states stores. + * Written during IncrementalExecution. `validateAndMaybeEvolveStateSchema` will initialize + * `columnFamilySchemas` and `stateVariableInfos` during `init()` call on driver. */ + private val driverProcessorHandle: DriverStatefulProcessorHandleImpl = + new DriverStatefulProcessorHandleImpl(timeMode, groupingKeyExprEncoder) + /** * Distribute by grouping attributes - We need the underlying data and the initial state data * to have the same grouping so that the data are co-located on the same task. @@ -129,12 +148,43 @@ case class TransformWithStateInPandasExec( groupingAttributes.map(SortOrder(_, Ascending)), initialStateGroupingAttrs.map(SortOrder(_, Ascending))) + override def operatorStateMetadata( + stateSchemaPaths: List[String]): OperatorStateMetadata = { + getOperatorStateMetadata(stateSchemaPaths, getStateInfo, shortName, timeMode, outputMode) + } + + override def validateNewMetadata( + oldOperatorMetadata: OperatorStateMetadata, + newOperatorMetadata: OperatorStateMetadata): Unit = { + validateNewMetadataForTWS(oldOperatorMetadata, newOperatorMetadata) + } + override def validateAndMaybeEvolveStateSchema( hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): List[StateSchemaValidationResult] = { - // TODO(SPARK-49212): Implement schema evolution support - List.empty + // Start a python runner on driver, and execute pre-init UDF on the runner + val runner = new TransformWithStateInPandasPythonPreInitRunner( + pythonFunction, + "pyspark.sql.streaming.transform_with_state_driver_worker", + sessionLocalTimeZone, + groupingKeySchema, + driverProcessorHandle + ) + // runner initialization + runner.init() + try { + // execute UDF on the python runner + runner.process() + } catch { + case e: Throwable => + throw new SparkException("TransformWithStateInPandas driver worker " + + "exited unexpectedly (crashed)", e) + } + runner.stop() + + validateAndWriteStateSchema(hadoopConf, batchId, stateSchemaVersion, getStateInfo, + session, operatorStateMetadataVersion) } override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = { @@ -315,7 +365,6 @@ case class TransformWithStateInPandasExec( val currentTimeNs = System.nanoTime val updatesStartTimeNs = currentTimeNs - val (dedupAttributes, argOffsets) = resolveArgOffsets(child.output, groupingAttributes) // If timeout is based on event time, then filter late data based on watermark val filteredIter = watermarkPredicateForDataForLateEvents match { case Some(predicate) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala index 641e49657ca95..79e199f598b0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.python -import java.io.DataOutputStream +import java.io.{DataInputStream, DataOutputStream} import java.net.ServerSocket import scala.concurrent.ExecutionContext @@ -25,13 +25,13 @@ import scala.concurrent.ExecutionContext import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.ipc.ArrowStreamWriter -import org.apache.spark.TaskContext -import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions, PythonRDD} +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions, PythonFunction, PythonRDD, PythonWorkerUtils, StreamingPythonRunner} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.TransformWithStateInPandasPythonRunner.{GroupedInType, InType} -import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleImpl +import org.apache.spark.sql.execution.streaming.{DriverStatefulProcessorHandleImpl, StatefulProcessorHandleImpl} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -177,13 +177,12 @@ abstract class TransformWithStateInPandasPythonBaseRunner[I]( funcs.map(_._1), evalType, argOffsets, jobArtifactUUID, pythonMetrics) with PythonArrowInput[I] with BasicPythonArrowOutput + with TransformWithStateInPandasPythonRunnerUtils with Logging { protected val sqlConf = SQLConf.get protected val arrowMaxRecordsPerBatch = sqlConf.arrowMaxRecordsPerBatch - private var stateServerSocketPort: Int = 0 - override protected val workerConf: Map[String, String] = initialWorkerConf + (SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key -> arrowMaxRecordsPerBatch.toString) @@ -205,21 +204,7 @@ abstract class TransformWithStateInPandasPythonBaseRunner[I]( inputIterator: Iterator[I], partitionIndex: Int, context: TaskContext): Iterator[ColumnarBatch] = { - var stateServerSocket: ServerSocket = null - var failed = false - try { - stateServerSocket = new ServerSocket( /* port = */ 0, - /* backlog = */ 1) - stateServerSocketPort = stateServerSocket.getLocalPort - } catch { - case e: Throwable => - failed = true - throw e - } finally { - if (failed) { - closeServerSocketChannelSilently(stateServerSocket) - } - } + initStateServer() val executor = ThreadUtils.newDaemonSingleThreadExecutor("stateConnectionListenerThread") val executionContext = ExecutionContext.fromExecutor(executor) @@ -239,7 +224,108 @@ abstract class TransformWithStateInPandasPythonBaseRunner[I]( super.compute(inputIterator, partitionIndex, context) } - private def closeServerSocketChannelSilently(stateServerSocket: ServerSocket): Unit = { + override protected def writeUDF(dataOut: DataOutputStream): Unit = { + PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets, None) + } +} + +/** + * TransformWithStateInPandas driver side Python runner. Similar as executor side runner, + * will start a new daemon thread on the Python runner to run state server. + */ +class TransformWithStateInPandasPythonPreInitRunner( + func: PythonFunction, + workerModule: String, + timeZoneId: String, + groupingKeySchema: StructType, + processorHandleImpl: DriverStatefulProcessorHandleImpl) + extends StreamingPythonRunner(func, "", "", workerModule) + with TransformWithStateInPandasPythonRunnerUtils + with Logging { + protected val sqlConf = SQLConf.get + + private var dataOut: DataOutputStream = _ + private var dataIn: DataInputStream = _ + + private var daemonThread: Thread = _ + + override def init(): (DataOutputStream, DataInputStream) = { + val result = super.init() + dataOut = result._1 + dataIn = result._2 + + // start state server, update socket port + startStateServer() + (dataOut, dataIn) + } + + def process(): Unit = { + // Also write the port number for state server + dataOut.writeInt(stateServerSocketPort) + PythonWorkerUtils.writeUTF(groupingKeySchema.json, dataOut) + dataOut.flush() + + val resFromPython = dataIn.readInt() + if (resFromPython != 0) { + val errMessage = PythonWorkerUtils.readUTF(dataIn) + throw streamingPythonRunnerInitializationFailure(resFromPython, errMessage) + } + } + + override def stop(): Unit = { + super.stop() + closeServerSocketChannelSilently(stateServerSocket) + daemonThread.stop() + } + + private def startStateServer(): Unit = { + initStateServer() + + daemonThread = new Thread { + override def run(): Unit = { + try { + new TransformWithStateInPandasStateServer(stateServerSocket, processorHandleImpl, + groupingKeySchema, timeZoneId, errorOnDuplicatedFieldNames = true, + largeVarTypes = sqlConf.arrowUseLargeVarTypes, + sqlConf.arrowTransformWithStateInPandasMaxRecordsPerBatch).run() + } catch { + case e: Exception => + throw new SparkException("TransformWithStateInPandas state server " + + "daemon thread exited unexpectedly (crashed)", e) + } + } + } + daemonThread.setDaemon(true) + daemonThread.setName("stateConnectionListenerThread") + daemonThread.start() + } +} + +/** + * TransformWithStateInPandas Python runner utils functions for handling a state server + * in a new daemon thread. + */ +trait TransformWithStateInPandasPythonRunnerUtils extends Logging { + protected var stateServerSocketPort: Int = 0 + protected var stateServerSocket: ServerSocket = null + protected def initStateServer(): Unit = { + var failed = false + try { + stateServerSocket = new ServerSocket(/* port = */ 0, + /* backlog = */ 1) + stateServerSocketPort = stateServerSocket.getLocalPort + } catch { + case e: Throwable => + failed = true + throw e + } finally { + if (failed) { + closeServerSocketChannelSilently(stateServerSocket) + } + } + } + + protected def closeServerSocketChannelSilently(stateServerSocket: ServerSocket): Unit = { try { logInfo(log"closing the state server socket") stateServerSocket.close() @@ -248,10 +334,6 @@ abstract class TransformWithStateInPandasPythonBaseRunner[I]( logError(log"failed to close state server socket", e) } } - - override protected def writeUDF(dataOut: DataOutputStream): Unit = { - PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets, None) - } } object TransformWithStateInPandasPythonRunner { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala index d03c75620df8a..a48d0bfd15034 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleState, StateVariableType} +import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleImplBase, StatefulProcessorHandleState, StateVariableType} import org.apache.spark.sql.execution.streaming.state.StateMessage.{HandleState, ImplicitGroupingKeyRequest, ListStateCall, MapStateCall, StatefulProcessorCall, StateRequest, StateResponse, StateResponseWithLongTypeVal, StateResponseWithStringTypeVal, StateVariableRequest, TimerRequest, TimerStateCallCommand, TimerValueRequest, UtilsRequest, ValueStateCall} import org.apache.spark.sql.streaming.{ListState, MapState, TTLConfig, ValueState} import org.apache.spark.sql.types.{BinaryType, LongType, StructField, StructType} @@ -53,7 +53,7 @@ import org.apache.spark.util.Utils */ class TransformWithStateInPandasStateServer( stateServerSocket: ServerSocket, - statefulProcessorHandle: StatefulProcessorHandleImpl, + statefulProcessorHandle: StatefulProcessorHandleImplBase, groupingKeySchema: StructType, timeZoneId: String, errorOnDuplicatedFieldNames: Boolean, @@ -228,11 +228,13 @@ class TransformWithStateInPandasStateServer( // API and it will only be used by `group_ops` once per partition, we won't // need to worry about different function calls will interleaved and hence // this implementation is safe + assert(statefulProcessorHandle.isInstanceOf[StatefulProcessorHandleImpl]) val expiryRequest = message.getExpiryTimerRequest() val expiryTimestamp = expiryRequest.getExpiryTimestampMs if (!expiryTimestampIter.isDefined) { expiryTimestampIter = - Option(statefulProcessorHandle.getExpiredTimers(expiryTimestamp)) + Option(statefulProcessorHandle + .asInstanceOf[StatefulProcessorHandleImpl].getExpiredTimers(expiryTimestamp)) } // expiryTimestampIter could be None in the TWSPandasServerSuite if (!expiryTimestampIter.isDefined || !expiryTimestampIter.get.hasNext) { @@ -281,6 +283,9 @@ class TransformWithStateInPandasStateServer( case StatefulProcessorCall.MethodCase.SETHANDLESTATE => val requestedState = message.getSetHandleState.getState requestedState match { + case HandleState.PRE_INIT => + logInfo(log"set handle state to Pre-init") + statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.PRE_INIT) case HandleState.CREATED => logInfo(log"set handle state to Created") statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CREATED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 719c4da14d729..511f4421e16ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -214,7 +214,7 @@ class IncrementalExecution( // filepath, and write this path out in the OperatorStateMetadata file case statefulOp: StatefulOperator if isFirstBatch => val stateSchemaVersion = statefulOp match { - case _: TransformWithStateExec => + case _: TransformWithStateExec | _: TransformWithStateInPandasExec => sparkSession.sessionState.conf. getConf(SQLConf.STREAMING_TRANSFORM_WITH_STATE_OP_STATE_SCHEMA_VERSION) case _ => STATE_SCHEMA_DEFAULT_VERSION diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala index 5716242afc152..aabbb5f8cacef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala @@ -20,7 +20,6 @@ import java.util.UUID import java.util.concurrent.TimeUnit.NANOSECONDS import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -76,7 +75,11 @@ case class TransformWithStateExec( initialStateDataAttrs: Seq[Attribute], initialStateDeserializer: Expression, initialState: SparkPlan) - extends BinaryExecNode with StateStoreWriter with WatermarkSupport with ObjectProducerExec { + extends BinaryExecNode + with StateStoreWriter + with WatermarkSupport + with ObjectProducerExec + with TransformWithStateMetadataUtils { override def shortName: String = "transformWithStateExec" @@ -122,32 +125,32 @@ case class TransformWithStateExec( driverProcessorHandle } + /** + * This method is used for the driver-side stateful processor after we + * have collected all the necessary schemas. + * This instance of the stateful processor won't be used again. + */ + private def closeProcessorHandle(): Unit = { + statefulProcessor.close() + statefulProcessor.setHandle(null) + } + /** * Fetching the columnFamilySchemas from the StatefulProcessorHandle * after init is called. */ - private def getColFamilySchemas(): Map[String, StateStoreColFamilySchema] = { + override def getColFamilySchemas(): Map[String, StateStoreColFamilySchema] = { val columnFamilySchemas = getDriverProcessorHandle().getColumnFamilySchemas closeProcessorHandle() columnFamilySchemas } - private def getStateVariableInfos(): Map[String, TransformWithStateVariableInfo] = { + override def getStateVariableInfos(): Map[String, TransformWithStateVariableInfo] = { val stateVariableInfos = getDriverProcessorHandle().getStateVariableInfos closeProcessorHandle() stateVariableInfos } - /** - * This method is used for the driver-side stateful processor after we - * have collected all the necessary schemas. - * This instance of the stateful processor won't be used again. - */ - private def closeProcessorHandle(): Unit = { - statefulProcessor.close() - statefulProcessor.setHandle(null) - } - /** * Controls watermark propagation to downstream modes. If timeMode is * ProcessingTime, the output rows cannot be interpreted in eventTime, hence @@ -457,84 +460,22 @@ case class TransformWithStateExec( hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): List[StateSchemaValidationResult] = { - assert(stateSchemaVersion >= 3) - val newSchemas = getColFamilySchemas() - val stateSchemaDir = stateSchemaDirPath() - val newStateSchemaFilePath = - new Path(stateSchemaDir, s"${batchId}_${UUID.randomUUID().toString}") - val metadataPath = new Path(getStateInfo.checkpointLocation, s"${getStateInfo.operatorId}") - val metadataReader = OperatorStateMetadataReader.createReader( - metadataPath, hadoopConf, operatorStateMetadataVersion, batchId) - val operatorStateMetadata = try { - metadataReader.read() - } catch { - // If this is the first time we are running the query, there will be no metadata - // and this error is expected. In this case, we return None. - case ex: Exception if batchId == 0 => - None - } - - val oldStateSchemaFilePath: Option[Path] = operatorStateMetadata match { - case Some(metadata) => - metadata match { - case v2: OperatorStateMetadataV2 => - Some(new Path(v2.stateStoreInfo.head.stateSchemaFilePath)) - case _ => None - } - case None => None - } - List(StateSchemaCompatibilityChecker. - validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, - newSchemas.values.toList, session.sessionState, stateSchemaVersion, - storeName = StateStoreId.DEFAULT_STORE_NAME, - oldSchemaFilePath = oldStateSchemaFilePath, - newSchemaFilePath = Some(newStateSchemaFilePath))) + val info = getStateInfo + validateAndWriteStateSchema(hadoopConf, batchId, stateSchemaVersion, + info, session, operatorStateMetadataVersion) } /** Metadata of this stateful operator and its states stores. */ override def operatorStateMetadata( stateSchemaPaths: List[String]): OperatorStateMetadata = { val info = getStateInfo - val operatorInfo = OperatorInfoV1(info.operatorId, shortName) - // stateSchemaFilePath should be populated at this point - val stateStoreInfo = - Array(StateStoreMetadataV2( - StateStoreId.DEFAULT_STORE_NAME, 0, info.numPartitions, stateSchemaPaths.head)) - - val operatorProperties = TransformWithStateOperatorProperties( - timeMode.toString, - outputMode.toString, - getStateVariableInfos().values.toList - ) - OperatorStateMetadataV2(operatorInfo, stateStoreInfo, operatorProperties.json) - } - - private def stateSchemaDirPath(): Path = { - val storeName = StateStoreId.DEFAULT_STORE_NAME - val stateCheckpointPath = - new Path(getStateInfo.checkpointLocation, - s"${getStateInfo.operatorId.toString}") - - val stateSchemaPath = new Path(stateCheckpointPath, "_stateSchema") - val storeNamePath = new Path(stateSchemaPath, storeName) - storeNamePath + getOperatorStateMetadata(stateSchemaPaths, info, shortName, timeMode, outputMode) } override def validateNewMetadata( oldOperatorMetadata: OperatorStateMetadata, newOperatorMetadata: OperatorStateMetadata): Unit = { - (oldOperatorMetadata, newOperatorMetadata) match { - case ( - oldMetadataV2: OperatorStateMetadataV2, - newMetadataV2: OperatorStateMetadataV2) => - val oldOperatorProps = TransformWithStateOperatorProperties.fromJson( - oldMetadataV2.operatorPropertiesJson) - val newOperatorProps = TransformWithStateOperatorProperties.fromJson( - newMetadataV2.operatorPropertiesJson) - TransformWithStateOperatorProperties.validateOperatorProperties( - oldOperatorProps, newOperatorProps) - case (_, _) => - } + validateNewMetadataForTWS(oldOperatorMetadata, newOperatorMetadata) } override protected def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala index bc67cee57fef8..34dddeab59d29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala @@ -16,6 +16,10 @@ */ package org.apache.spark.sql.execution.streaming +import java.util.UUID + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats import org.json4s.JsonAST._ import org.json4s.JsonDSL._ @@ -23,9 +27,10 @@ import org.json4s.jackson.JsonMethods import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType -import org.apache.spark.sql.execution.streaming.state.StateStoreErrors -import org.apache.spark.sql.streaming.TimeMode +import org.apache.spark.sql.execution.streaming.state.{OperatorInfoV1, OperatorStateMetadata, OperatorStateMetadataReader, OperatorStateMetadataV2, StateSchemaCompatibilityChecker, StateSchemaValidationResult, StateStoreColFamilySchema, StateStoreErrors, StateStoreId, StateStoreMetadataV2} +import org.apache.spark.sql.streaming.{OutputMode, TimeMode} /** * This file contains utility classes and functions for managing state variables in @@ -158,3 +163,104 @@ object TransformWithStateOperatorProperties extends Logging { } } } + +/** + * This trait contains utils functions related to TransformWithState metadata. + * This is used both in Scala and Python side of TransformWithState metadata support when calling + * `init()` with DriverStatefulProcessorHandleImpl, and get the state schema and state metadata + * on driver during physical planning phase. + */ +trait TransformWithStateMetadataUtils extends Logging { + def getColFamilySchemas(): Map[String, StateStoreColFamilySchema] + + def getStateVariableInfos(): Map[String, TransformWithStateVariableInfo] + + def getOperatorStateMetadata( + stateSchemaPaths: List[String], + info: StatefulOperatorStateInfo, + shortName: String, + timeMode: TimeMode, + outputMode: OutputMode): OperatorStateMetadata = { + val operatorInfo = OperatorInfoV1(info.operatorId, shortName) + // stateSchemaFilePath should be populated at this point + val stateStoreInfo = + Array(StateStoreMetadataV2( + StateStoreId.DEFAULT_STORE_NAME, 0, info.numPartitions, stateSchemaPaths.head)) + + val operatorProperties = TransformWithStateOperatorProperties( + timeMode.toString, + outputMode.toString, + getStateVariableInfos().values.toList + ) + OperatorStateMetadataV2(operatorInfo, stateStoreInfo, operatorProperties.json) + } + + def validateAndWriteStateSchema( + hadoopConf: Configuration, + batchId: Long, + stateSchemaVersion: Int, + info: StatefulOperatorStateInfo, + session: SparkSession, + operatorStateMetadataVersion: Int = 2): List[StateSchemaValidationResult] = { + assert(stateSchemaVersion >= 3) + val newSchemas = getColFamilySchemas() + val stateSchemaDir = stateSchemaDirPath(info) + val newStateSchemaFilePath = + new Path(stateSchemaDir, s"${batchId}_${UUID.randomUUID().toString}") + val metadataPath = new Path(info.checkpointLocation, s"${info.operatorId}") + val metadataReader = OperatorStateMetadataReader.createReader( + metadataPath, hadoopConf, operatorStateMetadataVersion, batchId) + val operatorStateMetadata = try { + metadataReader.read() + } catch { + // If this is the first time we are running the query, there will be no metadata + // and this error is expected. In this case, we return None. + case _: Exception if batchId == 0 => + None + } + + val oldStateSchemaFilePath: Option[Path] = operatorStateMetadata match { + case Some(metadata) => + metadata match { + case v2: OperatorStateMetadataV2 => + Some(new Path(v2.stateStoreInfo.head.stateSchemaFilePath)) + case _ => None + } + case None => None + } + // state schema file written here, writing the new schema list we passed here + List(StateSchemaCompatibilityChecker. + validateAndMaybeEvolveStateSchema(info, hadoopConf, + newSchemas.values.toList, session.sessionState, stateSchemaVersion, + storeName = StateStoreId.DEFAULT_STORE_NAME, + oldSchemaFilePath = oldStateSchemaFilePath, + newSchemaFilePath = Some(newStateSchemaFilePath))) + } + + def validateNewMetadataForTWS( + oldOperatorMetadata: OperatorStateMetadata, + newOperatorMetadata: OperatorStateMetadata): Unit = { + (oldOperatorMetadata, newOperatorMetadata) match { + case ( + oldMetadataV2: OperatorStateMetadataV2, + newMetadataV2: OperatorStateMetadataV2) => + val oldOperatorProps = TransformWithStateOperatorProperties.fromJson( + oldMetadataV2.operatorPropertiesJson) + val newOperatorProps = TransformWithStateOperatorProperties.fromJson( + newMetadataV2.operatorPropertiesJson) + TransformWithStateOperatorProperties.validateOperatorProperties( + oldOperatorProps, newOperatorProps) + case (_, _) => + } + } + + private def stateSchemaDirPath(info: StatefulOperatorStateInfo): Path = { + val storeName = StateStoreId.DEFAULT_STORE_NAME + val stateCheckpointPath = + new Path(info.checkpointLocation, s"${info.operatorId.toString}") + + val stateSchemaPath = new Path(stateCheckpointPath, "_stateSchema") + val storeNamePath = new Path(stateSchemaPath, storeName) + storeNamePath + } +} From 94f9bb0b61a42489104564209ede0f11af7c5ccf Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 27 Dec 2024 08:31:30 +0800 Subject: [PATCH 261/438] [SPARK-50673][ML] Avoid traversing model coefficients twice in `Word2VecModel` constructor ### What changes were proposed in this pull request? Avoid traversing model twice in `Word2VecModel` constructor ### Why are the changes needed? In public constructor `def this(model: Map[String, Array[Float]])`, the implementation traverses the model coefficients (which is a Map) twice to build the `Word2VecModel`, for `wordIndex` and `wordVectors`, respectively. I am not sure whether it might be problematic, since the two traversals needs the same ordering. Generating the result with single pass can guarantee the correctness, anyway. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49298 from zhengruifeng/ml_w2v_build. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../apache/spark/mllib/feature/Word2Vec.scala | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index b5b2233ecb756..100fa13db5180 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -520,9 +520,15 @@ class Word2VecModel private[spark] ( } } + // Auxiliary constructor must begin with call to 'this'. + // Helper constructor for `def this(model: Map[String, Array[Float]])`. + private def this(model: (Map[String, Int], Array[Float])) = { + this(model._1, model._2) + } + @Since("1.5.0") def this(model: Map[String, Array[Float]]) = { - this(Word2VecModel.buildWordIndex(model), Word2VecModel.buildWordVectors(model)) + this(Word2VecModel.buildFromVecMap(model)) } @Since("1.4.0") @@ -642,21 +648,22 @@ class Word2VecModel private[spark] ( @Since("1.4.0") object Word2VecModel extends Loader[Word2VecModel] { - private def buildWordIndex(model: Map[String, Array[Float]]): Map[String, Int] = { - CUtils.toMapWithIndex(model.keys) - } - - private def buildWordVectors(model: Map[String, Array[Float]]): Array[Float] = { + private def buildFromVecMap( + model: Map[String, Array[Float]]): (Map[String, Int], Array[Float]) = { require(model.nonEmpty, "Word2VecMap should be non-empty") + val (vectorSize, numWords) = (model.head._2.length, model.size) - val wordList = model.keys.toArray val wordVectors = new Array[Float](vectorSize * numWords) - var i = 0 - while (i < numWords) { - Array.copy(model(wordList(i)), 0, wordVectors, i * vectorSize, vectorSize) - i += 1 + + val wordIndex = collection.immutable.Map.newBuilder[String, Int] + wordIndex.sizeHint(numWords) + + model.iterator.zipWithIndex.foreach { + case ((word, vector), i) => + wordIndex += ((word, i)) + Array.copy(vector, 0, wordVectors, i * vectorSize, vectorSize) } - wordVectors + (wordIndex.result(), wordVectors) } private object SaveLoadV1_0 { From f3426b7b45dc9484894270018ec55982e5c4b174 Mon Sep 17 00:00:00 2001 From: Stevo Mitric Date: Fri, 27 Dec 2024 10:23:30 +0900 Subject: [PATCH 262/438] [SPARK-50669][SQL] Change the signature of TimestampAdd expression ### What changes were proposed in this pull request? In this PR the signature of `TimestampAdd` function was changed from `IntegerType` to `LongType` for quantity parameter. ### Why are the changes needed? To allow long types to be passed to timestamp_add function. Currently passing anything larger then `Int32.MaxValue` will cause a cast overflow exception. There is no need to restrict this method to only in32 values. ### Does this PR introduce _any_ user-facing change? Yes, the following query passes: ```sql select timestampadd( millisecond, 1733011227403, timestamp '1970-01-01T00:00:00.000Z' ) ``` Above query now returns `2024-12-01T00:00:27.000+00:00` instead of throwing a cast exception. ### How was this patch tested? New tests in this PR. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49291 from stevomitric/stevomitric/timestampadd-long. Authored-by: Stevo Mitric Signed-off-by: Hyukjin Kwon --- .../expressions/datetimeExpressions.scala | 4 +- .../sql/catalyst/util/DateTimeUtils.scala | 31 ++++--- .../sql/errors/QueryExecutionErrors.scala | 4 +- .../expressions/DateExpressionsSuite.scala | 87 ++++++++++++++----- .../function_timestamp_add.explain | 2 +- .../timestampNTZ/timestamp-ansi.sql.out | 4 +- .../timestampNTZ/timestamp.sql.out | 4 +- .../errors/QueryExecutionErrorsSuite.scala | 2 +- 8 files changed, 96 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 12fe456eace8d..81be40b3b6474 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -3431,7 +3431,7 @@ case class TimestampAdd( override def left: Expression = quantity override def right: Expression = timestamp - override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType, AnyTimestampType) + override def inputTypes: Seq[AbstractDataType] = Seq(LongType, AnyTimestampType) override def dataType: DataType = timestamp.dataType override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = @@ -3440,7 +3440,7 @@ case class TimestampAdd( @transient private lazy val zoneIdInEval: ZoneId = zoneIdForType(timestamp.dataType) override def nullSafeEval(q: Any, micros: Any): Any = { - DateTimeUtils.timestampAdd(unit, q.asInstanceOf[Int], micros.asInstanceOf[Long], zoneIdInEval) + DateTimeUtils.timestampAdd(unit, q.asInstanceOf[Long], micros.asInstanceOf[Long], zoneIdInEval) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index c9ca3ed864c16..1f741169898e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -667,33 +667,44 @@ object DateTimeUtils extends SparkDateTimeUtils { * @param zoneId The time zone ID at which the operation is performed. * @return A timestamp value, expressed in microseconds since 1970-01-01 00:00:00Z. */ - def timestampAdd(unit: String, quantity: Int, micros: Long, zoneId: ZoneId): Long = { + def timestampAdd(unit: String, quantity: Long, micros: Long, zoneId: ZoneId): Long = { try { unit.toUpperCase(Locale.ROOT) match { case "MICROSECOND" => timestampAddInterval(micros, 0, 0, quantity, zoneId) case "MILLISECOND" => timestampAddInterval(micros, 0, 0, - Math.multiplyExact(quantity.toLong, MICROS_PER_MILLIS), zoneId) + Math.multiplyExact(quantity, MICROS_PER_MILLIS), zoneId) case "SECOND" => timestampAddInterval(micros, 0, 0, - Math.multiplyExact(quantity.toLong, MICROS_PER_SECOND), zoneId) + Math.multiplyExact(quantity, MICROS_PER_SECOND), zoneId) case "MINUTE" => timestampAddInterval(micros, 0, 0, - Math.multiplyExact(quantity.toLong, MICROS_PER_MINUTE), zoneId) + Math.multiplyExact(quantity, MICROS_PER_MINUTE), zoneId) case "HOUR" => timestampAddInterval(micros, 0, 0, - Math.multiplyExact(quantity.toLong, MICROS_PER_HOUR), zoneId) + Math.multiplyExact(quantity, MICROS_PER_HOUR), zoneId) case "DAY" | "DAYOFYEAR" => - timestampAddInterval(micros, 0, quantity, 0, zoneId) + // Given that more than `Int32.MaxValue` days will cause an `ArithmeticException` due to + // overflow, we can safely cast the quantity to an `Int` here. Same follows for larger + // unites. + timestampAddInterval(micros, 0, Math.toIntExact(quantity), 0, zoneId) case "WEEK" => - timestampAddInterval(micros, 0, Math.multiplyExact(quantity, DAYS_PER_WEEK), 0, zoneId) + timestampAddInterval( + micros, + 0, + Math.multiplyExact(Math.toIntExact(quantity), DAYS_PER_WEEK), + 0, + zoneId) case "MONTH" => - timestampAddMonths(micros, quantity, zoneId) + timestampAddMonths(micros, Math.toIntExact(quantity), zoneId) case "QUARTER" => - timestampAddMonths(micros, Math.multiplyExact(quantity, 3), zoneId) + timestampAddMonths(micros, Math.multiplyExact(Math.toIntExact(quantity), 3), zoneId) case "YEAR" => - timestampAddMonths(micros, Math.multiplyExact(quantity, MONTHS_PER_YEAR), zoneId) + timestampAddMonths( + micros, + Math.multiplyExact(Math.toIntExact(quantity), MONTHS_PER_YEAR), + zoneId) } } catch { case _: scala.MatchError => 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 2ec85a38723cb..e500f5e3cbd78 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 @@ -2474,11 +2474,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE ) } - def timestampAddOverflowError(micros: Long, amount: Int, unit: String): ArithmeticException = { + def timestampAddOverflowError(micros: Long, amount: Long, unit: String): ArithmeticException = { new SparkArithmeticException( errorClass = "DATETIME_OVERFLOW", messageParameters = Map( - "operation" -> (s"add ${toSQLValue(amount, IntegerType)} $unit to " + + "operation" -> (s"add ${toSQLValue(amount, LongType)} $unit to " + s"${toSQLValue(DateTimeUtils.microsToInstant(micros), TimestampType)}")), context = Array.empty, summary = "") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 5cd974838fa24..09650a0dcc022 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1893,26 +1893,26 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-38195: add a quantity of interval units to a timestamp") { // Check case-insensitivity checkEvaluation( - TimestampAdd("Hour", Literal(1), Literal(LocalDateTime.of(2022, 2, 15, 12, 57, 0))), + TimestampAdd("Hour", Literal(1L), Literal(LocalDateTime.of(2022, 2, 15, 12, 57, 0))), LocalDateTime.of(2022, 2, 15, 13, 57, 0)) // Check nulls as input values checkEvaluation( TimestampAdd( "MINUTE", - Literal.create(null, IntegerType), + Literal.create(null, LongType), Literal(LocalDateTime.of(2022, 2, 15, 12, 57, 0))), null) checkEvaluation( TimestampAdd( "MINUTE", - Literal(1), + Literal(1L), Literal.create(null, TimestampType)), null) // Check crossing the daylight saving time checkEvaluation( TimestampAdd( "HOUR", - Literal(6), + Literal(6L), Literal(Instant.parse("2022-03-12T23:30:00Z")), Some("America/Los_Angeles")), Instant.parse("2022-03-13T05:30:00Z")) @@ -1920,7 +1920,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( TimestampAdd( "DAY", - Literal(2), + Literal(2L), Literal(LocalDateTime.of(2020, 2, 28, 10, 11, 12)), Some("America/Los_Angeles")), LocalDateTime.of(2020, 3, 1, 10, 11, 12)) @@ -1940,7 +1940,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { quantity, timestamp, Some(tz)), - IntegerType, tsType) + LongType, tsType) } } } @@ -1961,84 +1961,127 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // timestampadd(DAY, 1, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 checkEvaluation( - TimestampAdd("DAY", Literal(1), Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + TimestampAdd("DAY", Literal(1L), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), skippedTime) // timestampadd(HOUR, 24, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 checkEvaluation( - TimestampAdd("HOUR", Literal(24), + TimestampAdd("HOUR", Literal(24L), Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), skippedTime + MICROS_PER_HOUR) // timestampadd(HOUR, 23, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 checkEvaluation( - TimestampAdd("HOUR", Literal(23), + TimestampAdd("HOUR", Literal(23L), Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), skippedTime) // timestampadd(SECOND, SECONDS_PER_DAY, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 checkEvaluation( TimestampAdd( - "SECOND", Literal(SECONDS_PER_DAY.toInt), + "SECOND", Literal(SECONDS_PER_DAY), Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), skippedTime + MICROS_PER_HOUR) // timestampadd(SECOND, SECONDS_PER_DAY, 2011-03-12 03:00:00) = 2011-03-13 03:59:59 checkEvaluation( TimestampAdd( - "SECOND", Literal(SECONDS_PER_DAY.toInt - 1), + "SECOND", Literal(SECONDS_PER_DAY - 1), Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), skippedTime + MICROS_PER_HOUR - MICROS_PER_SECOND) // timestampadd(DAY, 1, 2011-11-05 02:00:00) = 2011-11-06 02:00:00 checkEvaluation( - TimestampAdd("DAY", Literal(1), + TimestampAdd("DAY", Literal(1L), Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), repeatedTime + MICROS_PER_HOUR) // timestampadd(DAY, 1, 2011-11-05 01:00:00) = 2011-11-06 01:00:00 (pre-transition) checkEvaluation( - TimestampAdd("DAY", Literal(1), + TimestampAdd("DAY", Literal(1L), Literal(repeatedTime - 25 * MICROS_PER_HOUR, TimestampType)), repeatedTime - MICROS_PER_HOUR) // timestampadd(DAY, -1, 2011-11-07 01:00:00) = 2011-11-06 01:00:00 (post-transition) checkEvaluation( - TimestampAdd("DAY", Literal(-1), + TimestampAdd("DAY", Literal(-1L), Literal(repeatedTime + 24 * MICROS_PER_HOUR, TimestampType)), repeatedTime) // timestampadd(MONTH, 1, 2011-10-06 01:00:00) = 2011-11-06 01:00:00 (pre-transition) checkEvaluation( TimestampAdd( - "MONTH", Literal(1), + "MONTH", Literal(1L), Literal(repeatedTime - MICROS_PER_HOUR - 31 * MICROS_PER_DAY, TimestampType)), repeatedTime - MICROS_PER_HOUR) // timestampadd(MONTH, -1, 2011-12-06 01:00:00) = 2011-11-06 01:00:00 (post-transition) checkEvaluation( TimestampAdd( - "MONTH", Literal(-1), + "MONTH", Literal(-1L), Literal(repeatedTime + 30 * MICROS_PER_DAY, TimestampType)), repeatedTime) // timestampadd(HOUR, 23, 2011-11-05 02:00:00) = 2011-11-06 01:00:00 (pre-transition) checkEvaluation( - TimestampAdd("HOUR", Literal(23), + TimestampAdd("HOUR", Literal(23L), Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), repeatedTime - MICROS_PER_HOUR) // timestampadd(HOUR, 24, 2011-11-05 02:00:00) = 2011-11-06 01:00:00 (post-transition) checkEvaluation( - TimestampAdd("HOUR", Literal(24), + TimestampAdd("HOUR", Literal(24L), Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), repeatedTime) } } + test("SPARK-50669: timestampadd with long types") { + // A value that is larger than Int.MaxValue. + val longValue = 10_000_000_000L + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + checkEvaluation( + TimestampAdd("MICROSECOND", Literal(longValue), Literal(0L, TimestampType)), + longValue) + checkEvaluation( + TimestampAdd("MILLISECOND", Literal(longValue), Literal(0L, TimestampType)), + longValue * MICROS_PER_MILLIS) + checkEvaluation( + TimestampAdd("SECOND", Literal(longValue), Literal(0L, TimestampType)), + longValue * MICROS_PER_SECOND) + checkEvaluation( + TimestampAdd("MINUTE", Literal(longValue), Literal(0L, TimestampType)), + longValue * MICROS_PER_MINUTE) + + // Add a smaller value so overflow doesn't happen. + val valueToAdd = 1_000L + checkEvaluation( + TimestampAdd("HOUR", Literal(valueToAdd), Literal(0L, TimestampType)), + valueToAdd * MICROS_PER_HOUR) + checkEvaluation( + TimestampAdd("DAY", Literal(valueToAdd), Literal(0L, TimestampType)), + valueToAdd * MICROS_PER_DAY) + checkEvaluation( + TimestampAdd("WEEK", Literal(valueToAdd), Literal(0L, TimestampType)), + valueToAdd * MICROS_PER_DAY * DAYS_PER_WEEK) + + // Make sure overflow are thrown for larger values. + val overflowVal = Long.MaxValue + Seq("MILLISECOND", "SECOND", "MINUTE", "HOUR", "DAY", "WEEK").foreach { interval => + checkErrorInExpression[SparkArithmeticException](TimestampAdd(interval, + Literal(overflowVal), + Literal(0L, TimestampType)), + condition = "DATETIME_OVERFLOW", + parameters = Map("operation" -> + s"add ${overflowVal}L $interval to TIMESTAMP '1970-01-01 00:00:00'")) + } + } + } + test("SPARK-42635: timestampadd unit conversion overflow") { withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { checkErrorInExpression[SparkArithmeticException](TimestampAdd("DAY", - Literal(106751992), + Literal(106751992L), Literal(0L, TimestampType)), condition = "DATETIME_OVERFLOW", - parameters = Map("operation" -> "add 106751992 DAY to TIMESTAMP '1970-01-01 00:00:00'")) + parameters = Map("operation" -> "add 106751992L DAY to TIMESTAMP '1970-01-01 00:00:00'")) checkErrorInExpression[SparkArithmeticException](TimestampAdd("QUARTER", - Literal(1431655764), + Literal(1431655764L), Literal(0L, TimestampType)), condition = "DATETIME_OVERFLOW", parameters = Map("operation" -> - "add 1431655764 QUARTER to TIMESTAMP '1970-01-01 00:00:00'")) + "add 1431655764L QUARTER to TIMESTAMP '1970-01-01 00:00:00'")) } } diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_timestamp_add.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_timestamp_add.explain index 36dde1393cdb2..4b46e8453a1c0 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_timestamp_add.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_timestamp_add.explain @@ -1,2 +1,2 @@ -Project [timestampadd(week, cast(x#0L as int), t#0, Some(America/Los_Angeles)) AS timestampadd(week, x, t)#0] +Project [timestampadd(week, x#0L, t#0, Some(America/Los_Angeles)) AS timestampadd(week, x, t)#0] +- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out index ec227afc87fe1..f900293918c69 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out @@ -759,14 +759,14 @@ Project [from_csv(StructField(t,TimestampNTZType,true), (timestampFormat,dd/MMMM -- !query select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') -- !query analysis -Project [timestampadd(MONTH, -1, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MONTH, -1, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +Project [timestampadd(MONTH, cast(-1 as bigint), 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MONTH, -1, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +- OneRowRelation -- !query select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') -- !query analysis -Project [timestampadd(MINUTE, 58, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MINUTE, 58, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +Project [timestampadd(MINUTE, cast(58 as bigint), 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MINUTE, 58, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out index 7475f837250d5..6dd91c8cdccf6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out @@ -819,14 +819,14 @@ Project [from_csv(StructField(t,TimestampNTZType,true), (timestampFormat,dd/MMMM -- !query select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') -- !query analysis -Project [timestampadd(MONTH, -1, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MONTH, -1, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +Project [timestampadd(MONTH, cast(-1 as bigint), 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MONTH, -1, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +- OneRowRelation -- !query select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') -- !query analysis -Project [timestampadd(MINUTE, 58, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MINUTE, 58, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +Project [timestampadd(MINUTE, cast(58 as bigint), 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MINUTE, 58, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] +- OneRowRelation diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 9abda729c02ed..17c3c1e1e2a70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -351,7 +351,7 @@ class QueryExecutionErrorsSuite sql("select timestampadd(YEAR, 1000000, timestamp'2022-03-09 01:02:03')").collect() }, condition = "DATETIME_OVERFLOW", - parameters = Map("operation" -> "add 1000000 YEAR to TIMESTAMP '2022-03-09 01:02:03'"), + parameters = Map("operation" -> "add 1000000L YEAR to TIMESTAMP '2022-03-09 01:02:03'"), sqlState = "22008") } From 928655beb9bfa8c4260b0bd855e4c4dfc0229a55 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 27 Dec 2024 10:39:28 +0900 Subject: [PATCH 263/438] [SPARK-50677][BUILD] Upgrade jupiter-interface to 0.13.3 and Junit5 to 5.11.4 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This pr aims to upgrade `jupiter-interface` from 0.13.1 to 0.13.3 and Junit5 to the latest version(Platform 1.11.4 + Jupiter 5.11.4). ### Why are the changes needed? The new version of `jupiter-interface` brings two fixes: - https://github.com/sbt/sbt-jupiter-interface/pull/126 - https://github.com/sbt/sbt-jupiter-interface/pull/128 The full release notes of `jupiter-interface` as follows: - https://github.com/sbt/sbt-jupiter-interface/releases/tag/v0.13.3 and the full release notes between Junit 5.11.3 to 5.11.4 as follows: - https://junit.org/junit5/docs/5.11.4/release-notes/#release-notes-5.11.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 #49301 from LuciferYang/SPARK-50677. Authored-by: yangjie01 Signed-off-by: Hyukjin Kwon --- pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- project/plugins.sbt | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index bfbfdeebd1eb2..b6f0f317c43cc 100644 --- a/pom.xml +++ b/pom.xml @@ -217,13 +217,13 @@ 4.1.115.Final 2.0.69.Final 76.1 - 5.11.3 - 1.11.3 + 5.11.4 + 1.11.4 - 0.13.1 + 0.13.3 org.fusesource.leveldbjni - 6.13.4 - 1.17.6 + 7.0.1 ${java.home} @@ -2594,11 +2593,6 @@ javax.servlet-api ${javaxservlet.version} - - com.squareup.okio - okio - ${okio.version} - diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 211c6c93b9674..814a3e1c595b8 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -37,7 +37,7 @@ io.fabric8 - volcano-model-v1beta1 + volcano-model ${kubernetes-client.version} @@ -105,11 +105,6 @@ test - - io.fabric8 - kubernetes-httpclient-okhttp - ${kubernetes-client.version} - io.fabric8 kubernetes-client diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 2c28dc380046c..557bf01cbdbae 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -24,10 +24,7 @@ import com.google.common.io.Files import io.fabric8.kubernetes.client.{ConfigBuilder, KubernetesClient, KubernetesClientBuilder} import io.fabric8.kubernetes.client.Config.KUBERNETES_REQUEST_RETRY_BACKOFFLIMIT_SYSTEM_PROPERTY import io.fabric8.kubernetes.client.Config.autoConfigure -import io.fabric8.kubernetes.client.okhttp.OkHttpClientFactory import io.fabric8.kubernetes.client.utils.Utils.getSystemPropertyOrEnvVar -import okhttp3.Dispatcher -import okhttp3.OkHttpClient import org.apache.spark.SparkConf import org.apache.spark.annotation.{DeveloperApi, Since, Stable} @@ -35,7 +32,6 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.K8S_CONTEXT import org.apache.spark.internal.config.ConfigEntry -import org.apache.spark.util.ThreadUtils /** * :: DeveloperApi :: @@ -78,10 +74,6 @@ object SparkKubernetesClientFactory extends Logging { .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") val clientCertFile = sparkConf .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") - // TODO(SPARK-37687): clean up direct usage of OkHttpClient, see also: - // https://github.com/fabric8io/kubernetes-client/issues/3547 - val dispatcher = new Dispatcher( - ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) // Allow for specifying a context used to auto-configure from the users K8S config file val kubeContext = sparkConf.get(KUBERNETES_CONTEXT).filter(_.nonEmpty) @@ -117,17 +109,9 @@ object SparkKubernetesClientFactory extends Logging { }.withOption(namespace) { (ns, configBuilder) => configBuilder.withNamespace(ns) }.build() - val factoryWithCustomDispatcher = new OkHttpClientFactory() { - override protected def additionalConfig(builder: OkHttpClient.Builder): Unit = { - builder.dispatcher(dispatcher) - } - } logDebug("Kubernetes client config: " + new ObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(config)) - new KubernetesClientBuilder() - .withHttpClientFactory(factoryWithCustomDispatcher) - .withConfig(config) - .build() + new KubernetesClientBuilder().withConfig(config).build() } private implicit class OptionConfigurableConfigBuilder(val configBuilder: ConfigBuilder) diff --git a/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala b/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala index 314550713ef16..046d268df4e29 100644 --- a/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala +++ b/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala @@ -17,8 +17,8 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model._ +import io.fabric8.volcano.api.model.scheduling.v1beta1.{PodGroup, PodGroupSpec} import io.fabric8.volcano.client.DefaultVolcanoClient -import io.fabric8.volcano.scheduling.v1beta1.{PodGroup, PodGroupSpec} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverConf, KubernetesExecutorConf, SparkPod} import org.apache.spark.internal.Logging diff --git a/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala b/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala index dab414e0e19e7..c92164993ef80 100644 --- a/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.features import java.io.File -import io.fabric8.volcano.scheduling.v1beta1.PodGroup +import io.fabric8.volcano.api.model.scheduling.v1beta1.PodGroup import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ diff --git a/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala b/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala index 2f414b72ee195..4a9dc135ecfdc 100644 --- a/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala @@ -28,8 +28,8 @@ import scala.concurrent.Future import scala.jdk.CollectionConverters._ import io.fabric8.kubernetes.api.model.{HasMetadata, Pod, Quantity} +import io.fabric8.volcano.api.model.scheduling.v1beta1.{Queue, QueueBuilder} import io.fabric8.volcano.client.VolcanoClient -import io.fabric8.volcano.scheduling.v1beta1.{Queue, QueueBuilder} import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually From 98dc76364ffab660b52fb8a22e5d2fbce4bd8c47 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Sat, 4 Jan 2025 13:28:23 -0800 Subject: [PATCH 311/438] [SPARK-50716][CORE] Fix the cleanup logic for symbolic links in `JavaUtils.deleteRecursivelyUsingJavaIO` method ### What changes were proposed in this pull request? To address the cleanup logic for symbolic links in the `JavaUtils.deleteRecursivelyUsingJavaIO` method, the following changes have been made in this pr: 1. Change to use `Files.readAttributes(file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS)` to read the `BasicFileAttributes` of the file. By specifying `LinkOption.NOFOLLOW_LINKS`, the attributes of the symbolic link itself are read, rather than those of the file it points to. This allows us to use `fileAttributes.isSymbolicLink()` to check if a file is a symbolic link. 2. After the above change, it is no longer possible for `fileAttributes.isDirectory()` and `fileAttributes.isSymbolicLink()` to be true simultaneously. Therefore, when `fileAttributes.isDirectory()` is true, there is no need to check `!fileAttributes.isSymbolicLink()`. 3. When `fileAttributes.isSymbolicLink()` is true, deletion behavior for the symbolic link has been added. 4. When `!file.exists()` is true, an additional check for `!fileAttributes.isSymbolicLink()` has been added. This is because for a broken symbolic link, `file.exists()` will also return false, but in such cases, we should proceed with the cleanup. 5. The previously handwritten `isSymlink` method in JavaUtils has been removed, as it is no longer needed after the above changes. ### Why are the changes needed? Fix the cleanup logic for symbolic links in `JavaUtils.deleteRecursivelyUsingJavaIO` method. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - New test cases have been added - Check with existing test cases which named `PipedRDDSuite`: Run `build/sbt "core/testOnly org.apache.spark.rdd.PipedRDDSuite"` Before ``` git status On branch upmaster Your branch is up to date with 'upstream/master'. Untracked files: (use "git add ..." to include in what will be committed) core/tasks/ ls -l core/tasks total 0 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 099f2492-acef-4556-8a34-1318dccf7ad2 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 47d46196-2f7b-4c7b-acf3-7e1d26584c12 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 5e23fe20-1e3f-49b8-8404-5cd3b1033e37 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 a2cbf5a9-3ebf-4332-be87-c9501830750e drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 ddf45bf5-d0fa-4970-9094-930f382b675c drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 e25fe5ad-a0be-48d0-81f6-605542f447b5 ls -l core/tasks/099f2492-acef-4556-8a34-1318dccf7ad2 total 0 lrwxr-xr-x 1 yangjie01 staff 59 1 3 18:15 benchmarks -> /Users/yangjie01/SourceCode/git/spark-sbt/core/./benchmarks lrwxr-xr-x 1 yangjie01 staff 52 1 3 18:15 src -> /Users/yangjie01/SourceCode/git/spark-sbt/core/./src lrwxr-xr-x 1 yangjie01 staff 55 1 3 18:15 target -> /Users/yangjie01/SourceCode/git/spark-sbt/core/./target ``` We noticed that symbolic links are left behind after the tests, even though manual cleanup has been invoked in the test code: https://github.com/apache/spark/blob/b210f422b0078d535eddc696ebba8d92f67b81fb/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala#L214-L232 After ``` git status On branch deleteRecursivelyUsingJavaIO-SymbolicLink Your branch is up to date with 'origin/deleteRecursivelyUsingJavaIO-SymbolicLink'. nothing to commit, working tree clean ``` We observe that there are no residual symbolic links left after the tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49347 from LuciferYang/deleteRecursivelyUsingJavaIO-SymbolicLink. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: Dongjoon Hyun --- .../apache/spark/network/util/JavaUtils.java | 23 ++++------- .../org/apache/spark/util/UtilsSuite.scala | 38 +++++++++++++++++++ 2 files changed, 45 insertions(+), 16 deletions(-) diff --git a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java index 90dddc2cb08c1..027a0d5fb402e 100644 --- a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -22,6 +22,7 @@ import java.nio.channels.ReadableByteChannel; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.nio.file.LinkOption; import java.nio.file.attribute.BasicFileAttributes; import java.util.*; import java.util.concurrent.TimeUnit; @@ -125,10 +126,11 @@ public static void deleteRecursively(File file, FilenameFilter filter) throws IO private static void deleteRecursivelyUsingJavaIO( File file, FilenameFilter filter) throws IOException { - if (!file.exists()) return; BasicFileAttributes fileAttributes = - Files.readAttributes(file.toPath(), BasicFileAttributes.class); - if (fileAttributes.isDirectory() && !isSymlink(file)) { + Files.readAttributes(file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); + // SPARK-50716: If the file does not exist and not a broken symbolic link, return directly. + if (!file.exists() && !fileAttributes.isSymbolicLink()) return; + if (fileAttributes.isDirectory()) { IOException savedIOException = null; for (File child : listFilesSafely(file, filter)) { try { @@ -143,8 +145,8 @@ private static void deleteRecursivelyUsingJavaIO( } } - // Delete file only when it's a normal file or an empty directory. - if (fileAttributes.isRegularFile() || + // Delete file only when it's a normal file, a symbolic link, or an empty directory. + if (fileAttributes.isRegularFile() || fileAttributes.isSymbolicLink() || (fileAttributes.isDirectory() && listFilesSafely(file, null).length == 0)) { boolean deleted = file.delete(); // Delete can also fail if the file simply did not exist. @@ -192,17 +194,6 @@ private static File[] listFilesSafely(File file, FilenameFilter filter) throws I } } - private static boolean isSymlink(File file) throws IOException { - Objects.requireNonNull(file); - File fileInCanonicalDir = null; - if (file.getParent() == null) { - fileInCanonicalDir = file; - } else { - fileInCanonicalDir = new File(file.getParentFile().getCanonicalFile(), file.getName()); - } - return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); - } - private static final Map timeSuffixes; private static final Map byteSuffixes; diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index febc59e5c5369..baa748573d75b 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -22,6 +22,7 @@ import java.lang.reflect.Field import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.{Files => JFiles} import java.text.DecimalFormatSymbols import java.util.Locale import java.util.concurrent.TimeUnit @@ -731,6 +732,43 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties { assert(!sourceFile2.exists()) } + test("SPARK-50716: deleteRecursively - SymbolicLink To File") { + val tempDir = Utils.createTempDir() + val sourceFile = new File(tempDir, "foo.txt") + JFiles.write(sourceFile.toPath, "Some content".getBytes) + assert(sourceFile.exists()) + + val symlinkFile = new File(tempDir, "bar.txt") + JFiles.createSymbolicLink(symlinkFile.toPath, sourceFile.toPath) + + // Check that the symlink was created successfully + assert(JFiles.isSymbolicLink(symlinkFile.toPath)) + Utils.deleteRecursively(tempDir) + + // Verify that everything is deleted + assert(!tempDir.exists) + } + + test("SPARK-50716: deleteRecursively - SymbolicLink To Dir") { + val tempDir = Utils.createTempDir() + val sourceDir = new File(tempDir, "sourceDir") + assert(sourceDir.mkdir()) + val sourceFile = new File(sourceDir, "file.txt") + JFiles.write(sourceFile.toPath, "Some content".getBytes) + + val symlinkDir = new File(tempDir, "targetDir") + JFiles.createSymbolicLink(symlinkDir.toPath, sourceDir.toPath) + + // Check that the symlink was created successfully + assert(JFiles.isSymbolicLink(symlinkDir.toPath)) + + // Now delete recursively + Utils.deleteRecursively(tempDir) + + // Verify that everything is deleted + assert(!tempDir.exists) + } + test("loading properties from file") { withTempDir { tmpDir => val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) From 0d0fa86a03119608159078b0f6474db96485a80f Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Sat, 4 Jan 2025 13:30:32 -0800 Subject: [PATCH 312/438] [SPARK-50578][PYTHON][SS][FOLLOWUP] Change to use `Thread.interrupt` instead of `Thread.stop` to interrupt the execution of TransformWithStateInPandasPythonPreInitRunner#daemonThread ### What changes were proposed in this pull request? This PR change to use `Thread.interrupt()` instead of `Thread.stop()` to attempt to interrupt the execution of `TransformWithStateInPandasPythonPreInitRunner#daemonThread`. Additionally, logic has been added in `TransformWithStateInPandasStateServer#run` to respond to the interrupt by setting the `CLOSED` state and exiting. ### Why are the changes needed? The `Thread.stop` method in Java 21 directly throws an `UnsupportedOperationException`, which led to the failure of the Java 21 daily tests: - https://github.com/apache/spark/actions/runs/12511573912/job/34903859772 - https://github.com/apache/spark/actions/runs/12523542188/job/34933207012 - https://github.com/apache/spark/actions/runs/12592534465/job/35097321533 ![image](https://github.com/user-attachments/assets/75cef6d7-d66a-4652-b01d-38412d6db3b0) So the primary purpose of this change is to restore the daily tests for Java 21. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Pass Java 21 GitHub Action test: https://github.com/LuciferYang/spark/actions/runs/12606699142/job/35137180872 ![image](https://github.com/user-attachments/assets/9e5e8b08-d167-4f7a-959c-8ebe6e22f9bc) ### Was this patch authored or co-authored using generative AI tooling? No Closes #49354 from LuciferYang/java21-test-2. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../python/TransformWithStateInPandasPythonRunner.scala | 2 +- .../python/TransformWithStateInPandasStateServer.scala | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala index 79e199f598b0c..f415ae2543d34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala @@ -275,7 +275,7 @@ class TransformWithStateInPandasPythonPreInitRunner( override def stop(): Unit = { super.stop() closeServerSocketChannelSilently(stateServerSocket) - daemonThread.stop() + daemonThread.interrupt() } private def startStateServer(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala index a48d0bfd15034..fe1bbdd66ac17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala @@ -146,6 +146,9 @@ class TransformWithStateInPandasStateServer( while (listeningSocket.isConnected && statefulProcessorHandle.getHandleState != StatefulProcessorHandleState.CLOSED) { + if (Thread.currentThread().isInterrupted) { + throw new InterruptedException("Thread was interrupted") + } try { val version = inputStream.readInt() if (version != -1) { @@ -159,6 +162,11 @@ class TransformWithStateInPandasStateServer( logWarning(log"No more data to read from the socket") statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CLOSED) return + case _: InterruptedException => + logInfo(log"Thread interrupted, shutting down state server") + Thread.currentThread().interrupt() + statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CLOSED) + return case e: Exception => logError(log"Error reading message: ${MDC(LogKeys.ERROR, e.getMessage)}", e) sendResponse(1, e.getMessage) From 91f3fdd25852b43095dd5273358fc394ffd11b66 Mon Sep 17 00:00:00 2001 From: Zhihong Yu Date: Mon, 6 Jan 2025 00:08:11 +0800 Subject: [PATCH 313/438] [SPARK-50578][PYTHON][SS][FOLLOWUP] Remove unnecessary if block ### What changes were proposed in this pull request? This PR removes the `if` block at the beginning of the while loop since `InterruptedException` is handled in the try / catch block. ### Why are the changes needed? This change makes the code easier to follow. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing Java 21 daily tests ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49356 from tedyu/tws-if. Authored-by: Zhihong Yu Signed-off-by: yangjie01 --- .../python/TransformWithStateInPandasStateServer.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala index fe1bbdd66ac17..e37e4266b46b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala @@ -146,9 +146,6 @@ class TransformWithStateInPandasStateServer( while (listeningSocket.isConnected && statefulProcessorHandle.getHandleState != StatefulProcessorHandleState.CLOSED) { - if (Thread.currentThread().isInterrupted) { - throw new InterruptedException("Thread was interrupted") - } try { val version = inputStream.readInt() if (version != -1) { From 24c9247208c20baf3985d9248eca4098cfe5bce3 Mon Sep 17 00:00:00 2001 From: William Hyun Date: Sun, 5 Jan 2025 15:18:22 -0800 Subject: [PATCH 314/438] [SPARK-50730][SQL][TESTS] Skip TPC-DS collation benchmark in branch-3.5 ### What changes were proposed in this pull request? This PR aims to skip TPC-DS collation benchmark in branch-3.5 ### Why are the changes needed? TPS-DS collation benchmark was added by the following PR at Spark 4.0 - https://github.com/apache/spark/pull/45739 However, branch-3.5 daily CI is trying to run non-existing tests. - https://github.com/apache/spark/actions/runs/12618956363/job/35162950623 Screenshot 2025-01-05 at 2 48 54 PM ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Check manually after merge. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49360 from williamhyun/SPARK-50730. Authored-by: William Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 13f4e66842ef1..29621930b1ffa 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1094,6 +1094,7 @@ jobs: spark.sql.autoBroadcastJoinThreshold=-1 spark.sql.join.forceApplyShuffledHashJoin=true - name: Run TPC-DS queries on collated data + if: inputs.branch != 'branch-3.5' run: | SPARK_TPCDS_DATA=`pwd`/tpcds-sf-1 build/sbt "sql/testOnly org.apache.spark.sql.TPCDSCollationQueryTestSuite" - name: Upload test results to report From b613b228892d7c450c91112f14d1ace85cbf4c0b Mon Sep 17 00:00:00 2001 From: Evan Wu Date: Mon, 6 Jan 2025 10:16:09 +0800 Subject: [PATCH 315/438] [SPARK-50703][PYTHON] Refine the docstring of regexp_replace, regexp_substr, and regexp_instr ### What changes were proposed in this pull request? Refined docstring for `regexp_replace`, `regexp_substr`, and `regexp_instr`. ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? doctests ### Was this patch authored or co-authored using generative AI tooling? No Closes #49348 from drexler-sky/docstring2. Authored-by: Evan Wu Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/builtin.py | 149 +++++++++++++++++++----- 1 file changed, 121 insertions(+), 28 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 5d557c7277a30..0ff9b17d68a2c 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15535,43 +15535,102 @@ def regexp_replace( Examples -------- - >>> df = spark.createDataFrame([("100-200", r"(\d+)", "--")], ["str", "pattern", "replacement"]) - >>> df.select(regexp_replace('str', r'(\d+)', '--').alias('d')).collect() - [Row(d='-----')] - >>> df.select(regexp_replace("str", col("pattern"), col("replacement")).alias('d')).collect() - [Row(d='-----')] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame( + ... [("100-200", r"(\d+)", "--")], + ... ["str", "pattern", "replacement"] + ... ) + + Example 1: Replaces all the substrings in the `str` column name that + match the regex pattern `(\d+)` (one or more digits) with the replacement + string "--". + + >>> df.select('*', sf.regexp_replace('str', r'(\d+)', '--')).show() + +-------+-------+-----------+---------------------------------+ + | str|pattern|replacement|regexp_replace(str, (\d+), --, 1)| + +-------+-------+-----------+---------------------------------+ + |100-200| (\d+)| --| -----| + +-------+-------+-----------+---------------------------------+ + + Example 2: Replaces all the substrings in the `str` Column that match + the regex pattern in the `pattern` Column with the string in the `replacement` + column. + + >>> df.select('*', \ + ... sf.regexp_replace(sf.col("str"), sf.col("pattern"), sf.col("replacement")) \ + ... ).show() + +-------+-------+-----------+--------------------------------------------+ + | str|pattern|replacement|regexp_replace(str, pattern, replacement, 1)| + +-------+-------+-----------+--------------------------------------------+ + |100-200| (\d+)| --| -----| + +-------+-------+-----------+--------------------------------------------+ """ return _invoke_function_over_columns("regexp_replace", string, lit(pattern), lit(replacement)) @_try_remote_functions def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - r"""Returns the substring that matches the Java regex `regexp` within the string `str`. + r"""Returns the first substring that matches the Java regex `regexp` within the string `str`. If the regular expression is not found, the result is null. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - regexp : :class:`~pyspark.sql.Column` or str + regexp : :class:`~pyspark.sql.Column` or column name regex pattern to apply. Returns ------- :class:`~pyspark.sql.Column` - the substring that matches a Java regex within the string `str`. + the first substring that matches a Java regex within the string `str`. Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("1a 2b 14m", r"\d+")], ["str", "regexp"]) - >>> df.select(regexp_substr('str', lit(r'\d+')).alias('d')).collect() - [Row(d='1')] - >>> df.select(regexp_substr('str', lit(r'mmm')).alias('d')).collect() - [Row(d=None)] - >>> df.select(regexp_substr("str", col("regexp")).alias('d')).collect() - [Row(d='1')] + + Example 1: Returns the first substring in the `str` column name that + matches the regex pattern `(\d+)` (one or more digits). + + >>> df.select('*', sf.regexp_substr('str', sf.lit(r'\d+'))).show() + +---------+------+-----------------------+ + | str|regexp|regexp_substr(str, \d+)| + +---------+------+-----------------------+ + |1a 2b 14m| \d+| 1| + +---------+------+-----------------------+ + + Example 2: Returns the first substring in the `str` column name that + matches the regex pattern `(mmm)` (three consecutive 'm' characters) + + >>> df.select('*', sf.regexp_substr('str', sf.lit(r'mmm'))).show() + +---------+------+-----------------------+ + | str|regexp|regexp_substr(str, mmm)| + +---------+------+-----------------------+ + |1a 2b 14m| \d+| NULL| + +---------+------+-----------------------+ + + Example 3: Returns the first substring in the `str` column name that + matches the regex pattern in `regexp` Column. + + >>> df.select('*', sf.regexp_substr("str", sf.col("regexp"))).show() + +---------+------+--------------------------+ + | str|regexp|regexp_substr(str, regexp)| + +---------+------+--------------------------+ + |1a 2b 14m| \d+| 1| + +---------+------+--------------------------+ + + Example 4: Returns the first substring in the `str` Column that + matches the regex pattern in `regexp` column name. + + >>> df.select('*', sf.regexp_substr(sf.col("str"), "regexp")).show() + +---------+------+--------------------------+ + | str|regexp|regexp_substr(str, regexp)| + +---------+------+--------------------------+ + |1a 2b 14m| \d+| 1| + +---------+------+--------------------------+ """ return _invoke_function_over_columns("regexp_substr", str, regexp) @@ -15580,36 +15639,70 @@ def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: def regexp_instr( str: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None ) -> Column: - r"""Extract all strings in the `str` that match the Java regex `regexp` + r"""Returns the position of the first substring in the `str` that match the Java regex `regexp` and corresponding to the regex group index. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - regexp : :class:`~pyspark.sql.Column` or str + regexp : :class:`~pyspark.sql.Column` or column name regex pattern to apply. - idx : int, optional + idx : :class:`~pyspark.sql.Column` or int, optional matched group id. Returns ------- :class:`~pyspark.sql.Column` - all strings in the `str` that match a Java regex and corresponding to the regex group index. + the position of the first substring in the `str` that match a Java regex and corresponding + to the regex group index. Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("1a 2b 14m", r"\d+(a|b|m)")], ["str", "regexp"]) - >>> df.select(regexp_instr('str', lit(r'\d+(a|b|m)')).alias('d')).collect() - [Row(d=1)] - >>> df.select(regexp_instr('str', lit(r'\d+(a|b|m)'), 1).alias('d')).collect() - [Row(d=1)] - >>> df.select(regexp_instr('str', lit(r'\d+(a|b|m)'), 2).alias('d')).collect() - [Row(d=1)] - >>> df.select(regexp_instr('str', col("regexp")).alias('d')).collect() - [Row(d=1)] + + Example 1: Returns the position of the first substring in the `str` column name that + match the regex pattern `(\d+(a|b|m))` (one or more digits followed by 'a', 'b', or 'm'). + + >>> df.select('*', sf.regexp_instr('str', sf.lit(r'\d+(a|b|m)'))).show() + +---------+----------+--------------------------------+ + | str| regexp|regexp_instr(str, \d+(a|b|m), 0)| + +---------+----------+--------------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+--------------------------------+ + + Example 2: Returns the position of the first substring in the `str` column name that + match the regex pattern `(\d+(a|b|m))` (one or more digits followed by 'a', 'b', or 'm'), + + >>> df.select('*', sf.regexp_instr('str', sf.lit(r'\d+(a|b|m)'), sf.lit(1))).show() + +---------+----------+--------------------------------+ + | str| regexp|regexp_instr(str, \d+(a|b|m), 1)| + +---------+----------+--------------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+--------------------------------+ + + Example 3: Returns the position of the first substring in the `str` column name that + match the regex pattern in `regexp` Column. + + >>> df.select('*', sf.regexp_instr('str', sf.col("regexp"))).show() + +---------+----------+----------------------------+ + | str| regexp|regexp_instr(str, regexp, 0)| + +---------+----------+----------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+----------------------------+ + + Example 4: Returns the position of the first substring in the `str` Column that + match the regex pattern in `regexp` column name. + + >>> df.select('*', sf.regexp_instr(sf.col("str"), "regexp")).show() + +---------+----------+----------------------------+ + | str| regexp|regexp_instr(str, regexp, 0)| + +---------+----------+----------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+----------------------------+ """ if idx is None: return _invoke_function_over_columns("regexp_instr", str, regexp) From 3ecb2908dea68b96a0ec8a00ffd69ad872a6d42a Mon Sep 17 00:00:00 2001 From: cashmand Date: Mon, 6 Jan 2025 10:32:54 +0800 Subject: [PATCH 316/438] [SPARK-50614][SQL] Revert API change for shredding ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/49234 changed the `setSchema` method of ParquetWriteSupport to add a shredding schema parameter. To avoid the risk of breaking libraries that call `setSchema`, this PR instead creates a separate `setShreddingSchema` method to set the shredding schema. If not called, shredding will not be used. ### Why are the changes needed? Avoid breaking API. ### Does this PR introduce _any_ user-facing change? No, the feature has not been released yet. ### How was this patch tested? Existing unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49353 from cashmand/fix_set_schema. Authored-by: cashmand Signed-off-by: Wenchen Fan --- .../execution/datasources/parquet/ParquetUtils.scala | 3 ++- .../datasources/parquet/ParquetWriteSupport.scala | 10 +++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index a609a4e0a25f3..663182d8d1820 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -485,7 +485,8 @@ object ParquetUtils extends Logging { } // This metadata is useful for keeping UDTs like Vector/Matrix. - ParquetWriteSupport.setSchema(dataSchema, conf, shreddingSchema) + ParquetWriteSupport.setSchema(dataSchema, conf) + shreddingSchema.foreach(ParquetWriteSupport.setShreddingSchema(_, conf)) // Sets flags for `ParquetWriteSupport`, which converts Catalyst schema to Parquet // schema and writes actual rows to Parquet files. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 02b432f98d7d8..35eb57a2e4fb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -539,14 +539,14 @@ object ParquetWriteSupport { val SPARK_VARIANT_SHREDDING_SCHEMA: String = "org.apache.spark.sql.parquet.variant.shredding.attributes" - def setSchema(schema: StructType, configuration: Configuration, - shreddingSchema: Option[StructType]): Unit = { + def setSchema(schema: StructType, configuration: Configuration): Unit = { configuration.set(SPARK_ROW_SCHEMA, schema.json) configuration.setIfUnset( ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) - shreddingSchema.foreach { s => - configuration.set(SPARK_VARIANT_SHREDDING_SCHEMA, s.json) - } + } + + def setShreddingSchema(shreddingSchema: StructType, configuration: Configuration): Unit = { + configuration.set(SPARK_VARIANT_SHREDDING_SCHEMA, shreddingSchema.json) } } From 1793a20dbdcb10eab4fe2eecbba4ebc5258aa637 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 6 Jan 2025 11:28:51 +0800 Subject: [PATCH 317/438] [SPARK-50700][SQL] `spark.sql.catalog.spark_catalog` supports `builtin` magic value ### What changes were proposed in this pull request? This PR adds a magic value `builtin`(and sets it to the default value) for `spark.sql.catalog.spark_catalog`. ### Why are the changes needed? Currently, `spark.sql.catalog.spark_catalog` is optional and has `None` as the default value. When `spark.sql.catalog.spark_catalog=a.bad.catalog.impl` is wrongly set in `spark-defaults.conf`, the user has no way to overwrite it in `spark-submit`. Note that, explicitly setting it to `o.a.s.sql.execution.datasources.v2.V2SessionCatalog` does not work either, because `V2SessionCatalog` does not have a zero-args constructor. ``` spark-submit \ --conf spark.sql.catalog.spark_catalog=org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog \ ... ``` To fix the above issue, similar to what we did for `spark.sql.hive.metastore.jars`, just use "builtin" to represent the built-in `V2SessionCatalog`. ### Does this PR introduce _any_ user-facing change? No change for default behavior, and users are allowed to use `spark.sql.catalog.spark_catalog=builtin` to set `spark_catalog` as the built-in `V2SessionCatalog`. ### How was this patch tested? Code in UTs like ``` // unset this config to use the default v2 session catalog. spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) ``` are replaced with ``` spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49332 from pan3793/SPARK-50700. Authored-by: Cheng Pan Signed-off-by: Kent Yao --- .../connector/catalog/CatalogManager.scala | 7 ++-- .../apache/spark/sql/internal/SQLConf.scala | 10 ++++- .../analysis/ResolveSessionCatalog.scala | 2 +- .../datasources/v2/V2SessionCatalog.scala | 2 +- .../sql/internal/DataFrameWriterImpl.scala | 8 +++- .../sql/connector/DataSourceV2SQLSuite.scala | 42 +++++++++---------- .../spark/sql/connector/DeleteFromTests.scala | 4 +- 7 files changed, 43 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index db94659b1033b..9b8584604d32f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -85,9 +85,10 @@ class CatalogManager( * in the fallback configuration, spark.sql.sources.useV1SourceList */ private[sql] def v2SessionCatalog: CatalogPlugin = { - conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { _ => - catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) - }.getOrElse(defaultSessionCatalog) + conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) match { + case "builtin" => defaultSessionCatalog + case _ => catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) + } } private var _currentNamespace: Option[Array[String]] = None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6a45380b7a99d..7bc4051b45d4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4404,10 +4404,16 @@ object SQLConf { s"the $SESSION_CATALOG_NAME and must be consistent with it; for example, if a table can " + s"be loaded by the $SESSION_CATALOG_NAME, this catalog must also return the table " + s"metadata. To delegate operations to the $SESSION_CATALOG_NAME, implementations can " + - "extend 'CatalogExtension'.") + "extend 'CatalogExtension'. The value should be either 'builtin' which represents the " + + "spark's builit-in V2SessionCatalog, or a fully qualified class name of the catalog " + + "implementation.") .version("3.0.0") .stringConf - .createOptional + .transform { + case builtin if builtin.equalsIgnoreCase("builtin") => "builtin" + case fullClassName => fullClassName + } + .createWithDefault("builtin") object MapKeyDedupPolicy extends Enumeration { val EXCEPTION, LAST_WIN = Value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index fa28a2cb9ead6..87ea3071f490b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -726,7 +726,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) private def supportsV1Command(catalog: CatalogPlugin): Boolean = { isSessionCatalog(catalog) && ( - SQLConf.get.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).isEmpty || + SQLConf.get.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) == "builtin" || catalog.isInstanceOf[CatalogExtension]) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index e9927cdcc7a33..0a533645648e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -83,7 +83,7 @@ class V2SessionCatalog(catalog: SessionCatalog) } private def hasCustomSessionCatalog: Boolean = { - catalog.conf.contains(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key) + catalog.conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) != "builtin" } override def loadTable(ident: Identifier): Table = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala index 0069062e63078..5a96db5e34bbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala @@ -383,6 +383,11 @@ final class DataFrameWriterImpl[T] private[sql](ds: Dataset[T]) extends DataFram } } + private def hasCustomSessionCatalog: Boolean = { + df.sparkSession.sessionState.conf + .getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) != "builtin" + } + /** * Saves the content of the `DataFrame` as the specified table. * @@ -426,8 +431,7 @@ final class DataFrameWriterImpl[T] private[sql](ds: Dataset[T]) extends DataFram import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val session = df.sparkSession - val canUseV2 = lookupV2Provider().isDefined || (df.sparkSession.sessionState.conf.getConf( - SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).isDefined && + val canUseV2 = lookupV2Provider().isDefined || (hasCustomSessionCatalog && !df.sparkSession.sessionState.catalogManager.catalog(CatalogManager.SESSION_CATALOG_NAME) .isInstanceOf[CatalogExtension]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 87d0a1ff4e7bf..8d255e9efda54 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -278,8 +278,8 @@ class DataSourceV2SQLSuiteV1Filter test("CreateTable: without USING clause") { withSQLConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key -> "false") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val testCatalog = catalog("testcat").asTableCatalog sql("CREATE TABLE testcat.t1 (id int)") @@ -785,8 +785,8 @@ class DataSourceV2SQLSuiteV1Filter } test("CreateTableAsSelect: v2 session catalog can load v1 source table") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -846,8 +846,8 @@ class DataSourceV2SQLSuiteV1Filter // TODO: ignored by SPARK-31707, restore the test after create table syntax unification ignore("CreateTableAsSelect: without USING clause") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val testCatalog = catalog("testcat").asTableCatalog sql("CREATE TABLE testcat.t1 AS SELECT 1 i") @@ -1086,11 +1086,11 @@ class DataSourceV2SQLSuiteV1Filter Seq(true, false).foreach { useV1Table => val format = if (useV1Table) "json" else v2Format if (useV1Table) { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") } else { spark.conf.set( - V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[InMemoryTableSessionCatalog].getName) + V2_SESSION_CATALOG_IMPLEMENTATION, classOf[InMemoryTableSessionCatalog].getName) } withTable("t") { @@ -1815,8 +1815,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-46972: asymmetrical replacement for char/varchar in V2SessionCatalog.createTable") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") withTable("t") { sql(s"CREATE TABLE t(c char(1), v varchar(2)) USING $v2Source") } @@ -2533,8 +2533,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-30001: session catalog name can be specified in SQL statements") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") withTable("t") { sql("CREATE TABLE t USING json AS SELECT 1 AS i") @@ -2598,8 +2598,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-30094: current namespace is used during table resolution") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") withTable("spark_catalog.default.t", "testcat.ns.t") { sql("CREATE TABLE t USING parquet AS SELECT 1") @@ -2613,8 +2613,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-30284: CREATE VIEW should track the current catalog and namespace") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val sessionCatalogName = CatalogManager.SESSION_CATALOG_NAME sql("CREATE NAMESPACE testcat.ns1.ns2") @@ -2651,8 +2651,8 @@ class DataSourceV2SQLSuiteV1Filter } test("COMMENT ON NAMESPACE") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") // Session catalog is used. sql("CREATE NAMESPACE ns") checkNamespaceComment("ns", "minor revision") @@ -2685,8 +2685,8 @@ class DataSourceV2SQLSuiteV1Filter } test("COMMENT ON TABLE") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") // Session catalog is used. withTable("t") { sql("CREATE TABLE t(k int) USING json") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala index fd022580db42b..26f64ceb33fe3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala @@ -100,8 +100,8 @@ trait DeleteFromTests extends DatasourceV2SQLBase { } test("DeleteFrom: DELETE is only supported with v2 tables") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val v1Table = "tbl" withTable(v1Table) { sql(s"CREATE TABLE $v1Table" + From aed5d110c2aabe8e6176c64f93d4a91bf07acc62 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 11:30:26 +0800 Subject: [PATCH 318/438] [SPARK-50731][SQL][TESTS] Mark `*TransformWithStateSuite` as `SlowSQLTest` ### What changes were proposed in this pull request? This PR aims to mark `*TransformWithStateSuite` as `SlowSQLTest`. For example, `StateDataSourceTransformWithStateSuite` took almost 2 minutes. ``` 54950 2025-01-05T19:48:22.9043468Z [info] StateDataSourceTransformWithStateSuite: ... 56139 2025-01-05T19:50:15.1404546Z [info] BloomFilterAggregateQuerySuite: ``` ### Why are the changes needed? To balance the test pipelines by moving newly added long-running test suites. Currently, - https://github.com/apache/spark/actions/runs/12613864071/job/35152182685 (`sql - other tests` took 1h 38 mins) - https://github.com/apache/spark/actions/runs/12613864071/job/35152182636 (`sql - slow tests` took 47 mins) ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Manually check the CI logs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49361 from dongjoon-hyun/SPARK-50731. Authored-by: Dongjoon Hyun Signed-off-by: Kent Yao --- .../v2/state/StateDataSourceTransformWithStateSuite.scala | 2 ++ .../apache/spark/sql/streaming/TransformWithStateSuite.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala index b402344a73177..fe224608fffd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.functions.{col, explode, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{InputMapRow, ListState, MapInputEvent, MapOutputEvent, MapStateTTLProcessor, MaxEventTimeStatefulProcessor, OutputMode, RunningCountStatefulProcessor, RunningCountStatefulProcessorWithProcTimeTimerUpdates, StatefulProcessor, StateStoreMetricsTest, TestMapStateProcessor, TimeMode, TimerValues, TransformWithStateSuiteUtils, Trigger, TTLConfig, ValueState} import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.tags.SlowSQLTest import org.apache.spark.util.Utils /** Stateful processor of single value state var with non-primitive type */ @@ -125,6 +126,7 @@ class SessionGroupsStatefulProcessorWithTTL extends /** * Test suite to verify integration of state data source reader with the transformWithState operator */ +@SlowSQLTest class StateDataSourceTransformWithStateSuite extends StateStoreMetricsTest with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 601699c8320fb..97dad5fe78a19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} +import org.apache.spark.tags.SlowSQLTest object TransformWithStateSuiteUtils { val NUM_SHUFFLE_PARTITIONS = 5 @@ -441,6 +442,7 @@ class SleepingTimerProcessor extends StatefulProcessor[String, String, String] { /** * Class that adds tests for transformWithState stateful streaming operator */ +@SlowSQLTest class TransformWithStateSuite extends StateStoreMetricsTest with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { From 7ebd3950008694f5ed85420fbe620d3e2247bb90 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 11:33:23 +0800 Subject: [PATCH 319/438] [SPARK-50729][SQL][TESTS] Remove `DebugFilesystem` from `TPCDS(Collation)?QueryTestSuite` ### What changes were proposed in this pull request? This PR aims to remove `DebugFilesystem` from `TPCDS(Collation)?QueryTestSuite` while keeping it in all other places. ### Why are the changes needed? `TPCDS(Collation)?QueryTestSuite` extends `TPCDSBase` and `TPCHBase` respectively and they are based on `TPCBase` which extends `SharedSparkSession`. https://github.com/apache/spark/blob/91f3fdd25852b43095dd5273358fc394ffd11b66/sql/core/src/test/scala/org/apache/spark/sql/TPCBase.scala#L24 And, `SharedSparkSession` always uses `DebugFilesystem`. https://github.com/apache/spark/blob/91f3fdd25852b43095dd5273358fc394ffd11b66/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala#L68 This PR aims to remove this resetting. ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49359 from dongjoon-hyun/SPARK-50729. Authored-by: Dongjoon Hyun Signed-off-by: Kent Yao --- .../org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala | 1 + .../test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala index 46a24acb475c4..43e6111fc99ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala @@ -67,6 +67,7 @@ class TPCDSCollationQueryTestSuite extends QueryTest with TPCDSBase with SQLQuer // To make output results deterministic override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.SHUFFLE_PARTITIONS.key, "1") + .remove("spark.hadoop.fs.file.impl") protected override def createSparkSession: TestSparkSession = { new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, sparkConf)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala index bde6155529872..c1246a167b8cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala @@ -62,6 +62,7 @@ class TPCDSQueryTestSuite extends QueryTest with TPCDSBase with SQLQueryTestHelp // To make output results deterministic override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.SHUFFLE_PARTITIONS.key, "1") + .remove("spark.hadoop.fs.file.impl") protected override def createSparkSession: TestSparkSession = { new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, sparkConf)) From 2cc45c2d027301275b19f8e79d1f714c95d8a28d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 5 Jan 2025 21:38:35 -0800 Subject: [PATCH 320/438] [SPARK-50733][K8S][INFRA] Update K8s IT CI to use K8s 1.32 ### What changes were proposed in this pull request? As a part of Apache Spark 4.0.0 preparation, this PR aims to use the latest K8s v1.32.0 (2024-12-11) for K8s integration tests instead of v1.31.0 (August 13, 2024). ### Why are the changes needed? K8s v1.32 was released on last December. - https://kubernetes.io/releases/#release-v1-32 Previously, we has been depending on the default version of Minikube because Minikube has a rapid release cycle. However, the release cycle of Minikube is a little slow in these days. We had better control the test target K8s version explicitly for now. ### Does this PR introduce _any_ user-facing change? No, this is a infra-only change. ### How was this patch tested? Pass the CIs and check the K8s CI log. The following is the current log of this PR. ``` Kubelet Version: v1.32.0 Kube-Proxy Version: v1.32.0 ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49364 from dongjoon-hyun/k8s_1.32. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 29621930b1ffa..3d08c338f053e 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1226,6 +1226,7 @@ jobs: - name: Start Minikube uses: medyagh/setup-minikube@v0.0.18 with: + kubernetes-version: "1.32.0" # Github Action limit cpu:2, memory: 6947MB, limit to 2U6G for better resource statistic cpus: 2 memory: 6144m From 54b4c26504c321ee30e65de7d674c2b3ad99ff7e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 5 Jan 2025 21:43:02 -0800 Subject: [PATCH 321/438] [SPARK-50732][SQL][TESTS] Mark `DataFrameAggregateSuite` and its derived classes as `SlowSQLTest` ### What changes were proposed in this pull request? This PR aims to mark `DataFrameAggregateSuite` and its derived classes as `SlowSQLTest`. ``` $ git grep DataFrameAggregateSuite sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala:class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala:class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala: extends DataFrameAggregateSuite sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala:class DataFrameAggregateSuite extends QueryTest ``` ### Why are the changes needed? When we are adding more tests into `DataFrameAggregateSuite`, its derived three classes also grow together. To balance the test pipelines by moving this long-running test suite group. The following is the AS-IS run time before this PR. Screenshot 2025-01-05 at 20 58 15 ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Manually check the CI logs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49363 from dongjoon-hyun/SPARK-50732. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/AggregateHashMapSuite.scala | 4 ++++ .../scala/org/apache/spark/sql/DataFrameAggregateSuite.scala | 2 ++ 2 files changed, 6 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala index b253c4a70bbf9..e1b0676831549 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala @@ -21,7 +21,9 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.SlowSQLTest +@SlowSQLTest class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.CODEGEN_FALLBACK.key, "false") @@ -37,6 +39,7 @@ class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with Befo } } +@SlowSQLTest class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.CODEGEN_FALLBACK.key, "false") @@ -52,6 +55,7 @@ class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeA } } +@SlowSQLTest class TwoLevelAggregateHashMapWithVectorizedMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index ad80dc65926bc..141d6b219f2a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -39,10 +39,12 @@ import org.apache.spark.sql.test.SQLTestData.DecimalData import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DayTimeIntervalType.{DAY, HOUR, MINUTE, SECOND} import org.apache.spark.sql.types.YearMonthIntervalType.{MONTH, YEAR} +import org.apache.spark.tags.SlowSQLTest import org.apache.spark.unsafe.types.CalendarInterval case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) +@SlowSQLTest class DataFrameAggregateSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { From 56284bfb1e894f5f80b53b1dd528090f9a9b3427 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Mon, 6 Jan 2025 15:00:11 +0900 Subject: [PATCH 322/438] [SPARK-48459][FOLLOWUP][MINOR] Cleanup unused global variable ### What changes were proposed in this pull request? This PR followups for https://github.com/apache/spark/pull/46789 to remove unused global variable ### Why are the changes needed? Code cleanup ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The existing CI should pass ### Was this patch authored or co-authored using generative AI tooling? No Closes #49365 from itholic/minor_48459. Authored-by: Haejoon Lee Signed-off-by: Hyukjin Kwon --- python/pyspark/errors/utils.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/pyspark/errors/utils.py b/python/pyspark/errors/utils.py index 5488940645a13..0d01cbb961bb6 100644 --- a/python/pyspark/errors/utils.py +++ b/python/pyspark/errors/utils.py @@ -259,8 +259,6 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: if spark is not None and hasattr(func, "__name__") and is_debugging_enabled(): if is_remote(): - global current_origin - # Getting the configuration requires RPC call. Uses the default value for now. depth = 1 set_current_origin(func.__name__, _capture_call_site(depth)) From 459483acb45e44592c81e0f449c09c4607a680a4 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Mon, 6 Jan 2025 16:33:19 +0900 Subject: [PATCH 323/438] [SPARK-50357][PYTHON] Support Interrupt(Tag|All) APIs for PySpark ### What changes were proposed in this pull request? This PR proposes to support `Interrupt(Tag|All)` for PySpark ### Why are the changes needed? To improve the compatibility between Spark Connect and Spark Classic. ### Does this PR introduce _any_ user-facing change? New APIs are added - InterruptTag - InterruptAll ### How was this patch tested? Added UTs ### Was this patch authored or co-authored using generative AI tooling? No Closes #49014 from itholic/SPARK-50357. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- .../reference/pyspark.sql/spark_session.rst | 4 +-- python/pyspark/sql/session.py | 34 +++++++++++++------ .../connect/test_parity_job_cancellation.py | 22 ------------ .../sql/tests/test_connect_compatibility.py | 2 -- .../sql/tests/test_job_cancellation.py | 22 ++++++++++++ python/pyspark/sql/tests/test_session.py | 1 - 6 files changed, 48 insertions(+), 37 deletions(-) diff --git a/python/docs/source/reference/pyspark.sql/spark_session.rst b/python/docs/source/reference/pyspark.sql/spark_session.rst index 1677d3e8e0209..a35fccbcffe99 100644 --- a/python/docs/source/reference/pyspark.sql/spark_session.rst +++ b/python/docs/source/reference/pyspark.sql/spark_session.rst @@ -52,6 +52,8 @@ See also :class:`SparkSession`. SparkSession.dataSource SparkSession.getActiveSession SparkSession.getTags + SparkSession.interruptAll + SparkSession.interruptTag SparkSession.newSession SparkSession.profile SparkSession.removeTag @@ -86,8 +88,6 @@ Spark Connect Only SparkSession.clearProgressHandlers SparkSession.client SparkSession.copyFromLocalToFs - SparkSession.interruptAll SparkSession.interruptOperation - SparkSession.interruptTag SparkSession.registerProgressHandler SparkSession.removeProgressHandler diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index f3a1639fddafa..fc434cd16bfbd 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -2197,13 +2197,15 @@ def copyFromLocalToFs(self, local_path: str, dest_path: str) -> None: messageParameters={"feature": "SparkSession.copyFromLocalToFs"}, ) - @remote_only def interruptAll(self) -> List[str]: """ Interrupt all operations of this session currently running on the connected server. .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Returns ------- list of str @@ -2213,18 +2215,25 @@ def interruptAll(self) -> List[str]: ----- There is still a possibility of operation finishing just as it is interrupted. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.interruptAll"}, - ) + java_list = self._jsparkSession.interruptAll() + python_list = list() + + # Use iterator to manually iterate through Java list + java_iterator = java_list.iterator() + while java_iterator.hasNext(): + python_list.append(str(java_iterator.next())) + + return python_list - @remote_only def interruptTag(self, tag: str) -> List[str]: """ Interrupt all operations of this session with the given operation tag. .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Returns ------- list of str @@ -2234,10 +2243,15 @@ def interruptTag(self, tag: str) -> List[str]: ----- There is still a possibility of operation finishing just as it is interrupted. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.interruptTag"}, - ) + java_list = self._jsparkSession.interruptTag(tag) + python_list = list() + + # Use iterator to manually iterate through Java list + java_iterator = java_list.iterator() + while java_iterator.hasNext(): + python_list.append(str(java_iterator.next())) + + return python_list @remote_only def interruptOperation(self, op_id: str) -> List[str]: diff --git a/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py index c5184b04d6aa5..ddb4554afa55a 100644 --- a/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py +++ b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py @@ -32,28 +32,6 @@ def func(target): create_thread=lambda target, session: threading.Thread(target=func, args=(target,)) ) - def test_interrupt_tag(self): - thread_ids = range(4) - self.check_job_cancellation( - lambda job_group: self.spark.addTag(job_group), - lambda job_group: self.spark.interruptTag(job_group), - thread_ids, - [i for i in thread_ids if i % 2 == 0], - [i for i in thread_ids if i % 2 != 0], - ) - self.spark.clearTags() - - def test_interrupt_all(self): - thread_ids = range(4) - self.check_job_cancellation( - lambda job_group: None, - lambda job_group: self.spark.interruptAll(), - thread_ids, - thread_ids, - [], - ) - self.spark.clearTags() - if __name__ == "__main__": import unittest diff --git a/python/pyspark/sql/tests/test_connect_compatibility.py b/python/pyspark/sql/tests/test_connect_compatibility.py index ef83dc3834d0c..25b8be1f9ac7a 100644 --- a/python/pyspark/sql/tests/test_connect_compatibility.py +++ b/python/pyspark/sql/tests/test_connect_compatibility.py @@ -266,9 +266,7 @@ def test_spark_session_compatibility(self): "addArtifacts", "clearProgressHandlers", "copyFromLocalToFs", - "interruptAll", "interruptOperation", - "interruptTag", "newSession", "registerProgressHandler", "removeProgressHandler", diff --git a/python/pyspark/sql/tests/test_job_cancellation.py b/python/pyspark/sql/tests/test_job_cancellation.py index a046c9c01811b..3f30f78808892 100644 --- a/python/pyspark/sql/tests/test_job_cancellation.py +++ b/python/pyspark/sql/tests/test_job_cancellation.py @@ -166,6 +166,28 @@ def get_outer_local_prop(): self.assertEqual(first, {"a", "b"}) self.assertEqual(second, {"a", "b", "c"}) + def test_interrupt_tag(self): + thread_ids = range(4) + self.check_job_cancellation( + lambda job_group: self.spark.addTag(job_group), + lambda job_group: self.spark.interruptTag(job_group), + thread_ids, + [i for i in thread_ids if i % 2 == 0], + [i for i in thread_ids if i % 2 != 0], + ) + self.spark.clearTags() + + def test_interrupt_all(self): + thread_ids = range(4) + self.check_job_cancellation( + lambda job_group: None, + lambda job_group: self.spark.interruptAll(), + thread_ids, + thread_ids, + [], + ) + self.spark.clearTags() + class JobCancellationTests(JobCancellationTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/tests/test_session.py b/python/pyspark/sql/tests/test_session.py index 3fbc0be943e45..a22fe777e3c9a 100644 --- a/python/pyspark/sql/tests/test_session.py +++ b/python/pyspark/sql/tests/test_session.py @@ -227,7 +227,6 @@ def test_unsupported_api(self): (lambda: session.client, "client"), (session.addArtifacts, "addArtifact(s)"), (lambda: session.copyFromLocalToFs("", ""), "copyFromLocalToFs"), - (lambda: session.interruptTag(""), "interruptTag"), (lambda: session.interruptOperation(""), "interruptOperation"), ] From b3d2f28962dbabfe6632a3c9f945bd20770184ba Mon Sep 17 00:00:00 2001 From: morvenhuang Date: Mon, 6 Jan 2025 16:24:11 +0800 Subject: [PATCH 324/438] [SPARK-50671][BUILD] Error compiling spark-protobuf module using user-defined protoc ### What changes were proposed in this pull request? People may use user-defined protoc(as below) for various reasons, for example, if they're on macOS 11, they have to use it since the default version of protoc and protoc-gen-grpc-java plugin do not work on macOS 11. ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ``` But this results in compilation failure: ``` [ERROR] Failed to execute goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run (default) on project spark-protobuf_2.13: Execution default of goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run failed: Cannot read the array length because "" is null ``` This PR aims to fix this compilation failure, the error is caused by `protocol-jar-maven-plugin` bug: - os72/protoc-jar-maven-plugin#104 ### Why are the changes needed? Before: ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ... [ERROR] Failed to execute goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run (default) on project spark-protobuf_2.13: Execution default of goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run failed: Cannot read the array length because "" is null ``` After: ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ... [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 4.0.0-SNAPSHOT: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 7.539 s] [INFO] Spark Project Tags ................................. SUCCESS [ 7.527 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 8.413 s] [INFO] Spark Project Common Utils ......................... SUCCESS [ 29.923 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 10.732 s] [INFO] Spark Project Networking ........................... SUCCESS [ 16.691 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 12.653 s] [INFO] Spark Project Variant .............................. SUCCESS [ 5.262 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 16.246 s] [INFO] Spark Project Connect Shims ........................ SUCCESS [ 4.356 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 8.666 s] [INFO] Spark Project Core ................................. SUCCESS [02:56 min] [INFO] Spark Project ML Local Library ..................... SUCCESS [ 30.063 s] [INFO] Spark Project GraphX ............................... SUCCESS [ 33.989 s] [INFO] Spark Project Streaming ............................ SUCCESS [ 51.897 s] [INFO] Spark Project SQL API .............................. SUCCESS [ 39.508 s] [INFO] Spark Project Catalyst ............................. SUCCESS [03:15 min] [INFO] Spark Project SQL .................................. SUCCESS [04:56 min] [INFO] Spark Project ML Library ........................... SUCCESS [02:30 min] [INFO] Spark Project Tools ................................ SUCCESS [ 4.463 s] [INFO] Spark Project Hive ................................. SUCCESS [01:33 min] [INFO] Spark Project Connect Common ....................... SUCCESS [ 55.308 s] [INFO] Spark Avro ......................................... SUCCESS [ 33.350 s] [INFO] Spark Protobuf ..................................... SUCCESS [ 38.981 s] [INFO] Spark Project REPL ................................. SUCCESS [ 21.165 s] [INFO] Spark Project Connect Server ....................... SUCCESS [ 58.033 s] [INFO] Spark Project Connect Client ....................... SUCCESS [ 59.078 s] [INFO] Spark Project Assembly ............................. SUCCESS [ 6.703 s] [INFO] Kafka 0.10+ Token Provider for Streaming ........... SUCCESS [ 20.047 s] [INFO] Spark Integration for Kafka 0.10 ................... SUCCESS [ 26.882 s] [INFO] Kafka 0.10+ Source for Structured Streaming ........ SUCCESS [ 41.397 s] [INFO] Spark Project Examples ............................. SUCCESS [ 41.308 s] [INFO] Spark Integration for Kafka 0.10 Assembly .......... SUCCESS [ 11.300 s] [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Run command: ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49296 from morvenhuang/SPARK-50671. Authored-by: morvenhuang Signed-off-by: yangjie01 --- connector/protobuf/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index e85481ef9e1c8..22d24a7cdb62d 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -189,7 +189,7 @@ src/test/resources/protobuf - true + direct java From a1a2251ed7e4adaf275db2f2470777ec9d9b5ea1 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 6 Jan 2025 19:23:21 +0900 Subject: [PATCH 325/438] [SPARK-50738][PYTHON] Upgrade black to 23.12.1 ### What changes were proposed in this pull request? This PR proposes to upgrade `black` to 23.12.1 ### Why are the changes needed? To leverage bug fixes in `black`. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49369 from HyukjinKwon/upgrade-black. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_and_test.yml | 2 +- .github/workflows/pages.yml | 2 +- dev/create-release/spark-rm/Dockerfile | 2 +- dev/pyproject.toml | 2 +- dev/reformat-python | 2 +- dev/requirements.txt | 2 +- dev/spark-test-image/docs/Dockerfile | 2 +- dev/spark-test-image/lint/Dockerfile | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 3d08c338f053e..a46b90b4913ee 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -743,7 +743,7 @@ jobs: python-version: '3.11' - name: Install dependencies for Python CodeGen check run: | - python3.11 -m pip install 'black==23.9.1' 'protobuf==5.29.1' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' + python3.11 -m pip install 'black==23.12.1' 'protobuf==5.29.1' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' python3.11 -m pip list - name: Python CodeGen check for branch-3.5 if: inputs.branch == 'branch-3.5' diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml index 637abb86b36cd..4bcc275064d3c 100644 --- a/.github/workflows/pages.yml +++ b/.github/workflows/pages.yml @@ -62,7 +62,7 @@ jobs: run: | pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow 'pandas==2.2.3' 'plotly>=4.8' 'docutils<0.18.0' \ - 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ + 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' - name: Install Ruby for documentation generation diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 190c82429e29e..7b97aba21205b 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -130,7 +130,7 @@ RUN python3.9 -m pip install --force $BASIC_PIP_PKGS unittest-xml-reporting $CON # See 'docutils<0.18.0' in SPARK-39421 RUN python3.9 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ -'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ +'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' RUN python3.9 -m pip list diff --git a/dev/pyproject.toml b/dev/pyproject.toml index f19107b3782a6..8b91943009555 100644 --- a/dev/pyproject.toml +++ b/dev/pyproject.toml @@ -27,7 +27,7 @@ testpaths = [ [tool.black] # When changing the version, we have to update # GitHub workflow version and dev/reformat-python -required-version = "23.9.1" +required-version = "23.12.1" line-length = 100 target-version = ['py39'] include = '\.pyi?$' diff --git a/dev/reformat-python b/dev/reformat-python index 46b7efc931aae..9a1199faa938e 100755 --- a/dev/reformat-python +++ b/dev/reformat-python @@ -22,7 +22,7 @@ FWDIR="$( cd "$DIR"/.. && pwd )" cd "$FWDIR" BLACK_BUILD="${PYTHON_EXECUTABLE} -m black" -BLACK_VERSION="23.9.1" +BLACK_VERSION="23.12.1" $PYTHON_EXECUTABLE -c 'import black' 2> /dev/null if [ $? -ne 0 ]; then echo "The Python library providing the 'black' module was not found. Please install Black, for example, via 'pip install black==$BLACK_VERSION'." diff --git a/dev/requirements.txt b/dev/requirements.txt index c1ae808c5e291..168025cf1db5a 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -54,7 +54,7 @@ jira>=3.5.2 PyGithub # pandas API on Spark Code formatter. -black==23.9.1 +black==23.12.1 py # Spark Connect (required) diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index 6b087f08281c3..f1e33763df468 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -85,7 +85,7 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 # See 'docutils<0.18.0' in SPARK-39421 RUN python3.9 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ - 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ + 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' \ && python3.9 -m pip cache purge diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index c663c3dac3b9f..c3ffd7ba4e4b2 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -72,7 +72,7 @@ RUN apt-get update && apt-get install -y python3.9 python3.9-distutils \ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 RUN python3.9 -m pip install \ - 'black==23.9.1' \ + 'black==23.12.1' \ 'flake8==3.9.0' \ 'googleapis-common-protos-stubs==2.2.0' \ 'grpc-stubs==1.24.11' \ From 51501dfaa7aee3b370e796de54be1b05e6432987 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 6 Jan 2025 19:23:56 +0900 Subject: [PATCH 326/438] [SPARK-50596][PYTHON][FOLLOW-UP] Pin Py4J version in dev/requirements.txt ### What changes were proposed in this pull request? This PR is a followup PR that also fixes `dev/requirementst.txt` ### Why are the changes needed? To match the Py4J version with ours. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49368 from HyukjinKwon/SPARK-50596-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- dev/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/requirements.txt b/dev/requirements.txt index 168025cf1db5a..845778f0e9b99 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -1,5 +1,5 @@ # PySpark dependencies (required) -py4j>=0.10.9.7 +py4j>=0.10.9.8 # PySpark dependencies (optional) numpy>=1.21 From 0d3f4cddf0f5acd169c8221b552836f6c088b3bc Mon Sep 17 00:00:00 2001 From: vicennial Date: Mon, 6 Jan 2025 19:33:09 +0900 Subject: [PATCH 327/438] [SPARK-50708][CONNECT] Delete Artifact resources on GC of `ArtifactManager` instance ### What changes were proposed in this pull request? Registers a `java.lang.ref.Cleaner` for `ArtifactManager`. `ArtifactManager#cleanUpResources` is further made thread-safe and swallows FileSystem exceptions in order to let the remaining cleanup go through. ### Why are the changes needed? Currently, an instance of ArtifactManager's resources isn't cleaned up automatically during its GC. Previously, Artifact Manager was only used in Spark Connect, where its lifetime was controlled by the Spark Connect Session which would manually call `ArtifactManager#cleanUpResources` when it itself is closed. With the recent changes allowing the artifact manager to be used in 'Classic Spark', we should GC related resources when the SparkSession gets GCed (and thus, the ArtifactManager instance is no longer reachable) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49341 from vicennial/artifactGC. Authored-by: vicennial Signed-off-by: Hyukjin Kwon --- .../CheckConnectJvmClientCompatibility.scala | 4 + .../sql/connect/service/SessionHolder.scala | 2 +- .../spark/sql/artifact/ArtifactManager.scala | 105 +++++++++++++----- .../sql/artifact/ArtifactManagerSuite.scala | 6 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../sql/hive/test/TestHiveSingleton.scala | 2 +- 6 files changed, 89 insertions(+), 32 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index 4ec84a4087eb7..be659a0fa95ba 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -183,6 +183,10 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.ExtendedExplainGenerator"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDTFRegistration"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataSourceRegistration"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.artifact.ArtifactStateForCleanup"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.artifact.ArtifactStateForCleanup$"), // DataFrameNaFunctions ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameNaFunctions.fillValue"), diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 5dced7acfb0d2..36f3bcd1e6cd7 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -301,7 +301,7 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio // Clean up all artifacts. // Note: there can be concurrent AddArtifact calls still adding something. - artifactManager.cleanUpResources() + artifactManager.close() // Clean up running streaming queries. // Note: there can be concurrent streaming queries being started. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index 65f361cbae3f2..6394cef9fc760 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.artifact -import java.io.File +import java.io.{File, IOException} +import java.lang.ref.Cleaner import java.net.{URI, URL, URLClassLoader} import java.nio.ByteBuffer import java.nio.file.{CopyOption, Files, Path, Paths, StandardCopyOption} @@ -30,8 +31,8 @@ import scala.reflect.ClassTag import org.apache.commons.io.{FilenameUtils, FileUtils} import org.apache.hadoop.fs.{LocalFileSystem, Path => FSPath} -import org.apache.spark.{JobArtifactSet, JobArtifactState, SparkEnv, SparkException, SparkUnsupportedOperationException} -import org.apache.spark.internal.Logging +import org.apache.spark.{JobArtifactSet, JobArtifactState, SparkContext, SparkEnv, SparkException, SparkUnsupportedOperationException} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config.{CONNECT_SCALA_UDF_STUB_PREFIXES, EXECUTOR_USER_CLASS_PATH_FIRST} import org.apache.spark.sql.{Artifact, SparkSession} import org.apache.spark.sql.internal.SQLConf @@ -51,7 +52,7 @@ import org.apache.spark.util.{ChildFirstURLClassLoader, StubClassLoader, Utils} * * @param session The object used to hold the Spark Connect session state. */ -class ArtifactManager(session: SparkSession) extends Logging { +class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging { import ArtifactManager._ // The base directory where all artifacts are stored. @@ -373,34 +374,28 @@ class ArtifactManager(session: SparkSession) extends Logging { newArtifactManager } + private val cleanUpStateForGlobalResources = ArtifactStateForCleanup( + session.sessionUUID, + session.sparkContext, + state, + artifactPath) + // Ensure that no reference to `this` is captured/help by the cleanup lambda + private def getCleanable: Cleaner.Cleanable = cleaner.register( + this, + () => ArtifactManager.cleanUpGlobalResources(cleanUpStateForGlobalResources) + ) + private var cleanable = getCleanable + /** * Cleans up all resources specific to this `session`. */ - private[sql] def cleanUpResources(): Unit = { + private def cleanUpResources(): Unit = { logDebug( s"Cleaning up resources for session with sessionUUID ${session.sessionUUID}") - // Clean up added files - val fileserver = SparkEnv.get.rpcEnv.fileServer - val sparkContext = session.sparkContext - if (state != null) { - val shouldUpdateEnv = sparkContext.addedFiles.contains(state.uuid) || - sparkContext.addedArchives.contains(state.uuid) || - sparkContext.addedJars.contains(state.uuid) - if (shouldUpdateEnv) { - sparkContext.addedFiles.remove(state.uuid).foreach(_.keys.foreach(fileserver.removeFile)) - sparkContext.addedArchives.remove(state.uuid).foreach(_.keys.foreach(fileserver.removeFile)) - sparkContext.addedJars.remove(state.uuid).foreach(_.keys.foreach(fileserver.removeJar)) - sparkContext.postEnvironmentUpdate() - } - } - - // Clean up cached relations - val blockManager = sparkContext.env.blockManager - blockManager.removeCache(session.sessionUUID) - - // Clean up artifacts folder - FileUtils.deleteDirectory(artifactPath.toFile) + // Clean up global resources via the Cleaner process. + // Note that this will only be run once per instance. + cleanable.clean() // Clean up internal trackers jarsList.clear() @@ -412,6 +407,17 @@ class ArtifactManager(session: SparkSession) extends Logging { cachedClassLoader = None } + override def close(): Unit = { + cleanUpResources() + } + + private[sql] def cleanUpResourcesForTesting(): Unit = { + cleanUpResources() + // Tests reuse the same instance so we need to re-register the cleanable otherwise, it is run + // only once per instance. + cleanable = getCleanable + } + def uploadArtifactToFs( remoteRelativePath: Path, serverLocalStagingPath: Path): Unit = { @@ -481,4 +487,51 @@ object ArtifactManager extends Logging { throw SparkException.internalError(s"Block $fromId not found in the block manager.") } } + + // Shared cleaner instance + private val cleaner: Cleaner = Cleaner.create() + + /** + * Helper method to clean up global resources (i.e. resources associated with the calling + * instance but held externally in sparkContext, blockManager, disk etc.) + */ + private def cleanUpGlobalResources(cleanupState: ArtifactStateForCleanup): Unit = { + // Clean up added files + val (sparkSessionUUID, sparkContext, state, artifactPath) = ( + cleanupState.sparkSessionUUID, + cleanupState.sparkContext, + cleanupState.jobArtifactState, + cleanupState.artifactPath) + val fileServer = SparkEnv.get.rpcEnv.fileServer + if (state != null) { + val shouldUpdateEnv = sparkContext.addedFiles.contains(state.uuid) || + sparkContext.addedArchives.contains(state.uuid) || + sparkContext.addedJars.contains(state.uuid) + if (shouldUpdateEnv) { + sparkContext.addedFiles.remove(state.uuid).foreach(_.keys.foreach(fileServer.removeFile)) + sparkContext.addedArchives.remove(state.uuid).foreach(_.keys.foreach(fileServer.removeFile)) + sparkContext.addedJars.remove(state.uuid).foreach(_.keys.foreach(fileServer.removeJar)) + sparkContext.postEnvironmentUpdate() + } + } + + // Clean up cached relations + val blockManager = sparkContext.env.blockManager + blockManager.removeCache(sparkSessionUUID) + + // Clean up artifacts folder + try { + FileUtils.deleteDirectory(artifactPath.toFile) + } catch { + case e: IOException => + logWarning(log"Failed to delete directory ${MDC(LogKeys.PATH, artifactPath.toFile)}: " + + log"${MDC(LogKeys.EXCEPTION, e.getMessage)}", e) + } + } } + +private[artifact] case class ArtifactStateForCleanup( + sparkSessionUUID: String, + sparkContext: SparkContext, + jobArtifactState: JobArtifactState, + artifactPath: Path) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala index f09e896fb39a3..a24982aea1585 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala @@ -48,7 +48,7 @@ class ArtifactManagerSuite extends SharedSparkSession { private def sessionUUID: String = spark.sessionUUID override def afterEach(): Unit = { - artifactManager.cleanUpResources() + artifactManager.cleanUpResourcesForTesting() super.afterEach() } @@ -209,7 +209,7 @@ class ArtifactManagerSuite extends SharedSparkSession { assert(expectedPath.toFile.exists()) // Remove resources - artifactManager.cleanUpResources() + artifactManager.cleanUpResourcesForTesting() assert(blockManager.getLocalBytes(blockId).isEmpty) assert(!expectedPath.toFile.exists()) @@ -294,7 +294,7 @@ class ArtifactManagerSuite extends SharedSparkSession { val sessionDirectory = artifactManager.artifactPath.toFile assert(sessionDirectory.exists()) - artifactManager.cleanUpResources() + artifactManager.cleanUpResourcesForTesting() assert(!sessionDirectory.exists()) assert(ArtifactManager.artifactRootDirectory.toFile.exists()) } 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 c41370c96241a..5431066c30a9f 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 @@ -72,7 +72,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd override def afterEach(): Unit = { try { - spark.artifactManager.cleanUpResources() + spark.artifactManager.cleanUpResourcesForTesting() } finally { super.afterEach() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala index 770e1da94a1c7..7a0599cda2fe7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala @@ -42,7 +42,7 @@ trait TestHiveSingleton extends SparkFunSuite with BeforeAndAfterAll { protected override def afterEach(): Unit = { try { - spark.artifactManager.cleanUpResources() + spark.artifactManager.cleanUpResourcesForTesting() } finally { super.afterEach() } From 47825e502a22ff8c14a27f05a09b953bf403ae65 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 6 Jan 2025 21:33:13 +0800 Subject: [PATCH 328/438] [SPARK-50736][INFRA] Enable testing module `pyspark-logger` ### What changes were proposed in this pull request? Enable testing module `pyspark-logger` ### Why are the changes needed? this testing module is missing in CI, so following tests were always skipped https://github.com/apache/spark/blob/85d92d7c3a6a38b1b6cfc667caac9176fab5813b/dev/sparktestsupport/modules.py#L1441-L1450 ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #49367 from zhengruifeng/py_test_logging. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 2 +- dev/sparktestsupport/modules.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index a46b90b4913ee..614fdd49d83df 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -499,7 +499,7 @@ jobs: - >- pyspark-sql, pyspark-resource, pyspark-testing - >- - pyspark-core, pyspark-errors, pyspark-streaming + pyspark-core, pyspark-errors, pyspark-streaming, pyspark-logger - >- pyspark-mllib, pyspark-ml, pyspark-ml-connect - >- diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 2a19439bd92f9..71817a64b3581 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1438,7 +1438,7 @@ def __hash__(self): ], ) -pyspark_logging = Module( +pyspark_logger = Module( name="pyspark-logger", dependencies=[], source_file_regexes=["python/pyspark/logger"], From fbb4502070dab7f6480de7e8c5d088453eff3eac Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 6 Jan 2025 08:47:55 -0800 Subject: [PATCH 329/438] [SPARK-50716][CORE][FOLLOWUP] Fix the scenario in `JavaUtils#deleteRecursivelyUsingJavaIO` where `BasicFileAttributes` cannot be read ### What changes were proposed in this pull request? This PR adds protection against IOException (IOE) scenarios when reading the `BasicFileAttributes` of a file in the `deleteRecursivelyUsingJavaIO` method: it catches the IOE and returns null, and silently handles the scenario where `fileAttributes` is null in the subsequent logic. ### Why are the changes needed? When the inode itself does not exist, it is impossible to read its `BasicFileAttributes`, and an IOException (IOE) will be thrown, which caused the failure of the MacOS daily test: - https://github.com/apache/spark/actions/runs/12622568770/job/35170636435 ``` - JobArtifactSet uses resources from SparkContext *** FAILED *** java.nio.file.NoSuchFileException: /Users/runner/work/spark/spark/core/target/tmp/spark-6a6b2d5d-1371-4801-a6c4-59dc9d69c2f2/userFiles-e450317a-136c-49ff-8099-9e8282c766b5/testFile661537940680128228.zip at java.base/sun.nio.fs.UnixException.translateToIOException(UnixException.java:92) at java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:106) at java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:111) at java.base/sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(UnixFileAttributeViews.java:55) at java.base/sun.nio.fs.UnixFileSystemProvider.readAttributes(UnixFileSystemProvider.java:171) at java.base/java.nio.file.Files.readAttributes(Files.java:1853) at org.apache.spark.network.util.JavaUtils.deleteRecursivelyUsingJavaIO(JavaUtils.java:130) at org.apache.spark.network.util.JavaUtils.deleteRecursively(JavaUtils.java:123) at org.apache.spark.network.util.JavaUtils.deleteRecursively(JavaUtils.java:94) at org.apache.spark.util.SparkFileUtils.deleteRecursively(SparkFileUtils.scala:121) ... ``` https://github.com/apache/spark/pull/49347 aimed to fix the cleanup of symbolic links by moving the operation to read `BasicFileAttributes` before `!file.exists` to add a check for broken symbolic links. However, in Spark, there is a logic that first cleans up the potentially existing destination path before overwriting it. The target path being cleaned up may itself be a non-existent inode, such as: https://github.com/apache/spark/blob/91f3fdd25852b43095dd5273358fc394ffd11b66/core/src/main/scala/org/apache/spark/SparkContext.scala#L1879-L1888 Therefore, additional protection is needed for this scenario to maintain compatibility with the old behavior. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass Github Actions - Pass MacOs 15 & Java 21 Github Actions: https://github.com/LuciferYang/spark/runs/35170478542 - Pass Macos PySpark Github Actions: https://github.com/LuciferYang/spark/runs/35178442154 ### Was this patch authored or co-authored using generative AI tooling? No Closes #49357 from LuciferYang/SPARK-50716-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../apache/spark/network/util/JavaUtils.java | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java index 027a0d5fb402e..3482c6addfee3 100644 --- a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -126,10 +126,10 @@ public static void deleteRecursively(File file, FilenameFilter filter) throws IO private static void deleteRecursivelyUsingJavaIO( File file, FilenameFilter filter) throws IOException { - BasicFileAttributes fileAttributes = - Files.readAttributes(file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); - // SPARK-50716: If the file does not exist and not a broken symbolic link, return directly. - if (!file.exists() && !fileAttributes.isSymbolicLink()) return; + BasicFileAttributes fileAttributes = readFileAttributes(file); + // SPARK-50716: If the file attributes are null, that is, the file attributes cannot be read, + // or if the file does not exist and is not a broken symbolic link, then return directly. + if (fileAttributes == null || (!file.exists() && !fileAttributes.isSymbolicLink())) return; if (fileAttributes.isDirectory()) { IOException savedIOException = null; for (File child : listFilesSafely(file, filter)) { @@ -156,6 +156,18 @@ private static void deleteRecursivelyUsingJavaIO( } } + /** + * Reads basic attributes of a given file, of return null if an I/O error occurs. + */ + private static BasicFileAttributes readFileAttributes(File file) { + try { + return Files.readAttributes( + file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); + } catch (IOException e) { + return null; + } + } + private static void deleteRecursivelyUsingUnixNative(File file) throws IOException { ProcessBuilder builder = new ProcessBuilder("rm", "-rf", file.getAbsolutePath()); Process process = null; From d44fd2b5711095a3ea39b6d6e0fcc0dbc7118727 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Mon, 6 Jan 2025 12:27:47 -0800 Subject: [PATCH 330/438] [SPARK-50694][SQL] Support renames in subqueries ### What changes were proposed in this pull request? Supports renames in subqueries: - `sub.toDF(...)` - `sub.alias(...)` ### Why are the changes needed? When the query is used as a subquery by adding `col.outer()`, `toDF` or `alias` doesn't work because they need analyzed plans. ### Does this PR introduce _any_ user-facing change? Yes, those APIs are available in subqueries. ### How was this patch tested? Added / modified the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49336 from ueshin/issues/SPARK-50694/renames. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../spark/sql/DataFrameSubquerySuite.scala | 37 +++++++++- .../DataFrameTableValuedFunctionsSuite.scala | 73 ++++++++++--------- python/pyspark/sql/dataframe.py | 3 +- python/pyspark/sql/tests/test_subquery.py | 42 ++++++++++- python/pyspark/sql/tests/test_tvf.py | 58 ++++++++------- .../connect/planner/SparkConnectPlanner.scala | 9 +-- .../scala/org/apache/spark/sql/Dataset.scala | 67 +++++++++++------ .../spark/sql/DataFrameSubquerySuite.scala | 40 +++++++++- .../DataFrameTableValuedFunctionsSuite.scala | 47 ++++++------ 9 files changed, 255 insertions(+), 121 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index fc37444f77197..4b36d36983a5d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -354,6 +354,28 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { } } + test("lateral join with star expansion") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(spark.range(1).select().select($"*")), + sql("SELECT * FROM t1, LATERAL (SELECT *)")) + checkAnswer( + t1.lateralJoin(t2.select($"*")).toDF("c1", "c2", "c3", "c4"), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM t2)").toDF("c1", "c2", "c3", "c4")) + checkAnswer( + t1.lateralJoin(t2.select($"t1.*".outer(), $"t2.*")) + .toDF("c1", "c2", "c3", "c4", "c5", "c6"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2)") + .toDF("c1", "c2", "c3", "c4", "c5", "c6")) + checkAnswer( + t1.lateralJoin(t2.alias("t1").select($"t1.*")).toDF("c1", "c2", "c3", "c4"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1)").toDF("c1", "c2", "c3", "c4")) + } + } + test("lateral join with different join types") { withView("t1") { val t1 = table1() @@ -375,6 +397,17 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { } } + test("lateral join with subquery alias") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c1".outer(), $"c2".outer()).toDF("a", "b").as("s")) + .select("a", "b"), + sql("SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b)")) + } + } + test("lateral join with correlated equality / non-equality predicates") { withView("t1", "t2") { val t1 = table1() @@ -441,8 +474,8 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { val t2 = table2() checkAnswer( - t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2"), "left") - .join(t1.as("t3"), $"t2.c2" === $"t3.c2", "left") + t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2").as("s"), "left") + .join(t1.as("t3"), $"s.c2" === $"t3.c2", "left") .toDF("c1", "c2", "c3", "c4", "c5"), sql(""" |SELECT * FROM t1 diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala index aeef2e8f0fcf2..12a49ad21676e 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala @@ -61,10 +61,11 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin( + spark.tvf.explode(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)")) checkAnswer( - t3.lateralJoin(spark.tvf.explode($"c2".outer())), + t3.lateralJoin(spark.tvf.explode($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)")) checkAnswer( spark.tvf @@ -113,10 +114,11 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin( + spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)")) checkAnswer( - t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)")) checkAnswer( spark.tvf @@ -161,7 +163,10 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi arrayStruct.lateralJoin(spark.tvf.inline($"arr".outer())), sql("SELECT * FROM array_struct JOIN LATERAL INLINE(arr)")) checkAnswer( - arrayStruct.lateralJoin(spark.tvf.inline($"arr".outer()), $"id" === $"col1", "left"), + arrayStruct.lateralJoin( + spark.tvf.inline($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", + "left"), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k")) } } @@ -202,8 +207,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi sql("SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr)")) checkAnswer( arrayStruct.lateralJoin( - spark.tvf.inline_outer($"arr".outer()), - $"id" === $"col1", + spark.tvf.inline_outer($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", "left"), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k")) } @@ -238,30 +243,27 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi jsonTable .as("t1") .lateralJoin( - spark.tvf.json_tuple( - $"t1.jstring".outer(), - lit("f1"), - lit("f2"), - lit("f3"), - lit("f4"), - lit("f5"))) - .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + spark.tvf + .json_tuple( + $"t1.jstring".outer(), + lit("f1"), + lit("f2"), + lit("f3"), + lit("f4"), + lit("f5")) + .as("t2")) + .select($"t1.key", $"t2.*"), sql( "SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2")) checkAnswer( jsonTable .as("t1") - .lateralJoin( - spark.tvf.json_tuple( - $"jstring".outer(), - lit("f1"), - lit("f2"), - lit("f3"), - lit("f4"), - lit("f5"))) - .where($"c0".isNotNull) - .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + .lateralJoin(spark.tvf + .json_tuple($"jstring".outer(), lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) + .as("t2")) + .where($"t2.c0".isNotNull) + .select($"t1.key", $"t2.*"), sql( "SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 " + @@ -390,17 +392,18 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer( t1.lateralJoin( - spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer())) - .select($"col0", $"col1"), + spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer()).as("t")) + .select($"t.*"), sql("SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t")) checkAnswer( - t1.lateralJoin(spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer())) - .select($"col0".as("x"), $"col1".as("y")), + t1.lateralJoin( + spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer()).toDF("x", "y").as("t")) + .select($"t.*"), sql("SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)")) checkAnswer( t1.join(t3, $"t1.c1" === $"t3.c1") - .lateralJoin(spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer())) - .select($"col0", $"col1"), + .lateralJoin(spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer()).as("t")) + .select($"t.*"), sql("SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t")) } } @@ -463,8 +466,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer( variantTable .as("t1") - .lateralJoin(spark.tvf.variant_explode($"v".outer())) - .select($"id", $"pos", $"key", $"value"), + .lateralJoin(spark.tvf.variant_explode($"v".outer()).as("t")) + .select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode(v) AS t")) } } @@ -515,8 +518,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer( variantTable .as("t1") - .lateralJoin(spark.tvf.variant_explode_outer($"v".outer())) - .select($"id", $"pos", $"key", $"value"), + .lateralJoin(spark.tvf.variant_explode_outer($"v".outer()).as("t")) + .select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode_outer(v) AS t")) } } diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 660f577f56f8b..e321f2c8d755b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -2713,11 +2713,10 @@ def lateralJoin( >>> customers.alias("c").lateralJoin( ... orders.alias("o") ... .where(sf.col("o.customer_id") == sf.col("c.customer_id").outer()) + ... .select("order_id", "order_date") ... .orderBy(sf.col("order_date").desc()) ... .limit(2), ... how="left" - ... ).select( - ... "c.customer_id", "name", "order_id", "order_date" ... ).orderBy("customer_id", "order_id").show() +-----------+-------+--------+----------+ |customer_id| name|order_id|order_date| diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 0f431589b4616..99a22d7c29664 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -518,6 +518,28 @@ def test_lateral_join_with_single_column_select(self): self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2)"""), ) + def test_lateral_join_with_star_expansion(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin(self.spark.range(1).select().select(sf.col("*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT *)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.select(sf.col("*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT * FROM t2)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.select(sf.col("t1.*").outer(), sf.col("t2.*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.alias("t1").select(sf.col("t1.*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1)"""), + ) + def test_lateral_join_with_different_join_types(self): with self.tempView("t1"): t1 = self.table1() @@ -572,6 +594,20 @@ def test_lateral_join_with_different_join_types(self): }, ) + def test_lateral_join_with_subquery_alias(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1) + .select(sf.col("c1").outer(), sf.col("c2").outer()) + .toDF("a", "b") + .alias("s") + ).select("a", "b"), + self.spark.sql("""SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b)"""), + ) + def test_lateral_join_with_correlated_predicates(self): with self.tempView("t1", "t2"): t1 = self.table1() @@ -661,9 +697,11 @@ def test_lateral_join_in_between_regular_joins(self): assertDataFrameEqual( t1.lateralJoin( - t2.where(sf.col("t1.c1").outer() == sf.col("t2.c1")).select(sf.col("c2")), + t2.where(sf.col("t1.c1").outer() == sf.col("t2.c1")) + .select(sf.col("c2")) + .alias("s"), how="left", - ).join(t1.alias("t3"), sf.col("t2.c2") == sf.col("t3.c2"), how="left"), + ).join(t1.alias("t3"), sf.col("s.c2") == sf.col("t3.c2"), how="left"), self.spark.sql( """ SELECT * FROM t1 diff --git a/python/pyspark/sql/tests/test_tvf.py b/python/pyspark/sql/tests/test_tvf.py index ea20cbf9b8f3a..c7274c0810cfb 100644 --- a/python/pyspark/sql/tests/test_tvf.py +++ b/python/pyspark/sql/tests/test_tvf.py @@ -65,11 +65,13 @@ def test_explode_with_lateral_join(self): assertDataFrameEqual( t1.lateralJoin( self.spark.tvf.explode(sf.array(sf.col("c1").outer(), sf.col("c2").outer())) - ).toDF("c1", "c2", "c3"), + .toDF("c3") + .alias("t2") + ), self.spark.sql("""SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)"""), ) assertDataFrameEqual( - t3.lateralJoin(self.spark.tvf.explode(sf.col("c2").outer())).toDF("c1", "c2", "v"), + t3.lateralJoin(self.spark.tvf.explode(sf.col("c2").outer()).toDF("v").alias("t2")), self.spark.sql("""SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)"""), ) assertDataFrameEqual( @@ -127,12 +129,14 @@ def test_explode_outer_with_lateral_join(self): self.spark.tvf.explode_outer( sf.array(sf.col("c1").outer(), sf.col("c2").outer()) ) - ).toDF("c1", "c2", "c3"), + .toDF("c3") + .alias("t2") + ), self.spark.sql("""SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)"""), ) assertDataFrameEqual( - t3.lateralJoin(self.spark.tvf.explode_outer(sf.col("c2").outer())).toDF( - "c1", "c2", "v" + t3.lateralJoin( + self.spark.tvf.explode_outer(sf.col("c2").outer()).toDF("v").alias("t2") ), self.spark.sql("""SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)"""), ) @@ -193,10 +197,10 @@ def test_inline_with_lateral_join(self): ) assertDataFrameEqual( array_struct.lateralJoin( - self.spark.tvf.inline(sf.col("arr").outer()), - sf.col("id") == sf.col("col1"), + self.spark.tvf.inline(sf.col("arr").outer()).toDF("k", "v").alias("t"), + sf.col("id") == sf.col("k"), "left", - ).toDF("id", "arr", "k", "v"), + ), self.spark.sql( """ SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k @@ -252,10 +256,10 @@ def test_inline_outer_with_lateral_join(self): ) assertDataFrameEqual( array_struct.lateralJoin( - self.spark.tvf.inline_outer(sf.col("arr").outer()), - sf.col("id") == sf.col("col1"), + self.spark.tvf.inline_outer(sf.col("arr").outer()).toDF("k", "v").alias("t"), + sf.col("id") == sf.col("k"), "left", - ).toDF("id", "arr", "k", "v"), + ), self.spark.sql( """ SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k @@ -302,9 +306,9 @@ def test_json_tuple_with_lateral_join(self): sf.lit("f3"), sf.lit("f4"), sf.lit("f5"), - ) + ).alias("t2") ) - .select("key", "c0", "c1", "c2", "c3", "c4"), + .select("t1.key", "t2.*"), self.spark.sql( """ SELECT t1.key, t2.* FROM json_table t1, @@ -322,10 +326,10 @@ def test_json_tuple_with_lateral_join(self): sf.lit("f3"), sf.lit("f4"), sf.lit("f5"), - ) + ).alias("t2") ) - .where(sf.col("c0").isNotNull()) - .select("key", "c0", "c1", "c2", "c3", "c4"), + .where(sf.col("t2.c0").isNotNull()) + .select("t1.key", "t2.*"), self.spark.sql( """ SELECT t1.key, t2.* FROM json_table t1, @@ -485,8 +489,8 @@ def test_stack_with_lateral_join(self): sf.col("c1").outer(), sf.lit("Value"), sf.col("c2").outer(), - ) - ).select("col0", "col1"), + ).alias("t") + ).select("t.*"), self.spark.sql( """SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t""" ), @@ -494,17 +498,19 @@ def test_stack_with_lateral_join(self): assertDataFrameEqual( t1.lateralJoin( self.spark.tvf.stack(sf.lit(1), sf.col("c1").outer(), sf.col("c2").outer()) - ).select("col0", "col1"), - self.spark.sql("""SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t"""), + .toDF("x", "y") + .alias("t") + ).select("t.*"), + self.spark.sql("""SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)"""), ) assertDataFrameEqual( t1.join(t3, sf.col("t1.c1") == sf.col("t3.c1")) .lateralJoin( self.spark.tvf.stack( sf.lit(1), sf.col("t1.c2").outer(), sf.col("t3.c2").outer() - ) + ).alias("t") ) - .select("col0", "col1"), + .select("t.*"), self.spark.sql( """ SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 @@ -570,8 +576,8 @@ def test_variant_explode_with_lateral_join(self): assertDataFrameEqual( variant_table.alias("t1") - .lateralJoin(self.spark.tvf.variant_explode(sf.col("v").outer())) - .select("id", "pos", "key", "value"), + .lateralJoin(self.spark.tvf.variant_explode(sf.col("v").outer()).alias("t")) + .select("t1.id", "t.*"), self.spark.sql( """ SELECT t1.id, t.* FROM variant_table AS t1, @@ -629,8 +635,8 @@ def test_variant_explode_outer_with_lateral_join(self): assertDataFrameEqual( variant_table.alias("t1") - .lateralJoin(self.spark.tvf.variant_explode_outer(sf.col("v").outer())) - .select("id", "pos", "key", "value"), + .lateralJoin(self.spark.tvf.variant_explode_outer(sf.col("v").outer()).alias("t")) + .select("t1.id", "t.*"), self.spark.sql( """ SELECT t1.id, t.* FROM variant_table AS t1, diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 490ae473a6e4c..c0b4384af8b6d 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -45,7 +45,7 @@ import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTableValuedFunction, UnresolvedTranspose} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ @@ -566,10 +566,9 @@ class SparkConnectPlanner( } private def transformToDF(rel: proto.ToDF): LogicalPlan = { - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .toDF(rel.getColumnNamesList.asScala.toSeq: _*) - .logicalPlan + UnresolvedSubqueryColumnAliases( + rel.getColumnNamesList.asScala.toSeq, + transformRelation(rel.getInput)) } private def transformMapPartitions(rel: proto.MapPartitions): LogicalPlan = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index b9ae0e5b91318..287628f2cbef3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -92,6 +92,23 @@ private[sql] object Dataset { dataset } + def apply[T]( + sparkSession: SparkSession, + logicalPlan: LogicalPlan, + encoderGenerator: () => Encoder[T]): Dataset[T] = { + val dataset = new Dataset(sparkSession, logicalPlan, encoderGenerator) + // Eagerly bind the encoder so we verify that the encoder matches the underlying + // schema. The user will get an error if this is not the case. + // optimization: it is guaranteed that [[InternalRow]] can be converted to [[Row]] so + // do not do this check in that case. this check can be expensive since it requires running + // the whole [[Analyzer]] to resolve the deserializer + if (!dataset.queryExecution.isLazyAnalysis + && dataset.encoder.clsTag.runtimeClass != classOf[Row]) { + dataset.resolvedEnc + } + dataset + } + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = sparkSession.withActive { val qe = sparkSession.sessionState.executePlan(logicalPlan) @@ -241,8 +258,13 @@ class Dataset[T] private[sql]( this(queryExecution, () => encoder) } + def this( + sparkSession: SparkSession, logicalPlan: LogicalPlan, encoderGenerator: () => Encoder[T]) = { + this(sparkSession.sessionState.executePlan(logicalPlan), encoderGenerator) + } + def this(sparkSession: SparkSession, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { - this(sparkSession.sessionState.executePlan(logicalPlan), encoder) + this(sparkSession, logicalPlan, () => encoder) } def this(sqlContext: SQLContext, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { @@ -508,16 +530,8 @@ class Dataset[T] private[sql]( /** @inheritdoc */ @scala.annotation.varargs - def toDF(colNames: String*): DataFrame = { - require(schema.size == colNames.size, - "The number of columns doesn't match.\n" + - s"Old column names (${schema.size}): " + schema.fields.map(_.name).mkString(", ") + "\n" + - s"New column names (${colNames.size}): " + colNames.mkString(", ")) - - val newCols = logicalPlan.output.zip(colNames).map { case (oldAttribute, newName) => - Column(oldAttribute).as(newName) - } - select(newCols : _*) + def toDF(colNames: String*): DataFrame = withPlan { + UnresolvedSubqueryColumnAliases(colNames, logicalPlan) } /** @inheritdoc */ @@ -854,7 +868,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def as(alias: String): Dataset[T] = withTypedPlan { + def as(alias: String): Dataset[T] = withSameTypedPlan { SubqueryAlias(alias, logicalPlan) } @@ -909,7 +923,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def filter(condition: Column): Dataset[T] = withTypedPlan { + def filter(condition: Column): Dataset[T] = withSameTypedPlan { Filter(condition.expr, logicalPlan) } @@ -1038,7 +1052,7 @@ class Dataset[T] private[sql]( /** @inheritdoc */ @scala.annotation.varargs - def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = withTypedPlan { + def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = withSameTypedPlan { CollectMetrics(name, (expr +: exprs).map(_.named), logicalPlan, id) } @@ -1050,12 +1064,12 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def limit(n: Int): Dataset[T] = withTypedPlan { + def limit(n: Int): Dataset[T] = withSameTypedPlan { Limit(Literal(n), logicalPlan) } /** @inheritdoc */ - def offset(n: Int): Dataset[T] = withTypedPlan { + def offset(n: Int): Dataset[T] = withSameTypedPlan { Offset(Literal(n), logicalPlan) } @@ -1142,7 +1156,7 @@ class Dataset[T] private[sql]( /** @inheritdoc */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = { - withTypedPlan { + withSameTypedPlan { Sample(0.0, fraction, withReplacement, seed, logicalPlan) } } @@ -1340,7 +1354,7 @@ class Dataset[T] private[sql]( def dropDuplicates(): Dataset[T] = dropDuplicates(this.columns) /** @inheritdoc */ - def dropDuplicates(colNames: Seq[String]): Dataset[T] = withTypedPlan { + def dropDuplicates(colNames: Seq[String]): Dataset[T] = withSameTypedPlan { val groupCols = groupColsFromDropDuplicates(colNames) Deduplicate(groupCols, logicalPlan) } @@ -1351,7 +1365,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = withTypedPlan { + def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = withSameTypedPlan { val groupCols = groupColsFromDropDuplicates(colNames) // UnsupportedOperationChecker will fail the query if this is called with batch Dataset. DeduplicateWithinWatermark(groupCols, logicalPlan) @@ -1511,7 +1525,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def repartition(numPartitions: Int): Dataset[T] = withTypedPlan { + def repartition(numPartitions: Int): Dataset[T] = withSameTypedPlan { Repartition(numPartitions, shuffle = true, logicalPlan) } @@ -1526,7 +1540,7 @@ class Dataset[T] private[sql]( s"""Invalid partitionExprs specified: $sortOrders |For range partitioning use repartitionByRange(...) instead. """.stripMargin) - withTypedPlan { + withSameTypedPlan { RepartitionByExpression(partitionExprs.map(_.expr), logicalPlan, numPartitions) } } @@ -1539,13 +1553,13 @@ class Dataset[T] private[sql]( case expr: SortOrder => expr case expr: Expression => SortOrder(expr, Ascending) }) - withTypedPlan { + withSameTypedPlan { RepartitionByExpression(sortOrder, logicalPlan, numPartitions) } } /** @inheritdoc */ - def coalesce(numPartitions: Int): Dataset[T] = withTypedPlan { + def coalesce(numPartitions: Int): Dataset[T] = withSameTypedPlan { Repartition(numPartitions, shuffle = false, logicalPlan) } @@ -2240,7 +2254,7 @@ class Dataset[T] private[sql]( SortOrder(expr, Ascending) } } - withTypedPlan { + withSameTypedPlan { Sort(sortOrder, global = global, logicalPlan) } } @@ -2255,6 +2269,11 @@ class Dataset[T] private[sql]( Dataset(sparkSession, logicalPlan) } + /** A convenient function to wrap a logical plan and produce a Dataset. */ + @inline private def withSameTypedPlan(logicalPlan: LogicalPlan): Dataset[T] = { + Dataset(sparkSession, logicalPlan, encoderGenerator) + } + /** A convenient function to wrap a set based logical plan and produce a Dataset. */ @inline private def withSetOperator[U : Encoder](logicalPlan: LogicalPlan): Dataset[U] = { if (isUnTyped) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index f94cf89276ec4..fdfb909d9ba73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -418,6 +418,30 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } } + test("lateral join with star expansion") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(spark.range(1).select().select($"*")), + sql("SELECT * FROM t1, LATERAL (SELECT *)") + ) + checkAnswer( + t1.lateralJoin(t2.select($"*")), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM t2)") + ) + checkAnswer( + t1.lateralJoin(t2.select($"t1.*".outer(), $"t2.*")), + sql("SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2)") + ) + checkAnswer( + t1.lateralJoin(t2.alias("t1").select($"t1.*")), + sql("SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1)") + ) + } + } + test("lateral join with different join types") { withView("t1") { val t1 = table1() @@ -444,6 +468,18 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } } + test("lateral join with subquery alias") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c1".outer(), $"c2".outer()).toDF("a", "b").as("s")) + .select("a", "b"), + sql("SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b)") + ) + } + } + test("lateral join with correlated equality / non-equality predicates") { withView("t1", "t2") { val t1 = table1() @@ -516,8 +552,8 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { checkAnswer( t1.lateralJoin( - t2.where($"t1.c1".outer() === $"t2.c1").select($"c2"), "left" - ).join(t1.as("t3"), $"t2.c2" === $"t3.c2", "left"), + t2.where($"t1.c1".outer() === $"t2.c1").select($"c2").as("s"), "left" + ).join(t1.as("t3"), $"s.c2" === $"t3.c2", "left"), sql( """ |SELECT * FROM t1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala index 4f2cd275ffdfd..637e0cf964fe5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala @@ -60,11 +60,11 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)") ) checkAnswer( - t3.lateralJoin(spark.tvf.explode($"c2".outer())), + t3.lateralJoin(spark.tvf.explode($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)") ) checkAnswer( @@ -112,11 +112,12 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin( + spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)") ) checkAnswer( - t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)") ) checkAnswer( @@ -164,8 +165,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi ) checkAnswer( arrayStruct.lateralJoin( - spark.tvf.inline($"arr".outer()), - $"id" === $"col1", + spark.tvf.inline($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", "left" ), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k") @@ -210,8 +211,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi ) checkAnswer( arrayStruct.lateralJoin( - spark.tvf.inline_outer($"arr".outer()), - $"id" === $"col1", + spark.tvf.inline_outer($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", "left" ), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k") @@ -249,8 +250,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi jsonTable.as("t1").lateralJoin( spark.tvf.json_tuple( $"t1.jstring".outer(), - lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) - ).select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")).as("t2") + ).select($"t1.key", $"t2.*"), sql("SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2") ) @@ -258,9 +259,9 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi jsonTable.as("t1").lateralJoin( spark.tvf.json_tuple( $"jstring".outer(), - lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) - ).where($"c0".isNotNull) - .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")).as("t2") + ).where($"t2.c0".isNotNull) + .select($"t1.key", $"t2.*"), sql("SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 " + "WHERE t2.c0 IS NOT NULL") @@ -388,21 +389,21 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer( t1.lateralJoin( - spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer()) - ).select($"col0", $"col1"), + spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer()).as("t") + ).select($"t.*"), sql("SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t") ) checkAnswer( t1.lateralJoin( - spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer()) - ).select($"col0".as("x"), $"col1".as("y")), + spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer()).toDF("x", "y").as("t") + ).select($"t.*"), sql("SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)") ) checkAnswer( t1.join(t3, $"t1.c1" === $"t3.c1") .lateralJoin( - spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer()) - ).select($"col0", $"col1"), + spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer()).as("t") + ).select($"t.*"), sql("SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t") ) } @@ -466,8 +467,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer( variantTable.as("t1").lateralJoin( - spark.tvf.variant_explode($"v".outer()) - ).select($"id", $"pos", $"key", $"value"), + spark.tvf.variant_explode($"v".outer()).as("t") + ).select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode(v) AS t") ) } @@ -519,8 +520,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer( variantTable.as("t1").lateralJoin( - spark.tvf.variant_explode_outer($"v".outer()) - ).select($"id", $"pos", $"key", $"value"), + spark.tvf.variant_explode_outer($"v".outer()).as("t") + ).select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode_outer(v) AS t") ) } From 2f0340701f5c8c958324b5a801b69b3acc39a21f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 15:39:13 -0800 Subject: [PATCH 331/438] [SPARK-50243][SQL][TESTS][FOLLOWUP] Remove `ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER` setting from `AdaptiveQueryExecSuite` ### What changes were proposed in this pull request? This PR is a follow-up to revert one line from the irrelevant test case of `AdaptiveQueryExecSuite`. - #49007 ### Why are the changes needed? `AdaptiveQueryExecSuite` has been a flaky test and was designed to repeat the test logic many times. It's irrelevant to #49007 . ### Does this PR introduce _any_ user-facing change? No, this is a revert to the original code. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49374 from dongjoon-hyun/SPARK-50243. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala | 1 - 1 file changed, 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 d298b98aaa8da..ad28fd5176d99 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 @@ -749,7 +749,6 @@ class AdaptiveQueryExecSuite // so retry several times here to avoid unit test failure. eventually(timeout(15.seconds), interval(500.milliseconds)) { withSQLConf( - SQLConf.ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5") { // `testData` is small enough to be broadcast but has empty partition ratio over the config. From 91bc50930dc9ed57e33479b2e4595621d765d4f1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:08:30 -0800 Subject: [PATCH 332/438] [SPARK-50741][BUILD] Upgrade `zstd-jni` to 1.5.6-9 ### What changes were proposed in this pull request? This PR aims to upgrade `zstd-jni` to 1.5.6-9. ### Why are the changes needed? To bring the latest bug fix: - https://github.com/luben/zstd-jni/issues/336 ### 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 #49375 from dongjoon-hyun/SPARK-50741. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- 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 d6db12457eeeb..aa171f0cbfffd 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -284,4 +284,4 @@ xz/1.10//xz-1.10.jar zjsonpatch/7.0.1//zjsonpatch-7.0.1.jar zookeeper-jute/3.9.3//zookeeper-jute-3.9.3.jar zookeeper/3.9.3//zookeeper-3.9.3.jar -zstd-jni/1.5.6-8//zstd-jni-1.5.6-8.jar +zstd-jni/1.5.6-9//zstd-jni-1.5.6-9.jar diff --git a/pom.xml b/pom.xml index ba8f42a3b181d..3a8cbd1f033c2 100644 --- a/pom.xml +++ b/pom.xml @@ -838,7 +838,7 @@ com.github.luben zstd-jni - 1.5.6-8 + 1.5.6-9 com.clearspring.analytics From 32e3df7fea181bbf9fad8f92acd4149d505d92dc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:09:19 -0800 Subject: [PATCH 333/438] [SPARK-50742][CORE] Remove `spark.hadoop.fs.s3a.connection.establish.timeout` setting ### What changes were proposed in this pull request? This PR aims to remove `spark.hadoop.fs.s3a.connection.establish.timeout` setting from `SparkContext` because Apache Spark 4.0.0 uses Apache Hadoop 3.4.1 which has the same default value. - #48295 ### Why are the changes needed? This is a logical cleanup by reverting two patches. - #45710 - #46874 ### Does this PR introduce _any_ user-facing change? No. There is no behavior change because we will use the same `fs.s3a.connection.establish.timeout` value. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49376 from dongjoon-hyun/SPARK-50742. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5d3a9c2690c41..30d772bd62d77 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -423,9 +423,6 @@ class SparkContext(config: SparkConf) extends Logging { if (!_conf.contains("spark.app.name")) { throw new SparkException("An application name must be set in your configuration") } - // HADOOP-19097 Set fs.s3a.connection.establish.timeout to 30s - // We can remove this after Apache Hadoop 3.4.1 releases - conf.setIfMissing("spark.hadoop.fs.s3a.connection.establish.timeout", "30000") // This should be set as early as possible. SparkContext.fillMissingMagicCommitterConfsIfNeeded(_conf) From 4aa389fdb16a8d5e6811cb0ad0e53e94ce319a68 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:10:07 -0800 Subject: [PATCH 334/438] [SPARK-50745][SQL][TESTS] Use `eventually` in `BufferHolderSparkSubmitSuite` ### What changes were proposed in this pull request? This PR aims to use `eventually` in `BufferHolderSparkSubmitSuite` to stabilize the long-standing flakiness. ### Why are the changes needed? - https://github.com/dongjoon-hyun/spark/actions/runs/12638427790/job/35214696371 ``` [info] BufferHolderSparkSubmitSuite: [info] - SPARK-22222: Buffer holder should be able to allocate memory larger than 1GB *** FAILED *** (3 seconds, 721 milliseconds) ``` ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49380 from dongjoon-hyun/SPARK-50745. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../expressions/codegen/BufferHolderSparkSubmitSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala index 891e2d048b7a8..b0ed1ecabb8d6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.scalatest.{Assertions, BeforeAndAfterEach} +import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} import org.scalatest.matchers.must.Matchers +import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkIllegalArgumentException, TestUtils} import org.apache.spark.deploy.SparkSubmitTestUtils @@ -46,7 +48,10 @@ class BufferHolderSparkSubmitSuite "--conf", "spark.master.rest.enabled=false", "--conf", "spark.driver.extraJavaOptions=-ea", unusedJar.toString) - runSparkSubmit(argsForSparkSubmit) + // Given that the default timeout of runSparkSubmit is 60 seconds, try 3 times in total. + eventually(timeout(210.seconds), interval(70.seconds)) { + runSparkSubmit(argsForSparkSubmit) + } } } From 1419f38bd50e737f0df78f66119c84373610f18b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:11:20 -0800 Subject: [PATCH 335/438] [SPARK-39903][SQL][TESTS] Reenable TPCDS q72 of `TPCDSQuerySuite` in GitHub Actions ### What changes were proposed in this pull request? This PR aims to re-enable TPCDS q72 test of `TPCDSQuerySuite` in GitHub Actions. ### Why are the changes needed? We have been running q72 query already in other test suites like `TPCDSQueryTestSuite`) in these days. - https://github.com/apache/spark/actions/runs/12624895360/job/35175796403 ``` [info] - q72 (57 seconds, 895 milliseconds) [info] - q72-v2.7 (58 seconds, 223 milliseconds) ``` ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Pass the CIs and check the log. - https://github.com/dongjoon-hyun/spark/actions/runs/12639834198/job/35219025146 ``` [info] TPCDSQuerySuite: ... [info] - q72 (40 milliseconds) ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49382 from dongjoon-hyun/SPARK-39903. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala index ffd15eb46a48e..e8b36d8b130cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala @@ -33,8 +33,7 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSBase { // Disable read-side char padding so that the generated code is less than 8000. super.sparkConf.set(SQLConf.READ_SIDE_CHAR_PADDING, false) - // q72 is skipped due to GitHub Actions' memory limit. - tpcdsQueries.filterNot(sys.env.contains("GITHUB_ACTIONS") && _ == "q72").foreach { name => + tpcdsQueries.foreach { name => val queryString = resourceToString(s"tpcds/$name.sql", classLoader = Thread.currentThread().getContextClassLoader) test(name) { @@ -44,8 +43,7 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSBase { } } - // q72 is skipped due to GitHub Actions' memory limit. - tpcdsQueriesV2_7_0.filterNot(sys.env.contains("GITHUB_ACTIONS") && _ == "q72").foreach { name => + tpcdsQueriesV2_7_0.foreach { name => val queryString = resourceToString(s"tpcds-v2.7.0/$name.sql", classLoader = Thread.currentThread().getContextClassLoader) test(s"$name-v2.7") { From 9820108dbc1ec6a0e3576d404a0e0623bd3b8355 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:14:08 -0800 Subject: [PATCH 336/438] [MINOR][DOCS] Use SBT slash syntax in `ExternalAppendOnlyUnsafeRowArrayBenchmark` doc ### What changes were proposed in this pull request? This PR aims to use `slash` syntax in `ExternalAppendOnlyUnsafeRowArrayBenchmark` doc instead of the deprecated syntax. - https://www.scala-sbt.org/1.x/docs/Migrating-from-sbt-013x.html#slash ### Why are the changes needed? This PR is aligned with our previous efforts. It seems that this is a leftover which we missed the documentation. - #32115 - #37326 - #37754 ### Does this PR introduce _any_ user-facing change? No. This is a doc-only update. ### How was this patch tested? Manual review. ``` $ build/sbt "project sql; set Test / javaOptions += \"-Dspark.memory.debugFill=false\";Test/runMain org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49384 from dongjoon-hyun/minor. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../ExternalAppendOnlyUnsafeRowArrayBenchmark.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala index 0078c3f9f65de..31b002a1e245d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala @@ -32,10 +32,10 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter * {{{ * 1. without sbt: * bin/spark-submit --class --jars - * 2. build/sbt build/sbt ";project sql;set javaOptions - * in Test += \"-Dspark.memory.debugFill=false\";Test/runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt ";project sql;set javaOptions - * in Test += \"-Dspark.memory.debugFill=false\";Test/runMain " + * 2. build/sbt build/sbt ";project sql; + * set Test / javaOptions += \"-Dspark.memory.debugFill=false\";Test/runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt ";project sql; + * set Test / javaOptions += \"-Dspark.memory.debugFill=false\";Test/runMain " * Results will be written to * "benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt". * }}} From e24e7b4640b548a11c0e89f805a96ee33359fbe9 Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Tue, 7 Jan 2025 09:34:37 +0800 Subject: [PATCH 337/438] [SPARK-50392][PYTHON] DataFrame conversion to table argument in Spark Classic ### What changes were proposed in this pull request? Support DataFrame conversion to table arguments in Spark Classic, and enable UDTFs to accept table arguments in both PySpark and Scala. Spark Connect support will be a follow-up, with the goal of completing it by the end of this month. ### Why are the changes needed? Table-Valued Functions (TVFs) and User-Defined Table Functions (UDTFs) are widely used in Spark workflows. These functions often require a table argument, which Spark internally represents as a Catalyst expression. While Spark SQL supports constructs like TABLE() for this purpose, **there is no direct API in PySpark or Scala to convert a DataFrame into a table argument**. So we propose to support DataFrame conversion to table arguments (in Spark Classic first), and enable UDTFs to accept table arguments in both PySpark and Scala.. ### Does this PR introduce _any_ user-facing change? Yes DataFrame conversion to table argument is supported in Spark Classic, and UDTFs accept table arguments in both PySpark and Scala. ```py >>> from pyspark.sql.functions import udtf >>> from pyspark.sql import Row >>> >>> udtf(returnType="a: int") ... class TestUDTF: ... def eval(self, row: Row): ... if row[0] > 5: ... yield row[0], ... >>> df = spark.range(8) >>> >>> TestUDTF(df.asTable()).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().partitionBy(df.id)).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().partitionBy(df.id).orderBy(df.id)).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().withSinglePartition()).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().partitionBy(df.id).withSinglePartition()).show() Traceback (most recent call last): ... pyspark.errors.exceptions.captured.IllegalArgumentException: Cannot call withSinglePartition() after partitionBy() has been called. ``` ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49055 from xinrong-meng/TableArg. Authored-by: Xinrong Meng Signed-off-by: Xinrong Meng --- .../CheckConnectJvmClientCompatibility.scala | 1 + python/pyspark/sql/_typing.pyi | 2 + python/pyspark/sql/classic/dataframe.py | 4 + python/pyspark/sql/column.py | 3 +- python/pyspark/sql/connect/dataframe.py | 8 ++ python/pyspark/sql/dataframe.py | 24 ++++ python/pyspark/sql/table_arg.py | 52 ++++++++ .../sql/tests/connect/test_parity_udtf.py | 8 ++ python/pyspark/sql/tests/test_udtf.py | 112 ++++++++++++++++++ python/pyspark/sql/tvf_argument.py | 27 +++++ python/pyspark/sql/udtf.py | 33 ++++-- .../scala/org/apache/spark/sql/Column.scala | 2 +- .../sql/TableValuedFunctionArgument.scala | 20 ++++ .../scala/org/apache/spark/sql/Dataset.scala | 23 ++++ .../scala/org/apache/spark/sql/TableArg.scala | 74 ++++++++++++ .../python/UserDefinedPythonFunction.scala | 14 ++- 16 files changed, 395 insertions(+), 12 deletions(-) create mode 100644 python/pyspark/sql/table_arg.py create mode 100644 python/pyspark/sql/tvf_argument.py create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index be659a0fa95ba..7bac10e79d0b4 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -183,6 +183,7 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.ExtendedExplainGenerator"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDTFRegistration"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataSourceRegistration"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.TableArg"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.artifact.ArtifactStateForCleanup"), ProblemFilters.exclude[MissingClassProblem]( diff --git a/python/pyspark/sql/_typing.pyi b/python/pyspark/sql/_typing.pyi index 4969268939adf..27fa0f2a90133 100644 --- a/python/pyspark/sql/_typing.pyi +++ b/python/pyspark/sql/_typing.pyi @@ -36,8 +36,10 @@ from pyspark._typing import PrimitiveType from pyspark.profiler import CodeMapDict import pyspark.sql.types from pyspark.sql.column import Column +from pyspark.sql.tvf_argument import TableValuedFunctionArgument ColumnOrName = Union[Column, str] +TVFArgumentOrName = Union[TableValuedFunctionArgument, str] ColumnOrNameOrOrdinal = Union[Column, str, int] DecimalLiteral = decimal.Decimal DateTimeLiteral = Union[datetime.datetime, datetime.date] diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py index cc44d2c9942d2..8beabda6c135d 100644 --- a/python/pyspark/sql/classic/dataframe.py +++ b/python/pyspark/sql/classic/dataframe.py @@ -74,6 +74,7 @@ from pyspark.sql.utils import get_active_spark_context, to_java_array, to_scala_map from pyspark.sql.pandas.conversion import PandasConversionMixin from pyspark.sql.pandas.map_ops import PandasMapOpsMixin +from pyspark.sql.table_arg import TableArg if TYPE_CHECKING: @@ -1787,6 +1788,9 @@ def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> ParentDataF else: return DataFrame(self._jdf.transpose(), self.sparkSession) + def asTable(self) -> TableArg: + return TableArg(self._jdf.asTable()) + def scalar(self) -> Column: return Column(self._jdf.scalar()) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 285d30fad3bc2..e5640dd81b1fb 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -25,6 +25,7 @@ Union, ) +from pyspark.sql.tvf_argument import TableValuedFunctionArgument from pyspark.sql.utils import dispatch_col_method from pyspark.sql.types import DataType from pyspark.errors import PySparkValueError @@ -37,7 +38,7 @@ __all__ = ["Column"] -class Column: +class Column(TableValuedFunctionArgument): """ A column in a DataFrame. diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 3d8f0eced34b2..789292bdd56f6 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -86,6 +86,7 @@ from pyspark.sql.connect.functions import builtin as F from pyspark.sql.pandas.types import from_arrow_schema, to_arrow_schema from pyspark.sql.pandas.functions import _validate_pandas_udf # type: ignore[attr-defined] +from pyspark.sql.table_arg import TableArg if TYPE_CHECKING: @@ -1801,6 +1802,13 @@ def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> ParentDataF self._session, ) + def asTable(self) -> TableArg: + # TODO(SPARK-50393): Support DataFrame conversion to table argument in Spark Connect + raise PySparkNotImplementedError( + errorClass="NOT_IMPLEMENTED", + messageParameters={"feature": "asTable()"}, + ) + def scalar(self) -> Column: from pyspark.sql.connect.column import Column as ConnectColumn diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index e321f2c8d755b..394ac6bdb69c9 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -42,6 +42,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.merge import MergeIntoWriter from pyspark.sql.streaming import DataStreamWriter +from pyspark.sql.table_arg import TableArg from pyspark.sql.types import StructType, Row from pyspark.sql.utils import dispatch_df_method @@ -6578,6 +6579,29 @@ def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> "DataFrame" """ ... + def asTable(self) -> TableArg: + """ + Converts the DataFrame into a `TableArg` object, which can be used as a table argument + in a user-defined table function (UDTF). + + After obtaining a TableArg from a DataFrame using this method, you can specify partitioning + and ordering for the table argument by calling methods such as `partitionBy`, `orderBy`, and + `withSinglePartition` on the `TableArg` instance. + - partitionBy(*cols): Partitions the data based on the specified columns. This method cannot + be called after withSinglePartition() has been called. + - orderBy(*cols): Orders the data within partitions based on the specified columns. + - withSinglePartition(): Indicates that the data should be treated as a single partition. + This method cannot be called after partitionBy() has been called. + + .. versionadded:: 4.0.0 + + Returns + ------- + :class:`TableArg` + A `TableArg` object representing a table argument. + """ + ... + def scalar(self) -> Column: """ Return a `Column` object for a SCALAR Subquery containing exactly one row and one column. diff --git a/python/pyspark/sql/table_arg.py b/python/pyspark/sql/table_arg.py new file mode 100644 index 0000000000000..d4b5e1653c7a1 --- /dev/null +++ b/python/pyspark/sql/table_arg.py @@ -0,0 +1,52 @@ +# +# 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. +# + +from typing import TYPE_CHECKING + +from pyspark.sql.classic.column import _to_java_column, _to_seq +from pyspark.sql.tvf_argument import TableValuedFunctionArgument +from pyspark.sql.utils import get_active_spark_context + + +if TYPE_CHECKING: + from py4j.java_gateway import JavaObject + from pyspark.sql._typing import ColumnOrName + + +class TableArg(TableValuedFunctionArgument): + def __init__(self, j_table_arg: "JavaObject"): + self._j_table_arg = j_table_arg + + def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": + sc = get_active_spark_context() + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + j_cols = _to_seq(sc, cols, _to_java_column) + new_j_table_arg = self._j_table_arg.partitionBy(j_cols) + return TableArg(new_j_table_arg) + + def orderBy(self, *cols: "ColumnOrName") -> "TableArg": + sc = get_active_spark_context() + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + j_cols = _to_seq(sc, cols, _to_java_column) + new_j_table_arg = self._j_table_arg.orderBy(j_cols) + return TableArg(new_j_table_arg) + + def withSinglePartition(self) -> "TableArg": + new_j_table_arg = self._j_table_arg.withSinglePartition() + return TableArg(new_j_table_arg) diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py b/python/pyspark/sql/tests/connect/test_parity_udtf.py index 6955e7377b4c4..6f4e4133335eb 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udtf.py +++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py @@ -76,6 +76,14 @@ def test_udtf_with_analyze_using_file(self): def test_udtf_access_spark_session(self): super().test_udtf_access_spark_session() + @unittest.skip("TODO(SPARK-50393): support df.asTable() in Spark Connect") + def test_df_asTable(self): + super().test_df_asTable() + + @unittest.skip("TODO(SPARK-50393): support df.asTable() in Spark Connect") + def test_df_asTable_chaining_methods(self): + super().test_df_asTable_chaining_methods() + def _add_pyfile(self, path): self.spark.addArtifacts(path, pyfile=True) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 2424f74f93d36..eca3ab0013b92 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -27,6 +27,7 @@ PySparkTypeError, AnalysisException, PySparkPicklingError, + IllegalArgumentException, ) from pyspark.util import PythonEvalType from pyspark.sql.functions import ( @@ -1055,6 +1056,15 @@ def test_udtf_with_table_argument_query(self): [Row(a=6), Row(a=7)], ) + def test_df_asTable(self): + func = self.udtf_for_table_argument() + self.spark.udtf.register("test_udtf", func) + df = self.spark.range(8) + assertDataFrameEqual( + func(df.asTable()), + self.spark.sql("SELECT * FROM test_udtf(TABLE (SELECT id FROM range(0, 8)))"), + ) + def udtf_for_table_argument(self): class TestUDTF: def eval(self, row: Row): @@ -1064,6 +1074,108 @@ def eval(self, row: Row): func = udtf(TestUDTF, returnType="a: int") return func + def test_df_asTable_chaining_methods(self): + class TestUDTF: + def eval(self, row: Row): + yield row["key"], row["value"] + + func = udtf(TestUDTF, returnType="key: int, value: string") + df = self.spark.createDataFrame( + [(1, "a", 3), (1, "b", 3), (2, "c", 4), (2, "d", 4)], ["key", "value", "number"] + ) + assertDataFrameEqual( + func(df.asTable().partitionBy("key").orderBy(df.value)), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().partitionBy(["key", "number"]).orderBy(df.value)), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().partitionBy("key").orderBy(df.value.desc())), + [ + Row(key=1, value="b"), + Row(key=1, value="a"), + Row(key=2, value="d"), + Row(key=2, value="c"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().partitionBy("key").orderBy(["number", "value"])), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().withSinglePartition()), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + ) + + assertDataFrameEqual( + func(df.asTable().withSinglePartition().orderBy("value")), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + ) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Cannot call withSinglePartition\(\) after partitionBy\(\)" + r" or withSinglePartition\(\) has been called", + ): + df.asTable().partitionBy(df.key).withSinglePartition() + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Cannot call partitionBy\(\) after partitionBy\(\)" + r" or withSinglePartition\(\) has been called", + ): + df.asTable().withSinglePartition().partitionBy(df.key) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Please call partitionBy\(\) or withSinglePartition\(\) before orderBy\(\)", + ): + df.asTable().orderBy(df.key) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Please call partitionBy\(\) or withSinglePartition\(\) before orderBy\(\)", + ): + df.asTable().partitionBy().orderBy(df.key) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Cannot call partitionBy\(\) after partitionBy\(\)" + r" or withSinglePartition\(\) has been called", + ): + df.asTable().partitionBy(df.key).partitionBy() + def test_udtf_with_int_and_table_argument_query(self): class TestUDTF: def eval(self, i: int, row: Row): diff --git a/python/pyspark/sql/tvf_argument.py b/python/pyspark/sql/tvf_argument.py new file mode 100644 index 0000000000000..cb373d35d9ec2 --- /dev/null +++ b/python/pyspark/sql/tvf_argument.py @@ -0,0 +1,27 @@ +# +# 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. +# + + +class TableValuedFunctionArgument: + """ + Base class for arguments passed to Table Valued Functions. + + This class is intentionally left empty and serves as a marker to achieve + parity with the Scala `TableValuedFunctionArgument` trait. + """ + + pass diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index 12c3e985fb89d..3ea32349d81d2 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -32,7 +32,7 @@ if TYPE_CHECKING: from py4j.java_gateway import JavaObject - from pyspark.sql._typing import ColumnOrName + from pyspark.sql._typing import TVFArgumentOrName from pyspark.sql.dataframe import DataFrame from pyspark.sql.session import SparkSession @@ -372,22 +372,41 @@ def _create_judtf(self, func: Type) -> "JavaObject": )(self._name, wrapped_func, jdt, self.evalType, self.deterministic) return judtf - def __call__(self, *args: "ColumnOrName", **kwargs: "ColumnOrName") -> "DataFrame": + def __call__(self, *args: "TVFArgumentOrName", **kwargs: "TVFArgumentOrName") -> "DataFrame": from pyspark.sql.classic.column import _to_java_column, _to_seq from pyspark.sql import DataFrame, SparkSession + from pyspark.sql.table_arg import TableArg spark = SparkSession._getActiveSessionOrCreate() sc = spark.sparkContext assert sc._jvm is not None - jcols = [_to_java_column(arg) for arg in args] + [ - sc._jvm.PythonSQLUtils.namedArgumentExpression(key, _to_java_column(value)) - for key, value in kwargs.items() - ] + # Process positional arguments + jargs = [] + for arg in args: + if isinstance(arg, TableArg): + # If the argument is a TableArg, get the Java TableArg object + jargs.append(arg._j_table_arg) + else: + # Otherwise, convert it to a Java column + jargs.append(_to_java_column(arg)) # type: ignore[arg-type] + + # Process keyword arguments + jkwargs = [] + for key, value in kwargs.items(): + if isinstance(value, TableArg): + # If the value is a TableArg, get the Java TableArg object + j_arg = value._j_table_arg + else: + # Otherwise, convert it to a Java column + j_arg = _to_java_column(value) # type: ignore[arg-type] + # Create a named argument expression + j_named_arg = sc._jvm.PythonSQLUtils.namedArgumentExpression(key, j_arg) + jkwargs.append(j_named_arg) judtf = self._judtf - jPythonUDTF = judtf.apply(spark._jsparkSession, _to_seq(sc, jcols)) + jPythonUDTF = judtf.apply(spark._jsparkSession, _to_seq(sc, jargs + jkwargs)) return DataFrame(jPythonUDTF, spark) def asDeterministic(self) -> "UserDefinedTableFunction": diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala index 50ef61d4a7a19..f13b340e5e9c8 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala @@ -137,7 +137,7 @@ class TypedColumn[-T, U](node: ColumnNode, private[sql] val encoder: Encoder[U]) * @since 1.3.0 */ @Stable -class Column(val node: ColumnNode) extends Logging { +class Column(val node: ColumnNode) extends Logging with TableValuedFunctionArgument { private[sql] def this(name: String, planId: Option[Long]) = this(withOrigin { name match { case "*" => internal.UnresolvedStar(None, planId) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala b/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala new file mode 100644 index 0000000000000..f99c4ecd48554 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala @@ -0,0 +1,20 @@ +/* + * 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 + +trait TableValuedFunctionArgument diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 287628f2cbef3..e4e782a50e3d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1040,6 +1040,29 @@ class Dataset[T] private[sql]( ) } + /** + * Converts the DataFrame into a `TableArg` object, which can be used as a table argument + * in a user-defined table function (UDTF). + * + * After obtaining a `TableArg` from a DataFrame using this method, you can specify + * partitioning and ordering for the table argument by calling methods such as `partitionBy`, + * `orderBy`, and `withSinglePartition` on the `TableArg` instance. + * - partitionBy(*cols): Partitions the data based on the specified columns. + * This method cannot be called after withSinglePartition() has been called. + * - orderBy(*cols): Orders the data within partitions based on the specified columns. + * - withSinglePartition(): Indicates that the data should be treated as a single partition. + * This method cannot be called after partitionBy() has been called. + * + * @group untypedrel + * @since 4.0.0 + */ + def asTable(): TableArg = { + new TableArg( + FunctionTableSubqueryArgumentExpression(plan = logicalPlan), + sparkSession + ) + } + /** @inheritdoc */ def scalar(): Column = { Column(ExpressionColumnNode(ScalarSubqueryExpr(logicalPlan))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala b/sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala new file mode 100644 index 0000000000000..133775c0b666c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala @@ -0,0 +1,74 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.{Ascending, Expression, FunctionTableSubqueryArgumentExpression, SortOrder} + +class TableArg( + private[sql] val expression: FunctionTableSubqueryArgumentExpression, + sparkSession: SparkSession) + extends TableValuedFunctionArgument { + import sparkSession.toRichColumn + + private def isPartitioned: Boolean = + expression.partitionByExpressions.nonEmpty || expression.withSinglePartition + + @scala.annotation.varargs + def partitionBy(cols: Column*): TableArg = { + if (isPartitioned) { + throw new IllegalArgumentException( + "Cannot call partitionBy() after partitionBy() or withSinglePartition() has been called." + ) + } + val partitionByExpressions = cols.map(_.expr) + new TableArg( + expression.copy( + partitionByExpressions = partitionByExpressions), + sparkSession) + } + + @scala.annotation.varargs + def orderBy(cols: Column*): TableArg = { + if (!isPartitioned) { + throw new IllegalArgumentException( + "Please call partitionBy() or withSinglePartition() before orderBy()." + ) + } + val orderByExpressions = cols.map { col => + col.expr match { + case sortOrder: SortOrder => sortOrder + case expr: Expression => SortOrder(expr, Ascending) + } + } + new TableArg( + expression.copy(orderByExpressions = orderByExpressions), + sparkSession) + } + + def withSinglePartition(): TableArg = { + if (isPartitioned) { + throw new IllegalArgumentException( + "Cannot call withSinglePartition() after partitionBy() or " + + "withSinglePartition() has been called." + ) + } + new TableArg( + expression.copy(withSinglePartition = true), + sparkSession) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala index 388ede5d062e5..575e3d4072b8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import net.razorvine.pickle.Pickler import org.apache.spark.api.python.{PythonEvalType, PythonFunction, PythonWorkerUtils, SpecialLengths} -import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession, TableArg, TableValuedFunctionArgument} import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Descending, Expression, FunctionTableSubqueryArgumentExpression, NamedArgumentExpression, NullsFirst, NullsLast, PythonUDAF, PythonUDF, PythonUDTF, PythonUDTFAnalyzeResult, PythonUDTFSelectedExpression, SortOrder, UnresolvedPolymorphicPythonUDTF} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, NamedParametersSupport, OneRowRelation} @@ -160,8 +160,16 @@ case class UserDefinedPythonTableFunction( } /** Returns a [[DataFrame]] that will evaluate to calling this UDTF with the given input. */ - def apply(session: SparkSession, exprs: Column*): DataFrame = { - val udtf = builder(exprs.map(session.expression), session.sessionState.sqlParser) + def apply(session: SparkSession, exprs: TableValuedFunctionArgument*): DataFrame = { + val parser = session.sessionState.sqlParser + val expressions = exprs.map { + case col: Column => session.expression(col) + case tableArg: TableArg => tableArg.expression + case other => throw new IllegalArgumentException( + s"Unsupported argument type: ${other.getClass.getName}" + ) + } + val udtf = builder(expressions, parser) Dataset.ofRows(session, udtf) } } From 5be9587a3ae587678680359f88f84d8554a70a66 Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Tue, 7 Jan 2025 10:14:43 +0800 Subject: [PATCH 338/438] [SPARK-48730][SQL] Implement CreateSQLFunctionCommand for SQL Scalar and Table Functions ### What changes were proposed in this pull request? This PR implements the `CreateSQLFunctionCommand` to support the creation of SQL scalar and table functions. Note that the logic for storing and resolving SQL UDFs will be implemented in subsequent PRs. And more SQL tests will be added once it can work E2E. ### Why are the changes needed? To support SQL UDFs. ### Does this PR introduce _any_ user-facing change? Yes. After this PR, users can create persistent SQL UDFs. ### How was this patch tested? New UTs. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49126 from allisonwang-db/spark-48730-create-sql-udf. Authored-by: Allison Wang Signed-off-by: Allison Wang --- .../resources/error/error-conditions.json | 66 ++++ .../apache/spark/sql/types/StructField.scala | 12 + .../sql/catalyst/analysis/unresolved.scala | 3 +- .../catalog/UserDefinedFunctionErrors.scala | 68 +++- .../catalyst/plans/logical/v2Commands.scala | 22 +- .../sql/errors/QueryCompilationErrors.scala | 9 + .../sql/catalyst/analysis/AnalysisTest.scala | 15 + .../analysis/ResolveSessionCatalog.scala | 21 ++ .../sql/catalyst/catalog/SQLFunction.scala | 84 +++++ .../catalog/UserDefinedFunction.scala | 2 + .../plans/logical/SQLFunctionNode.scala | 45 +++ .../spark/sql/execution/SparkSqlParser.scala | 15 +- .../command/CreateSQLFunctionCommand.scala | 328 +++++++++++++++++- .../CreateUserDefinedFunctionCommand.scala | 110 ++++++ .../spark/sql/execution/command/views.scala | 15 +- .../CreateSQLFunctionParserSuite.scala | 203 +++++++++++ .../execution/command/DDLParserSuite.scala | 44 --- 17 files changed, 1004 insertions(+), 58 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 573e7f3a6a384..52c0315bd0732 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1229,6 +1229,18 @@ }, "sqlState" : "4274K" }, + "DUPLICATE_ROUTINE_PARAMETER_NAMES" : { + "message" : [ + "Found duplicate name(s) in the parameter list of the user-defined routine : ." + ], + "sqlState" : "42734" + }, + "DUPLICATE_ROUTINE_RETURNS_COLUMNS" : { + "message" : [ + "Found duplicate column(s) in the RETURNS clause column list of the user-defined routine : ." + ], + "sqlState" : "42711" + }, "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : { "message" : [ "Previous node emitted a row with eventTime= which is older than current_watermark_value=", @@ -4695,6 +4707,12 @@ ], "sqlState" : "42P01" }, + "TABLE_VALUED_ARGUMENTS_NOT_YET_IMPLEMENTED_FOR_SQL_FUNCTIONS" : { + "message" : [ + "Cannot SQL user-defined function with TABLE arguments because this functionality is not yet implemented." + ], + "sqlState" : "0A000" + }, "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON" : { "message" : [ "Failed to analyze the Python user defined table function: " @@ -5827,6 +5845,54 @@ ], "sqlState" : "42K0E" }, + "USER_DEFINED_FUNCTIONS" : { + "message" : [ + "User defined function is invalid:" + ], + "subClass" : { + "CANNOT_CONTAIN_COMPLEX_FUNCTIONS" : { + "message" : [ + "SQL scalar function cannot contain aggregate/window/generate functions: " + ] + }, + "CANNOT_REPLACE_NON_SQL_UDF_WITH_SQL_UDF" : { + "message" : [ + "Cannot replace the non-SQL function with a SQL function." + ] + }, + "NOT_A_VALID_DEFAULT_EXPRESSION" : { + "message" : [ + "The DEFAULT expression of ``.`` is not supported because it contains a subquery." + ] + }, + "NOT_A_VALID_DEFAULT_PARAMETER_POSITION" : { + "message" : [ + "In routine `` parameter `` with DEFAULT must not be followed by parameter `` without DEFAULT." + ] + }, + "NOT_NULL_ON_FUNCTION_PARAMETERS" : { + "message" : [ + "Cannot specify NOT NULL on function parameters: " + ] + }, + "RETURN_COLUMN_COUNT_MISMATCH" : { + "message" : [ + "The number of columns produced by the RETURN clause (num: ``) does not match the number of column names specified by the RETURNS clause (num: ``) of ." + ] + }, + "SQL_TABLE_UDF_BODY_MUST_BE_A_QUERY" : { + "message" : [ + "SQL table function body must be a query." + ] + }, + "SQL_TABLE_UDF_MISSING_COLUMN_NAMES" : { + "message" : [ + "The relation returned by the query in the CREATE FUNCTION statement for with RETURNS TABLE clause lacks explicit names for one or more output columns; please rewrite the function body to provide explicit column names or add column names to the RETURNS TABLE clause, and re-run the command." + ] + } + }, + "sqlState" : "42601" + }, "USER_RAISED_EXCEPTION" : { "message" : [ "" diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala index d4e590629921c..f33a49e686a59 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -147,6 +147,18 @@ case class StructField( if (metadata.contains("comment")) Option(metadata.getString("comment")) else None } + /** + * Return the default value of this StructField. This is used for storing the default value of a + * function parameter. + */ + private[sql] def getDefault(): Option[String] = { + if (metadata.contains("default")) { + Option(metadata.getString("default")) + } else { + None + } + } + /** * Updates the StructField with a new current default value. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 87a5e94d9f630..b47af90c651a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -206,7 +206,8 @@ case class ResolvedInlineTable(rows: Seq[Seq[Expression]], output: Seq[Attribute */ case class UnresolvedTableValuedFunction( name: Seq[String], - functionArgs: Seq[Expression]) + functionArgs: Seq[Expression], + override val isStreaming: Boolean = false) extends UnresolvedLeafNode { final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_TABLE_VALUED_FUNCTION) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala index a5381669caea8..e8cfa8d74e83f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.SparkException +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.errors.QueryErrorsBase /** - * Errors during registering and executing [[UserDefinedFunction]]s. + * Errors during registering and executing + * [[org.apache.spark.sql.expressions.UserDefinedFunction]]s. */ object UserDefinedFunctionErrors extends QueryErrorsBase { def unsupportedUserDefinedFunction(language: RoutineLanguage): Throwable = { @@ -31,4 +33,68 @@ object UserDefinedFunctionErrors extends QueryErrorsBase { def unsupportedUserDefinedFunction(language: String): Throwable = { SparkException.internalError(s"Unsupported user defined function type: $language") } + + def duplicateParameterNames(routineName: String, names: String): Throwable = { + new AnalysisException( + errorClass = "DUPLICATE_ROUTINE_PARAMETER_NAMES", + messageParameters = Map("routineName" -> routineName, "names" -> names)) + } + + def duplicateReturnsColumns(routineName: String, columns: String): Throwable = { + new AnalysisException( + errorClass = "DUPLICATE_ROUTINE_RETURNS_COLUMNS", + messageParameters = Map("routineName" -> routineName, "columns" -> columns)) + } + + def cannotSpecifyNotNullOnFunctionParameters(input: String): Throwable = { + new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.NOT_NULL_ON_FUNCTION_PARAMETERS", + messageParameters = Map("input" -> input)) + } + + def bodyIsNotAQueryForSqlTableUdf(functionName: String): Throwable = { + new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.SQL_TABLE_UDF_BODY_MUST_BE_A_QUERY", + messageParameters = Map("name" -> functionName)) + } + + def missingColumnNamesForSqlTableUdf(functionName: String): Throwable = { + new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.SQL_TABLE_UDF_MISSING_COLUMN_NAMES", + messageParameters = Map("functionName" -> toSQLId(functionName))) + } + + def invalidTempViewReference(routineName: Seq[String], tempViewName: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INVALID_TEMP_OBJ_REFERENCE", + messageParameters = Map( + "obj" -> "FUNCTION", + "objName" -> toSQLId(routineName), + "tempObj" -> "VIEW", + "tempObjName" -> toSQLId(tempViewName) + ) + ) + } + + def invalidTempFuncReference(routineName: Seq[String], tempFuncName: String): Throwable = { + new AnalysisException( + errorClass = "INVALID_TEMP_OBJ_REFERENCE", + messageParameters = Map( + "obj" -> "FUNCTION", + "objName" -> toSQLId(routineName), + "tempObj" -> "FUNCTION", + "tempObjName" -> toSQLId(tempFuncName) + ) + ) + } + + def invalidTempVarReference(routineName: Seq[String], varName: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INVALID_TEMP_OBJ_REFERENCE", + messageParameters = Map( + "obj" -> "FUNCTION", + "objName" -> toSQLId(routineName), + "tempObj" -> "VARIABLE", + "tempObjName" -> toSQLId(varName))) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 85b5e8379d3d5..58c62a90225aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -21,8 +21,8 @@ import org.apache.spark.{SparkIllegalArgumentException, SparkUnsupportedOperatio import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AssignmentUtils, EliminateSubqueryAliases, FieldName, NamedRelation, PartitionSpec, ResolvedIdentifier, ResolvedProcedure, TypeCheckResult, UnresolvedException, UnresolvedProcedure, ViewSchemaMode} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.catalog.{FunctionResource, RoutineLanguage} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.catalog.FunctionResource import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, MetadataAttribute, NamedExpression, UnaryExpression, Unevaluable, V2ExpressionUtils} import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema import org.apache.spark.sql.catalyst.trees.BinaryLike @@ -1072,6 +1072,26 @@ case class CreateFunction( copy(child = newChild) } +/** + * The logical plan of the CREATE FUNCTION command for SQL Functions. + */ +case class CreateUserDefinedFunction( + child: LogicalPlan, + inputParamText: Option[String], + returnTypeText: String, + exprText: Option[String], + queryText: Option[String], + comment: Option[String], + isDeterministic: Option[Boolean], + containsSQL: Option[Boolean], + language: RoutineLanguage, + isTableFunc: Boolean, + ignoreIfExists: Boolean, + replace: Boolean) extends UnaryCommand { + override protected def withNewChildInternal(newChild: LogicalPlan): CreateUserDefinedFunction = + copy(child = newChild) +} + /** * The logical plan of the DROP FUNCTION command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index d38c7a01e1c44..65ae8da3c4da1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2172,6 +2172,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "ability" -> ability)) } + def tableValuedArgumentsNotYetImplementedForSqlFunctions( + action: String, functionName: String): Throwable = { + new AnalysisException( + errorClass = "TABLE_VALUED_ARGUMENTS_NOT_YET_IMPLEMENTED_FOR_SQL_FUNCTIONS", + messageParameters = Map( + "action" -> action, + "functionName" -> functionName)) + } + def tableValuedFunctionTooManyTableArgumentsError(num: Int): Throwable = { new AnalysisException( errorClass = "TABLE_VALUED_FUNCTION_TOO_MANY_TABLE_ARGUMENTS", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 71744f4d15105..58e6cd7fe1695 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -198,6 +198,21 @@ trait AnalysisTest extends PlanTest { } } + protected def assertParseErrorClass( + parser: String => Any, + sqlCommand: String, + errorClass: String, + parameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty): Unit = { + val e = parseException(parser)(sqlCommand) + checkError( + exception = e, + condition = errorClass, + parameters = parameters, + queryContext = queryContext + ) + } + protected def interceptParseException(parser: String => Any)( sqlCommand: String, messages: String*)(condition: Option[String] = None): Unit = { val e = parseException(parser)(sqlCommand) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 87ea3071f490b..6a388a7849f75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -497,6 +497,27 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case CreateFunction(ResolvedIdentifier(catalog, _), _, _, _, _) => throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "CREATE FUNCTION") + + case c @ CreateUserDefinedFunction( + ResolvedIdentifierInSessionCatalog(ident), _, _, _, _, _, _, _, _, _, _, _) => + CreateUserDefinedFunctionCommand( + FunctionIdentifier(ident.table, ident.database, ident.catalog), + c.inputParamText, + c.returnTypeText, + c.exprText, + c.queryText, + c.comment, + c.isDeterministic, + c.containsSQL, + c.language, + c.isTableFunc, + isTemp = false, + c.ignoreIfExists, + c.replace) + + case CreateUserDefinedFunction( + ResolvedIdentifier(catalog, _), _, _, _, _, _, _, _, _, _, _, _) => + throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "CREATE FUNCTION") } private def constructV1TableCmd( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala index 8ae0341e5646c..c0bd4ac80f5ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala @@ -17,9 +17,16 @@ package org.apache.spark.sql.catalyst.catalog +import scala.collection.mutable + +import org.json4s.JsonAST.{JArray, JString} +import org.json4s.jackson.JsonMethods.{compact, render} + import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.catalog.UserDefinedFunction._ +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalarSubquery} import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, Project} import org.apache.spark.sql.types.{DataType, StructType} /** @@ -56,10 +63,48 @@ case class SQLFunction( assert((isTableFunc && returnType.isRight) || (!isTableFunc && returnType.isLeft)) override val language: RoutineLanguage = LanguageSQL + + /** + * Optionally get the function body as an expression or query using the given parser. + */ + def getExpressionAndQuery( + parser: ParserInterface, + isTableFunc: Boolean): (Option[Expression], Option[LogicalPlan]) = { + // The RETURN clause of the CREATE FUNCTION statement looks like this in the parser: + // RETURN (query | expression) + // If the 'query' matches and parses as a SELECT clause of one item with no FROM clause, and + // this is a scalar function, we skip a level of subquery expression wrapping by using the + // referenced expression directly. + val parsedExpression = exprText.map(parser.parseExpression) + val parsedQuery = queryText.map(parser.parsePlan) + (parsedExpression, parsedQuery) match { + case (None, Some(Project(expr :: Nil, _: OneRowRelation))) + if !isTableFunc => + (Some(expr), None) + case (Some(ScalarSubquery(Project(expr :: Nil, _: OneRowRelation), _, _, _, _, _, _)), None) + if !isTableFunc => + (Some(expr), None) + case (_, _) => + (parsedExpression, parsedQuery) + } + } } object SQLFunction { + private val SQL_FUNCTION_PREFIX = "sqlFunction." + + private val FUNCTION_CATALOG_AND_NAMESPACE = "catalogAndNamespace.numParts" + private val FUNCTION_CATALOG_AND_NAMESPACE_PART_PREFIX = "catalogAndNamespace.part." + + private val FUNCTION_REFERRED_TEMP_VIEW_NAMES = "referredTempViewNames" + private val FUNCTION_REFERRED_TEMP_FUNCTION_NAMES = "referredTempFunctionsNames" + private val FUNCTION_REFERRED_TEMP_VARIABLE_NAMES = "referredTempVariableNames" + + def parseDefault(text: String, parser: ParserInterface): Expression = { + parser.parseExpression(text) + } + /** * This method returns an optional DataType indicating, when present, either the return type for * scalar user-defined functions, or a StructType indicating the names and types of the columns in @@ -92,4 +137,43 @@ object SQLFunction { } } } + + def isSQLFunction(className: String): Boolean = className == SQL_FUNCTION_PREFIX + + /** + * Convert the current catalog and namespace to properties. + */ + def catalogAndNamespaceToProps( + currentCatalog: String, + currentNamespace: Seq[String]): Map[String, String] = { + val props = new mutable.HashMap[String, String] + val parts = currentCatalog +: currentNamespace + if (parts.nonEmpty) { + props.put(FUNCTION_CATALOG_AND_NAMESPACE, parts.length.toString) + parts.zipWithIndex.foreach { case (name, index) => + props.put(s"$FUNCTION_CATALOG_AND_NAMESPACE_PART_PREFIX$index", name) + } + } + props.toMap + } + + /** + * Convert the temporary object names to properties. + */ + def referredTempNamesToProps( + viewNames: Seq[Seq[String]], + functionsNames: Seq[String], + variableNames: Seq[Seq[String]]): Map[String, String] = { + val viewNamesJson = + JArray(viewNames.map(nameParts => JArray(nameParts.map(JString).toList)).toList) + val functionsNamesJson = JArray(functionsNames.map(JString).toList) + val variableNamesJson = + JArray(variableNames.map(nameParts => JArray(nameParts.map(JString).toList)).toList) + + val props = new mutable.HashMap[String, String] + props.put(FUNCTION_REFERRED_TEMP_VIEW_NAMES, compact(render(viewNamesJson))) + props.put(FUNCTION_REFERRED_TEMP_FUNCTION_NAMES, compact(render(functionsNamesJson))) + props.put(FUNCTION_REFERRED_TEMP_VARIABLE_NAMES, compact(render(variableNamesJson))) + props.toMap + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala index 1473f19cb71bd..6567062841de5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala @@ -56,6 +56,8 @@ trait UserDefinedFunction { } object UserDefinedFunction { + val SQL_CONFIG_PREFIX = "sqlConfig." + def parseTableSchema(text: String, parser: ParserInterface): StructType = { val parsed = parser.parseTableSchema(text) CharVarcharUtils.failIfHasCharVarchar(parsed).asInstanceOf[StructType] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala new file mode 100644 index 0000000000000..0a3274af33b5b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala @@ -0,0 +1,45 @@ +/* + * 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.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.catalog.SQLFunction +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.trees.TreePattern.FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * A container for holding a SQL function query plan and its function identifier. + * + * @param function: the SQL function that this node represents. + * @param child: the SQL function body. + */ +case class SQLFunctionNode( + function: SQLFunction, + child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output + override def stringArgs: Iterator[Any] = Iterator(function.name, child) + override protected def withNewChildInternal(newChild: LogicalPlan): SQLFunctionNode = + copy(child = newChild) + + // Throw a reasonable error message when trying to call a SQL UDF with TABLE argument(s). + if (child.containsPattern(FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION)) { + throw QueryCompilationErrors + .tableValuedArgumentsNotYetImplementedForSqlFunctions("call", toSQLId(function.name.funcName)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8d5ddb2d85c4e..744ab03d5d037 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -723,8 +723,19 @@ class SparkSqlAstBuilder extends AstBuilder { withIdentClause(ctx.identifierReference(), functionIdentifier => { if (ctx.TEMPORARY == null) { - // TODO: support creating persistent UDFs. - operationNotAllowed(s"creating persistent SQL functions is not supported", ctx) + CreateUserDefinedFunction( + UnresolvedIdentifier(functionIdentifier), + inputParamText, + returnTypeText, + exprText, + queryText, + comment, + deterministic, + containsSQL, + language, + isTableFunc, + ctx.EXISTS != null, + ctx.REPLACE != null) } else { // Disallow to define a temporary function with `IF NOT EXISTS` if (ctx.EXISTS != null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala index d2aaa93fcca06..25598a12af222 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala @@ -17,9 +17,19 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.catalog.SQLFunction +import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation} +import org.apache.spark.sql.catalyst.catalog.{SessionCatalog, SQLFunction, UserDefinedFunctionErrors} +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Generator, LateralSubquery, Literal, ScalarSubquery, SubqueryExpression, WindowExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical.{LateralJoin, LogicalPlan, OneRowRelation, Project, SQLFunctionNode, UnresolvedWith} +import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_ATTRIBUTE +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.command.CreateUserDefinedFunctionCommand._ +import org.apache.spark.sql.types.{DataType, StructField, StructType} /** * The DDL command that creates a SQL function. @@ -52,10 +62,13 @@ case class CreateSQLFunctionCommand( replace: Boolean) extends CreateUserDefinedFunctionCommand { - override def run(sparkSession: SparkSession): Seq[Row] = { - import SQLFunction._ + import SQLFunction._ + override def run(sparkSession: SparkSession): Seq[Row] = { val parser = sparkSession.sessionState.sqlParser + val analyzer = sparkSession.sessionState.analyzer + val catalog = sparkSession.sessionState.catalog + val conf = sparkSession.sessionState.conf val inputParam = inputParamText.map(parser.parseTableSchema) val returnType = parseReturnTypeText(returnTypeText, isTableFunc, parser) @@ -72,8 +85,313 @@ case class CreateSQLFunctionCommand( isTableFunc, Map.empty) - // TODO: Implement the rest of the method. + val newFunction = { + val (expression, query) = function.getExpressionAndQuery(parser, isTableFunc) + assert(query.nonEmpty || expression.nonEmpty) + + // Check if the function can be replaced. + if (replace && catalog.functionExists(name)) { + checkFunctionSignatures(catalog, name) + } + + // Build function input. + val inputPlan = if (inputParam.isDefined) { + val param = inputParam.get + checkParameterNotNull(param, inputParamText.get) + checkParameterNameDuplication(param, conf, name) + checkDefaultsTrailing(param, name) + + // Qualify the input parameters with the function name so that attributes referencing + // the function input parameters can be resolved correctly. + val qualifier = Seq(name.funcName) + val input = param.map(p => Alias( + { + val defaultExpr = p.getDefault() + if (defaultExpr.isEmpty) { + Literal.create(null, p.dataType) + } else { + val defaultPlan = parseDefault(defaultExpr.get, parser) + if (SubqueryExpression.hasSubquery(defaultPlan)) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.NOT_A_VALID_DEFAULT_EXPRESSION", + messageParameters = + Map("functionName" -> name.funcName, "parameterName" -> p.name)) + } else if (defaultPlan.containsPattern(UNRESOLVED_ATTRIBUTE)) { + // TODO(SPARK-50698): use parsed expression instead of expression string. + defaultPlan.collect { + case a: UnresolvedAttribute => + throw QueryCompilationErrors.unresolvedAttributeError( + "UNRESOLVED_COLUMN", a.sql, Seq.empty, a.origin) + } + } + Cast(defaultPlan, p.dataType) + } + }, p.name)(qualifier = qualifier)) + Project(input, OneRowRelation()) + } else { + OneRowRelation() + } + + // Build the function body and check if the function body can be analyzed successfully. + val (unresolvedPlan, analyzedPlan, inferredReturnType) = if (!isTableFunc) { + // Build SQL scalar function plan. + val outputExpr = if (query.isDefined) ScalarSubquery(query.get) else expression.get + val plan: LogicalPlan = returnType.map { t => + val retType: DataType = t match { + case Left(t) => t + case _ => throw SparkException.internalError( + "Unexpected return type for a scalar SQL UDF.") + } + val outputCast = Seq(Alias(Cast(outputExpr, retType), name.funcName)()) + Project(outputCast, inputPlan) + }.getOrElse { + // If no explicit RETURNS clause is present, infer the result type from the function body. + val outputAlias = Seq(Alias(outputExpr, name.funcName)()) + Project(outputAlias, inputPlan) + } + + // Check the function body can be analyzed correctly. + val analyzed = analyzer.execute(plan) + val (resolved, resolvedReturnType) = analyzed match { + case p @ Project(expr :: Nil, _) if expr.resolved => + (p, Left(expr.dataType)) + case other => + (other, function.returnType) + } + + // Check if the SQL function body contains aggregate/window functions. + // This check needs to be performed before checkAnalysis to provide better error messages. + checkAggOrWindowOrGeneratorExpr(resolved) + + // Check if the SQL function body can be analyzed. + checkFunctionBodyAnalysis(analyzer, function, resolved) + + (plan, resolved, resolvedReturnType) + } else { + // Build SQL table function plan. + if (query.isEmpty) { + throw UserDefinedFunctionErrors.bodyIsNotAQueryForSqlTableUdf(name.funcName) + } + + // Construct a lateral join to analyze the function body. + val plan = LateralJoin(inputPlan, LateralSubquery(query.get), Inner, None) + val analyzed = analyzer.execute(plan) + val newPlan = analyzed match { + case Project(_, j: LateralJoin) => j + case j: LateralJoin => j + case _ => throw SparkException.internalError("Unexpected plan returned when " + + s"creating a SQL TVF: ${analyzed.getClass.getSimpleName}.") + } + val maybeResolved = newPlan.asInstanceOf[LateralJoin].right.plan + + // Check if the function body can be analyzed. + checkFunctionBodyAnalysis(analyzer, function, maybeResolved) + + // Get the function's return schema. + val returnParam: StructType = returnType.map { + case Right(t) => t + case Left(_) => throw SparkException.internalError( + "Unexpected return schema for a SQL table function.") + }.getOrElse { + // If no explicit RETURNS clause is present, infer the result type from the function body. + // To detect this, we search for instances of the UnresolvedAlias expression. Examples: + // CREATE TABLE t USING PARQUET AS VALUES (0, 1), (1, 2) AS tab(c1, c2); + // SELECT c1 FROM t --> UnresolvedAttribute: 'c1 + // SELECT c1 + 1 FROM t --> UnresolvedAlias: unresolvedalias(('c1 + 1), None) + // SELECT c1 + 1 AS a FROM t --> Alias: ('c1 + 1) AS a#2 + query.get match { + case Project(projectList, _) if projectList.exists(_.isInstanceOf[UnresolvedAlias]) => + throw UserDefinedFunctionErrors.missingColumnNamesForSqlTableUdf(name.funcName) + case _ => + StructType(analyzed.asInstanceOf[LateralJoin].right.plan.output.map { col => + StructField(col.name, col.dataType) + }) + } + } + + // Check the return columns cannot have NOT NULL specified. + checkParameterNotNull(returnParam, returnTypeText) + + // Check duplicated return column names. + checkReturnsColumnDuplication(returnParam, conf, name) + + // Check if the actual output size equals to the number of return parameters. + val outputSize = maybeResolved.output.size + if (outputSize != returnParam.size) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.RETURN_COLUMN_COUNT_MISMATCH", + messageParameters = Map( + "outputSize" -> s"$outputSize", + "returnParamSize" -> s"${returnParam.size}", + "name" -> s"$name" + ) + ) + } + + (plan, analyzed, Right(returnParam)) + } + + // A permanent function is not allowed to reference temporary objects. + // This should be called after `qe.assertAnalyzed()` (i.e., `plan` can be resolved) + verifyTemporaryObjectsNotExists(catalog, isTemp, name, unresolvedPlan, analyzedPlan) + + // Generate function properties. + val properties = generateFunctionProperties(sparkSession, unresolvedPlan, analyzedPlan) + + // Derive determinism of the SQL function. + val deterministic = analyzedPlan.deterministic + + function.copy( + // Assign the return type, inferring from the function body if needed. + returnType = inferredReturnType, + deterministic = Some(function.deterministic.getOrElse(deterministic)), + properties = properties + ) + } + + // TODO: create/register sql functions in catalog Seq.empty } + + /** + * Check if the function body can be analyzed. + */ + private def checkFunctionBodyAnalysis( + analyzer: Analyzer, + function: SQLFunction, + body: LogicalPlan): Unit = { + analyzer.checkAnalysis(SQLFunctionNode(function, body)) + } + + /** Check whether the new function is replacing an existing SQL function. */ + private def checkFunctionSignatures(catalog: SessionCatalog, name: FunctionIdentifier): Unit = { + val info = catalog.lookupFunctionInfo(name) + if (!isSQLFunction(info.getClassName)) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.CANNOT_REPLACE_NON_SQL_UDF_WITH_SQL_UDF", + messageParameters = Map("name" -> s"$name") + ) + } + } + + /** + * Collect all temporary views and functions and return the identifiers separately + * This func traverses the unresolved plan `child`. Below are the reasons: + * 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding + * logical plan. After replacement, it is impossible to detect whether the SubqueryAlias is + * added/generated from a temporary view. + * 2) The temp functions are represented by multiple classes. Most are inaccessible from this + * package (e.g., HiveGenericUDF). + * 3) Temporary SQL functions, once resolved, cannot be identified as temp functions. + */ + private def collectTemporaryObjectsInUnresolvedPlan( + catalog: SessionCatalog, + child: LogicalPlan): (Seq[Seq[String]], Seq[String]) = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + def collectTempViews(child: LogicalPlan): Seq[Seq[String]] = { + child.flatMap { + case UnresolvedRelation(nameParts, _, _) if catalog.isTempView(nameParts) => + Seq(nameParts) + case w: UnresolvedWith if !w.resolved => w.innerChildren.flatMap(collectTempViews) + case plan if !plan.resolved => plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempViews(e.plan) + case _ => Seq.empty + }) + case _ => Seq.empty + }.distinct + } + + def collectTempFunctions(child: LogicalPlan): Seq[String] = { + child.flatMap { + case w: UnresolvedWith if !w.resolved => w.innerChildren.flatMap(collectTempFunctions) + case plan if !plan.resolved => + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempFunctions(e.plan) + case e: UnresolvedFunction + if catalog.isTemporaryFunction(e.nameParts.asFunctionIdentifier) => + Seq(e.nameParts.asFunctionIdentifier.funcName) + case _ => Seq.empty + }) + case _ => Seq.empty + }.distinct + } + (collectTempViews(child), collectTempFunctions(child)) + } + + /** + * Permanent functions are not allowed to reference temp objects, including temp functions + * and temp views. + */ + private def verifyTemporaryObjectsNotExists( + catalog: SessionCatalog, + isTemporary: Boolean, + name: FunctionIdentifier, + child: LogicalPlan, + analyzed: LogicalPlan): Unit = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + if (!isTemporary) { + val (tempViews, tempFunctions) = collectTemporaryObjectsInUnresolvedPlan(catalog, child) + tempViews.foreach { nameParts => + throw UserDefinedFunctionErrors.invalidTempViewReference( + routineName = name.asMultipart, tempViewName = nameParts) + } + tempFunctions.foreach { funcName => + throw UserDefinedFunctionErrors.invalidTempFuncReference( + routineName = name.asMultipart, tempFuncName = funcName) + } + val tempVars = ViewHelper.collectTemporaryVariables(analyzed) + tempVars.foreach { varName => + throw UserDefinedFunctionErrors.invalidTempVarReference( + routineName = name.asMultipart, varName = varName) + } + } + } + + /** + * Check if the SQL function body contains aggregate/window/generate functions. + * Note subqueries inside the SQL function body can contain aggregate/window/generate functions. + */ + private def checkAggOrWindowOrGeneratorExpr(plan: LogicalPlan): Unit = { + if (plan.resolved) { + plan.transformAllExpressions { + case e if e.isInstanceOf[WindowExpression] || e.isInstanceOf[Generator] || + e.isInstanceOf[AggregateExpression] => + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.CANNOT_CONTAIN_COMPLEX_FUNCTIONS", + messageParameters = Map("queryText" -> s"${exprText.orElse(queryText).get}") + ) + } + } + } + + /** + * Generate the function properties, including: + * 1. the SQL configs when creating the function. + * 2. the catalog and database name when creating the function. This will be used to provide + * context during nested function resolution. + * 3. referred temporary object names if the function is a temp function. + */ + private def generateFunctionProperties( + session: SparkSession, + plan: LogicalPlan, + analyzed: LogicalPlan): Map[String, String] = { + val catalog = session.sessionState.catalog + val conf = session.sessionState.conf + val manager = session.sessionState.catalogManager + + // Only collect temporary object names when the function is a temp function. + val (tempViews, tempFunctions) = if (isTemp) { + collectTemporaryObjectsInUnresolvedPlan(catalog, plan) + } else { + (Nil, Nil) + } + val tempVars = ViewHelper.collectTemporaryVariables(analyzed) + + sqlConfigsToProps(conf) ++ + catalogAndNamespaceToProps( + manager.currentCatalog.name, + manager.currentNamespace.toIndexedSeq) ++ + referredTempNamesToProps(tempViews, tempFunctions, tempVars) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala index bebb0f5cf6c38..1ee3c8a4c388f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala @@ -17,9 +17,15 @@ package org.apache.spark.sql.execution.command +import java.util.Locale + +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.catalog.{LanguageSQL, RoutineLanguage, UserDefinedFunctionErrors} +import org.apache.spark.sql.catalyst.catalog.UserDefinedFunction._ import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType /** * The base class for CreateUserDefinedFunctionCommand @@ -74,4 +80,108 @@ object CreateUserDefinedFunctionCommand { throw UserDefinedFunctionErrors.unsupportedUserDefinedFunction(other) } } + + /** + * Convert SQL configs to properties by prefixing all configs with a key. + * When converting a function to [[org.apache.spark.sql.catalyst.catalog.CatalogFunction]] or + * [[org.apache.spark.sql.catalyst.expressions.ExpressionInfo]], all SQL configs and other + * function properties (such as the function parameters and the function return type) + * are saved together in a property map. + */ + def sqlConfigsToProps(conf: SQLConf): Map[String, String] = { + val modifiedConfs = ViewHelper.getModifiedConf(conf) + modifiedConfs.map { case (key, value) => s"$SQL_CONFIG_PREFIX$key" -> value } + } + + /** + * Check whether the function parameters contain duplicated column names. + * It takes the function input parameter struct as input and verifies that there is no duplicates + * in the parameter column names. + * If any duplicates are found, it throws an exception with helpful information for users to + * fix the wrong function parameters. + * + * Perform this check while registering the function to fail early. + * This check does not need to run the function itself. + */ + def checkParameterNameDuplication( + param: StructType, + conf: SQLConf, + name: FunctionIdentifier): Unit = { + val names = if (conf.caseSensitiveAnalysis) { + param.fields.map(_.name) + } else { + param.fields.map(_.name.toLowerCase(Locale.ROOT)) + } + if (names.distinct.length != names.length) { + val duplicateColumns = names.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"`$x`" + } + throw UserDefinedFunctionErrors.duplicateParameterNames( + routineName = name.funcName, + names = duplicateColumns.toSeq.sorted.mkString(", ")) + } + } + + /** + * Check whether the function has duplicate column names in the RETURNS clause. + */ + def checkReturnsColumnDuplication( + columns: StructType, + conf: SQLConf, + name: FunctionIdentifier): Unit = { + val names = if (conf.caseSensitiveAnalysis) { + columns.fields.map(_.name) + } else { + columns.fields.map(_.name.toLowerCase(Locale.ROOT)) + } + if (names.distinct.length != names.length) { + val duplicateColumns = names.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"`$x`" + } + throw UserDefinedFunctionErrors.duplicateReturnsColumns( + routineName = name.funcName, + columns = duplicateColumns.toSeq.sorted.mkString(", ")) + } + } + + /** + * Check whether the function parameters contain non trailing defaults. + * For languages that support default values for input parameters, + * this check ensures once a default value is given to a parameter, + * all subsequent parameters must also have a default value. It throws error if otherwise. + * + * Perform this check on function input parameters while registering the function to fail early. + * This check does not need to run the function itself. + */ + def checkDefaultsTrailing(param: StructType, name: FunctionIdentifier): Unit = { + var defaultFound = false + var previousParamName = ""; + param.fields.foreach { field => + if (field.getDefault().isEmpty && defaultFound) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.NOT_A_VALID_DEFAULT_PARAMETER_POSITION", + messageParameters = Map( + "functionName" -> name.funcName, + "parameterName" -> previousParamName, + "nextParameterName" -> field.name)) + } + defaultFound |= field.getDefault().isDefined + previousParamName = field.name + } + } + + /** + * Check whether the function input or return columns (for TABLE Return type) have NOT NULL + * specified. Throw exception if NOT NULL is found. + * + * Perform this check on function input and return parameters while registering the function + * to fail early. This check does not need to run the function itself. + */ + def checkParameterNotNull(param: StructType, input: String): Unit = { + param.fields.foreach { field => + if (!field.nullable) { + throw UserDefinedFunctionErrors.cannotSpecifyNotNullOnFunctionParameters(input) + } + } + } } 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 d5a72fd6c441a..f654c846c8a57 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 @@ -464,12 +464,19 @@ object ViewHelper extends SQLConfHelper with Logging { } /** - * Convert the view SQL configs to `properties`. + * Get all configurations that are modifiable and should be captured. */ - private def sqlConfigsToProps(conf: SQLConf): Map[String, String] = { - val modifiedConfs = conf.getAllConfs.filter { case (k, _) => + def getModifiedConf(conf: SQLConf): Map[String, String] = { + conf.getAllConfs.filter { case (k, _) => conf.isModifiable(k) && shouldCaptureConfig(k) } + } + + /** + * Convert the view SQL configs to `properties`. + */ + private def sqlConfigsToProps(conf: SQLConf): Map[String, String] = { + val modifiedConfs = getModifiedConf(conf) // Some configs have dynamic default values, such as SESSION_LOCAL_TIMEZONE whose // default value relies on the JVM system timezone. We need to always capture them to // to make sure we apply the same configs when reading the view. @@ -690,7 +697,7 @@ object ViewHelper extends SQLConfHelper with Logging { /** * Collect all temporary SQL variables and return the identifiers separately. */ - private def collectTemporaryVariables(child: LogicalPlan): Seq[Seq[String]] = { + def collectTemporaryVariables(child: LogicalPlan): Seq[Seq[String]] = { def collectTempVars(child: LogicalPlan): Seq[Seq[String]] = { child.flatMap { plan => plan.expressions.flatMap(_.flatMap { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala new file mode 100644 index 0000000000000..75b42c6440719 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala @@ -0,0 +1,203 @@ +/* + * 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.command + +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.catalog.LanguageSQL +import org.apache.spark.sql.catalyst.plans.logical.CreateUserDefinedFunction +import org.apache.spark.sql.execution.SparkSqlParser + +class CreateSQLFunctionParserSuite extends AnalysisTest { + private lazy val parser = new SparkSqlParser() + + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(parser.parsePlan)(sqlCommand, messages: _*)() + + private def checkParseError( + sqlCommand: String, + errorClass: String, + parameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty): Unit = + assertParseErrorClass(parser.parsePlan, sqlCommand, errorClass, parameters, queryContext) + + // scalastyle:off argcount + private def createSQLFunction( + nameParts: Seq[String], + inputParamText: Option[String] = None, + returnTypeText: String = "INT", + exprText: Option[String] = None, + queryText: Option[String] = None, + comment: Option[String] = None, + isDeterministic: Option[Boolean] = None, + containsSQL: Option[Boolean] = None, + isTableFunc: Boolean = false, + ignoreIfExists: Boolean = false, + replace: Boolean = false): CreateUserDefinedFunction = { + // scalastyle:on argcount + CreateUserDefinedFunction( + UnresolvedIdentifier(nameParts), + inputParamText = inputParamText, + returnTypeText = returnTypeText, + exprText = exprText, + queryText = queryText, + comment = comment, + isDeterministic = isDeterministic, + containsSQL = containsSQL, + language = LanguageSQL, + isTableFunc = isTableFunc, + ignoreIfExists = ignoreIfExists, + replace = replace) + } + + // scalastyle:off argcount + private def createSQLFunctionCommand( + name: String, + inputParamText: Option[String] = None, + returnTypeText: String = "INT", + exprText: Option[String] = None, + queryText: Option[String] = None, + comment: Option[String] = None, + isDeterministic: Option[Boolean] = None, + containsSQL: Option[Boolean] = None, + isTableFunc: Boolean = false, + ignoreIfExists: Boolean = false, + replace: Boolean = false): CreateSQLFunctionCommand = { + // scalastyle:on argcount + CreateSQLFunctionCommand( + FunctionIdentifier(name), + inputParamText = inputParamText, + returnTypeText = returnTypeText, + exprText = exprText, + queryText = queryText, + comment = comment, + isDeterministic = isDeterministic, + containsSQL = containsSQL, + isTableFunc = isTableFunc, + isTemp = true, + ignoreIfExists = ignoreIfExists, + replace = replace) + } + + test("create temporary SQL functions") { + comparePlans( + parser.parsePlan("CREATE TEMPORARY FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunctionCommand("a", exprText = Some("1"))) + + comparePlans( + parser.parsePlan( + "CREATE TEMPORARY FUNCTION a(x INT) RETURNS TABLE (a INT) RETURN SELECT x"), + createSQLFunctionCommand( + name = "a", + inputParamText = Some("x INT"), + returnTypeText = "a INT", + queryText = Some("SELECT x"), + isTableFunc = true)) + + comparePlans( + parser.parsePlan("CREATE OR REPLACE TEMPORARY FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunctionCommand("a", exprText = Some("1"), replace = true)) + + checkParseError( + "CREATE TEMPORARY FUNCTION a.b() RETURNS INT RETURN 1", + errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + parameters = Map("database" -> "`a`"), + queryContext = Array( + ExpectedContext("CREATE TEMPORARY FUNCTION a.b() RETURNS INT RETURN 1", 0, 51) + ) + ) + + checkParseError( + "CREATE TEMPORARY FUNCTION a.b.c() RETURNS INT RETURN 1", + errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + parameters = Map( + "statement" -> "CREATE TEMPORARY FUNCTION", + "name" -> "`a`.`b`.`c`"), + queryContext = Array( + ExpectedContext("CREATE TEMPORARY FUNCTION a.b.c() RETURNS INT RETURN 1", 0, 53) + ) + ) + + checkParseError( + "CREATE TEMPORARY FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1", + errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_IF_NOT_EXISTS", + parameters = Map.empty, + queryContext = Array( + ExpectedContext("CREATE TEMPORARY FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1", 0, 63) + ) + ) + } + + test("create persistent SQL functions") { + comparePlans( + parser.parsePlan("CREATE FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunction(Seq("a"), exprText = Some("1"))) + + comparePlans( + parser.parsePlan("CREATE FUNCTION a.b(x INT) RETURNS INT RETURN x"), + createSQLFunction(Seq("a", "b"), Some("x INT"), exprText = Some("x"))) + + comparePlans(parser.parsePlan( + "CREATE FUNCTION a.b.c(x INT) RETURNS TABLE (a INT) RETURN SELECT x"), + createSQLFunction(Seq("a", "b", "c"), Some("x INT"), returnTypeText = "a INT", None, + Some("SELECT x"), isTableFunc = true)) + + comparePlans(parser.parsePlan("CREATE FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1"), + createSQLFunction(Seq("a"), exprText = Some("1"), ignoreIfExists = true) + ) + + comparePlans(parser.parsePlan("CREATE OR REPLACE FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunction(Seq("a"), exprText = Some("1"), replace = true)) + + comparePlans( + parser.parsePlan( + """ + |CREATE FUNCTION a(x INT COMMENT 'x') RETURNS INT + |LANGUAGE SQL DETERMINISTIC CONTAINS SQL + |COMMENT 'function' + |RETURN x + |""".stripMargin), + createSQLFunction(Seq("a"), inputParamText = Some("x INT COMMENT 'x'"), + exprText = Some("x"), isDeterministic = Some(true), containsSQL = Some(true), + comment = Some("function")) + ) + + intercept("CREATE OR REPLACE FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1", + "Cannot create a routine with both IF NOT EXISTS and REPLACE specified") + } + + test("create SQL functions with unsupported routine characteristics") { + intercept("CREATE FUNCTION foo() RETURNS INT LANGUAGE blah RETURN 1", + "Operation not allowed: Unsupported language for user defined functions: blah") + + intercept("CREATE FUNCTION foo() RETURNS INT SPECIFIC foo1 RETURN 1", + "Operation not allowed: SQL function with SPECIFIC name is not supported") + + intercept("CREATE FUNCTION foo() RETURNS INT NO SQL RETURN 1", + "Operation not allowed: SQL function with NO SQL is not supported") + + intercept("CREATE FUNCTION foo() RETURNS INT NO SQL CONTAINS SQL RETURN 1", + "Found duplicate clauses: SQL DATA ACCESS") + + intercept("CREATE FUNCTION foo() RETURNS INT RETURNS NULL ON NULL INPUT RETURN 1", + "Operation not allowed: SQL function with RETURNS NULL ON NULL INPUT is not supported") + + intercept("CREATE FUNCTION foo() RETURNS INT SQL SECURITY INVOKER RETURN 1", + "Operation not allowed: SQL function with SQL SECURITY INVOKER is not supported") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index d38708ab3745c..3dea8593b428d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.command import org.apache.spark.SparkThrowable -import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, SchemaCompensation, UnresolvedAttribute, UnresolvedFunctionName, UnresolvedIdentifier} import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, FileResource, FunctionResource, JarResource} import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -37,9 +36,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { super.parseException(parser.parsePlan)(sqlText) } - private def intercept(sqlCommand: String, messages: String*): Unit = - interceptParseException(parser.parsePlan)(sqlCommand, messages: _*)() - private def compareTransformQuery(sql: String, expected: LogicalPlan): Unit = { val plan = parser.parsePlan(sql).asInstanceOf[ScriptTransformation].copy(ioschema = null) comparePlans(plan, expected, checkAnalysis = false) @@ -827,44 +823,4 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { parser.parsePlan("SHOW CATALOGS LIKE 'defau*'"), ShowCatalogsCommand(Some("defau*"))) } - - test("Create SQL functions") { - comparePlans( - parser.parsePlan("CREATE TEMP FUNCTION foo() RETURNS INT RETURN 1"), - CreateSQLFunctionCommand( - FunctionIdentifier("foo"), - inputParamText = None, - returnTypeText = "INT", - exprText = Some("1"), - queryText = None, - comment = None, - isDeterministic = None, - containsSQL = None, - isTableFunc = false, - isTemp = true, - ignoreIfExists = false, - replace = false)) - intercept("CREATE FUNCTION foo() RETURNS INT RETURN 1", - "Operation not allowed: creating persistent SQL functions is not supported") - } - - test("create SQL functions with unsupported routine characteristics") { - intercept("CREATE FUNCTION foo() RETURNS INT LANGUAGE blah RETURN 1", - "Operation not allowed: Unsupported language for user defined functions: blah") - - intercept("CREATE FUNCTION foo() RETURNS INT SPECIFIC foo1 RETURN 1", - "Operation not allowed: SQL function with SPECIFIC name is not supported") - - intercept("CREATE FUNCTION foo() RETURNS INT NO SQL RETURN 1", - "Operation not allowed: SQL function with NO SQL is not supported") - - intercept("CREATE FUNCTION foo() RETURNS INT NO SQL CONTAINS SQL RETURN 1", - "Found duplicate clauses: SQL DATA ACCESS") - - intercept("CREATE FUNCTION foo() RETURNS INT RETURNS NULL ON NULL INPUT RETURN 1", - "Operation not allowed: SQL function with RETURNS NULL ON NULL INPUT is not supported") - - intercept("CREATE FUNCTION foo() RETURNS INT SQL SECURITY INVOKER RETURN 1", - "Operation not allowed: SQL function with SQL SECURITY INVOKER is not supported") - } } From e68b98a35cfed0930f59f30b2f8a542936a6a480 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 7 Jan 2025 10:16:43 +0800 Subject: [PATCH 339/438] [MINOR][PYTHON][TESTS] Restore daily coverage build ### What changes were proposed in this pull request? Skip `test_value_state_ttl_expiration` in daily coverage build ### Why are the changes needed? to restore daily coverage build https://github.com/apache/spark/actions/runs/12630507164 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? PR build with ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11", "PYSPARK_CODECOV": "true"}' default: '{"pyspark": "true"}' ``` ### Was this patch authored or co-authored using generative AI tooling? no Closes #49388 from zhengruifeng/py_fix_cov. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../sql/tests/pandas/test_pandas_transform_with_state.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 7a00f2f43d149..516a95a91a5e7 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -314,6 +314,9 @@ def check_results(batch_df, batch_id): SimpleTTLStatefulProcessor(), check_results, False, "processingTime" ) + @unittest.skipIf( + "COVERAGE_PROCESS_START" in os.environ, "Flaky with coverage enabled, skipping for now." + ) def test_value_state_ttl_expiration(self): def check_results(batch_df, batch_id): if batch_id == 0: From 0467aca97120e493a3b02c5ff9460042dcdf233d Mon Sep 17 00:00:00 2001 From: Junfan Zhang Date: Mon, 6 Jan 2025 20:00:08 -0800 Subject: [PATCH 340/438] [SPARK-49783][YARN] Fix resource leak of yarn allocator ### What changes were proposed in this pull request? Fix the resource leak of yarn allocator ### Why are the changes needed? When the target < running containers number, the assigned containers from the resource manager will be skipped, but these containers are not released by invoking the amClient.releaseAssignedContainer , that will make these containers reserved into the Yarn resourceManager at least 10 minutes. And so, the cluster resource will be wasted at a high ratio. And this will reflect that the vcore * seconds statistics from yarn side will be greater than the result from the spark event logs. From my statistics, the cluster resource waste ratio is ~25% if the spark jobs are exclusive in this cluster. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? In our internal hadoop cluster ### Was this patch authored or co-authored using generative AI tooling? No Closes #48238 from zuston/patch-1. Authored-by: Junfan Zhang Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 1 + 1 file changed, 1 insertion(+) 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 b593d73f85721..911ce2e27f96d 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 @@ -820,6 +820,7 @@ private[yarn] class YarnAllocator( logInfo(log"Skip launching executorRunnable as running executors count: " + log"${MDC(LogKeys.COUNT, rpRunningExecs)} reached target executors count: " + log"${MDC(LogKeys.NUM_EXECUTOR_TARGET, getOrUpdateTargetNumExecutorsForRPId(rpId))}.") + internalReleaseContainer(container) } } } From b3182e5521b9aca70e9dc029fcb47e1185602629 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Tue, 7 Jan 2025 12:08:36 +0800 Subject: [PATCH 341/438] [SPARK-50744][SQL] Add a test case for view/CTE name resolution precedence MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add an important test case for CTE resolution: ``` CREATE VIEW v1 AS SELECT 1; CREATE VIEW v2 AS SELECT * FROM v1; – The result is 1. – The `v2` body will be inlined in the main query tree during the analysis, but upper `v1` – CTE definition won't take precedence over the lower `v1` view. WITH v1 AS ( SELECT 2 ) SELECT * FROM v2; ``` This is an exception to the usual "CTE name takes precedence over the table/view name". ### Why are the changes needed? To harden Spark testing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New test case. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49378 from vladimirg-db/vladimirg-db/add-cte-vs-view-test-case. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../sql-tests/analyzer-results/cte.sql.out | 40 +++++++++++++++++++ .../test/resources/sql-tests/inputs/cte.sql | 5 +++ .../resources/sql-tests/results/cte.sql.out | 26 ++++++++++++ 3 files changed, 71 insertions(+) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index ea09573db51a5..cdd3698ce9af4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -17,6 +17,19 @@ CreateViewCommand `t2`, select * from values 0, 1 as t(id), false, false, LocalT +- LocalRelation [id#x] +-- !query +create temporary view t3 as select * from t +-- !query analysis +CreateViewCommand `t3`, select * from t, false, false, LocalTempView, UNSUPPORTED, true + +- Project [id#x] + +- SubqueryAlias t + +- View (`t`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias t + +- LocalRelation [id#x] + + -- !query WITH s AS (SELECT 1 FROM s) SELECT * FROM s -- !query analysis @@ -76,6 +89,27 @@ WithCTE +- CTERelationRef xxxx, true, [1#x], false, false +-- !query +WITH t AS (SELECT 1) SELECT * FROM t3 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [id#x] + +- SubqueryAlias t3 + +- View (`t3`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias t + +- View (`t`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias t + +- LocalRelation [id#x] + + -- !query WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 -- !query analysis @@ -778,3 +812,9 @@ DropTempViewCommand t DROP VIEW IF EXISTS t2 -- !query analysis DropTempViewCommand t2 + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query analysis +DropTempViewCommand t3 diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte.sql b/sql/core/src/test/resources/sql-tests/inputs/cte.sql index 67a94ce61617d..1e17529d545bf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cte.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cte.sql @@ -1,5 +1,6 @@ create temporary view t as select * from values 0, 1, 2 as t(id); create temporary view t2 as select * from values 0, 1 as t(id); +create temporary view t3 as select * from t; -- WITH clause should not fall into infinite loop by referencing self WITH s AS (SELECT 1 FROM s) SELECT * FROM s; @@ -10,6 +11,9 @@ SELECT * FROM r; -- WITH clause should reference the base table WITH t AS (SELECT 1 FROM t) SELECT * FROM t; +-- Table `t` referenced by a view should take precedence over the top CTE `t` +WITH t AS (SELECT 1) SELECT * FROM t3; + -- WITH clause should not allow cross reference WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2; @@ -175,3 +179,4 @@ with cte as (select * from cte) select * from cte; -- Clean up DROP VIEW IF EXISTS t; DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS t3; diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index 754a8832ef6c5..4367ae1d5f2d0 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -15,6 +15,14 @@ struct<> +-- !query +create temporary view t3 as select * from t +-- !query schema +struct<> +-- !query output + + + -- !query WITH s AS (SELECT 1 FROM s) SELECT * FROM s -- !query schema @@ -70,6 +78,16 @@ struct<1:int> 1 +-- !query +WITH t AS (SELECT 1) SELECT * FROM t3 +-- !query schema +struct +-- !query output +0 +1 +2 + + -- !query WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 -- !query schema @@ -580,3 +598,11 @@ DROP VIEW IF EXISTS t2 struct<> -- !query output + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query schema +struct<> +-- !query output + From 22cbb9694ca53efef1d57387e14976d3906c2b15 Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Tue, 7 Jan 2025 13:55:46 +0800 Subject: [PATCH 342/438] [SPARK-50746][SQL] Replace Either with VariantPathSegment ### What changes were proposed in this pull request? It replaces `type PathSegment = Either[String, Int]` with a dedicated class `VariantPathSegment`. There is no semantic change, but the code has clear naming. ### Why are the changes needed? To make the code easier to understand. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49385 from chenhao-db/VariantPathSegment. Authored-by: Chenhao Li Signed-off-by: Wenchen Fan --- .../variant/variantExpressions.scala | 30 +++++++++++-------- .../datasources/PushVariantIntoScan.scala | 4 +-- .../parquet/SparkShreddingUtils.scala | 15 +++++----- 3 files changed, 26 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala index ba910b8c7e5fd..ff8b168793b5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala @@ -184,33 +184,37 @@ case class ToVariantObject(child: Expression) } } -object VariantPathParser extends RegexParsers { - // A path segment in the `VariantGet` expression represents either an object key access or an - // array index access. - type PathSegment = Either[String, Int] +// A path segment in the `VariantGet` expression represents either an object key access or an array +// index access. +sealed abstract class VariantPathSegment extends Serializable + +case class ObjectExtraction(key: String) extends VariantPathSegment +case class ArrayExtraction(index: Int) extends VariantPathSegment + +object VariantPathParser extends RegexParsers { private def root: Parser[Char] = '$' // Parse index segment like `[123]`. - private def index: Parser[PathSegment] = + private def index: Parser[VariantPathSegment] = for { index <- '[' ~> "\\d+".r <~ ']' } yield { - scala.util.Right(index.toInt) + ArrayExtraction(index.toInt) } // Parse key segment like `.name`, `['name']`, or `["name"]`. - private def key: Parser[PathSegment] = + private def key: Parser[VariantPathSegment] = for { key <- '.' ~> "[^\\.\\[]+".r | "['" ~> "[^\\'\\?]+".r <~ "']" | "[\"" ~> "[^\\\"\\?]+".r <~ "\"]" } yield { - scala.util.Left(key) + ObjectExtraction(key) } - private val parser: Parser[List[PathSegment]] = phrase(root ~> rep(key | index)) + private val parser: Parser[List[VariantPathSegment]] = phrase(root ~> rep(key | index)) - def parse(str: String): Option[Array[PathSegment]] = { + def parse(str: String): Option[Array[VariantPathSegment]] = { this.parseAll(parser, str) match { case Success(result, _) => Some(result.toArray) case _ => None @@ -349,14 +353,14 @@ case object VariantGet { /** The actual implementation of the `VariantGet` expression. */ def variantGet( input: VariantVal, - parsedPath: Array[VariantPathParser.PathSegment], + parsedPath: Array[VariantPathSegment], dataType: DataType, castArgs: VariantCastArgs): Any = { var v = new Variant(input.getValue, input.getMetadata) for (path <- parsedPath) { v = path match { - case scala.util.Left(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) - case scala.util.Right(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) + case ObjectExtraction(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) + case ArrayExtraction(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) case _ => null } if (v == null) return null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala index 83d219c28983b..33ba4f772a13a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.HashMap import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.variant.{VariantGet, VariantPathParser} +import org.apache.spark.sql.catalyst.expressions.variant._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project, Subquery} import org.apache.spark.sql.catalyst.rules.Rule @@ -54,7 +54,7 @@ case class VariantMetadata( .build() ).build() - def parsedPath(): Array[VariantPathParser.PathSegment] = { + def parsedPath(): Array[VariantPathSegment] = { VariantPathParser.parse(path).getOrElse { val name = if (failOnError) "variant_get" else "try_variant_get" throw QueryExecutionErrors.invalidVariantGetPath(path, name) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index c0c4900344154..ffb6704061e66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.expressions.variant._ -import org.apache.spark.sql.catalyst.expressions.variant.VariantPathParser.PathSegment import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.RowToColumnConverter @@ -56,9 +55,9 @@ case class SparkShreddedRow(row: SpecializedGetters) extends ShreddingUtils.Shre override def numElements(): Int = row.asInstanceOf[ArrayData].numElements() } -// The search result of a `PathSegment` in a `VariantSchema`. +// The search result of a `VariantPathSegment` in a `VariantSchema`. case class SchemaPathSegment( - rawPath: PathSegment, + rawPath: VariantPathSegment, // Whether this path segment is an object or array extraction. isObject: Boolean, // `schema.typedIdx`, if the path exists in the schema (for object extraction, the schema @@ -714,11 +713,11 @@ case object SparkShreddingUtils { // found at a certain level of the file type, then `typedIdx` will be -1 starting from // this position, and the final `schema` will be null. for (i <- rawPath.indices) { - val isObject = rawPath(i).isLeft + val isObject = rawPath(i).isInstanceOf[ObjectExtraction] var typedIdx = -1 var extractionIdx = -1 rawPath(i) match { - case scala.util.Left(key) if schema != null && schema.objectSchema != null => + case ObjectExtraction(key) if schema != null && schema.objectSchema != null => val fieldIdx = schema.objectSchemaMap.get(key) if (fieldIdx != null) { typedIdx = schema.typedIdx @@ -727,7 +726,7 @@ case object SparkShreddingUtils { } else { schema = null } - case scala.util.Right(index) if schema != null && schema.arraySchema != null => + case ArrayExtraction(index) if schema != null && schema.arraySchema != null => typedIdx = schema.typedIdx extractionIdx = index schema = schema.arraySchema @@ -770,8 +769,8 @@ case object SparkShreddingUtils { var v = new Variant(row.getBinary(variantIdx), topLevelMetadata) while (pathIdx < pathLen) { v = pathList(pathIdx).rawPath match { - case scala.util.Left(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) - case scala.util.Right(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) + case ObjectExtraction(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) + case ArrayExtraction(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) case _ => null } if (v == null) return null From 36d23eff4b4c3a2b8fd301672e532132c96fdd68 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 7 Jan 2025 16:23:38 +0800 Subject: [PATCH 343/438] [SPARK-50541] Describe Table As JSON ### What changes were proposed in this pull request? Support `DESCRIBE TABLE ... [AS JSON]` to optionally display table metadata in JSON format. **SQL Ref Spec:** { DESC | DESCRIBE } [ TABLE ] [ EXTENDED | FORMATTED ] table_name { [ PARTITION clause ] | [ column_name ] } **[ AS JSON ]** Output: json_metadata: String ### Why are the changes needed? The Spark SQL command `DESCRIBE TABLE` displays table metadata in a DataFrame format geared toward human consumption. This format causes parsing challenges, e.g. if fields contain special characters or the format changes as new features are added. The new `AS JSON` option would return the table metadata as a JSON string that supports parsing via machine, while being extensible with a minimized risk of breaking changes. It is not meant to be human-readable. ### Does this PR introduce _any_ user-facing change? Yes, this provides a new option to display DESCRIBE TABLE metadata in JSON format. See below (and updated golden files) for the JSON output schema: ``` { "table_name": "", "catalog_name": "", "schema_name": "", "namespace": [""], "type": "", "provider": "", "columns": [ { "name": "", "type": , "comment": "", "nullable": , "default": "" } ], "partition_values": { "": "" }, "location": "", "view_text": "", "view_original_text": "", "view_schema_mode": "", "view_catalog_and_namespace": "", "view_query_output_columns": ["col1", "col2"], "owner": "", "comment": "", "table_properties": { "property1": "", "property2": "" }, "storage_properties": { "property1": "", "property2": "" }, "serde_library": "", "input_format": "", "output_format": "", "num_buckets": , "bucket_columns": [""], "sort_columns": [""], "created_time": "", "last_access": "", "partition_provider": "" } ``` ### How was this patch tested? - Updated golden files for `describe.sql` - Added tests in `DescribeTableParserSuite.scala`, `DescribeTableSuite.scala`, `PlanResolutionSuite.scala` ### Was this patch authored or co-authored using generative AI tooling? Closes #49139 from asl3/asl3/describetableasjson. Authored-by: Amanda Liu Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 12 + docs/sql-ref-ansi-compliance.md | 1 + docs/sql-ref-syntax-aux-describe-table.md | 99 +++- .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 1 + .../sql/catalyst/parser/SqlBaseParser.g4 | 4 +- .../spark/sql/errors/CompilationErrors.scala | 12 + .../sql/catalyst/catalog/interface.scala | 235 +++++--- .../sql/catalyst/parser/AstBuilder.scala | 13 +- .../plans/DescribeCommandSchema.scala | 20 +- .../catalyst/plans/logical/v2Commands.scala | 13 + .../sql/errors/QueryCompilationErrors.scala | 4 + .../analysis/ResolveSessionCatalog.scala | 8 +- .../spark/sql/execution/command/tables.scala | 259 ++++++++ .../datasources/v2/DataSourceV2Strategy.scala | 3 + .../analyzer-results/describe.sql.out | 41 ++ .../resources/sql-tests/inputs/describe.sql | 12 + .../sql-tests/results/describe.sql.out | 53 ++ .../results/keywords-enforced.sql.out | 1 + .../sql-tests/results/keywords.sql.out | 1 + .../results/nonansi/keywords.sql.out | 1 + .../sql-tests/results/show-tables.sql.out | 2 - .../apache/spark/sql/SQLQueryTestHelper.scala | 13 +- .../command/DescribeTableParserSuite.scala | 7 + .../command/PlanResolutionSuite.scala | 39 +- .../command/v1/DescribeTableSuite.scala | 558 ++++++++++++++++++ .../ThriftServerWithSparkContextSuite.scala | 2 +- 26 files changed, 1313 insertions(+), 101 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 52c0315bd0732..97c8f059bcdec 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1155,6 +1155,13 @@ ], "sqlState" : "42623" }, + "DESCRIBE_JSON_NOT_EXTENDED" : { + "message" : [ + "DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.", + "For example: DESCRIBE EXTENDED AS JSON is supported but DESCRIBE AS JSON is not." + ], + "sqlState" : "0A000" + }, "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED" : { "message" : [ "Distinct window functions are not supported: ." @@ -5283,6 +5290,11 @@ "Attach a comment to the namespace ." ] }, + "DESC_TABLE_COLUMN_JSON" : { + "message" : [ + "DESC TABLE COLUMN AS JSON not supported for individual columns." + ] + }, "DESC_TABLE_COLUMN_PARTITION" : { "message" : [ "DESC TABLE COLUMN for a specific partition." diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 50cdcd6d09790..3b1138b9ee0e5 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -568,6 +568,7 @@ Below is a list of all the keywords in Spark SQL. |ITEMS|non-reserved|non-reserved|non-reserved| |ITERATE|non-reserved|non-reserved|non-reserved| |JOIN|reserved|strict-non-reserved|reserved| +|JSON|non-reserved|non-reserved|non-reserved| |KEYS|non-reserved|non-reserved|non-reserved| |LANGUAGE|non-reserved|non-reserved|reserved| |LAST|non-reserved|non-reserved|non-reserved| diff --git a/docs/sql-ref-syntax-aux-describe-table.md b/docs/sql-ref-syntax-aux-describe-table.md index 4b6e1e8c3461e..6a14da1e43801 100644 --- a/docs/sql-ref-syntax-aux-describe-table.md +++ b/docs/sql-ref-syntax-aux-describe-table.md @@ -29,16 +29,17 @@ to return the metadata pertaining to a partition or column respectively. ### Syntax ```sql -{ DESC | DESCRIBE } [ TABLE ] [ format ] table_identifier [ partition_spec ] [ col_name ] +{ DESC | DESCRIBE } [ TABLE ] [ format ] table_identifier [ partition_spec ] [ col_name ] [ AS JSON ] ``` ### Parameters * **format** - Specifies the optional format of describe output. If `EXTENDED` is specified + Specifies the optional format of describe output. If `EXTENDED` or `FORMATTED` is specified then additional metadata information (such as parent database, owner, and access time) - is returned. + is returned. Also if `EXTENDED` or `FORMATTED` is specified, then the metadata can be returned + in JSON format by specifying `AS JSON` at the end of the statement. * **table_identifier** @@ -60,8 +61,96 @@ to return the metadata pertaining to a partition or column respectively. and `col_name` are mutually exclusive and can not be specified together. Currently nested columns are not allowed to be specified. + JSON format is not currently supported for individual columns. + **Syntax:** `[ database_name. ] [ table_name. ] column_name` +* **AS JSON** + + An optional parameter to return the table metadata in JSON format. Only supported when `EXTENDED` + or `FORMATTED` format is specified (both produce equivalent JSON). + + **Syntax:** `[ AS JSON ]` + + **Schema:** + + Below is the full JSON schema. + In actual output, null fields are omitted and the JSON is not pretty-printed (see Examples). + + ```sql + { + "table_name": "", + "catalog_name": "", + "schema_name": "", + "namespace": [""], + "type": "", + "provider": "", + "columns": [ + { + "name": "", + "type": , + "comment": "", + "nullable": , + "default": "" + } + ], + "partition_values": { + "": "" + }, + "location": "", + "view_text": "", + "view_original_text": "", + "view_schema_mode": "", + "view_catalog_and_namespace": "", + "view_query_output_columns": ["col1", "col2"], + "comment": "", + "table_properties": { + "property1": "", + "property2": "" + }, + "storage_properties": { + "property1": "", + "property2": "" + }, + "serde_library": "", + "input_format": "", + "output_format": "", + "num_buckets": , + "bucket_columns": [""], + "sort_columns": [""], + "created_time": "", + "created_by": "", + "last_access": "", + "partition_provider": "" + } + ``` + + Below are the schema definitions for ``: + +| Spark SQL Data Types | JSON Representation | +|-----------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| ByteType | `{ "name" : "tinyint" }` | +| ShortType | `{ "name" : "smallint" }` | +| IntegerType | `{ "name" : "int" }` | +| LongType | `{ "name" : "bigint" }` | +| FloatType | `{ "name" : "float" }` | +| DoubleType | `{ "name" : "double" }` | +| DecimalType | `{ "name" : "decimal", "precision": p, "scale": s }` | +| StringType | `{ "name" : "string" }` | +| VarCharType | `{ "name" : "varchar", "length": n }` | +| CharType | `{ "name" : "char", "length": n }` | +| BinaryType | `{ "name" : "binary" }` | +| BooleanType | `{ "name" : "boolean" }` | +| DateType | `{ "name" : "date" }` | +| VariantType | `{ "name" : "variant" }` | +| TimestampType | `{ "name" : "timestamp_ltz" }` | +| TimestampNTZType | `{ "name" : "timestamp_ntz" }` | +| YearMonthIntervalType | `{ "name" : "interval", "start_unit": "", "end_unit": "" }` | +| DayTimeIntervalType | `{ "name" : "interval", "start_unit": "", "end_unit": "" }` | +| ArrayType | `{ "name" : "array", "element_type": , "element_nullable": }` | +| MapType | `{ "name" : "map", "key_type": , "value_type": , "value_nullable": }` | +| StructType | `{ "name" : "struct", "fields": [ {"name" : "field1", "type" : , “nullable”: , "comment": “”, "default": “”}, ... ] }` | + ### Examples ```sql @@ -173,6 +262,10 @@ DESCRIBE customer salesdb.customer.name; |data_type| string| | comment|Short name| +---------+----------+ + +-- Returns the table metadata in JSON format. +DESC FORMATTED customer AS JSON; +{"table_name":"customer","catalog_name":"spark_catalog","schema_name":"default","namespace":["default"],"columns":[{"name":"cust_id","type":{"name":"integer"},"nullable":true},{"name":"name","type":{"name":"string"},"comment":"Short name","nullable":true},{"name":"state","type":{"name":"varchar","length":20},"nullable":true}],"location": "file:/tmp/salesdb.db/custom...","created_time":"2020-04-07T14:05:43Z","last_access":"UNKNOWN","created_by":"None","type":"MANAGED","provider":"parquet","partition_provider":"Catalog","partition_columns":["state"]} ``` ### Related Statements diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 91a267364216c..dafeed48aef11 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -283,6 +283,7 @@ IS: 'IS'; ITEMS: 'ITEMS'; ITERATE: 'ITERATE'; JOIN: 'JOIN'; +JSON: 'JSON'; KEYS: 'KEYS'; LANGUAGE: 'LANGUAGE'; LAST: 'LAST'; diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 3ca120da98dd4..667d200268cf8 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -287,7 +287,7 @@ statement | (DESC | DESCRIBE) namespace EXTENDED? identifierReference #describeNamespace | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? - identifierReference partitionSpec? describeColName? #describeRelation + identifierReference partitionSpec? describeColName? (AS JSON)? #describeRelation | (DESC | DESCRIBE) QUERY? query #describeQuery | COMMENT ON namespace identifierReference IS comment #commentNamespace @@ -1680,6 +1680,7 @@ ansiNonReserved | INVOKER | ITEMS | ITERATE + | JSON | KEYS | LANGUAGE | LAST @@ -2039,6 +2040,7 @@ nonReserved | IS | ITEMS | ITERATE + | JSON | KEYS | LANGUAGE | LAST diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala index 3e63b8281f739..617cab4b2a39b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala @@ -41,6 +41,18 @@ private[sql] trait CompilationErrors extends DataTypeErrorsBase { cause = Option(cause)) } + def describeJsonNotExtendedError(tableName: String): AnalysisException = { + new AnalysisException( + errorClass = "DESCRIBE_JSON_NOT_EXTENDED", + messageParameters = Map("tableName" -> tableName)) + } + + def describeColJsonUnsupportedError(): AnalysisException = { + new AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_JSON", + messageParameters = Map.empty) + } + def cannotFindDescriptorFileError(filePath: String, cause: Throwable): AnalysisException = { new AnalysisException( errorClass = "PROTOBUF_DESCRIPTOR_FILE_NOT_FOUND", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index e199452a2da67..858e2cf25b6fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -28,7 +28,7 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule} import org.apache.commons.lang3.StringUtils -import org.json4s.JsonAST.{JArray, JString} +import org.json4s.JsonAST.{JArray, JBool, JDouble, JInt, JNull, JObject, JString, JValue} import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkException @@ -51,6 +51,48 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils} import org.apache.spark.util.ArrayImplicits._ +/** + * Interface providing util to convert JValue to String representation of catalog entities. + */ +trait MetadataMapSupport { + def toJsonLinkedHashMap: mutable.LinkedHashMap[String, JValue] + + def toLinkedHashMap: mutable.LinkedHashMap[String, String] = { + jsonToString(toJsonLinkedHashMap) + } + + protected def jsonToString( + jsonMap: mutable.LinkedHashMap[String, JValue]): mutable.LinkedHashMap[String, String] = { + def removeWhitespace(str: String): String = { + str.replaceAll("\\s+$", "") + } + + val map = new mutable.LinkedHashMap[String, String]() + jsonMap.foreach { case (key, jValue) => + val stringValue = jValue match { + case JString(value) => removeWhitespace(value) + case JArray(values) => + values.map(_.values) + .map { + case str: String => quoteIdentifier(removeWhitespace(str)) + case other => removeWhitespace(other.toString) + } + .mkString("[", ", ", "]") + case JObject(fields) => + fields.map { case (k, v) => + s"$k=${removeWhitespace(v.values.toString)}" + } + .mkString("[", ", ", "]") + case JInt(value) => value.toString + case JDouble(value) => value.toString + case _ => removeWhitespace(jValue.values.toString) + } + map.put(key, stringValue) + } + map + } +} + /** * A function defined in the catalog. @@ -74,25 +116,31 @@ case class CatalogStorageFormat( outputFormat: Option[String], serde: Option[String], compressed: Boolean, - properties: Map[String, String]) { + properties: Map[String, String]) extends MetadataMapSupport { override def toString: String = { - toLinkedHashMap.map { case ((key, value)) => + toLinkedHashMap.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("Storage(", ", ", ")") } - def toLinkedHashMap: mutable.LinkedHashMap[String, String] = { - val map = new mutable.LinkedHashMap[String, String]() - locationUri.foreach(l => map.put("Location", l.toString)) - serde.foreach(map.put("Serde Library", _)) - inputFormat.foreach(map.put("InputFormat", _)) - outputFormat.foreach(map.put("OutputFormat", _)) - if (compressed) map.put("Compressed", "") + def toJsonLinkedHashMap: mutable.LinkedHashMap[String, JValue] = { + val map = mutable.LinkedHashMap[String, JValue]() + + locationUri.foreach(l => map += ("Location" -> JString(l.toString))) + serde.foreach(s => map += ("Serde Library" -> JString(s))) + inputFormat.foreach(format => map += ("InputFormat" -> JString(format))) + outputFormat.foreach(format => map += ("OutputFormat" -> JString(format))) + + if (compressed) map += ("Compressed" -> JBool(true)) + SQLConf.get.redactOptions(properties) match { case props if props.isEmpty => // No-op case props => - map.put("Storage Properties", props.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]")) + val storagePropsJson = JObject( + props.map { case (k, v) => k -> JString(v) }.toList + ) + map += ("Storage Properties" -> storagePropsJson) } map } @@ -120,35 +168,44 @@ case class CatalogTablePartition( parameters: Map[String, String] = Map.empty, createTime: Long = System.currentTimeMillis, lastAccessTime: Long = -1, - stats: Option[CatalogStatistics] = None) { + stats: Option[CatalogStatistics] = None) extends MetadataMapSupport { + def toJsonLinkedHashMap: mutable.LinkedHashMap[String, JValue] = { + val map = mutable.LinkedHashMap[String, JValue]() - def toLinkedHashMap: mutable.LinkedHashMap[String, String] = { - val map = new mutable.LinkedHashMap[String, String]() - val specString = spec.map { case (k, v) => s"$k=$v" }.mkString(", ") - map.put("Partition Values", s"[$specString]") - map ++= storage.toLinkedHashMap - if (parameters.nonEmpty) { - map.put("Partition Parameters", s"{" + - s"${SQLConf.get.redactOptions(parameters).map(p => p._1 + "=" + p._2).mkString(", ")}}") + val specJson = JObject(spec.map { case (k, v) => k -> JString(v) }.toList) + map += ("Partition Values" -> specJson) + + storage.toJsonLinkedHashMap.foreach { case (k, v) => + map += (k -> v) } - map.put("Created Time", new Date(createTime).toString) - val lastAccess = { - if (lastAccessTime <= 0) "UNKNOWN" else new Date(lastAccessTime).toString + + if (parameters.nonEmpty) { + val paramsJson = JObject(SQLConf.get.redactOptions(parameters).map { + case (k, v) => k -> JString(v) + }.toList) + map += ("Partition Parameters" -> paramsJson) } - map.put("Last Access", lastAccess) - stats.foreach(s => map.put("Partition Statistics", s.simpleString)) + + map += ("Created Time" -> JString(new Date(createTime).toString)) + + val lastAccess = if (lastAccessTime <= 0) JString("UNKNOWN") + else JString(new Date(lastAccessTime).toString) + map += ("Last Access" -> lastAccess) + + stats.foreach(s => map += ("Partition Statistics" -> JString(s.simpleString))) + map } override def toString: String = { - toLinkedHashMap.map { case ((key, value)) => + toLinkedHashMap.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("CatalogPartition(\n\t", "\n\t", ")") } /** Readable string representation for the CatalogTablePartition. */ def simpleString: String = { - toLinkedHashMap.map { case ((key, value)) => + toLinkedHashMap.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("", "\n", "") } @@ -284,7 +341,7 @@ object ClusterBySpec { case class BucketSpec( numBuckets: Int, bucketColumnNames: Seq[String], - sortColumnNames: Seq[String]) extends SQLConfHelper { + sortColumnNames: Seq[String]) extends SQLConfHelper with MetadataMapSupport { if (numBuckets <= 0 || numBuckets > conf.bucketingMaxBuckets) { throw QueryCompilationErrors.invalidBucketNumberError( @@ -301,11 +358,11 @@ case class BucketSpec( s"$numBuckets buckets, $bucketString$sortString" } - def toLinkedHashMap: mutable.LinkedHashMap[String, String] = { - mutable.LinkedHashMap[String, String]( - "Num Buckets" -> numBuckets.toString, - "Bucket Columns" -> bucketColumnNames.map(quoteIdentifier).mkString("[", ", ", "]"), - "Sort Columns" -> sortColumnNames.map(quoteIdentifier).mkString("[", ", ", "]") + def toJsonLinkedHashMap: mutable.LinkedHashMap[String, JValue] = { + mutable.LinkedHashMap[String, JValue]( + "Num Buckets" -> JInt(numBuckets), + "Bucket Columns" -> JArray(bucketColumnNames.map(JString).toList), + "Sort Columns" -> JArray(sortColumnNames.map(JString).toList) ) } } @@ -355,7 +412,7 @@ case class CatalogTable( tracksPartitionsInCatalog: Boolean = false, schemaPreservesCase: Boolean = true, ignoredProperties: Map[String, String] = Map.empty, - viewOriginalText: Option[String] = None) { + viewOriginalText: Option[String] = None) extends MetadataMapSupport { import CatalogTable._ @@ -524,67 +581,81 @@ case class CatalogTable( locationUri, inputFormat, outputFormat, serde, compressed, properties)) } + def toJsonLinkedHashMap: mutable.LinkedHashMap[String, JValue] = { + val filteredTableProperties = SQLConf.get + .redactOptions(properties.filter { case (k, v) => + !k.startsWith(VIEW_PREFIX) && v.nonEmpty + }) - def toLinkedHashMap: mutable.LinkedHashMap[String, String] = { - val map = new mutable.LinkedHashMap[String, String]() - val tableProperties = - SQLConf.get.redactOptions(properties.filter { case (k, _) => !k.startsWith(VIEW_PREFIX) }) - .toSeq.sortBy(_._1) - .map(p => p._1 + "=" + p._2) - val partitionColumns = partitionColumnNames.map(quoteIdentifier).mkString("[", ", ", "]") - val lastAccess = { - if (lastAccessTime <= 0) "UNKNOWN" else new Date(lastAccessTime).toString + val tableProperties: JValue = + if (filteredTableProperties.isEmpty) JNull + else JObject( + filteredTableProperties.toSeq.sortBy(_._1).map { case (k, v) => k -> JString(v) }: _*) + + val partitionColumns: JValue = + if (partitionColumnNames.nonEmpty) JArray(partitionColumnNames.map(JString).toList) + else JNull + + val lastAccess: JValue = + if (lastAccessTime <= 0) JString("UNKNOWN") else JString( + DateTimeUtils.microsToInstant(DateTimeUtils.millisToMicros(lastAccessTime)).toString) + + val viewQueryOutputColumns: JValue = + if (viewQueryColumnNames.nonEmpty) JArray(viewQueryColumnNames.map(JString).toList) + else JNull + + val map = mutable.LinkedHashMap[String, JValue]() + + if (identifier.catalog.isDefined) map += "Catalog" -> JString(identifier.catalog.get) + if (identifier.database.isDefined) map += "Database" -> JString(identifier.database.get) + map += "Table" -> JString(identifier.table) + if (Option(owner).exists(_.nonEmpty)) map += "Owner" -> JString(owner) + map += "Created Time" -> + JString(DateTimeUtils.microsToInstant(DateTimeUtils.millisToMicros(createTime)).toString) + if (lastAccess != JNull) map += "Last Access" -> lastAccess + map += "Created By" -> JString(s"Spark $createVersion") + map += "Type" -> JString(tableType.name) + if (provider.isDefined) map += "Provider" -> JString(provider.get) + bucketSpec.foreach { spec => + map ++= spec.toJsonLinkedHashMap.map { case (k, v) => k -> v } } - - identifier.catalog.foreach(map.put("Catalog", _)) - identifier.database.foreach(map.put("Database", _)) - map.put("Table", identifier.table) - if (owner != null && owner.nonEmpty) map.put("Owner", owner) - map.put("Created Time", new Date(createTime).toString) - map.put("Last Access", lastAccess) - map.put("Created By", "Spark " + createVersion) - map.put("Type", tableType.name) - provider.foreach(map.put("Provider", _)) - bucketSpec.foreach(map ++= _.toLinkedHashMap) - comment.foreach(map.put("Comment", _)) - collation.foreach(map.put("Collation", _)) - if (tableType == CatalogTableType.VIEW) { - viewText.foreach(map.put("View Text", _)) - viewOriginalText.foreach(map.put("View Original Text", _)) - if (SQLConf.get.viewSchemaBindingEnabled) { - map.put("View Schema Mode", viewSchemaMode.toString) - } - if (viewCatalogAndNamespace.nonEmpty) { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - map.put("View Catalog and Namespace", viewCatalogAndNamespace.quoted) - } - if (viewQueryColumnNames.nonEmpty) { - map.put("View Query Output Columns", - viewQueryColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")) - } + if (comment.isDefined) map += "Comment" -> JString(comment.get) + if (collation.isDefined) map += "Collation" -> JString(collation.get) + if (tableType == CatalogTableType.VIEW && viewText.isDefined) { + map += "View Text" -> JString(viewText.get) } - - if (tableProperties.nonEmpty) { - map.put("Table Properties", tableProperties.mkString("[", ", ", "]")) + if (tableType == CatalogTableType.VIEW && viewOriginalText.isDefined) { + map += "View Original Text" -> JString(viewOriginalText.get) } - stats.foreach(s => map.put("Statistics", s.simpleString)) - map ++= storage.toLinkedHashMap - if (tracksPartitionsInCatalog) map.put("Partition Provider", "Catalog") - if (partitionColumnNames.nonEmpty) map.put("Partition Columns", partitionColumns) - if (schema.nonEmpty) map.put("Schema", schema.treeString) - - map + if (SQLConf.get.viewSchemaBindingEnabled && tableType == CatalogTableType.VIEW) { + map += "View Schema Mode" -> JString(viewSchemaMode.toString) + } + if (viewCatalogAndNamespace.nonEmpty && tableType == CatalogTableType.VIEW) { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + map += "View Catalog and Namespace" -> JString(viewCatalogAndNamespace.quoted) + } + if (viewQueryOutputColumns != JNull) { + map += "View Query Output Columns" -> viewQueryOutputColumns + } + if (tableProperties != JNull) map += "Table Properties" -> tableProperties + if (stats.isDefined) map += "Statistics" -> JString(stats.get.simpleString) + map ++= storage.toJsonLinkedHashMap.map { case (k, v) => k -> v } + if (tracksPartitionsInCatalog) map += "Partition Provider" -> JString("Catalog") + if (partitionColumns != JNull) map += "Partition Columns" -> partitionColumns + if (schema.nonEmpty) map += "Schema" -> JString(schema.treeString) + + map.filterNot(_._2 == JNull) } override def toString: String = { - toLinkedHashMap.map { case ((key, value)) => + toLinkedHashMap.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("CatalogTable(\n", "\n", ")") } /** Readable string representation for the CatalogTable. */ def simpleString: String = { - toLinkedHashMap.map { case ((key, value)) => + toLinkedHashMap.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("", "\n", "") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 0a300cea03ffe..1f9c148303647 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -5200,10 +5200,17 @@ class AstBuilder extends DataTypeAstBuilder */ override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null + val asJson = ctx.JSON != null + if (asJson && !isExtended) { + val tableName = ctx.identifierReference.getText.split("\\.").lastOption.getOrElse("table") + throw QueryCompilationErrors.describeJsonNotExtendedError(tableName) + } val relation = createUnresolvedTableOrView(ctx.identifierReference, "DESCRIBE TABLE") if (ctx.describeColName != null) { if (ctx.partitionSpec != null) { throw QueryParsingErrors.descColumnForPartitionUnsupportedError(ctx) + } else if (asJson) { + throw QueryCompilationErrors.describeColJsonUnsupportedError() } else { DescribeColumn( relation, @@ -5221,7 +5228,11 @@ class AstBuilder extends DataTypeAstBuilder } else { Map.empty[String, String] } - DescribeRelation(relation, partitionSpec, isExtended) + if (asJson) { + DescribeRelationJson(relation, partitionSpec, isExtended) + } else { + DescribeRelation(relation, partitionSpec, isExtended) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeCommandSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeCommandSchema.scala index 99d2ea7751959..a6ec6f5736300 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeCommandSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeCommandSchema.scala @@ -21,13 +21,19 @@ import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.types.{MetadataBuilder, StringType} private[sql] object DescribeCommandSchema { - def describeTableAttributes(): Seq[AttributeReference] = Seq( - AttributeReference("col_name", StringType, nullable = false, - new MetadataBuilder().putString("comment", "name of the column").build())(), - AttributeReference("data_type", StringType, nullable = false, - new MetadataBuilder().putString("comment", "data type of the column").build())(), - AttributeReference("comment", StringType, nullable = true, - new MetadataBuilder().putString("comment", "comment of the column").build())()) + def describeJsonTableAttributes(): Seq[AttributeReference] = + Seq( + AttributeReference("json_metadata", StringType, nullable = false, + new MetadataBuilder().putString("comment", "JSON metadata of the table").build())() + ) + def describeTableAttributes(): Seq[AttributeReference] = { + Seq(AttributeReference("col_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column").build())(), + AttributeReference("data_type", StringType, nullable = false, + new MetadataBuilder().putString("comment", "data type of the column").build())(), + AttributeReference("comment", StringType, nullable = true, + new MetadataBuilder().putString("comment", "comment of the column").build())()) + } def describeColumnAttributes(): Seq[AttributeReference] = Seq( AttributeReference("info_name", StringType, nullable = false, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 58c62a90225aa..b486a1fd0a72a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -691,6 +691,19 @@ object DescribeRelation { def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes() } +/** + * The logical plan of the DESCRIBE relation_name AS JSON command. + */ +case class DescribeRelationJson( + relation: LogicalPlan, + partitionSpec: TablePartitionSpec, + isExtended: Boolean) extends UnaryCommand { + override val output: Seq[Attribute] = DescribeCommandSchema.describeJsonTableAttributes() + override def child: LogicalPlan = relation + override protected def withNewChildInternal(newChild: LogicalPlan): DescribeRelationJson = + copy(relation = newChild) +} + /** * The logical plan of the DESCRIBE relation_name col_name command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 65ae8da3c4da1..ac419fd150ae3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1615,6 +1615,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat notSupportedForV2TablesError("ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]") } + def describeAsJsonNotSupportedForV2TablesError(): Throwable = { + notSupportedForV2TablesError("DESCRIBE TABLE AS JSON") + } + def loadDataNotSupportedForV2TablesError(): Throwable = { notSupportedForV2TablesError("LOAD DATA") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 6a388a7849f75..3b58518b98da9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -152,13 +152,17 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case RenameTable(ResolvedV1TableOrViewIdentifier(oldIdent), newName, isView) => AlterTableRenameCommand(oldIdent, newName.asTableIdentifier, isView) + case DescribeRelationJson( + ResolvedV1TableOrViewIdentifier(ident), partitionSpec, isExtended) => + DescribeTableJsonCommand(ident, partitionSpec, isExtended) + // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. case DescribeRelation( - ResolvedV1TableOrViewIdentifier(ident), partitionSpec, isExtended, output) => + ResolvedV1TableOrViewIdentifier(ident), partitionSpec, isExtended, output) => DescribeTableCommand(ident, partitionSpec, isExtended, output) case DescribeColumn( - ResolvedViewIdentifier(ident), column: UnresolvedAttribute, isExtended, output) => + ResolvedViewIdentifier(ident), column: UnresolvedAttribute, isExtended, output) => // For views, the column will not be resolved by `ResolveReferences` because // `ResolvedView` stores only the identifier. DescribeColumnCommand(ident, column.nameParts, isExtended, output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index a58e8fac6e36d..e69e05ba7decd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -19,12 +19,16 @@ package org.apache.spark.sql.execution.command import java.net.{URI, URISyntaxException} +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileContext, FsConstants, Path} import org.apache.hadoop.fs.permission.{AclEntry, AclEntryScope, AclEntryType, FsAction, FsPermission} +import org.json4s._ +import org.json4s.JsonAST.JObject +import org.json4s.jackson.JsonMethods._ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{SQLConfHelper, TableIdentifier} @@ -746,6 +750,261 @@ case class DescribeTableCommand( } } +/** + * Command that looks like + * {{{ + * DESCRIBE [EXTENDED|FORMATTED] table_name partitionSpec? [AS JSON]; + * }}} + */ +case class DescribeTableJsonCommand( + table: TableIdentifier, + partitionSpec: TablePartitionSpec, + isExtended: Boolean) extends LeafRunnableCommand { + override val output = DescribeCommandSchema.describeJsonTableAttributes() + // Already added to jsonMap in DescribeTableJsonCommand + private val excludedKeys = Set("catalog", "schema", "database", "table") + + override def run(sparkSession: SparkSession): Seq[Row] = { + val jsonMap = mutable.LinkedHashMap[String, JValue]() + val catalog = sparkSession.sessionState.catalog + + if (catalog.isTempView(table)) { + if (partitionSpec.nonEmpty) { + throw QueryCompilationErrors.descPartitionNotAllowedOnTempView(table.identifier) + } + val schema = catalog.getTempViewOrPermanentTableMetadata(table).schema + describeColsJson(schema, jsonMap, header = false) + } else { + val metadata = catalog.getTableRawMetadata(table) + val schema = if (metadata.schema.isEmpty) { + // In older versions of Spark, + // the table schema can be empty and should be inferred at runtime. + sparkSession.table(metadata.identifier).schema + } else { + metadata.schema + } + + addKeyValueToMap("table_name", JString(metadata.identifier.table), jsonMap) + table.catalog.foreach(catalog => addKeyValueToMap("catalog_name", JString(catalog), jsonMap)) + table.database.foreach { db => + addKeyValueToMap("namespace", JArray(List(JString(db))), jsonMap) + addKeyValueToMap("schema_name", JString(db), jsonMap) + } + + describeColsJson(schema, jsonMap, header = false) + describeClusteringInfoJson(metadata, jsonMap) + + if (partitionSpec.nonEmpty) { + // Outputs the partition-specific info for the DDL command: + // "DESCRIBE [EXTENDED|FORMATTED] table_name PARTITION (partitionVal*)" + describePartitionInfoJson(sparkSession, catalog, metadata, jsonMap) + } else { + describeFormattedTableInfoJson(metadata, jsonMap) + } + } + + Seq(Row(compact(render(JObject(jsonMap.toList))))) + } + + private def addKeyValueToMap( + key: String, + value: JValue, + jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + // Rename some JSON keys that are pre-named in describe table implementation + val renames = Map( + "inputformat" -> "input_format", + "outputformat" -> "output_format" + ) + + val normalizedKey = key.toLowerCase().replace(" ", "_") + val renamedKey = renames.getOrElse(normalizedKey, normalizedKey) + + if (!jsonMap.contains(renamedKey) && !excludedKeys.contains(renamedKey)) { + jsonMap += renamedKey -> value + } + } + + /** + * Util to recursively form JSON string representation of data type, used for DESCRIBE AS JSON. + * Differs from `json` in DataType.scala by providing additional fields for some types. + */ + private def jsonType( + dataType: DataType): JValue = { + dataType match { + case arrayType: ArrayType => + JObject( + "name" -> JString("array"), + "element_type" -> jsonType(arrayType.elementType), + "element_nullable" -> JBool(arrayType.containsNull) + ) + + case mapType: MapType => + JObject( + "name" -> JString("map"), + "key_type" -> jsonType(mapType.keyType), + "value_type" -> jsonType(mapType.valueType), + "value_nullable" -> JBool(mapType.valueContainsNull) + ) + + case structType: StructType => + val fieldsJson = structType.fields.map { field => + val baseJson = List( + "name" -> JString(field.name), + "type" -> jsonType(field.dataType), + "nullable" -> JBool(field.nullable) + ) + val commentJson = field.getComment().map(comment => "comment" -> JString(comment)).toList + val defaultJson = + field.getCurrentDefaultValue().map(default => "default" -> JString(default)).toList + + JObject(baseJson ++ commentJson ++ defaultJson: _*) + }.toList + + JObject( + "name" -> JString("struct"), + "fields" -> JArray(fieldsJson) + ) + + case decimalType: DecimalType => + JObject( + "name" -> JString("decimal"), + "precision" -> JInt(decimalType.precision), + "scale" -> JInt(decimalType.scale) + ) + + case varcharType: VarcharType => + JObject( + "name" -> JString("varchar"), + "length" -> JInt(varcharType.length) + ) + + case charType: CharType => + JObject( + "name" -> JString("char"), + "length" -> JInt(charType.length) + ) + + // Only override TimestampType; TimestampType_NTZ type is already timestamp_ntz + case _: TimestampType => + JObject("name" -> JString("timestamp_ltz")) + + case yearMonthIntervalType: YearMonthIntervalType => + def getFieldName(field: Byte): String = YearMonthIntervalType.fieldToString(field) + + JObject( + "name" -> JString("interval"), + "start_unit" -> JString(getFieldName(yearMonthIntervalType.startField)), + "end_unit" -> JString(getFieldName(yearMonthIntervalType.endField)) + ) + + case dayTimeIntervalType: DayTimeIntervalType => + def getFieldName(field: Byte): String = DayTimeIntervalType.fieldToString(field) + + JObject( + "name" -> JString("interval"), + "start_unit" -> JString(getFieldName(dayTimeIntervalType.startField)), + "end_unit" -> JString(getFieldName(dayTimeIntervalType.endField)) + ) + + case _ => + JObject("name" -> JString(dataType.typeName)) + } + } + + private def describeColsJson( + schema: StructType, + jsonMap: mutable.LinkedHashMap[String, JValue], + header: Boolean): Unit = { + val columnsJson = jsonType(StructType(schema.fields)) + .asInstanceOf[JObject].find(_.isInstanceOf[JArray]).get + addKeyValueToMap("columns", columnsJson, jsonMap) + } + + private def describeClusteringInfoJson( + table: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + table.clusterBySpec.foreach { clusterBySpec => + val clusteringColumnsJson: JValue = JArray( + clusterBySpec.columnNames.map { fieldNames => + val nestedFieldOpt = table.schema.findNestedField(fieldNames.fieldNames.toIndexedSeq) + assert(nestedFieldOpt.isDefined, + "The clustering column " + + s"${fieldNames.fieldNames.map(quoteIfNeeded).mkString(".")} " + + s"was not found in the table schema ${table.schema.catalogString}." + ) + val (path, field) = nestedFieldOpt.get + JObject( + "name" -> JString((path :+ field.name).map(quoteIfNeeded).mkString(".")), + "type" -> jsonType(field.dataType), + "comment" -> field.getComment().map(JString).getOrElse(JNull) + ) + }.toList + ) + addKeyValueToMap("clustering_information", clusteringColumnsJson, jsonMap) + } + } + + private def describeFormattedTableInfoJson( + table: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + table.bucketSpec match { + case Some(spec) => + spec.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + case _ => + } + table.storage.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + + val filteredTableInfo = table.toJsonLinkedHashMap + + filteredTableInfo.map { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + } + + private def describePartitionInfoJson( + spark: SparkSession, + catalog: SessionCatalog, + metadata: CatalogTable, + jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + if (metadata.tableType == CatalogTableType.VIEW) { + throw QueryCompilationErrors.descPartitionNotAllowedOnView(table.identifier) + } + + DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION") + val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( + partitionSpec, + metadata.partitionSchema, + table.quotedString, + spark.sessionState.conf.resolver) + val partition = catalog.getPartition(table, normalizedPartSpec) + + // First add partition details to jsonMap. + // `addKeyValueToMap` only adds unique keys, so this ensures the + // more detailed partition information is added + // in the case of duplicated key names (e.g. storage_information). + partition.toJsonLinkedHashMap.map { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + + metadata.toJsonLinkedHashMap.map { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + + metadata.bucketSpec match { + case Some(spec) => + spec.toJsonLinkedHashMap.map { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + case _ => + } + metadata.storage.toJsonLinkedHashMap.map { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + } +} + /** * Command that looks like * {{{ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 499721fbae4e8..f7a3be9254758 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -344,6 +344,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DescribeNamespace(ResolvedNamespace(catalog, ns, _), extended, output) => DescribeNamespaceExec(output, catalog.asNamespaceCatalog, ns, extended) :: Nil + case DescribeRelationJson(_, _, _) => + throw QueryCompilationErrors.describeAsJsonNotSupportedForV2TablesError() + case DescribeRelation(r: ResolvedTable, partitionSpec, isExtended, output) => if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.describeDoesNotSupportPartitionForV2TablesError() diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out index ff0935bfd03ec..f52f69a5ff808 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -56,6 +56,35 @@ DESCRIBE t DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] +-- !query +DESCRIBE EXTENDED t AS JSON +-- !query analysis +DescribeTableJsonCommand `spark_catalog`.`default`.`t`, true + + +-- !query +DESCRIBE t AS JSON +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "DESCRIBE_JSON_NOT_EXTENDED", + "sqlState" : "0A000", + "messageParameters" : { + "tableName" : "t" + } +} + + +-- !query +DESC FORMATTED t a AS JSON +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_JSON", + "sqlState" : "0A000" +} + + -- !query DESC default.t -- !query analysis @@ -110,6 +139,12 @@ DESC t PARTITION (c='Us', d=1) DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=1], false, [col_name#x, data_type#x, comment#x] +-- !query +DESC EXTENDED t PARTITION (c='Us', d=1) AS JSON +-- !query analysis +DescribeTableJsonCommand `spark_catalog`.`default`.`t`, [c=Us, d=1], true + + -- !query DESC EXTENDED t PARTITION (c='Us', d=1) -- !query analysis @@ -290,6 +325,12 @@ EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) ExplainCommand 'DescribeRelation [c=Us, d=2], false, [col_name#x, data_type#x, comment#x], SimpleMode +-- !query +EXPLAIN DESCRIBE EXTENDED t PARTITION (c='Us', d=2) AS JSON +-- !query analysis +ExplainCommand 'DescribeRelationJson [c=Us, d=2], true, SimpleMode + + -- !query DROP TABLE t -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql index b37931456d00c..aa6f38defdecc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/describe.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql @@ -21,6 +21,14 @@ ALTER TABLE t ADD PARTITION (c='Us', d=1); DESCRIBE t; +DESCRIBE EXTENDED t AS JSON; + +-- AnalysisException: describe table as json must be extended +DESCRIBE t AS JSON; + +-- AnalysisException: describe col as json unsupported +DESC FORMATTED t a AS JSON; + DESC default.t; DESC TABLE t; @@ -39,6 +47,8 @@ DESC EXTENDED t; DESC t PARTITION (c='Us', d=1); +DESC EXTENDED t PARTITION (c='Us', d=1) AS JSON; + DESC EXTENDED t PARTITION (c='Us', d=1); DESC FORMATTED t PARTITION (c='Us', d=1); @@ -88,6 +98,7 @@ EXPLAIN DESC EXTENDED t; EXPLAIN EXTENDED DESC t; EXPLAIN DESCRIBE t b; EXPLAIN DESCRIBE t PARTITION (c='Us', d=2); +EXPLAIN DESCRIBE EXTENDED t PARTITION (c='Us', d=2) AS JSON; -- DROP TEST TABLES/VIEWS DROP TABLE t; @@ -119,3 +130,4 @@ DESC EXTENDED e; DESC TABLE EXTENDED e; DESC FORMATTED e; + diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 0f51816c145e8..015b0ceff335e 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -71,6 +71,41 @@ c string d string +-- !query +DESCRIBE EXTENDED t AS JSON +-- !query schema +struct +-- !query output +{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string"},"nullable":true},{"name":"b","type":{"name":"integer"},"nullable":true},{"name":"c","type":{"name":"string"},"nullable":true},{"name":"d","type":{"name":"string"},"nullable":true}],"num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"location":"file:[not included in comparison]/{warehouse_dir}/t","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","comment":"table_comment","table_properties":{"e":"3","password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"]} + + +-- !query +DESCRIBE t AS JSON +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "DESCRIBE_JSON_NOT_EXTENDED", + "sqlState" : "0A000", + "messageParameters" : { + "tableName" : "t" + } +} + + +-- !query +DESC FORMATTED t a AS JSON +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_JSON", + "sqlState" : "0A000" +} + + -- !query DESC default.t -- !query schema @@ -263,6 +298,14 @@ c string d string +-- !query +DESC EXTENDED t PARTITION (c='Us', d=1) AS JSON +-- !query schema +struct +-- !query output +{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string"},"nullable":true},{"name":"b","type":{"name":"integer"},"nullable":true},{"name":"c","type":{"name":"string"},"nullable":true},{"name":"d","type":{"name":"string"},"nullable":true}],"partition_values":{"c":"Us","d":"1"},"location":"file:[not included in comparison]/{warehouse_dir}/t/c=Us/d=1","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"table_properties":{"password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"]} + + -- !query DESC EXTENDED t PARTITION (c='Us', d=1) -- !query schema @@ -644,6 +687,16 @@ Execute DescribeTableCommand +- DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=2], false, [col_name#x, data_type#x, comment#x] +-- !query +EXPLAIN DESCRIBE EXTENDED t PARTITION (c='Us', d=2) AS JSON +-- !query schema +struct +-- !query output +== Physical Plan == +Execute DescribeTableJsonCommand + +- DescribeTableJsonCommand `spark_catalog`.`default`.`t`, [c=Us, d=2], true + + -- !query DROP TABLE t -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out index f9c9df3f9bf5f..521b0afe19264 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out @@ -170,6 +170,7 @@ IS true ITEMS false ITERATE false JOIN true +JSON false KEYS false LANGUAGE false LAST false diff --git a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out index 67e5e4170d789..4d702588ad2b3 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out @@ -170,6 +170,7 @@ IS false ITEMS false ITERATE false JOIN false +JSON false KEYS false LANGUAGE false LAST false diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out index 67e5e4170d789..4d702588ad2b3 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out @@ -170,6 +170,7 @@ IS false ITEMS false ITERATE false JOIN false +JSON false KEYS false LANGUAGE false LAST false diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index cc32e2eff2551..a4b967ca61f08 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -128,7 +128,6 @@ View Schema Mode: BINDING Schema: root |-- e: integer (nullable = true) - showdb show_t1 false Catalog: spark_catalog Database: showdb Table: show_t1 @@ -146,7 +145,6 @@ Schema: root |-- c: string (nullable = true) |-- d: string (nullable = true) - showdb show_t2 false Catalog: spark_catalog Database: showdb Table: show_t2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index 7daf2c6b1b58b..04f274e4af592 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -60,7 +60,18 @@ trait SQLQueryTestHelper extends Logging { .replaceAll("CTERelationDef \\d+,", s"CTERelationDef xxxx,") .replaceAll("CTERelationRef \\d+,", s"CTERelationRef xxxx,") .replaceAll("@\\w*,", s"@xxxxxxxx,") - .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds + .replaceAll("\\*\\(\\d+\\) ", "*") + .replaceAll( + s""""location":.*?$clsName/""", + s""""location": "$notIncludedMsg/{warehouse_dir}/""") + .replaceAll(s""""created_by":".*?"""", s""""created_by $notIncludedMsg":"None"""") + .replaceAll(s""""created_time":".*?"""", s""""created_time $notIncludedMsg":"None"""") + .replaceAll(s""""last_access":".*?"""", s""""last_access $notIncludedMsg":"None"""") + .replaceAll(s""""owner":".*?"""", s""""owner $notIncludedMsg":"None"""") + .replaceAll(s""""partition_statistics":"\\d+"""", + s""""partition_statistics $notIncludedMsg":"None"""") + .replaceAll("cterelationdef \\d+,", "cterelationdef xxxx,") + .replaceAll("cterelationref \\d+,", "cterelationref xxxx,") } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala index 944f20bf8e924..d81f007e2a4d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.command +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedTableOrView} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumn, DescribeRelation} @@ -75,6 +76,12 @@ class DescribeTableParserSuite extends AnalysisTest { UnresolvedAttribute(Seq("col")), isExtended = true)) + val error = intercept[AnalysisException](parsePlan("DESCRIBE EXTENDED t col AS JSON")) + + checkError( + exception = error, + condition = "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_JSON") + val sql = "DESCRIBE TABLE t PARTITION (ds='1970-01-01') col" checkError( exception = parseException(parsePlan)(sql), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 92467cbcb6c05..541fec1cb3740 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, Literal, StringLiteral} import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} -import org.apache.spark.sql.catalyst.plans.logical.{AlterColumn, AnalysisOnlyCommand, AppendData, Assignment, CreateTable, CreateTableAsSelect, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, OverwriteByExpression, OverwritePartitionsDynamic, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterColumn, AnalysisOnlyCommand, AppendData, Assignment, CreateTable, CreateTableAsSelect, DeleteAction, DeleteFromTable, DescribeRelation, DescribeRelationJson, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, OverwriteByExpression, OverwritePartitionsDynamic, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.connector.FakeV2Provider @@ -961,6 +961,43 @@ class PlanResolutionSuite extends AnalysisTest { assert(parsed4.isInstanceOf[DescribeTableCommand]) } + test("DESCRIBE AS JSON relation") { + Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { + case (tblName, useV1Command) => + val sql = s"DESC TABLE EXTENDED $tblName AS JSON" + val parsed = parseAndResolve(sql) + if (useV1Command) { + val expected2 = DescribeTableJsonCommand( + TableIdentifier(tblName, Some("default"), Some(SESSION_CATALOG_NAME)), + Map.empty, true) + + comparePlans(parsed, expected2) + } else { + parsed match { + case DescribeRelationJson(_: ResolvedTable, _, isExtended) => + assert(isExtended) + case _ => fail("Expect DescribeTable, but got:\n" + parsed.treeString) + } + } + + val sql2 = s"DESC TABLE EXTENDED $tblName PARTITION(a=1) AS JSON" + val parsed2 = parseAndResolve(sql2) + if (useV1Command) { + val expected2 = DescribeTableJsonCommand( + TableIdentifier(tblName, Some("default"), Some(SESSION_CATALOG_NAME)), + Map("a" -> "1"), true) + comparePlans(parsed2, expected2) + } else { + parsed2 match { + case DescribeRelationJson(_: ResolvedTable, partitionSpec, isExtended) => + assert(isExtended) + assert(partitionSpec == Map("a" -> "1")) + case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) + } + } + } + } + test("DELETE FROM") { Seq("v2Table", "testcat.tab").foreach { tblName => val sql1 = s"DELETE FROM $tblName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index 164ac2bff9f63..4413087e886e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.execution.command.v1 import java.util.Locale +import org.json4s._ +import org.json4s.jackson.JsonMethods.parse + import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.execution.command @@ -36,6 +39,7 @@ import org.apache.spark.sql.types.StringType */ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase with command.TestsV1AndV2Commands { + implicit val formats: org.json4s.DefaultFormats.type = org.json4s.DefaultFormats def getProvider(): String = defaultUsing.stripPrefix("USING").trim.toLowerCase(Locale.ROOT) @@ -276,4 +280,558 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { )) } } + + test("DESCRIBE AS JSON throws when not EXTENDED") { + withNamespaceAndTable("ns", "table") { t => + val tableCreationStr = + s""" + |CREATE TABLE $t ( + | employee_id INT, + | employee_name STRING, + | department STRING, + | hire_date DATE + |) USING parquet + |OPTIONS ('compression' = 'snappy', 'max_records' = '1000') + |PARTITIONED BY (department, hire_date) + |CLUSTERED BY (employee_id) SORTED BY (employee_name ASC) INTO 4 BUCKETS + |COMMENT 'Employee data table for testing partitions and buckets' + |TBLPROPERTIES ('version' = '1.0') + |""".stripMargin + spark.sql(tableCreationStr) + + val error = intercept[AnalysisException] { + spark.sql(s"DESCRIBE $t AS JSON") + } + + checkError( + exception = error, + condition = "DESCRIBE_JSON_NOT_EXTENDED", + parameters = Map("tableName" -> "table")) + } + } + + test("DESCRIBE AS JSON partitions, clusters, buckets") { + withNamespaceAndTable("ns", "table") { t => + val tableCreationStr = + s""" + |CREATE TABLE $t ( + | employee_id INT, + | employee_name STRING, + | department STRING, + | hire_date DATE + |) USING parquet + |OPTIONS ('compression' = 'snappy', 'max_records' = '1000') + |PARTITIONED BY (department, hire_date) + |CLUSTERED BY (employee_id) SORTED BY (employee_name ASC) INTO 4 BUCKETS + |COMMENT 'Employee data table for testing partitions and buckets' + |TBLPROPERTIES ('version' = '1.0') + |""".stripMargin + spark.sql(tableCreationStr) + val descriptionDf = spark.sql(s"DESCRIBE EXTENDED $t AS JSON") + val firstRow = descriptionDf.select("json_metadata").head() + val jsonValue = firstRow.getString(0) + val parsedOutput = parse(jsonValue).extract[DescribeTableJson] + + val expectedOutput = DescribeTableJson( + table_name = Some("table"), + catalog_name = Some(SESSION_CATALOG_NAME), + namespace = Some(List("ns")), + schema_name = Some("ns"), + columns = Some(List( + TableColumn("employee_id", Type("integer"), true), + TableColumn("employee_name", Type("string"), true), + TableColumn("department", Type("string"), true), + TableColumn("hire_date", Type("date"), true) + )), + owner = Some(""), + created_time = Some(""), + last_access = Some("UNKNOWN"), + created_by = Some("Spark 4.0.0-SNAPSHOT"), + `type` = Some("MANAGED"), + provider = Some("parquet"), + bucket_columns = Some(List("employee_id")), + sort_columns = Some(List("employee_name")), + comment = Some("Employee data table for testing partitions and buckets"), + table_properties = Some(Map( + "version" -> "1.0" + )), + location = Some(""), + serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), + inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + storage_properties = Some(Map( + "compression" -> "snappy", + "max_records" -> "1000" + )), + partition_provider = Some("Catalog"), + partition_columns = Some(List("department", "hire_date")) + ) + + if (getProvider() == "hive") { + assert(expectedOutput == parsedOutput.copy(owner = Some(""), + created_time = Some(""), + location = Some(""))) + } else { + assert(expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) + == parsedOutput.copy(owner = Some(""), created_time = Some(""), location = Some(""))) + } + } + } + + test("DESCRIBE AS JSON partition spec") { + withNamespaceAndTable("ns", "table") { t => + val tableCreationStr = + s""" + |CREATE TABLE $t ( + | id INT, + | name STRING, + | region STRING, + | category STRING + |) USING parquet + |PARTITIONED BY (region, category) + |COMMENT 'test partition spec' + |TBLPROPERTIES ('t' = 'test') + |""".stripMargin + spark.sql(tableCreationStr) + spark.sql(s"ALTER TABLE $t ADD PARTITION (region='USA', category='tech')") + + val descriptionDf = + spark.sql(s"DESCRIBE FORMATTED $t PARTITION (region='USA', category='tech') AS JSON") + val firstRow = descriptionDf.select("json_metadata").head() + val jsonValue = firstRow.getString(0) + val parsedOutput = parse(jsonValue).extract[DescribeTableJson] + + val expectedOutput = DescribeTableJson( + table_name = Some("table"), + catalog_name = Some("spark_catalog"), + namespace = Some(List("ns")), + schema_name = Some("ns"), + columns = Some(List( + TableColumn("id", Type("integer"), true), + TableColumn("name", Type("string"), true), + TableColumn("region", Type("string"), true), + TableColumn("category", Type("string"), true) + )), + last_access = Some("UNKNOWN"), + created_by = Some("Spark 4.0.0-SNAPSHOT"), + `type` = Some("MANAGED"), + provider = Some("parquet"), + bucket_columns = Some(Nil), + sort_columns = Some(Nil), + comment = Some("test partition spec"), + table_properties = Some(Map( + "t" -> "test" + )), + serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), + inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + storage_properties = Some(Map( + "serialization.format" -> "1" + )), + partition_provider = Some("Catalog"), + partition_columns = Some(List("region", "category")), + partition_values = Some(Map("region" -> "USA", "category" -> "tech")) + ) + + val filteredParsedStorageProperties = + parsedOutput.storage_properties.map(_.filterNot { case (key, _) => key == "path" }) + + if (getProvider() == "hive") { + assert(expectedOutput == + parsedOutput.copy(location = None, created_time = None, owner = None, + storage_properties = filteredParsedStorageProperties)) + } else { + assert(expectedOutput.copy( + inputformat = None, outputformat = None, serde_library = None, storage_properties = None) + == parsedOutput.copy(location = None, created_time = None, owner = None, + storage_properties = filteredParsedStorageProperties)) + } + } + } + + test("DESCRIBE AS JSON default values") { + withNamespaceAndTable("ns", "table") { t => + val tableCreationStr = + s""" + |CREATE TABLE $t ( + | id INT DEFAULT 1, + | name STRING DEFAULT 'unknown', + | created_at TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + | is_active BOOLEAN DEFAULT true + |) + |USING parquet COMMENT 'table_comment' + |""".stripMargin + spark.sql(tableCreationStr) + + val descriptionDf = spark.sql(s"DESC EXTENDED $t AS JSON") + val firstRow = descriptionDf.select("json_metadata").head() + val jsonValue = firstRow.getString(0) + val parsedOutput = parse(jsonValue).extract[DescribeTableJson] + + val expectedOutput = DescribeTableJson( + table_name = Some("table"), + catalog_name = Some("spark_catalog"), + namespace = Some(List("ns")), + schema_name = Some("ns"), + columns = Some(List( + TableColumn("id", Type("integer"), default = Some("1")), + TableColumn("name", Type("string"), default = Some("'unknown'")), + TableColumn("created_at", Type("timestamp_ltz"), default = Some("CURRENT_TIMESTAMP")), + TableColumn("is_active", Type("boolean"), default = Some("true")) + )), + last_access = Some("UNKNOWN"), + created_by = Some("Spark 4.0.0-SNAPSHOT"), + `type` = Some("MANAGED"), + storage_properties = None, + provider = Some("parquet"), + bucket_columns = Some(Nil), + sort_columns = Some(Nil), + comment = Some("table_comment"), + serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), + inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + table_properties = None + ) + if (getProvider() == "hive") { + assert( + expectedOutput == + parsedOutput.copy(location = None, created_time = None, owner = None) + ) + } else { + assert( + expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) == + parsedOutput.copy(location = None, created_time = None, owner = None) + ) + } + } + } + + test("DESCRIBE AS JSON temp view") { + withNamespaceAndTable("ns", "table") { t => + withTempView("temp_view") { + val tableCreationStr = + s""" + |CREATE TABLE $t (id INT, name STRING, created_at TIMESTAMP) + | USING parquet + | OPTIONS ('compression' 'snappy') + | CLUSTERED BY (id, name) SORTED BY (created_at) INTO 4 BUCKETS + | COMMENT 'test temp view' + | TBLPROPERTIES ('parquet.encryption' = 'true') + |""".stripMargin + spark.sql(tableCreationStr) + spark.sql(s"CREATE TEMPORARY VIEW temp_view AS SELECT * FROM $t") + val descriptionDf = spark.sql(s"DESCRIBE EXTENDED temp_view AS JSON") + val firstRow = descriptionDf.select("json_metadata").head() + val jsonValue = firstRow.getString(0) + val parsedOutput = parse(jsonValue).extract[DescribeTableJson] + + val expectedOutput = DescribeTableJson( + columns = Some(List( + TableColumn("id", Type("integer")), + TableColumn("name", Type("string")), + TableColumn("created_at", Type("timestamp_ltz")) + )) + ) + + assert(expectedOutput == parsedOutput) + } + } + } + + test("DESCRIBE AS JSON persistent view") { + withNamespaceAndTable("ns", "table") { t => + withView("view") { + val tableCreationStr = + s""" + |CREATE TABLE $t (id INT, name STRING, created_at TIMESTAMP) + | USING parquet + | OPTIONS ('compression' 'snappy') + | CLUSTERED BY (id, name) SORTED BY (created_at) INTO 4 BUCKETS + | COMMENT 'test temp view' + | TBLPROPERTIES ('parquet.encryption' = 'true') + |""".stripMargin + spark.sql(tableCreationStr) + spark.sql(s"CREATE VIEW view AS SELECT * FROM $t") + val descriptionDf = spark.sql(s"DESCRIBE EXTENDED view AS JSON") + val firstRow = descriptionDf.select("json_metadata").head() + val jsonValue = firstRow.getString(0) + val parsedOutput = parse(jsonValue).extract[DescribeTableJson] + + val expectedOutput = DescribeTableJson( + table_name = Some("view"), + catalog_name = Some("spark_catalog"), + namespace = Some(List("default")), + schema_name = Some("default"), + columns = Some(List( + TableColumn("id", Type("integer")), + TableColumn("name", Type("string")), + TableColumn("created_at", Type("timestamp_ltz")) + )), + serde_library = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"), + inputformat = Some("org.apache.hadoop.mapred.SequenceFileInputFormat"), + outputformat = Some("org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"), + storage_properties = Some(Map("serialization.format" -> "1")), + last_access = Some("UNKNOWN"), + created_by = Some("Spark 4.0.0-SNAPSHOT"), + `type` = Some("VIEW"), + view_text = Some("SELECT * FROM spark_catalog.ns.table"), + view_original_text = Some("SELECT * FROM spark_catalog.ns.table"), + view_schema_mode = Some("COMPENSATION"), + view_catalog_and_namespace = Some("spark_catalog.default"), + view_query_output_columns = Some(List("id", "name", "created_at")) + ) + + if (getProvider() == "hive") { + assert(expectedOutput == + parsedOutput.copy(table_properties = None, created_time = None, owner = None)) + } else { + assert(expectedOutput.copy(inputformat = None, + outputformat = None, serde_library = None, storage_properties = None) + == parsedOutput.copy(table_properties = None, created_time = None, owner = None)) + } + } + } + } + + test("DESCRIBE AS JSON for column throws Analysis Exception") { + withNamespaceAndTable("ns", "table") { t => + val tableCreationStr = + s""" + |CREATE TABLE ns.table( + | cust_id INT, + | state VARCHAR(20), + | name STRING COMMENT "Short name" + | ) + | USING parquet + | PARTITIONED BY (state) + |""".stripMargin + spark.sql(tableCreationStr) + spark.sql("INSERT INTO ns.table PARTITION (state = \"CA\") VALUES (100, \"Jane\")") + val error = intercept[AnalysisException] { + spark.sql("DESCRIBE FORMATTED ns.table ns.table.name AS JSON") + } + + checkError( + exception = error, + condition = "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_JSON") + } + } + + test("DESCRIBE AS JSON complex types") { + withNamespaceAndTable("ns", "table") { t => + val tableCreationStr = + s""" + |CREATE TABLE $t ( + | id STRING, + | logs VARIANT, + | nested_struct STRUCT< + | name: STRING, + | age: INT, + | contact: STRUCT< + | email: STRING, + | phone_numbers: ARRAY, + | addresses: ARRAY> + | > + | >, + | preferences MAP> + |) USING parquet + | OPTIONS (option1 'value1', option2 'value2') + | PARTITIONED BY (id) + | COMMENT 'A table with nested complex types' + | TBLPROPERTIES ('property1' = 'value1', 'password' = 'password') + """.stripMargin + spark.sql(tableCreationStr) + val descriptionDf = spark.sql(s"DESCRIBE EXTENDED $t AS JSON") + val firstRow = descriptionDf.select("json_metadata").head() + val jsonValue = firstRow.getString(0) + val parsedOutput = parse(jsonValue).extract[DescribeTableJson] + + val expectedOutput = DescribeTableJson( + table_name = Some("table"), + catalog_name = Some("spark_catalog"), + namespace = Some(List("ns")), + schema_name = Some("ns"), + columns = Some(List( + TableColumn( + name = "logs", + `type` = Type("variant"), + default = None + ), + TableColumn( + name = "nested_struct", + `type` = Type( + name = "struct", + fields = Some(List( + Field( + name = "name", + `type` = Type("string") + ), + Field( + name = "age", + `type` = Type("integer") + ), + Field( + name = "contact", + `type` = Type( + name = "struct", + fields = Some(List( + Field( + name = "email", + `type` = Type("string") + ), + Field( + name = "phone_numbers", + `type` = Type( + name = "array", + element_type = Some(Type("string")), + element_nullable = Some(true) + ) + ), + Field( + name = "addresses", + `type` = Type( + name = "array", + element_type = Some(Type( + name = "struct", + fields = Some(List( + Field( + name = "street", + `type` = Type("string") + ), + Field( + name = "city", + `type` = Type("string") + ), + Field( + name = "zip", + `type` = Type("integer") + ) + )) + )), + element_nullable = Some(true) + ) + ) + )) + ) + ) + )) + ), + default = None + ), + TableColumn( + name = "preferences", + `type` = Type( + name = "map", + key_type = Some(Type("string")), + value_type = Some(Type( + name = "array", + element_type = Some(Type("string")), + element_nullable = Some(true) + )), + value_nullable = Some(true) + ), + default = None + ), + TableColumn( + name = "id", + `type` = Type("string"), + default = None + ) + )), + serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), + inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + storage_properties = Some(Map( + "option1" -> "value1", + "option2" -> "value2" + )), + last_access = Some("UNKNOWN"), + created_by = Some("Spark 4.0.0-SNAPSHOT"), + `type` = Some("MANAGED"), + provider = Some("parquet"), + comment = Some("A table with nested complex types"), + table_properties = Some(Map( + "password" -> "*********(redacted)", + "property1" -> "value1" + )), + partition_provider = Some("Catalog"), + partition_columns = Some(List("id")) + ) + + if (getProvider() == "hive") { + assert(expectedOutput == + parsedOutput.copy(location = None, created_time = None, owner = None)) + } else { + assert(expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) + == parsedOutput.copy(location = None, created_time = None, owner = None)) + } + } + } } + +/** Represents JSON output of DESCRIBE TABLE AS JSON */ +case class DescribeTableJson( + table_name: Option[String] = None, + catalog_name: Option[String] = None, + namespace: Option[List[String]] = Some(Nil), + schema_name: Option[String] = None, + columns: Option[List[TableColumn]] = Some(Nil), + owner: Option[String] = None, + created_time: Option[String] = None, + last_access: Option[String] = None, + created_by: Option[String] = None, + `type`: Option[String] = None, + provider: Option[String] = None, + bucket_columns: Option[List[String]] = Some(Nil), + sort_columns: Option[List[String]] = Some(Nil), + comment: Option[String] = None, + table_properties: Option[Map[String, String]] = None, + location: Option[String] = None, + serde_library: Option[String] = None, + inputformat: Option[String] = None, + outputformat: Option[String] = None, + storage_properties: Option[Map[String, String]] = None, + partition_provider: Option[String] = None, + partition_columns: Option[List[String]] = Some(Nil), + partition_values: Option[Map[String, String]] = None, + view_text: Option[String] = None, + view_original_text: Option[String] = None, + view_schema_mode: Option[String] = None, + view_catalog_and_namespace: Option[String] = None, + view_query_output_columns: Option[List[String]] = None + ) + +/** Used for columns field of DescribeTableJson */ +case class TableColumn( + name: String, + `type`: Type, + element_nullable: Boolean = true, + comment: Option[String] = None, + default: Option[String] = None +) + +case class Type( + name: String, + fields: Option[List[Field]] = None, + `type`: Option[Type] = None, + element_type: Option[Type] = None, + key_type: Option[Type] = None, + value_type: Option[Type] = None, + comment: Option[String] = None, + default: Option[String] = None, + element_nullable: Option[Boolean] = Some(true), + value_nullable: Option[Boolean] = Some(true), + nullable: Option[Boolean] = Some(true) +) + +case class Field( + name: String, + `type`: Type, + element_nullable: Boolean = true, + comment: Option[String] = None, + default: Option[String] = None +) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index da0ddd3a156f7..254eda69e86e8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -214,7 +214,7 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { val sessionHandle = client.openSession(user, "") val infoValue = client.getInfo(sessionHandle, GetInfoType.CLI_ODBC_KEYWORDS) // scalastyle:off line.size.limit - assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,CONTAINS,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXECUTE,EXISTS,EXPLAIN,EXPORT,EXTEND,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NONE,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,RECURSIVE,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") + assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,CONTAINS,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXECUTE,EXISTS,EXPLAIN,EXPORT,EXTEND,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,JSON,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NONE,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,RECURSIVE,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") // scalastyle:on line.size.limit } } From 204c6729811789cd271627f5d45dfda92176e119 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 7 Jan 2025 17:13:21 +0800 Subject: [PATCH 344/438] [MINOR][TESTS] Fix comments on `runIfTestsEnabled` methods ### What changes were proposed in this pull request? This PR aims to fix comments on `runIfTestsEnabled` methods. ### Why are the changes needed? Apache Spark repository has two `runIfTestsEnabled` methods whose comments are wrong or ambiguous. ``` $ git grep -C1 'def runIfTestsEnabled' connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala- /** Run the give body of code only if Kinesis tests are enabled */ connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala: def runIfTestsEnabled(message: String)(body: => Unit): Unit = { connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala- if (shouldRunTests) { -- connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala- /** Run the give body of code only if Kinesis tests are enabled */ connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala: def runIfTestsEnabled(message: String)(body: => Unit): Unit = { connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala- if (shouldRunTests) { ``` ### Does this PR introduce _any_ user-facing change? No, this is a comment revision. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49393 from dongjoon-hyun/minor2. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- .../org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala | 2 +- .../org/apache/spark/streaming/kinesis/KinesisFunSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala index 9fbbc8ed2e0ff..b560f86ade38c 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerIntegrationFunSuite.scala @@ -43,7 +43,7 @@ trait DockerIntegrationFunSuite extends SparkFunSuite { } } - /** Run the give body of code only if Kinesis tests are enabled */ + /** Run the given body of code only if ENABLE_DOCKER_INTEGRATION_TESTS is 1. */ def runIfTestsEnabled(message: String)(body: => Unit): Unit = { if (shouldRunTests) { body diff --git a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala index 8dc4de1aa3609..7098840d62f91 100644 --- a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala +++ b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala @@ -35,7 +35,7 @@ trait KinesisFunSuite extends SparkFunSuite { } } - /** Run the give body of code only if Kinesis tests are enabled */ + /** Run the given body of code only if ENABLE_KINESIS_TESTS is 1. */ def runIfTestsEnabled(message: String)(body: => Unit): Unit = { if (shouldRunTests) { body From bba8cf48d14f91109eea04e22fd19be188fce5fb Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Tue, 7 Jan 2025 21:18:44 +0800 Subject: [PATCH 345/438] [SPARK-50734][SQL] Add catalog API for creating and registering SQL UDFs ### What changes were proposed in this pull request? This PR adds catalog APIs to support the creation and registration of SQL UDFs. It uses Hive Metastore to persist a SQL UDF by deserializing the function information into a FunctionResource and storing it in Hive (toCatalogFunction). During resolution, it retrieves the catalog function and deserializes it into a SQLFunction. This PR only adds the catalog API, and a subsequent PR will add the analyzer logic to resolve SQL UDFs. ### Why are the changes needed? To support SQL UDFs in Spark. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. End to end tests will be added in the next PR once we support SQL UDF resolution. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49389 from allisonwang-db/spark-50734-sql-udf-catalog-api. Authored-by: Allison Wang Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 11 + .../analysis/SQLFunctionExpression.scala | 41 ++++ .../catalyst/analysis/SQLFunctionNode.scala | 64 ++++++ .../sql/catalyst/catalog/SQLFunction.scala | 128 ++++++++++- .../sql/catalyst/catalog/SessionCatalog.scala | 206 ++++++++++++++++-- .../catalog/UserDefinedFunction.scala | 145 ++++++++++++ .../catalog/UserDefinedFunctionErrors.scala | 18 ++ .../sql/catalyst/catalog/interface.scala | 4 +- .../sql/catalyst/trees/TreePatterns.scala | 2 + .../catalog/UserDefinedFunction.scala | 70 ------ .../command/CreateSQLFunctionCommand.scala | 25 ++- 11 files changed, 624 insertions(+), 90 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionNode.scala rename sql/{core => catalyst}/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala (61%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 97c8f059bcdec..e47387d59fe15 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -758,6 +758,12 @@ ], "sqlState" : "22018" }, + "CORRUPTED_CATALOG_FUNCTION" : { + "message" : [ + "Cannot convert the catalog function '' into a SQL function due to corrupted function information in catalog. If the function is not a SQL function, please make sure the class name '' is loadable." + ], + "sqlState" : "0A000" + }, "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS" : { "message" : [ "Not allowed to create the permanent view without explicitly assigning an alias for the expression ." @@ -5892,6 +5898,11 @@ "The number of columns produced by the RETURN clause (num: ``) does not match the number of column names specified by the RETURNS clause (num: ``) of ." ] }, + "ROUTINE_PROPERTY_TOO_LARGE" : { + "message" : [ + "Cannot convert user defined routine to catalog function: routine properties are too large." + ] + }, "SQL_TABLE_UDF_BODY_MUST_BE_A_QUERY" : { "message" : [ "SQL table function body must be a query." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala new file mode 100644 index 0000000000000..fb6935d64d4c4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala @@ -0,0 +1,41 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.catalog.SQLFunction +import org.apache.spark.sql.catalyst.expressions.{Expression, Unevaluable} +import org.apache.spark.sql.catalyst.trees.TreePattern.{SQL_FUNCTION_EXPRESSION, TreePattern} +import org.apache.spark.sql.types.DataType + +/** + * Represent a SQL function expression resolved from the catalog SQL function builder. + */ +case class SQLFunctionExpression( + name: String, + function: SQLFunction, + inputs: Seq[Expression], + returnType: Option[DataType]) extends Expression with Unevaluable { + override def children: Seq[Expression] = inputs + override def dataType: DataType = returnType.get + override def nullable: Boolean = true + override def prettyName: String = name + override def toString: String = s"$name(${children.mkString(", ")})" + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): SQLFunctionExpression = copy(inputs = newChildren) + final override val nodePatterns: Seq[TreePattern] = Seq(SQL_FUNCTION_EXPRESSION) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionNode.scala new file mode 100644 index 0000000000000..38059d9810a7b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionNode.scala @@ -0,0 +1,64 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.catalog.SQLFunction +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, UnaryNode} +import org.apache.spark.sql.catalyst.trees.TreePattern.{FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION, SQL_TABLE_FUNCTION, TreePattern} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * A container for holding a SQL function query plan and its function identifier. + * + * @param function: the SQL function that this node represents. + * @param child: the SQL function body. + */ +case class SQLFunctionNode( + function: SQLFunction, + child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output + override def stringArgs: Iterator[Any] = Iterator(function.name, child) + override protected def withNewChildInternal(newChild: LogicalPlan): SQLFunctionNode = + copy(child = newChild) + + // Throw a reasonable error message when trying to call a SQL UDF with TABLE argument(s). + if (child.containsPattern(FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION)) { + throw QueryCompilationErrors + .tableValuedArgumentsNotYetImplementedForSqlFunctions("call", toSQLId(function.name.funcName)) + } +} + +/** + * Represent a SQL table function plan resolved from the catalog SQL table function builder. + */ +case class SQLTableFunction( + name: String, + function: SQLFunction, + inputs: Seq[Expression], + override val output: Seq[Attribute]) extends LeafNode { + final override val nodePatterns: Seq[TreePattern] = Seq(SQL_TABLE_FUNCTION) + + // Throw a reasonable error message when trying to call a SQL UDF with TABLE argument(s) because + // this functionality is not implemented yet. + if (inputs.exists(_.containsPattern(FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION))) { + throw QueryCompilationErrors + .tableValuedArgumentsNotYetImplementedForSqlFunctions("call", toSQLId(name)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala similarity index 61% rename from sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala index c0bd4ac80f5ed..923373c1856a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala @@ -22,9 +22,11 @@ import scala.collection.mutable import org.json4s.JsonAST.{JArray, JString} import org.json4s.jackson.JsonMethods.{compact, render} +import org.apache.spark.SparkException +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.catalog.UserDefinedFunction._ -import org.apache.spark.sql.catalyst.expressions.{Expression, ScalarSubquery} +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, ScalarSubquery} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, Project} import org.apache.spark.sql.types.{DataType, StructType} @@ -62,6 +64,8 @@ case class SQLFunction( assert(exprText.nonEmpty || queryText.nonEmpty) assert((isTableFunc && returnType.isRight) || (!isTableFunc && returnType.isLeft)) + import SQLFunction._ + override val language: RoutineLanguage = LanguageSQL /** @@ -88,12 +92,98 @@ case class SQLFunction( (parsedExpression, parsedQuery) } } + + /** Get scalar function return data type. */ + def getScalarFuncReturnType: DataType = returnType match { + case Left(dataType) => dataType + case Right(_) => + throw SparkException.internalError( + "This function is a table function, not a scalar function.") + } + + /** Get table function return columns. */ + def getTableFuncReturnCols: StructType = returnType match { + case Left(_) => + throw SparkException.internalError( + "This function is a scalar function, not a table function.") + case Right(columns) => columns + } + + /** + * Convert the SQL function to a [[CatalogFunction]]. + */ + def toCatalogFunction: CatalogFunction = { + val props = sqlFunctionToProps ++ properties + CatalogFunction( + identifier = name, + className = SQL_FUNCTION_PREFIX, + resources = propertiesToFunctionResources(props, name)) + } + + /** + * Convert the SQL function to an [[ExpressionInfo]]. + */ + def toExpressionInfo: ExpressionInfo = { + val props = sqlFunctionToProps ++ functionMetadataToProps ++ properties + val usage = mapper.writeValueAsString(props) + new ExpressionInfo( + SQL_FUNCTION_PREFIX, + name.database.orNull, + name.funcName, + usage, + "", + "", + "", + "", + "", + "", + "sql_udf") + } + + /** + * Convert the SQL function fields into properties. + */ + private def sqlFunctionToProps: Map[String, String] = { + val props = new mutable.HashMap[String, String] + val inputParamText = inputParam.map(_.fields.map(_.toDDL).mkString(", ")) + inputParamText.foreach(props.put(INPUT_PARAM, _)) + val returnTypeText = returnType match { + case Left(dataType) => dataType.sql + case Right(columns) => columns.toDDL + } + props.put(RETURN_TYPE, returnTypeText) + exprText.foreach(props.put(EXPRESSION, _)) + queryText.foreach(props.put(QUERY, _)) + comment.foreach(props.put(COMMENT, _)) + deterministic.foreach(d => props.put(DETERMINISTIC, d.toString)) + containsSQL.foreach(x => props.put(CONTAINS_SQL, x.toString)) + props.put(IS_TABLE_FUNC, isTableFunc.toString) + props.toMap + } + + private def functionMetadataToProps: Map[String, String] = { + val props = new mutable.HashMap[String, String] + owner.foreach(props.put(OWNER, _)) + props.put(CREATE_TIME, createTimeMs.toString) + props.toMap + } } object SQLFunction { private val SQL_FUNCTION_PREFIX = "sqlFunction." + private val INPUT_PARAM: String = SQL_FUNCTION_PREFIX + "inputParam" + private val RETURN_TYPE: String = SQL_FUNCTION_PREFIX + "returnType" + private val EXPRESSION: String = SQL_FUNCTION_PREFIX + "expression" + private val QUERY: String = SQL_FUNCTION_PREFIX + "query" + private val COMMENT: String = SQL_FUNCTION_PREFIX + "comment" + private val DETERMINISTIC: String = SQL_FUNCTION_PREFIX + "deterministic" + private val CONTAINS_SQL: String = SQL_FUNCTION_PREFIX + "containsSQL" + private val IS_TABLE_FUNC: String = SQL_FUNCTION_PREFIX + "isTableFunc" + private val OWNER: String = SQL_FUNCTION_PREFIX + "owner" + private val CREATE_TIME: String = SQL_FUNCTION_PREFIX + "createTime" + private val FUNCTION_CATALOG_AND_NAMESPACE = "catalogAndNamespace.numParts" private val FUNCTION_CATALOG_AND_NAMESPACE_PART_PREFIX = "catalogAndNamespace.part." @@ -101,6 +191,42 @@ object SQLFunction { private val FUNCTION_REFERRED_TEMP_FUNCTION_NAMES = "referredTempFunctionsNames" private val FUNCTION_REFERRED_TEMP_VARIABLE_NAMES = "referredTempVariableNames" + /** + * Convert a [[CatalogFunction]] into a SQL function. + */ + def fromCatalogFunction(function: CatalogFunction, parser: ParserInterface): SQLFunction = { + try { + val parts = function.resources.collect { case FunctionResource(FileResource, uri) => + val index = uri.substring(0, INDEX_LENGTH).toInt + val body = uri.substring(INDEX_LENGTH) + index -> body + } + val blob = parts.sortBy(_._1).map(_._2).mkString + val props = mapper.readValue(blob, classOf[Map[String, String]]) + val isTableFunc = props(IS_TABLE_FUNC).toBoolean + val returnType = parseReturnTypeText(props(RETURN_TYPE), isTableFunc, parser) + SQLFunction( + name = function.identifier, + inputParam = props.get(INPUT_PARAM).map(parseTableSchema(_, parser)), + returnType = returnType.get, + exprText = props.get(EXPRESSION), + queryText = props.get(QUERY), + comment = props.get(COMMENT), + deterministic = props.get(DETERMINISTIC).map(_.toBoolean), + containsSQL = props.get(CONTAINS_SQL).map(_.toBoolean), + isTableFunc = isTableFunc, + props.filterNot(_._1.startsWith(SQL_FUNCTION_PREFIX))) + } catch { + case e: Exception => + throw new AnalysisException( + errorClass = "CORRUPTED_CATALOG_FUNCTION", + messageParameters = Map( + "identifier" -> s"${function.identifier}", + "className" -> s"${function.className}"), cause = Some(e) + ) + } + } + def parseDefault(text: String, parser: ParserInterface): Expression = { parser.parseExpression(text) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index d87678ac34114..3c6dfe5ac8445 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -32,12 +32,15 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkThrowable} import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder -import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Expression, ExpressionInfo, NamedExpression, UpCast} +import org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry.TableFunctionBuilder +import org.apache.spark.sql.catalyst.catalog.SQLFunction.parseDefault +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, ExpressionInfo, NamedArgumentExpression, NamedExpression, UpCast} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter, LogicalPlan, NamedParametersSupport, Project, SubqueryAlias, View} import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils} import org.apache.spark.sql.connector.catalog.CatalogManager @@ -1532,10 +1535,49 @@ class SessionCatalog( } } + /** + * Create a user defined function. + */ + def createUserDefinedFunction(function: UserDefinedFunction, ignoreIfExists: Boolean): Unit = { + createFunction(function.toCatalogFunction, ignoreIfExists) + } + // ---------------------------------------------------------------- // | Methods that interact with temporary and metastore functions | // ---------------------------------------------------------------- + /** + * Constructs a [[FunctionBuilder]] based on the provided class that represents a function. + */ + private def makeSQLFunctionBuilder(function: SQLFunction): FunctionBuilder = { + if (function.isTableFunc) { + throw UserDefinedFunctionErrors.notAScalarFunction(function.name.nameParts) + } + (input: Seq[Expression]) => { + val args = rearrangeArguments(function.inputParam, input, function.name.toString) + val returnType = function.getScalarFuncReturnType + SQLFunctionExpression( + function.name.unquotedString, function, args, Some(returnType)) + } + } + + /** + * Constructs a [[TableFunctionBuilder]] based on the provided class that represents a function. + */ + private def makeSQLTableFunctionBuilder(function: SQLFunction): TableFunctionBuilder = { + if (!function.isTableFunc) { + throw UserDefinedFunctionErrors.notATableFunction(function.name.nameParts) + } + (input: Seq[Expression]) => { + val args = rearrangeArguments(function.inputParam, input, function.name.toString) + val returnParam = function.getTableFuncReturnCols + val output = returnParam.fields.map { param => + AttributeReference(param.name, param.dataType, param.nullable)() + } + SQLTableFunction(function.name.unquotedString, function, args, output.toSeq) + } + } + /** * Constructs a [[FunctionBuilder]] based on the provided function metadata. */ @@ -1550,6 +1592,24 @@ class SessionCatalog( (input: Seq[Expression]) => functionExpressionBuilder.makeExpression(name, clazz, input) } + private def makeUserDefinedScalarFuncBuilder(func: UserDefinedFunction): FunctionBuilder = { + func match { + case f: SQLFunction => makeSQLFunctionBuilder(f) + case _ => + val clsName = func.getClass.getSimpleName + throw UserDefinedFunctionErrors.unsupportedUserDefinedFunction(clsName) + } + } + + private def makeUserDefinedTableFuncBuilder(func: UserDefinedFunction): TableFunctionBuilder = { + func match { + case f: SQLFunction => makeSQLTableFunctionBuilder(f) + case _ => + val clsName = func.getClass.getSimpleName + throw UserDefinedFunctionErrors.unsupportedUserDefinedFunction(clsName) + } + } + /** * Loads resources such as JARs and Files for a function. Every resource is represented * by a tuple (resource type, resource uri). @@ -1597,6 +1657,81 @@ class SessionCatalog( "hive") } + /** + * Registers a temporary or persistent SQL scalar function into a session-specific + * [[FunctionRegistry]]. + */ + def registerSQLScalarFunction( + function: SQLFunction, + overrideIfExists: Boolean): Unit = { + registerUserDefinedFunction[Expression]( + function, + overrideIfExists, + functionRegistry, + makeSQLFunctionBuilder(function)) + } + + /** + * Registers a temporary or persistent SQL table function into a session-specific + * [[TableFunctionRegistry]]. + */ + def registerSQLTableFunction( + function: SQLFunction, + overrideIfExists: Boolean): Unit = { + registerUserDefinedFunction[LogicalPlan]( + function, + overrideIfExists, + tableFunctionRegistry, + makeSQLTableFunctionBuilder(function)) + } + + /** + * Rearranges the arguments of a UDF into positional order. + */ + private def rearrangeArguments( + inputParams: Option[StructType], + expressions: Seq[Expression], + functionName: String) : Seq[Expression] = { + val firstNamedArgumentExpressionIdx = + expressions.indexWhere(_.isInstanceOf[NamedArgumentExpression]) + if (firstNamedArgumentExpressionIdx == -1) { + return expressions + } + + val paramNames: Seq[InputParameter] = + if (inputParams.isDefined) { + inputParams.get.map { + p => p.getDefault() match { + case Some(defaultExpr) => + // This cast is needed to ensure the default value is of the target data type. + InputParameter(p.name, Some(Cast(parseDefault(defaultExpr, parser), p.dataType))) + case None => + InputParameter(p.name) + } + }.toSeq + } else { + Seq() + } + + NamedParametersSupport.defaultRearrange( + FunctionSignature(paramNames), expressions, functionName) + } + + /** + * Registers a temporary or permanent SQL function into a session-specific function registry. + */ + private def registerUserDefinedFunction[T]( + function: UserDefinedFunction, + overrideIfExists: Boolean, + registry: FunctionRegistryBase[T], + functionBuilder: Seq[Expression] => T): Unit = { + if (registry.functionExists(function.name) && !overrideIfExists) { + throw QueryCompilationErrors.functionAlreadyExistsError(function.name) + } + val info = function.toExpressionInfo + registry.registerFunction(function.name, info, functionBuilder) + } + /** * Unregister a temporary or permanent function from a session-specific [[FunctionRegistry]] * or [[TableFunctionRegistry]]. Return true if function exists. @@ -1753,7 +1888,11 @@ class SessionCatalog( requireDbExists(db) if (externalCatalog.functionExists(db, funcName)) { val metadata = externalCatalog.getFunction(db, funcName) - makeExprInfoForHiveFunction(metadata.copy(identifier = qualifiedIdent)) + if (metadata.isUserDefinedFunction) { + UserDefinedFunction.fromCatalogFunction(metadata, parser).toExpressionInfo + } else { + makeExprInfoForHiveFunction(metadata.copy(identifier = qualifiedIdent)) + } } else { failFunctionLookup(name) } @@ -1765,7 +1904,26 @@ class SessionCatalog( */ def resolvePersistentFunction( name: FunctionIdentifier, arguments: Seq[Expression]): Expression = { - resolvePersistentFunctionInternal(name, arguments, functionRegistry, makeFunctionBuilder) + resolvePersistentFunctionInternal[Expression]( + name, + arguments, + functionRegistry, + registerHiveFunc = func => + registerFunction( + func, + overrideIfExists = false, + registry = functionRegistry, + functionBuilder = makeFunctionBuilder(func) + ), + registerUserDefinedFunc = function => { + val builder = makeUserDefinedScalarFuncBuilder(function) + registerUserDefinedFunction[Expression]( + function = function, + overrideIfExists = false, + registry = functionRegistry, + functionBuilder = builder) + } + ) } /** @@ -1774,16 +1932,29 @@ class SessionCatalog( def resolvePersistentTableFunction( name: FunctionIdentifier, arguments: Seq[Expression]): LogicalPlan = { - // We don't support persistent table functions yet. - val builder = (func: CatalogFunction) => failFunctionLookup(name) - resolvePersistentFunctionInternal(name, arguments, tableFunctionRegistry, builder) + resolvePersistentFunctionInternal[LogicalPlan]( + name, + arguments, + tableFunctionRegistry, + // We don't support persistent Hive table functions yet. + registerHiveFunc = (func: CatalogFunction) => failFunctionLookup(name), + registerUserDefinedFunc = function => { + val builder = makeUserDefinedTableFuncBuilder(function) + registerUserDefinedFunction[LogicalPlan]( + function = function, + overrideIfExists = false, + registry = tableFunctionRegistry, + functionBuilder = builder) + } + ) } private def resolvePersistentFunctionInternal[T]( name: FunctionIdentifier, arguments: Seq[Expression], registry: FunctionRegistryBase[T], - createFunctionBuilder: CatalogFunction => FunctionRegistryBase[T]#FunctionBuilder): T = { + registerHiveFunc: CatalogFunction => Unit, + registerUserDefinedFunc: UserDefinedFunction => Unit): T = { // `synchronized` is used to prevent multiple threads from concurrently resolving the // same function that has not yet been loaded into the function registry. This is needed // because calling `registerFunction` twice with `overrideIfExists = false` can lead to @@ -1799,19 +1970,24 @@ class SessionCatalog( // The function has not been loaded to the function registry, which means // that the function is a persistent function (if it actually has been registered // in the metastore). We need to first put the function in the function registry. - val catalogFunction = externalCatalog.getFunction(db, funcName) - loadFunctionResources(catalogFunction.resources) + val catalogFunction = try { + externalCatalog.getFunction(db, funcName) + } catch { + case _: AnalysisException => failFunctionLookup(qualifiedIdent) + } // Please note that qualifiedName is provided by the user. However, // catalogFunction.identifier.unquotedString is returned by the underlying // catalog. So, it is possible that qualifiedName is not exactly the same as // catalogFunction.identifier.unquotedString (difference is on case-sensitivity). // At here, we preserve the input from the user. val funcMetadata = catalogFunction.copy(identifier = qualifiedIdent) - registerFunction( - funcMetadata, - overrideIfExists = false, - registry = registry, - functionBuilder = createFunctionBuilder(funcMetadata)) + if (!catalogFunction.isUserDefinedFunction) { + loadFunctionResources(catalogFunction.resources) + registerHiveFunc(funcMetadata) + } else { + val function = UserDefinedFunction.fromCatalogFunction(funcMetadata, parser) + registerUserDefinedFunc(function) + } // Now, we need to create the Expression. registry.lookupFunction(qualifiedIdent, arguments) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala new file mode 100644 index 0000000000000..fe00184e843a3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala @@ -0,0 +1,145 @@ +/* + * 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.catalyst.catalog + +import com.fasterxml.jackson.annotation.JsonInclude.Include +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.types.{DataType, StructType} + +/** + * The base class for all user defined functions registered via SQL. + */ +trait UserDefinedFunction { + + /** + * Qualified name of the function + */ + def name: FunctionIdentifier + + /** + * Additional properties to be serialized for the function. + * Use this to preserve the runtime configuration that should be used during the function + * execution, such as SQL configs etc. See [[SQLConf]] for more info. + */ + def properties: Map[String, String] + + /** + * Owner of the function + */ + def owner: Option[String] + + /** + * Function creation time in milliseconds since the linux epoch + */ + def createTimeMs: Long + + /** + * The language of the user defined function. + */ + def language: RoutineLanguage + + /** + * Convert the function to a [[CatalogFunction]]. + */ + def toCatalogFunction: CatalogFunction + + /** + * Convert the SQL function to an [[ExpressionInfo]]. + */ + def toExpressionInfo: ExpressionInfo +} + +object UserDefinedFunction { + val SQL_CONFIG_PREFIX = "sqlConfig." + val INDEX_LENGTH: Int = 3 + + // The default Hive Metastore SQL schema length for function resource uri. + private val HIVE_FUNCTION_RESOURCE_URI_LENGTH_THRESHOLD: Int = 4000 + + def parseTableSchema(text: String, parser: ParserInterface): StructType = { + val parsed = parser.parseTableSchema(text) + CharVarcharUtils.failIfHasCharVarchar(parsed).asInstanceOf[StructType] + } + + def parseDataType(text: String, parser: ParserInterface): DataType = { + val dataType = parser.parseDataType(text) + CharVarcharUtils.failIfHasCharVarchar(dataType) + } + + private val _mapper: ObjectMapper = getObjectMapper + + /** + * A shared [[ObjectMapper]] for serializations. + */ + def mapper: ObjectMapper = _mapper + + /** + * Convert the given properties to a list of function resources. + */ + def propertiesToFunctionResources( + props: Map[String, String], + name: FunctionIdentifier): Seq[FunctionResource] = { + val blob = mapper.writeValueAsString(props) + val threshold = HIVE_FUNCTION_RESOURCE_URI_LENGTH_THRESHOLD - INDEX_LENGTH + blob.grouped(threshold).zipWithIndex.map { case (part, i) => + // Add a sequence number to the part and pad it to a given length. + // E.g. 1 will become "001" if the given length is 3. + val index = s"%0${INDEX_LENGTH}d".format(i) + if (index.length > INDEX_LENGTH) { + throw UserDefinedFunctionErrors.routinePropertyTooLarge(name.funcName) + } + FunctionResource(FileResource, index + part) + }.toSeq + } + + /** + * Get a object mapper to serialize and deserialize function properties. + */ + private def getObjectMapper: ObjectMapper = { + val mapper = new ObjectMapper with ScalaObjectMapper + mapper.setSerializationInclusion(Include.NON_ABSENT) + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + mapper.registerModule(DefaultScalaModule) + mapper + } + + /** + * Convert a [[CatalogFunction]] into a corresponding UDF. + */ + def fromCatalogFunction(function: CatalogFunction, parser: ParserInterface) + : UserDefinedFunction = { + val className = function.className + if (SQLFunction.isSQLFunction(className)) { + SQLFunction.fromCatalogFunction(function, parser) + } else { + throw SparkException.internalError(s"Unsupported function type $className") + } + } + + /** + * Verify if the function is a [[UserDefinedFunction]]. + */ + def isUserDefinedFunction(className: String): Boolean = SQLFunction.isSQLFunction(className) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala index e8cfa8d74e83f..904a17bc8ce44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala @@ -97,4 +97,22 @@ object UserDefinedFunctionErrors extends QueryErrorsBase { "tempObj" -> "VARIABLE", "tempObjName" -> toSQLId(varName))) } + + def routinePropertyTooLarge(routineName: String): Throwable = { + new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.ROUTINE_PROPERTY_TOO_LARGE", + messageParameters = Map("name" -> toSQLId(routineName))) + } + + def notAScalarFunction(functionName: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "NOT_A_SCALAR_FUNCTION", + messageParameters = Map("functionName" -> toSQLId(functionName))) + } + + def notATableFunction(functionName: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "NOT_A_TABLE_FUNCTION", + messageParameters = Map("functionName" -> toSQLId(functionName))) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 858e2cf25b6fe..2ebfcf781b97b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -104,7 +104,9 @@ trait MetadataMapSupport { case class CatalogFunction( identifier: FunctionIdentifier, className: String, - resources: Seq[FunctionResource]) + resources: Seq[FunctionResource]) { + val isUserDefinedFunction: Boolean = UserDefinedFunction.isUserDefinedFunction(className) +} /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 1dfb0336ecf08..80531da4a0ab7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -90,6 +90,8 @@ object TreePattern extends Enumeration { val SCALA_UDF: Value = Value val SESSION_WINDOW: Value = Value val SORT: Value = Value + val SQL_FUNCTION_EXPRESSION: Value = Value + val SQL_TABLE_FUNCTION: Value = Value val SUBQUERY_ALIAS: Value = Value val SUM: Value = Value val TIME_WINDOW: Value = Value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala deleted file mode 100644 index 6567062841de5..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.catalyst.catalog - -import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.catalyst.util.CharVarcharUtils -import org.apache.spark.sql.types.{DataType, StructType} - -/** - * The base class for all user defined functions registered via SQL queries. - */ -trait UserDefinedFunction { - - /** - * Qualified name of the function - */ - def name: FunctionIdentifier - - /** - * Additional properties to be serialized for the function. - * Use this to preserve the runtime configuration that should be used during the function - * execution, such as SQL configs etc. See [[SQLConf]] for more info. - */ - def properties: Map[String, String] - - /** - * Owner of the function - */ - def owner: Option[String] - - /** - * Function creation time in milliseconds since the linux epoch - */ - def createTimeMs: Long - - /** - * The language of the user defined function. - */ - def language: RoutineLanguage -} - -object UserDefinedFunction { - val SQL_CONFIG_PREFIX = "sqlConfig." - - def parseTableSchema(text: String, parser: ParserInterface): StructType = { - val parsed = parser.parseTableSchema(text) - CharVarcharUtils.failIfHasCharVarchar(parsed).asInstanceOf[StructType] - } - - def parseDataType(text: String, parser: ParserInterface): DataType = { - val dataType = parser.parseDataType(text) - CharVarcharUtils.failIfHasCharVarchar(dataType) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala index 25598a12af222..fe4e6f121f57b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.execution.command import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, SQLFunctionNode, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{SessionCatalog, SQLFunction, UserDefinedFunctionErrors} import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Generator, LateralSubquery, Literal, ScalarSubquery, SubqueryExpression, WindowExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{LateralJoin, LogicalPlan, OneRowRelation, Project, SQLFunctionNode, UnresolvedWith} +import org.apache.spark.sql.catalyst.plans.logical.{LateralJoin, LogicalPlan, OneRowRelation, Project, UnresolvedWith} import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_ATTRIBUTE import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command.CreateUserDefinedFunctionCommand._ @@ -249,7 +249,26 @@ case class CreateSQLFunctionCommand( ) } - // TODO: create/register sql functions in catalog + if (isTemp) { + if (isTableFunc) { + catalog.registerSQLTableFunction(newFunction, overrideIfExists = replace) + } else { + catalog.registerSQLScalarFunction(newFunction, overrideIfExists = replace) + } + } else { + if (replace && catalog.functionExists(name)) { + // Hive metastore alter function method does not alter function resources + // so the existing function must be dropped first when replacing a SQL function. + assert(!ignoreIfExists) + catalog.dropFunction(name, ignoreIfExists) + } + // For a persistent function, we will store the metadata into underlying external catalog. + // This function will be loaded into the FunctionRegistry when a query uses it. + // We do not load it into FunctionRegistry right now, to avoid loading the resource + // immediately, as the Spark application to create the function may not have + // access to the function. + catalog.createUserDefinedFunction(newFunction, ignoreIfExists) + } Seq.empty } From 194aa18821c04f068864cc4cf9e3124c54ae7c44 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 7 Jan 2025 07:02:10 -0800 Subject: [PATCH 346/438] [SPARK-49868][DOC][FOLLOWUP] Update docs for executor failure tracking configrations ### What changes were proposed in this pull request? Previously, the executor failure tracking code was located at `ExecutorPodsAllocator`, which only takes effect when `spark.kubernetes.allocation.pods.allocator=direct`. https://github.com/apache/spark/pull/48344 moves the code to the `ExecutorPodsLifecycleManager` consequently removes this limitation. ### Why are the changes needed? Keep docs up-to-date with code. ### Does this PR introduce _any_ user-facing change? Yes, docs are updated. ### How was this patch tested? Review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48358 from pan3793/SPARK-49868-followup. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/internal/config/package.scala | 7 +++---- docs/configuration.md | 6 ++---- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 324ef701c4266..6d51424f0baff 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1023,8 +1023,7 @@ package object config { private[spark] val MAX_EXECUTOR_FAILURES = ConfigBuilder("spark.executor.maxNumFailures") .doc("The maximum number of executor failures before failing the application. " + - "This configuration only takes effect on YARN, or Kubernetes when " + - "`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.") + "This configuration only takes effect on YARN and Kubernetes.") .version("3.5.0") .intConf .createOptional @@ -1032,8 +1031,8 @@ package object config { private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS = ConfigBuilder("spark.executor.failuresValidityInterval") .doc("Interval after which executor failures will be considered independent and not " + - "accumulate towards the attempt count. This configuration only takes effect on YARN, " + - "or Kubernetes when `spark.kubernetes.allocation.pods.allocator` is set to 'direct'.") + "accumulate towards the attempt count. This configuration only takes effect on YARN " + + "and Kubernetes.") .version("3.5.0") .timeConf(TimeUnit.MILLISECONDS) .createOptional diff --git a/docs/configuration.md b/docs/configuration.md index 6957ca9a03d23..4a85c4f256a95 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -565,8 +565,7 @@ of the most common options to set are: numExecutors * 2, with minimum of 3 The maximum number of executor failures before failing the application. - This configuration only takes effect on YARN, or Kubernetes when - spark.kubernetes.allocation.pods.allocator is set to 'direct'. + This configuration only takes effect on YARN and Kubernetes. 3.5.0 @@ -576,8 +575,7 @@ of the most common options to set are: Interval after which executor failures will be considered independent and not accumulate towards the attempt count. - This configuration only takes effect on YARN, or Kubernetes when - spark.kubernetes.allocation.pods.allocator is set to 'direct'. + This configuration only takes effect on YARN and Kubernetes. 3.5.0 From be0ae1388cd87d8264bdcb41332d18961c1b592d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 7 Jan 2025 09:44:24 -0800 Subject: [PATCH 347/438] [SPARK-50541][SQL][TESTS][FOLLOWUP] Use `SPARK_VERSION` instead of hard-coded version strings ### What changes were proposed in this pull request? This is a follow-up to use `SPARK_VERSION` instead of hard-coded version strings. - #49139 ### Why are the changes needed? Hard-coded version strings will cause unit test failures from next week during Apache Spark 4.0.0 RC and maintenance releases like 4.0.1-SNAPSHOT. **BEFORE** ``` $ git grep 'created_by = Some("Spark ' sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala: created_by = Some("Spark 4.0.0-SNAPSHOT"), sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala: created_by = Some("Spark 4.0.0-SNAPSHOT"), sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala: created_by = Some("Spark 4.0.0-SNAPSHOT"), sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala: created_by = Some("Spark 4.0.0-SNAPSHOT"), sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala: created_by = Some("Spark 4.0.0-SNAPSHOT"), ``` **AFTER** ``` $ git grep 'created_by = Some("Spark ' $ ``` ### Does this PR introduce _any_ user-facing change? No, this is a test-case fix. ### How was this patch tested? Pass the CIs and check manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49401 from dongjoon-hyun/SPARK-50541. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../sql/execution/command/v1/DescribeTableSuite.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index 4413087e886e8..d5dd96f55c11a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -22,6 +22,7 @@ import java.util.Locale import org.json4s._ import org.json4s.jackson.JsonMethods.parse +import org.apache.spark.SPARK_VERSION import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.execution.command @@ -346,7 +347,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { owner = Some(""), created_time = Some(""), last_access = Some("UNKNOWN"), - created_by = Some("Spark 4.0.0-SNAPSHOT"), + created_by = Some(s"Spark $SPARK_VERSION"), `type` = Some("MANAGED"), provider = Some("parquet"), bucket_columns = Some(List("employee_id")), @@ -413,7 +414,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { TableColumn("category", Type("string"), true) )), last_access = Some("UNKNOWN"), - created_by = Some("Spark 4.0.0-SNAPSHOT"), + created_by = Some(s"Spark $SPARK_VERSION"), `type` = Some("MANAGED"), provider = Some("parquet"), bucket_columns = Some(Nil), @@ -480,7 +481,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { TableColumn("is_active", Type("boolean"), default = Some("true")) )), last_access = Some("UNKNOWN"), - created_by = Some("Spark 4.0.0-SNAPSHOT"), + created_by = Some(s"Spark $SPARK_VERSION"), `type` = Some("MANAGED"), storage_properties = None, provider = Some("parquet"), @@ -572,7 +573,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { outputformat = Some("org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"), storage_properties = Some(Map("serialization.format" -> "1")), last_access = Some("UNKNOWN"), - created_by = Some("Spark 4.0.0-SNAPSHOT"), + created_by = Some(s"Spark $SPARK_VERSION"), `type` = Some("VIEW"), view_text = Some("SELECT * FROM spark_catalog.ns.table"), view_original_text = Some("SELECT * FROM spark_catalog.ns.table"), @@ -750,7 +751,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { "option2" -> "value2" )), last_access = Some("UNKNOWN"), - created_by = Some("Spark 4.0.0-SNAPSHOT"), + created_by = Some(s"Spark $SPARK_VERSION"), `type` = Some("MANAGED"), provider = Some("parquet"), comment = Some("A table with nested complex types"), From ab5461cf54b01d091e8e48aef1c08a2b77b05dea Mon Sep 17 00:00:00 2001 From: Michael Zhang Date: Tue, 7 Jan 2025 14:46:38 -0800 Subject: [PATCH 348/438] [SPARK-50076][FOLLOW-UP] Fix multiple log key names ### What changes were proposed in this pull request? Adds more logkeys to be more specific about values being logged. Also fixes incorrect logkey usages. ### Why are the changes needed? To make structured logging more useful. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests pass. Changes are logging only. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49387 from michaelzhan-db/SPARK-50076-pt2. Lead-authored-by: Michael Zhang Co-authored-by: Sample User Signed-off-by: Gengliang Wang --- .../org/apache/spark/internal/LogKey.scala | 8 +++++- .../shuffle/sort/ShuffleExternalSorter.java | 2 +- .../org/apache/spark/BarrierTaskContext.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 26 +++++++++---------- .../scheduler/OutputCommitCoordinator.scala | 4 +-- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../org/apache/spark/scheduler/TaskSet.scala | 4 +-- .../spark/scheduler/TaskSetManager.scala | 2 +- .../spark/util/collection/Spillable.scala | 2 +- .../adaptive/ShufflePartitionsUtil.scala | 2 +- .../v2/WriteToDataSourceV2Exec.scala | 11 +++++--- .../streaming/state/StateStore.scala | 9 ++++--- .../streaming/receiver/BlockGenerator.scala | 2 +- 13 files changed, 43 insertions(+), 33 deletions(-) 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 9e681448aaaf2..c3a1af68d1c82 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 @@ -94,6 +94,7 @@ private[spark] object LogKeys { case object BATCH_TIMESTAMP extends LogKey case object BATCH_WRITE extends LogKey case object BIND_ADDRESS extends LogKey + case object BLOCK_GENERATOR_STATUS extends LogKey case object BLOCK_ID extends LogKey case object BLOCK_IDS extends LogKey case object BLOCK_MANAGER_ID extends LogKey @@ -549,7 +550,7 @@ private[spark] object LogKeys { case object NUM_RULE_OF_RUNS extends LogKey case object NUM_SEQUENCES extends LogKey case object NUM_SLOTS extends LogKey - case object NUM_SPILL_INFOS extends LogKey + case object NUM_SPILLS extends LogKey case object NUM_SPILL_WRITERS extends LogKey case object NUM_SUB_DIRS extends LogKey case object NUM_SUCCESSFUL_TASKS extends LogKey @@ -725,6 +726,7 @@ private[spark] object LogKeys { case object SHUFFLE_DB_BACKEND_KEY extends LogKey case object SHUFFLE_DB_BACKEND_NAME extends LogKey case object SHUFFLE_ID extends LogKey + case object SHUFFLE_IDS extends LogKey case object SHUFFLE_MERGE_ID extends LogKey case object SHUFFLE_MERGE_RECOVERY_FILE extends LogKey case object SHUFFLE_SERVICE_CONF_OVERLAY_URL extends LogKey @@ -755,14 +757,18 @@ private[spark] object LogKeys { case object STAGE extends LogKey case object STAGES extends LogKey case object STAGE_ATTEMPT extends LogKey + case object STAGE_ATTEMPT_ID extends LogKey case object STAGE_ID extends LogKey case object STAGE_NAME extends LogKey case object START_INDEX extends LogKey case object START_TIME extends LogKey case object STATEMENT_ID extends LogKey case object STATE_NAME extends LogKey + case object STATE_STORE_COORDINATOR extends LogKey case object STATE_STORE_ID extends LogKey case object STATE_STORE_PROVIDER extends LogKey + case object STATE_STORE_PROVIDER_ID extends LogKey + case object STATE_STORE_PROVIDER_IDS extends LogKey case object STATE_STORE_VERSION extends LogKey case object STATS extends LogKey case object STATUS extends LogKey diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index f96513f1b1097..de3c41a4b526b 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -165,7 +165,7 @@ private void writeSortedFile(boolean isFinalFile) { MDC.of(LogKeys.TASK_ATTEMPT_ID$.MODULE$, taskContext.taskAttemptId()), MDC.of(LogKeys.THREAD_ID$.MODULE$, Thread.currentThread().getId()), MDC.of(LogKeys.MEMORY_SIZE$.MODULE$, Utils.bytesToString(getMemoryUsage())), - MDC.of(LogKeys.NUM_SPILL_INFOS$.MODULE$, spills.size()), + MDC.of(LogKeys.NUM_SPILLS$.MODULE$, spills.size()), MDC.of(LogKeys.SPILL_TIMES$.MODULE$, spills.size() != 1 ? "times" : "time")); } diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index c8d6000cd6282..5b18ab95b07eb 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -62,7 +62,7 @@ class BarrierTaskContext private[spark] ( log"for ${MDC(TOTAL_TIME, System.currentTimeMillis() - st)} ms,") logInfo(log"Task ${MDC(TASK_ATTEMPT_ID, taskAttemptId())}" + log" from Stage ${MDC(STAGE_ID, stageId())}" + - log"(Attempt ${MDC(STAGE_ATTEMPT, stageAttemptNumber())}) " + + log"(Attempt ${MDC(STAGE_ATTEMPT_ID, stageAttemptNumber())}) " + msg + waitMsg + log" current barrier epoch is ${MDC(BARRIER_EPOCH, barrierEpoch)}.") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e06b7d86e1db0..aee92ba928b4a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1382,9 +1382,9 @@ private[spark] class DAGScheduler( logInfo( log"Got job ${MDC(JOB_ID, job.jobId)} (${MDC(CALL_SITE_SHORT_FORM, callSite.shortForm)}) " + log"with ${MDC(NUM_PARTITIONS, partitions.length)} output partitions") - logInfo(log"Final stage: ${MDC(STAGE_ID, finalStage)} " + + logInfo(log"Final stage: ${MDC(STAGE, finalStage)} " + log"(${MDC(STAGE_NAME, finalStage.name)})") - logInfo(log"Parents of final stage: ${MDC(STAGE_ID, finalStage.parents)}") + logInfo(log"Parents of final stage: ${MDC(STAGES, finalStage.parents)}") logInfo(log"Missing parents: ${MDC(MISSING_PARENT_STAGES, getMissingParentStages(finalStage))}") val jobSubmissionTime = clock.getTimeMillis() @@ -1465,7 +1465,7 @@ private[spark] class DAGScheduler( val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) if (missing.isEmpty) { - logInfo(log"Submitting ${MDC(STAGE_ID, stage)} (${MDC(RDD_ID, stage.rdd)}), " + + logInfo(log"Submitting ${MDC(STAGE, stage)} (${MDC(RDD_ID, stage.rdd)}), " + log"which has no missing parents") submitMissingTasks(stage, jobId.get) } else { @@ -1517,12 +1517,12 @@ private[spark] class DAGScheduler( val shuffleId = stage.shuffleDep.shuffleId val shuffleMergeId = stage.shuffleDep.shuffleMergeId if (stage.shuffleDep.shuffleMergeEnabled) { - logInfo(log"Shuffle merge enabled before starting the stage for ${MDC(STAGE_ID, stage)}" + + logInfo(log"Shuffle merge enabled before starting the stage for ${MDC(STAGE, stage)}" + log" with shuffle ${MDC(SHUFFLE_ID, shuffleId)} and shuffle merge" + log" ${MDC(SHUFFLE_MERGE_ID, shuffleMergeId)} with" + log" ${MDC(NUM_MERGER_LOCATIONS, stage.shuffleDep.getMergerLocs.size.toString)} merger locations") } else { - logInfo(log"Shuffle merge disabled for ${MDC(STAGE_ID, stage)} with " + + logInfo(log"Shuffle merge disabled for ${MDC(STAGE, stage)} with " + log"shuffle ${MDC(SHUFFLE_ID, shuffleId)} and " + log"shuffle merge ${MDC(SHUFFLE_MERGE_ID, shuffleMergeId)}, " + log"but can get enabled later adaptively once enough " + @@ -1583,7 +1583,7 @@ private[spark] class DAGScheduler( // merger locations but the corresponding shuffle map stage did not complete // successfully, we would still enable push for its retry. s.shuffleDep.setShuffleMergeAllowed(false) - logInfo(log"Push-based shuffle disabled for ${MDC(STAGE_ID, stage)} " + + logInfo(log"Push-based shuffle disabled for ${MDC(STAGE, stage)} " + log"(${MDC(STAGE_NAME, stage.name)}) since it is already shuffle merge finalized") } } @@ -1707,7 +1707,7 @@ private[spark] class DAGScheduler( if (tasks.nonEmpty) { logInfo(log"Submitting ${MDC(NUM_TASKS, tasks.size)} missing tasks from " + - log"${MDC(STAGE_ID, stage)} (${MDC(RDD_ID, stage.rdd)}) (first 15 tasks are " + + log"${MDC(STAGE, stage)} (${MDC(RDD_ID, stage.rdd)}) (first 15 tasks are " + log"for partitions ${MDC(PARTITION_IDS, tasks.take(15).map(_.partitionId))})") val shuffleId = stage match { case s: ShuffleMapStage => Some(s.shuffleDep.shuffleId) @@ -1964,7 +1964,7 @@ private[spark] class DAGScheduler( } catch { case e: UnsupportedOperationException => logWarning(log"Could not cancel tasks " + - log"for stage ${MDC(STAGE_ID, stageId)}", e) + log"for stage ${MDC(STAGE, stageId)}", e) } listenerBus.post( SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobSucceeded)) @@ -1996,7 +1996,7 @@ private[spark] class DAGScheduler( logDebug("ShuffleMapTask finished on " + execId) if (executorFailureEpoch.contains(execId) && smt.epoch <= executorFailureEpoch(execId)) { - logInfo(log"Ignoring possibly bogus ${MDC(STAGE_ID, smt)} completion from " + + logInfo(log"Ignoring possibly bogus ${MDC(STAGE, smt)} completion from " + log"executor ${MDC(EXECUTOR_ID, execId)}") } else { // The epoch of the task is acceptable (i.e., the task was launched after the most @@ -2026,8 +2026,8 @@ private[spark] class DAGScheduler( if (failedStage.latestInfo.attemptNumber() != task.stageAttemptId) { logInfo(log"Ignoring fetch failure from " + log"${MDC(TASK_ID, task)} as it's from " + - log"${MDC(STAGE_ID, failedStage)} attempt " + - log"${MDC(STAGE_ATTEMPT, task.stageAttemptId)} and there is a more recent attempt for " + + log"${MDC(FAILED_STAGE, failedStage)} attempt " + + log"${MDC(STAGE_ATTEMPT_ID, task.stageAttemptId)} and there is a more recent attempt for " + log"that stage (attempt " + log"${MDC(NUM_ATTEMPT, failedStage.latestInfo.attemptNumber())}) running") } else { @@ -2035,8 +2035,8 @@ private[spark] class DAGScheduler( isExecutorDecommissioningOrDecommissioned(taskScheduler, bmAddress) if (ignoreStageFailure) { logInfo(log"Ignoring fetch failure from ${MDC(TASK_NAME, task)} of " + - log"${MDC(STAGE, failedStage)} attempt " + - log"${MDC(STAGE_ATTEMPT, task.stageAttemptId)} when count " + + log"${MDC(FAILED_STAGE, failedStage)} attempt " + + log"${MDC(STAGE_ATTEMPT_ID, task.stageAttemptId)} when count " + log"${MDC(MAX_ATTEMPTS, config.STAGE_MAX_CONSECUTIVE_ATTEMPTS.key)} " + log"as executor ${MDC(EXECUTOR_ID, bmAddress.executorId)} is decommissioned and " + log"${MDC(CONFIG, config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE.key)}=true") diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index df28a97a349ea..a769c3fa14b62 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -149,7 +149,7 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) // The task output has been committed successfully case _: TaskCommitDenied => logInfo(log"Task was denied committing, stage: ${MDC(LogKeys.STAGE_ID, stage)}." + - log"${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)}, " + + log"${MDC(LogKeys.STAGE_ATTEMPT_ID, stageAttempt)}, " + log"partition: ${MDC(LogKeys.PARTITION_ID, partition)}, " + log"attempt: ${MDC(LogKeys.NUM_ATTEMPT, attemptNumber)}") case _ => @@ -181,7 +181,7 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) stageStates.get(stage) match { case Some(state) if attemptFailed(state, stageAttempt, partition, attemptNumber) => logInfo(log"Commit denied for stage=${MDC(LogKeys.STAGE_ID, stage)}." + - log"${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)}, partition=" + + log"${MDC(LogKeys.STAGE_ATTEMPT_ID, stageAttempt)}, partition=" + log"${MDC(LogKeys.PARTITION_ID, partition)}: task attempt " + log"${MDC(LogKeys.NUM_ATTEMPT, attemptNumber)} already marked as failed.") false diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 8e3cb1379339d..43193dc5366a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -325,7 +325,7 @@ private[spark] class TaskSchedulerImpl( } tsm.suspend() logInfo(log"Stage ${MDC(LogKeys.STAGE_ID, stageId)}." + - log"${MDC(LogKeys.STAGE_ATTEMPT, tsm.taskSet.stageAttemptId)} was cancelled") + log"${MDC(LogKeys.STAGE_ATTEMPT_ID, tsm.taskSet.stageAttemptId)} was cancelled") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 2474a1342eb2e..3513cb1f93764 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.internal.LogKeys.{STAGE_ATTEMPT, STAGE_ID} +import org.apache.spark.internal.LogKeys.{STAGE_ATTEMPT_ID, STAGE_ID} import org.apache.spark.internal.MessageWithContext /** @@ -42,7 +42,7 @@ private[spark] class TaskSet( lazy val logId: MessageWithContext = { val hashMap = new java.util.HashMap[String, String]() hashMap.put(STAGE_ID.name, stageId.toString) - hashMap.put(STAGE_ATTEMPT.name, stageAttemptId.toString) + hashMap.put(STAGE_ATTEMPT_ID.name, stageAttemptId.toString) MessageWithContext(id, hashMap) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index fdc82285b76bb..0eaf138d3eb8d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -1001,7 +1001,7 @@ private[spark] class TaskSetManager( logError( log"Task ${MDC(TASK_INDEX, info.index)}.${MDC(TASK_ATTEMPT_ID, info.attemptNumber)} " + log"in stage ${MDC(STAGE_ID, taskSet.stageId)}." + - log"${MDC(STAGE_ATTEMPT, taskSet.stageAttemptId)} (TID ${MDC(TASK_ID, tid)}) " + + log"${MDC(STAGE_ATTEMPT_ID, taskSet.stageAttemptId)} (TID ${MDC(TASK_ID, tid)}) " + log"can not write to output file: ${MDC(ERROR, ef.description)}; not retrying") emptyTaskInfoAccumulablesAndNotifyDagScheduler(tid, tasks(index), reason, null, accumUpdates, metricPeaks) diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index c3d648dccea73..7f2a1a8419a71 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -146,6 +146,6 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) logInfo(log"Thread ${MDC(LogKeys.THREAD_ID, threadId)} " + log"spilling in-memory map of ${MDC(LogKeys.BYTE_SIZE, org.apache.spark.util.Utils.bytesToString(size))} to disk " + - log"(${MDC(LogKeys.SPILL_TIMES, _spillCount)} times so far)") + log"(${MDC(LogKeys.NUM_SPILLS, _spillCount)} times so far)") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala index bb7d904402ded..1ea4df0254673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala @@ -61,7 +61,7 @@ object ShufflePartitionsUtil extends Logging { val targetSize = maxTargetSize.min(advisoryTargetSize).max(minPartitionSize) val shuffleIds = mapOutputStatistics.flatMap(_.map(_.shuffleId)).mkString(", ") - logInfo(log"For shuffle(${MDC(LogKeys.SHUFFLE_ID, shuffleIds)}, advisory target size: " + + logInfo(log"For shuffle(${MDC(LogKeys.SHUFFLE_IDS, shuffleIds)}, advisory target size: " + log"${MDC(LogKeys.ADVISORY_TARGET_SIZE, advisoryTargetSize)}, actual target size " + log"${MDC(LogKeys.TARGET_SIZE, targetSize)}, minimum partition size: " + log"${MDC(LogKeys.PARTITION_SIZE, minPartitionSize)}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index bdcf7b8260a7a..308b1bceca12a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -480,7 +480,7 @@ trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serial log"(task ${MDC(LogKeys.TASK_ID, taskId)}, " + log"attempt ${MDC(LogKeys.TASK_ATTEMPT_ID, attemptId)}, " + log"stage ${MDC(LogKeys.STAGE_ID, stageId)}." + - log"${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)})") + log"${MDC(LogKeys.STAGE_ATTEMPT_ID, stageAttempt)})") dataWriter.commit() } else { @@ -500,7 +500,8 @@ trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serial logInfo(log"Committed partition ${MDC(LogKeys.PARTITION_ID, partId)} " + log"(task ${MDC(LogKeys.TASK_ID, taskId)}, " + log"attempt ${MDC(LogKeys.TASK_ATTEMPT_ID, attemptId)}, " + - log"stage ${MDC(LogKeys.STAGE_ID, stageId)}.${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)})") + log"stage ${MDC(LogKeys.STAGE_ID, stageId)}." + + log"${MDC(LogKeys.STAGE_ATTEMPT_ID, stageAttempt)})") DataWritingSparkTaskResult(iterWithMetrics.count, msg) @@ -509,12 +510,14 @@ trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serial logError(log"Aborting commit for partition ${MDC(LogKeys.PARTITION_ID, partId)} " + log"(task ${MDC(LogKeys.TASK_ID, taskId)}, " + log"attempt ${MDC(LogKeys.TASK_ATTEMPT_ID, attemptId)}, " + - log"stage ${MDC(LogKeys.STAGE_ID, stageId)}.${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)})") + log"stage ${MDC(LogKeys.STAGE_ID, stageId)}." + + log"${MDC(LogKeys.STAGE_ATTEMPT_ID, stageAttempt)})") dataWriter.abort() logError(log"Aborted commit for partition ${MDC(LogKeys.PARTITION_ID, partId)} " + log"(task ${MDC(LogKeys.TASK_ID, taskId)}, " + log"attempt ${MDC(LogKeys.TASK_ATTEMPT_ID, attemptId)}, " + - log"stage ${MDC(LogKeys.STAGE_ID, stageId)}.${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)})") + log"stage ${MDC(LogKeys.STAGE_ID, stageId)}." + + log"${MDC(LogKeys.STAGE_ATTEMPT_ID, stageAttempt)})") }, finallyBlock = { dataWriter.close() }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 0f945af6ede9b..33df8ad42747c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -977,7 +977,8 @@ object StateStore extends Logging { } finally { val duration = System.currentTimeMillis() - startTime val logMsg = - log"Finished maintenance task for provider=${MDC(LogKeys.STATE_STORE_PROVIDER, id)}" + + log"Finished maintenance task for " + + log"provider=${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}" + log" in elapsed_time=${MDC(LogKeys.TIME_UNITS, duration)}\n" if (duration > 5000) { logInfo(logMsg) @@ -1007,9 +1008,9 @@ object StateStore extends Logging { .map(_.reportActiveInstance(storeProviderId, host, executorId, otherProviderIds)) .getOrElse(Seq.empty[StateStoreProviderId]) logInfo(log"Reported that the loaded instance " + - log"${MDC(LogKeys.STATE_STORE_PROVIDER, storeProviderId)} is active") + log"${MDC(LogKeys.STATE_STORE_PROVIDER_ID, storeProviderId)} is active") logDebug(log"The loaded instances are going to unload: " + - log"${MDC(LogKeys.STATE_STORE_PROVIDER, providerIdsToUnload.mkString(", "))}") + log"${MDC(LogKeys.STATE_STORE_PROVIDER_IDS, providerIdsToUnload)}") providerIdsToUnload } else { Seq.empty[StateStoreProviderId] @@ -1041,7 +1042,7 @@ object StateStore extends Logging { _coordRef = StateStoreCoordinatorRef.forExecutor(env) } logInfo(log"Retrieved reference to StateStoreCoordinator: " + - log"${MDC(LogKeys.STATE_STORE_PROVIDER, _coordRef)}") + log"${MDC(LogKeys.STATE_STORE_COORDINATOR, _coordRef)}") Some(_coordRef) } else { _coordRef = null diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index e0e85712a2301..fae68123773dd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -142,7 +142,7 @@ private[streaming] class BlockGenerator( state = StoppedAddingData } else { logWarning(log"Cannot stop BlockGenerator as its not in the Active state " + - log"[state = ${MDC(STATUS, state)}]") + log"[state = ${MDC(BLOCK_GENERATOR_STATUS, state)}]") return } } From cfb2e4054f77c80afe197ab85c51e12e79d5f821 Mon Sep 17 00:00:00 2001 From: Bo Zhang Date: Wed, 8 Jan 2025 09:03:11 +0900 Subject: [PATCH 349/438] [SPARK-50756][SQL] Use error class for exceptions in SparkConf.validateSettings ### What changes were proposed in this pull request? This change is to use error class for exceptions thrown in SparkConf.validateSettings. ### Why are the changes needed? To adapt to the error class framework. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49399 from bozhang2820/spark-50756. Authored-by: Bo Zhang Signed-off-by: Hyukjin Kwon --- .../main/resources/error/error-conditions.json | 12 ++++++++++++ .../scala/org/apache/spark/SparkConf.scala | 18 ++++++++++++------ 2 files changed, 24 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 e47387d59fe15..297e0ec1d3679 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2393,6 +2393,12 @@ ], "sqlState" : "42604" }, + "INVALID_EXECUTOR_HEARTBEAT_INTERVAL" : { + "message" : [ + "The value of =ms must be greater than the value of =ms." + ], + "sqlState" : "42616" + }, "INVALID_EXECUTOR_MEMORY" : { "message" : [ "Executor memory must be at least .", @@ -3693,6 +3699,12 @@ ], "sqlState" : "07501" }, + "NETWORK_AUTH_MUST_BE_ENABLED" : { + "message" : [ + " must be enabled when enabling encryption." + ], + "sqlState" : "42616" + }, "NONEXISTENT_FIELD_NAME_IN_LIST" : { "message" : [ "Field(s) do(es) not exist. Available fields: " diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index ae6ef1ee55608..aab636e8d8125 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -607,17 +607,23 @@ class SparkConf(loadDefaults: Boolean) } val encryptionEnabled = get(NETWORK_CRYPTO_ENABLED) || get(SASL_ENCRYPTION_ENABLED) - require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), - s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") + SparkException.require( + !encryptionEnabled || get(NETWORK_AUTH_ENABLED), + "NETWORK_AUTH_MUST_BE_ENABLED", + Map("networkAuthEnabledConf" -> NETWORK_AUTH_ENABLED.key)) val executorTimeoutThresholdMs = get(NETWORK_TIMEOUT) * 1000 val executorHeartbeatIntervalMs = get(EXECUTOR_HEARTBEAT_INTERVAL) - val networkTimeout = NETWORK_TIMEOUT.key // If spark.executor.heartbeatInterval bigger than spark.network.timeout, // it will almost always cause ExecutorLostFailure. See SPARK-22754. - require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The value of " + - s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be greater than the value of " + - s"${EXECUTOR_HEARTBEAT_INTERVAL.key}=${executorHeartbeatIntervalMs}ms.") + SparkException.require( + executorTimeoutThresholdMs > executorHeartbeatIntervalMs, + "INVALID_EXECUTOR_HEARTBEAT_INTERVAL", + Map( + "networkTimeoutKey" -> NETWORK_TIMEOUT.key, + "networkTimeoutValue" -> executorTimeoutThresholdMs.toString, + "executorHeartbeatIntervalKey" -> EXECUTOR_HEARTBEAT_INTERVAL.key, + "executorHeartbeatIntervalValue" -> executorHeartbeatIntervalMs.toString)) } /** From 0b443f4a5041943b25438822d3c1a76b3236e5cb Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 7 Jan 2025 16:30:09 -0800 Subject: [PATCH 350/438] [SPARK-50755][SQL] Pretty plan display for InsertIntoHiveTable ### What changes were proposed in this pull request? Add `toString` for `HiveFileFormat` and `HiveTempPath` to make the display of `InsertIntoHiveTable` plan pretty. ### Why are the changes needed? I found the current plan replacing rules does not handle tailing object hash properly https://github.com/apache/spark/blob/36d23eff4b4c3a2b8fd301672e532132c96fdd68/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala#L62 instead of fixing the replacing rule(see #49396, and please let me know if any reviewer think we should fix that too), seems we can override the `toString` of those classes to make it display pretty. Minor improvements of plan display for `InsertIntoHiveTable`, and make it consistent with `DataSource` plan like `InsertIntoHadoopFsRelationCommand` `InsertIntoHadoopFsRelationCommand`: ``` -- !query insert into t6 values (97) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t6, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t6], Append, `spark_catalog`.`default`.`t6`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t6), [ascii] +- Project [cast(col1#x as bigint) AS ascii#xL] +- LocalRelation [col1#x] ``` `InsertIntoHiveTable`: ```patch -- !query insert into table spark_test_json_2021_07_16_01 values(1, 'a') -- !query analysis -InsertIntoHiveTable `spark_catalog`.`default`.`spark_test_json_2021_07_16_01`, false, false, [c1, c2], org.apache.spark.sql.hive.execution.HiveFileFormatxxxxxxxx, org.apache.spark.sql.hive.execution.HiveTempPath69beda67 +InsertIntoHiveTable `spark_catalog`.`default`.`spark_test_json_2021_07_16_01`, false, false, [c1, c2], Hive, HiveTempPath(file:[not included in comparison]/{warehouse_dir}/spark_test_json_2021_07_16_01) +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as string) AS c2#x] +- LocalRelation [col1#x, col2#x] ``` ### Does this PR introduce _any_ user-facing change? It affects the `EXPLAIN` outputs and Spark UI `SQL/DataFrame` tab plan display ### How was this patch tested? See the above examples. Spark does not have SQL tests related to the `hive` module, I identified this issue when porting internal test cases to the 4.0. Since all existing SQL tests live on the `sql` module, adding hive-related tests is impossible. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49400 from pan3793/SPARK-50755. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/hive/execution/HiveFileFormat.scala | 2 ++ .../org/apache/spark/sql/hive/execution/HiveTempPath.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index cabdddd4c475d..0d4efd9e77742 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -55,6 +55,8 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) override def shortName(): String = "hive" + override def toString: String = "Hive" + override def inferSchema( sparkSession: SparkSession, options: Map[String, String], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala index 16edfea67e38e..d97d3cd6dd4a9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala @@ -165,4 +165,6 @@ class HiveTempPath(session: SparkSession, val hadoopConf: Configuration, path: P def deleteIfNotStagingDir(path: Path, fs: FileSystem): Unit = { if (Option(path) != stagingDirForCreating) fs.delete(path, true) } + + override def toString: String = s"HiveTempPath($path)" } From 96da1abc0f3a89470358445b2fa89f6ab9726983 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 7 Jan 2025 16:43:33 -0800 Subject: [PATCH 351/438] [MINOR] Remove TODO comment about `ExecutorAdded` message ### What changes were proposed in this pull request? This PR aims to remove `TODO` comment about `ExecutorAdded` message. ``` // TODO(matei): replace hostPort with host ``` ### Why are the changes needed? `ExecutorAdded` message has been stable over 12 years with no change. It's unlikely for us to change this in Spark 4 or in the future. Thus, we had better remove this TODO message from Spark 4. ### Does this PR introduce _any_ user-facing change? No, this is a comment removal. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49402 from dongjoon-hyun/todo. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index cb5996a5097d2..12e031711aa2a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -205,7 +205,6 @@ private[deploy] object DeployMessages { case class RegisteredApplication(appId: String, master: RpcEndpointRef) extends DeployMessage - // TODO(matei): replace hostPort with host case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { Utils.checkHostPort(hostPort) } From 8bd7789872b42c91fe9b3bbd73cc44fca865cf5c Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Wed, 8 Jan 2025 09:12:20 +0800 Subject: [PATCH 352/438] [SPARK-50722][SQL] Detect self-contained WITH nodes ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/48284 to make the CTE inline decision smarter but without introducing overly complex linage tracking. This PR is an alternative fix to https://github.com/apache/spark/pull/49352 with tracking CTE references a different way. First of all, this PR reverts https://github.com/apache/spark/pull/48284 and keeps tracking "contains" relation in `outgoingRefs`. An example for this relation is "Case 2" in https://github.com/apache/spark/pull/49352 in which `t2` contains `t3`. But while https://github.com/apache/spark/pull/48284 strores the reference from `t2` -> `t3` in `t2`'s `indirectOutgoingRefSources`, this PR reverts to store the reference in `t2`'s `outgoingRefs` similar to other usual references. Obviously, the problem with this revert is that queries with duplicate CTE definitions can fail, so let's revisit the problematic query from https://github.com/apache/spark/pull/48284: ``` WithCTE CTEDef r0 ... CTEDef r1 View v WithCTE CTEDef t1 OneRowRelation CTEDef t2 CTERef t1 CTERef t2 // main query of the inner WithCTE CTEDef r2 View v // exactly the same as the view v above WithCTE CTEDef t1 OneRowRelation CTEDef t2 CTERef t1 CTERef t2 CTERef r2 // main query of the outer WithCTE ``` The source of the problem is view `v`, which duplicates a `WithCTE` node and introduces conflicting CTE definitions. In the above example, tracking both the `r1` -> `t2` and the `r2` -> `t2` "contains" relations in `r1`'s and `r2`'s `outgoingRefs`s cause reference counting problems, as those `t2`s are actually diffrent instances and so they can have different inlining depending on how many times `r1` and `r2` are referenced. Fortunately, this duplicate issue can only happen when the conflicting `WithCTE` nodes are self-contained (as those nodes can't contain any references to any outer definitions like `r0`). The main advantage of self-contained `WithCTE` nodes is that we need don't need to track `r1` -> `t2` and `r2` -> `t2` references, as the inlining of `t2` (and `t1`) inside the `WithCTE` node don't depend on references coming from outer (`r1`, `r2`) definition. (Please note that what depends on `r1` and `r2` is the inlined result of the whole inner `WithCTE` node.) So to recap: - we do need to track references from "contains" relation when the inner `WithCTE` is not self-contained - we don't need to (and actually it causes issues if we do) when the inner `WithCTE` is self-contained and the node appears multiple times in the query - if the inner `WithCTE` is self-contained, but it appears only once in the query, then the both ways are fine This PR suggests the following: - As during the traversal of `buildCTEMap()` we don't know in advance if a `WithCTE` node is self-contained or not, let's: - track the references from "contains" relations - and also record the container of a definition. E.g. when we encounter `t2` in `r1`, then record `t2` in `r1`'s `outgoingRefs` and record `r1` as the container of `t2`. - If we encounter a duplicate definition, then we know it is self-contained so we: - don't need to visit it again - and we can also remove the reference from its container to the definition. E.g. when we encounter `t2` in `r2`, then clear `t2` from `r1`'s `outgoingRefs` (we know that `r1` was `t2`'s container). ### Why are the changes needed? Make sure we can inline CTE when we should. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests from https://github.com/apache/spark/pull/49352. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49379 from peter-toth/SPARK-50722-cte-fix. Lead-authored-by: Peter Toth Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/InlineCTE.scala | 74 +++++++++------- .../org/apache/spark/sql/CTEInlineSuite.scala | 85 ++++++++++++++++++- 2 files changed, 127 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala index b3384c4e29566..ad1a1a99b8257 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala @@ -71,53 +71,64 @@ case class InlineCTE( * @param plan The plan to collect the CTEs from * @param cteMap A mutable map that accumulates the CTEs and their reference information by CTE * ids. - * @param collectCTERefs A function to collect CTE references so that the caller side can do some - * bookkeeping work. + * @param outerCTEId While collecting the map we use this optional CTE id to identify the + * current outer CTE. */ private def buildCTEMap( plan: LogicalPlan, cteMap: mutable.Map[Long, CTEReferenceInfo], - collectCTERefs: CTERelationRef => Unit = _ => ()): Unit = { + outerCTEId: Option[Long] = None): Unit = { plan match { case WithCTE(child, cteDefs) => - cteDefs.foreach { cteDef => - cteMap(cteDef.id) = CTEReferenceInfo( - cteDef = cteDef, - refCount = 0, - outgoingRefs = mutable.Map.empty.withDefaultValue(0), - shouldInline = true - ) - } - cteDefs.foreach { cteDef => - buildCTEMap(cteDef, cteMap, ref => { - // A CTE relation can references CTE relations defined before it in the same `WithCTE`. - // Here we update the out-going-ref-count for it, in case this CTE relation is not - // referenced at all and can be optimized out, and we need to decrease the ref counts - // for CTE relations that are referenced by it. - if (cteDefs.exists(_.id == ref.cteId)) { - cteMap(cteDef.id).increaseOutgoingRefCount(ref.cteId, 1) - } - // Similarly, a CTE relation can reference CTE relations defined in the outer `WithCTE`. - // Here we call the `collectCTERefs` function so that the outer CTE can also update the - // out-going-ref-count if needed. - collectCTERefs(ref) - }) + val isDuplicated = cteDefs.forall(cteDef => cteMap.contains(cteDef.id)) + if (isDuplicated) { + // If we have seen this `WithCTE` node then it must be self-contained so we can clear + // the references from containers to the definitions, and we don't need to process it + // again + + cteDefs.foreach { cteDef => + cteMap(cteDef.id).container.foreach(c => cteMap(c).outgoingRefs -= cteDef.id) + } + } else { + cteDefs.foreach { cteDef => + cteMap(cteDef.id) = CTEReferenceInfo( + cteDef = cteDef, + refCount = 0, + outgoingRefs = mutable.Map.empty.withDefaultValue(0), + shouldInline = true, + container = outerCTEId + ) + } + + cteDefs.foreach { cteDef => + buildCTEMap(cteDef, cteMap, Some(cteDef.id)) + } + buildCTEMap(child, cteMap, outerCTEId) } - buildCTEMap(child, cteMap, collectCTERefs) case ref: CTERelationRef => cteMap(ref.cteId) = cteMap(ref.cteId).withRefCountIncreased(1) - collectCTERefs(ref) + + // The `outerCTEId` CTE definition can either reference `cteId` definition if `cteId` is in + // the same or in an outer `WithCTE` node, or `outerCTEId` can contain `cteId` definition if + // `cteId` is an inner `WithCTE` node inside `outerCTEId`. + // In both cases we can track the relations in `outgoingRefs` when we see a definition the + // first time. But if we encounter a conflicting duplicated contains relation later, then we + // will remove the references of the first contains relation. + outerCTEId.foreach { cteId => + cteMap(cteId).increaseOutgoingRefCount(ref.cteId, 1) + } + case _ => if (plan.containsPattern(CTE)) { plan.children.foreach { child => - buildCTEMap(child, cteMap, collectCTERefs) + buildCTEMap(child, cteMap, outerCTEId) } plan.expressions.foreach { expr => if (expr.containsAllPatterns(PLAN_EXPRESSION, CTE)) { expr.foreach { - case e: SubqueryExpression => buildCTEMap(e.plan, cteMap, collectCTERefs) + case e: SubqueryExpression => buildCTEMap(e.plan, cteMap, outerCTEId) case _ => } } @@ -225,12 +236,15 @@ case class InlineCTE( * from other CTE relations and regular places. * @param outgoingRefs A mutable map that tracks outgoing reference counts to other CTE relations. * @param shouldInline If true, this CTE relation should be inlined in the places that reference it. + * @param container The container of a CTE definition is another CTE definition in which the + * `WithCTE` node of the definition resides. */ case class CTEReferenceInfo( cteDef: CTERelationDef, refCount: Int, outgoingRefs: mutable.Map[Long, Int], - shouldInline: Boolean) { + shouldInline: Boolean, + container: Option[Long]) { def withRefCountIncreased(count: Int): CTEReferenceInfo = { copy(refCount = refCount + count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala index f22d90d9f35d7..e8b9ffe284940 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.{And, GreaterThan, LessThan, Literal, Or} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, GreaterThan, LessThan, Literal, Or, Rand} +import org.apache.spark.sql.catalyst.optimizer.InlineCTE import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.adaptive._ import org.apache.spark.sql.execution.exchange.ReusedExchangeExec @@ -715,7 +716,7 @@ abstract class CTEInlineSuiteBase checkAnswer(df, Row(1)) } - test("SPARK-49816: should only update out-going-ref-count for referenced outer CTE relation") { + test("SPARK-49816: detect self-contained WithCTE nodes") { withView("v") { sql( """ @@ -735,6 +736,86 @@ abstract class CTEInlineSuiteBase checkAnswer(df, Row(1)) } } + + test("SPARK-49816: complicated reference count") { + // Manually build the logical plan for + // WITH + // r1 AS (SELECT random()), + // r2 AS ( + // WITH + // t1 AS (SELECT * FROM r1), + // t2 AS (SELECT * FROM r1) + // SELECT * FROM t2 + // ) + // SELECT * FROM r2 + // r1 should be inlined as it's only referenced once: main query -> r2 -> t2 -> r1 + val r1 = CTERelationDef(Project(Seq(Alias(Rand(Literal(0)), "r")()), OneRowRelation())) + val r1Ref = CTERelationRef(r1.id, r1.resolved, r1.output, r1.isStreaming) + val t1 = CTERelationDef(Project(r1.output, r1Ref)) + val t2 = CTERelationDef(Project(r1.output, r1Ref)) + val t2Ref = CTERelationRef(t2.id, t2.resolved, t2.output, t2.isStreaming) + val r2 = CTERelationDef(WithCTE(Project(t2.output, t2Ref), Seq(t1, t2))) + val r2Ref = CTERelationRef(r2.id, r2.resolved, r2.output, r2.isStreaming) + val query = WithCTE(Project(r2.output, r2Ref), Seq(r1, r2)) + val inlined = InlineCTE().apply(query) + assert(!inlined.exists(_.isInstanceOf[WithCTE])) + } + + test("SPARK-49816: complicated reference count 2") { + // Manually build the logical plan for + // WITH + // r1 AS (SELECT random()), + // r2 AS ( + // WITH + // t1 AS (SELECT * FROM r1), + // t2 AS (SELECT * FROM t1) + // SELECT * FROM t2 + // ) + // SELECT * FROM r1 + // This is similar to the previous test case, but t2 reference t1 instead of r1, and the main + // query references r1. r1 should be inlined as r2 is not referenced at all. + val r1 = CTERelationDef(Project(Seq(Alias(Rand(Literal(0)), "r")()), OneRowRelation())) + val r1Ref = CTERelationRef(r1.id, r1.resolved, r1.output, r1.isStreaming) + val t1 = CTERelationDef(Project(r1.output, r1Ref)) + val t1Ref = CTERelationRef(t1.id, t1.resolved, t1.output, t1.isStreaming) + val t2 = CTERelationDef(Project(t1.output, t1Ref)) + val t2Ref = CTERelationRef(t2.id, t2.resolved, t2.output, t2.isStreaming) + val r2 = CTERelationDef(WithCTE(Project(t2.output, t2Ref), Seq(t1, t2))) + val query = WithCTE(Project(r1.output, r1Ref), Seq(r1, r2)) + val inlined = InlineCTE().apply(query) + assert(!inlined.exists(_.isInstanceOf[WithCTE])) + } + + test("SPARK-49816: complicated reference count 3") { + // Manually build the logical plan for + // WITH + // r1 AS ( + // WITH + // t1 AS (SELECT random()), + // t2 AS (SELECT * FROM t1) + // SELECT * FROM t2 + // ), + // r2 AS ( + // WITH + // t1 AS (SELECT random()), + // t2 AS (SELECT * FROM r1) + // SELECT * FROM t2 + // ) + // SELECT * FROM r1 UNION ALL SELECT * FROM r2 + // The inner WITH in r1 and r2 should become `SELECT random()` and r1/r2 should be inlined. + val t1 = CTERelationDef(Project(Seq(Alias(Rand(Literal(0)), "r")()), OneRowRelation())) + val t1Ref = CTERelationRef(t1.id, t1.resolved, t1.output, t1.isStreaming) + val t2 = CTERelationDef(Project(t1.output, t1Ref)) + val t2Ref = CTERelationRef(t2.id, t2.resolved, t2.output, t2.isStreaming) + val cte = WithCTE(Project(t2.output, t2Ref), Seq(t1, t2)) + val r1 = CTERelationDef(cte) + val r1Ref = CTERelationRef(r1.id, r1.resolved, r1.output, r1.isStreaming) + val r2 = CTERelationDef(cte) + val r2Ref = CTERelationRef(r2.id, r2.resolved, r2.output, r2.isStreaming) + val query = WithCTE(Union(r1Ref, r2Ref), Seq(r1, r2)) + val inlined = InlineCTE().apply(query) + assert(!inlined.exists(_.isInstanceOf[WithCTE])) + } } class CTEInlineSuiteAEOff extends CTEInlineSuiteBase with DisableAdaptiveExecutionSuite From c5a9bc79071d03d81047866f43f19f17fb5b5a1f Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 8 Jan 2025 13:31:40 +0900 Subject: [PATCH 353/438] [SPARK-50761][INFRA] Make `pyspark-pandas` can be tested alone ### What changes were proposed in this pull request? Make pyspark-pandas module no longer depend on pyspark module ### Why are the changes needed? in existing framework, `pyspark-pandas` depends on `pyspark`, so: 1, a PS-only PR will trigger the test of the whole pyspark (both PS and non-PS); 2, in daily test `Build / Python-only (master, Python PS with old dependencies)` cannot only test the PS part ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? PR builder with ``` default: '{"pyspark-pandas": "true"}' ``` https://github.com/zhengruifeng/spark/actions/runs/12651324677/job/35253582924 ### Was this patch authored or co-authored using generative AI tooling? no Closes #49404 from zhengruifeng/infra_split_py. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .github/workflows/build_and_test.yml | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 614fdd49d83df..14b9b87dd50a2 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -84,10 +84,11 @@ jobs: id: set-outputs run: | if [ -z "${{ inputs.jobs }}" ]; then - pyspark_modules=`cd dev && python -c "import sparktestsupport.modules as m; print(','.join(m.name for m in m.all_modules if m.name.startswith('pyspark')))"` + pyspark_modules=`cd dev && python -c "import sparktestsupport.modules as m; print(','.join(m.name for m in m.all_modules if m.name.startswith('pyspark') and not m.name.startswith('pyspark-pandas')))"` + pyspark_pandas_modules=`cd dev && python -c "import sparktestsupport.modules as m; print(','.join(m.name for m in m.all_modules if m.name.startswith('pyspark-pandas')))"` pyspark=`./dev/is-changed.py -m $pyspark_modules` + pandas=`./dev/is-changed.py -m $pyspark_pandas_modules` if [[ "${{ github.repository }}" != 'apache/spark' ]]; then - pandas=$pyspark yarn=`./dev/is-changed.py -m yarn` kubernetes=`./dev/is-changed.py -m kubernetes` sparkr=`./dev/is-changed.py -m sparkr` @@ -139,7 +140,7 @@ jobs: if: inputs.branch != 'branch-3.5' env: ${{ fromJSON(inputs.envs) }} run: | - if [[ "${{ fromJson(steps.set-outputs.outputs.required).pyspark }}" == 'true' ]]; then + if [[ "${{ fromJson(steps.set-outputs.outputs.required).pyspark }}" == 'true' || "${{ fromJson(steps.set-outputs.outputs.required).pyspark-pandas }}" == 'true' ]]; then if [[ "${{ env.PYSPARK_IMAGE_TO_TEST }}" == "" ]]; then echo "PYSPARK_IMAGE_TO_TEST is required when pyspark is enabled." exit 1 @@ -392,6 +393,7 @@ jobs: needs: precondition if: >- fromJson(needs.precondition.outputs.required).pyspark == 'true' || + fromJson(needs.precondition.outputs.required).pyspark-pandas == 'true' || fromJson(needs.precondition.outputs.required).lint == 'true' || fromJson(needs.precondition.outputs.required).docs == 'true' || fromJson(needs.precondition.outputs.required).sparkr == 'true' @@ -468,7 +470,7 @@ jobs: # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ inputs.branch }} - name: Build and push (PySpark with ${{ env.PYSPARK_IMAGE_TO_TEST }}) - if: ${{ inputs.branch != 'branch-3.5' && fromJson(needs.precondition.outputs.required).pyspark == 'true' && env.PYSPARK_IMAGE_TO_TEST != '' }} + if: ${{ inputs.branch != 'branch-3.5' && (fromJson(needs.precondition.outputs.required).pyspark == 'true' || fromJson(needs.precondition.outputs.required).pyspark-pandas == 'true') && env.PYSPARK_IMAGE_TO_TEST != '' }} id: docker_build_pyspark env: ${{ fromJSON(inputs.envs) }} uses: docker/build-push-action@v6 @@ -484,7 +486,7 @@ jobs: pyspark: needs: [precondition, infra-image] # always run if pyspark == 'true', even infra-image is skip (such as non-master job) - if: (!cancelled()) && fromJson(needs.precondition.outputs.required).pyspark == 'true' + if: (!cancelled()) && (fromJson(needs.precondition.outputs.required).pyspark == 'true' || fromJson(needs.precondition.outputs.required).pyspark-pandas == 'true') name: "Build modules: ${{ matrix.modules }}" runs-on: ubuntu-latest timeout-minutes: 180 @@ -517,6 +519,13 @@ jobs: - >- pyspark-pandas-connect-part3 exclude: + # Always run if pyspark == 'true', even infra-image is skip (such as non-master job) + # In practice, the build will run in individual PR, but not against the individual commit + # in Apache Spark repository. + - modules: ${{ fromJson(needs.precondition.outputs.required).pyspark != 'true' && 'pyspark-sql, pyspark-resource, pyspark-testing' }} + - modules: ${{ fromJson(needs.precondition.outputs.required).pyspark != 'true' && 'pyspark-core, pyspark-errors, pyspark-streaming, pyspark-logger' }} + - modules: ${{ fromJson(needs.precondition.outputs.required).pyspark != 'true' && 'pyspark-mllib, pyspark-ml, pyspark-ml-connect' }} + - modules: ${{ fromJson(needs.precondition.outputs.required).pyspark != 'true' && 'pyspark-connect' }} # Always run if pyspark-pandas == 'true', even infra-image is skip (such as non-master job) # In practice, the build will run in individual PR, but not against the individual commit # in Apache Spark repository. From e0c9ddcec9aee5c9ca12ef5ee731d977ff2e4f09 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Wed, 8 Jan 2025 12:58:53 +0800 Subject: [PATCH 354/438] [SPARK-50743][SQL] Normalize CTERelationDef and CTERelationRef IDs ### What changes were proposed in this pull request? Normalize CTERelationDef and CTERelationRef IDs. ### Why are the changes needed? This is necessary so that the output of single-pass Analyzer containing CTEs can be compared to the fixed-point one. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49377 from vladimirg-db/vladimirg-db/normalize-cte-def-ref-ids. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../sql/catalyst/plans/NormalizePlan.scala | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala index 38cf2730e9acd..d7ba596cf3995 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.plans +import java.util.HashMap + import org.apache.spark.sql.catalyst.analysis.GetViewColumnByNameAndOrdinal import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -68,8 +70,13 @@ object NormalizePlan extends PredicateHelper { * etc., will all now be equivalent. * - Sample the seed will replaced by 0L. * - Join conditions will be resorted by hashCode. + * - CTERelationDef ids will be rewritten using a monitonically increasing counter from 0. + * - CTERelationRef ids will be remapped based on the new CTERelationDef IDs. This is possible, + * because WithCTE returns cteDefs as first children, and the defs will be traversed before the + * refs. */ def normalizePlan(plan: LogicalPlan): LogicalPlan = { + val cteIdNormalizer = new CteIdNormalizer plan transform { case Filter(condition: Expression, child: LogicalPlan) => Filter( @@ -113,6 +120,10 @@ object NormalizePlan extends PredicateHelper { localRelation.copy(data = localRelation.data.map { row => unsafeProjection(row) }) + case cteRelationDef: CTERelationDef => + cteIdNormalizer.normalizeDef(cteRelationDef) + case cteRelationRef: CTERelationRef => + cteIdNormalizer.normalizeRef(cteRelationRef) } } @@ -143,3 +154,25 @@ object NormalizePlan extends PredicateHelper { case _ => condition // Don't reorder. } } + +class CteIdNormalizer { + private var cteIdCounter: Long = 0 + private val oldToNewIdMapping = new HashMap[Long, Long] + + def normalizeDef(cteRelationDef: CTERelationDef): CTERelationDef = { + try { + oldToNewIdMapping.put(cteRelationDef.id, cteIdCounter) + cteRelationDef.copy(id = cteIdCounter) + } finally { + cteIdCounter += 1 + } + } + + def normalizeRef(cteRelationRef: CTERelationRef): CTERelationRef = { + if (oldToNewIdMapping.containsKey(cteRelationRef.cteId)) { + cteRelationRef.copy(cteId = oldToNewIdMapping.get(cteRelationRef.cteId)) + } else { + cteRelationRef + } + } +} From eb02144cffb3999ec7b7c383eaa2c8fc50039eec Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 7 Jan 2025 21:18:11 -0800 Subject: [PATCH 355/438] [SPARK-50760][SQL][TESTS] Add negative test cases to `table-aliases.sql` in line with `HiveResolutionSuite` ### What changes were proposed in this pull request? This PR aims to add negative test cases to `table-aliases.sql` in line with `HiveResolutionSuite`. ### Why are the changes needed? To remove TODO task by completing test coverage between SQL and Hive module. https://github.com/apache/spark/blob/96da1abc0f3a89470358445b2fa89f6ab9726983/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala#L120 ### 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 #49403 from dongjoon-hyun/SPARK-50760. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../analyzer-results/table-aliases.sql.out | 42 +++++++++++++++++ .../sql-tests/inputs/table-aliases.sql | 5 ++ .../sql-tests/results/table-aliases.sql.out | 46 +++++++++++++++++++ .../hive/execution/HiveResolutionSuite.scala | 1 - 4 files changed, 93 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out index 8d4bffd868753..b7eacd1b1b0e9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out @@ -217,3 +217,45 @@ Project [a#x, b#x, c#x, d#x] +- Project [id#x, v2#x] +- SubqueryAlias src2 +- LocalRelation [id#x, v2#x] + + +-- !query +SELECT src1.* FROM src1 a ORDER BY id LIMIT 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", + "sqlState" : "42704", + "messageParameters" : { + "columns" : "`id`, `v1`", + "targetString" : "`src1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "src1.*" + } ] +} + + +-- !query +SELECT src1.id FROM (SELECT * FROM src1 ORDER BY id LIMIT 1) a +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`src1`.`id`", + "proposal" : "`a`.`id`, `a`.`v1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "src1.id" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/table-aliases.sql b/sql/core/src/test/resources/sql-tests/inputs/table-aliases.sql index 5b98f056ebc5a..d2aef1f83863b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/table-aliases.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/table-aliases.sql @@ -33,3 +33,8 @@ CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2) SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d); SELECT dst.* FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d); + +-- Negative examples after aliasing +SELECT src1.* FROM src1 a ORDER BY id LIMIT 1; + +SELECT src1.id FROM (SELECT * FROM src1 ORDER BY id LIMIT 1) a; diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index 5c05bb3f4c22b..1878452d1c20b 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -168,3 +168,49 @@ struct 1 a 1 8.5 2 b 2 1.0 3 c 3 3.2 + + +-- !query +SELECT src1.* FROM src1 a ORDER BY id LIMIT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", + "sqlState" : "42704", + "messageParameters" : { + "columns" : "`id`, `v1`", + "targetString" : "`src1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "src1.*" + } ] +} + + +-- !query +SELECT src1.id FROM (SELECT * FROM src1 ORDER BY id LIMIT 1) a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`src1`.`id`", + "proposal" : "`a`.`id`, `a`.`v1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "src1.id" + } ] +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index df6ef57a581d0..ecf89e59c501c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -117,7 +117,6 @@ class HiveResolutionSuite extends HiveComparisonTest { /** * Negative examples. Currently only left here for documentation purposes. - * TODO(marmbrus): Test that catalyst fails on these queries. */ /* SemanticException [Error 10009]: Line 1:7 Invalid table alias 'src' From 507de06137f1cae21628652e41033c8cc90e2705 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Wed, 8 Jan 2025 15:15:49 +0900 Subject: [PATCH 356/438] [SPARK-50083][SQL] Integrate `_LEGACY_ERROR_TEMP_1231` into `PARTITIONS_NOT_FOUND` ### What changes were proposed in this pull request? This PR proposes to Integrate `_LEGACY_ERROR_TEMP_1231 ` into `PARTITIONS_NOT_FOUND ` ### Why are the changes needed? To improve the error message by assigning proper error condition and SQLSTATE ### Does this PR introduce _any_ user-facing change? No, only user-facing error message improved ### How was this patch tested? Updated the existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #48614 from itholic/LEGACY_1231. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- .../resources/error/error-conditions.json | 5 -- .../sql/errors/QueryCompilationErrors.scala | 8 +-- .../analyzer-results/show-tables.sql.out | 7 ++- .../sql-tests/results/show-tables.sql.out | 7 ++- .../AlterTableAddPartitionSuiteBase.scala | 19 ++++-- .../AlterTableDropPartitionSuiteBase.scala | 19 ++++-- .../AlterTableRenamePartitionSuiteBase.scala | 19 ++++-- .../command/ShowPartitionsSuiteBase.scala | 19 ++++-- .../command/TruncateTableSuiteBase.scala | 63 +++++++++++++++---- .../v1/AlterTableSetLocationSuite.scala | 4 +- .../command/v2/ShowTablesSuite.scala | 4 +- .../spark/sql/hive/StatisticsSuite.scala | 41 +++++++----- .../sql/hive/execution/HiveDDLSuite.scala | 6 +- .../execution/command/ShowTablesSuite.scala | 4 +- 14 files changed, 159 insertions(+), 66 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 297e0ec1d3679..a8ff0809be735 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -6848,11 +6848,6 @@ "Decimal scale () cannot be greater than precision ()." ] }, - "_LEGACY_ERROR_TEMP_1231" : { - "message" : [ - " is not a valid partition column in table ." - ] - }, "_LEGACY_ERROR_TEMP_1232" : { "message" : [ "Partition spec is invalid. The spec () must match the partition spec () defined in table ''." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index ac419fd150ae3..0d5fe7bc14596 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2673,12 +2673,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "comment" -> comment)) } - def invalidPartitionColumnKeyInTableError(key: String, tblName: String): Throwable = { + def invalidPartitionColumnKeyInTableError(key: String, tableName: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1231", + errorClass = "PARTITIONS_NOT_FOUND", messageParameters = Map( - "key" -> key, - "tblName" -> toSQLId(tblName))) + "partitionList" -> toSQLId(key), + "tableName" -> toSQLId(tableName))) } def invalidPartitionSpecError( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out index a86cc72f0863c..8ae12b928d72e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out @@ -166,10 +166,11 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1) -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_1231", + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "428FT", "messageParameters" : { - "key" : "a", - "tblName" : "`spark_catalog`.`showdb`.`show_t1`" + "partitionList" : "`a`", + "tableName" : "`spark_catalog`.`showdb`.`show_t1`" } } diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index a4b967ca61f08..af1bb75aef88d 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -247,10 +247,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_1231", + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "428FT", "messageParameters" : { - "key" : "a", - "tblName" : "`spark_catalog`.`showdb`.`show_t1`" + "partitionList" : "`a`", + "tableName" : "`spark_catalog`.`showdb`.`show_t1`" } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala index cb25942822f46..13ea6f5a30536 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala @@ -23,6 +23,7 @@ import org.apache.spark.SparkNumberFormatException import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf /** @@ -97,10 +98,20 @@ trait AlterTableAddPartitionSuiteBase extends QueryTest with DDLCommandTestUtils withNamespaceAndTable("ns", "tbl") { t => spark.sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing PARTITIONED BY (id)") withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val errMsg = intercept[AnalysisException] { - spark.sql(s"ALTER TABLE $t ADD PARTITION (ID=1) LOCATION 'loc1'") - }.getMessage - assert(errMsg.contains("ID is not a valid partition column")) + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + s"`$SESSION_CATALOG_NAME`.`ns`.`tbl`" + } else { + "`test_catalog`.`ns`.`tbl`" + } + checkError( + exception = intercept[AnalysisException] { + spark.sql(s"ALTER TABLE $t ADD PARTITION (ID=1) LOCATION 'loc1'") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`ID`", + "tableName" -> expectedTableName) + ) } withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { spark.sql(s"ALTER TABLE $t ADD PARTITION (ID=1) LOCATION 'loc1'") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala index 279042f675cd5..a49a94174195c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionsException import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf /** @@ -103,10 +104,20 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil withNamespaceAndTable("ns", "tbl") { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing PARTITIONED BY (id)") withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val errMsg = intercept[AnalysisException] { - sql(s"ALTER TABLE $t DROP PARTITION (ID=1)") - }.getMessage - assert(errMsg.contains("ID is not a valid partition column")) + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + s"`$SESSION_CATALOG_NAME`.`ns`.`tbl`" + } else { + "`test_catalog`.`ns`.`tbl`" + } + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP PARTITION (ID=1)") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`ID`", + "tableName" -> expectedTableName) + ) } withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala index 905e6cfb9caaa..186f2b293ea81 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, PartitionsAlreadyExistException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf /** @@ -170,10 +171,20 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt checkPartitions(t, Map("id" -> "1")) withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val errMsg = intercept[AnalysisException] { - sql(s"ALTER TABLE $t PARTITION (ID = 1) RENAME TO PARTITION (id = 2)") - }.getMessage - assert(errMsg.contains("ID is not a valid partition column")) + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + s"`$SESSION_CATALOG_NAME`.`ns`.`tbl`" + } else { + "`test_catalog`.`ns`.`tbl`" + } + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $t PARTITION (ID = 1) RENAME TO PARTITION (id = 2)") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`ID`", + "tableName" -> expectedTableName) + ) checkPartitions(t, Map("id" -> "1")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 462b967a75900..f7d41556b4e6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StringType, StructType} @@ -66,10 +67,20 @@ trait ShowPartitionsSuiteBase extends QueryTest with DDLCommandTestUtils { test("non-partitioning columns") { withNamespaceAndTable("ns", "dateTable") { t => createDateTable(t) - val errMsg = intercept[AnalysisException] { - sql(s"SHOW PARTITIONS $t PARTITION(abcd=2015, xyz=1)") - }.getMessage - assert(errMsg.contains("abcd is not a valid partition column")) + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + s"`$SESSION_CATALOG_NAME`.`ns`.`datetable`" + } else { + "`test_catalog`.`ns`.`dateTable`" + } + checkError( + exception = intercept[AnalysisException] { + sql(s"SHOW PARTITIONS $t PARTITION(abcd=2015, xyz=1)") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`abcd`", + "tableName" -> expectedTableName) + ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala index 8c985ea1f0527..b61065f41c5e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf /** @@ -103,10 +104,20 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { } // throw exception if the column in partition spec is not a partition column. - val errMsg = intercept[AnalysisException] { - sql(s"TRUNCATE TABLE $t PARTITION (unknown = 1)") - }.getMessage - assert(errMsg.contains("unknown is not a valid partition column")) + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + s"`$SESSION_CATALOG_NAME`.`ns`.`parttable`" + } else { + "`test_catalog`.`ns`.`partTable`" + } + checkError( + exception = intercept[AnalysisException] { + sql(s"TRUNCATE TABLE $t PARTITION (unknown = 1)") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`unknown`", + "tableName" -> expectedTableName) + ) } } @@ -117,10 +128,28 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE TABLE $t (c0 INT) $defaultUsing") sql(s"INSERT INTO $t SELECT 0") - val errMsg = intercept[AnalysisException] { - sql(s"TRUNCATE TABLE $t PARTITION (c0=1)") - }.getMessage - assert(errMsg.contains(invalidPartColumnError)) + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + s"`$SESSION_CATALOG_NAME`.`ns`.`tbl`" + } else { + "`test_catalog`.`ns`.`tbl`" + } + val expectedCondition = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + "_LEGACY_ERROR_TEMP_1267" + } else { + "PARTITIONS_NOT_FOUND" + } + val expectedParameters = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + Map("tableIdentWithDB" -> expectedTableName) + } else { + Map("partitionList" -> "`c0`", "tableName" -> expectedTableName) + } + checkError( + exception = intercept[AnalysisException] { + sql(s"TRUNCATE TABLE $t PARTITION (c0=1)") + }, + condition = expectedCondition, + parameters = expectedParameters + ) } } @@ -145,10 +174,20 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"INSERT INTO $t PARTITION (id=0) SELECT 'abc'") sql(s"INSERT INTO $t PARTITION (id=1) SELECT 'def'") withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val errMsg = intercept[AnalysisException] { - sql(s"TRUNCATE TABLE $t PARTITION (ID=1)") - }.getMessage - assert(errMsg.contains("ID is not a valid partition column")) + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + s"`$SESSION_CATALOG_NAME`.`ns`.`tbl`" + } else { + "`test_catalog`.`ns`.`tbl`" + } + checkError( + exception = intercept[AnalysisException] { + sql(s"TRUNCATE TABLE $t PARTITION (ID=1)") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`ID`", + "tableName" -> expectedTableName) + ) } withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { sql(s"TRUNCATE TABLE $t PARTITION (ID=1)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala index 8f5af2e1f2e76..343a591fb5585 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala @@ -93,8 +93,8 @@ trait AlterTableSetLocationSuiteBase extends command.AlterTableSetLocationSuiteB exception = intercept[AnalysisException] { sql(s"ALTER TABLE $t PARTITION (A='1', B='2') SET LOCATION '/path/to/part/ways3'") }, - condition = "_LEGACY_ERROR_TEMP_1231", - parameters = Map("key" -> "A", "tblName" -> "`spark_catalog`.`ns`.`tbl`") + condition = "PARTITIONS_NOT_FOUND", + parameters = Map("partitionList" -> "`A`", "tableName" -> "`spark_catalog`.`ns`.`tbl`") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index d66dca20d77b8..5719fbee370a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -53,8 +53,8 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase catalog: String, namespace: String, table: String): (String, Map[String, String]) = { - ("_LEGACY_ERROR_TEMP_1231", - Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`")) + ("PARTITIONS_NOT_FOUND", + Map("partitionList" -> "`id`", "tableName" -> s"`$catalog`.`$namespace`.`$table`")) } protected override def namespaceKey: String = "Namespace" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 9c2f4461ff263..e2f0040afe57c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -609,12 +609,15 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val message = intercept[AnalysisException] { - sql(s"ANALYZE TABLE $tableName PARTITION (DS='2010-01-01') COMPUTE STATISTICS") - }.getMessage - assert(message.contains( - "DS is not a valid partition column in table " + - s"`$SESSION_CATALOG_NAME`.`default`.`$tableName`")) + checkError( + exception = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $tableName PARTITION (DS='2010-01-01') COMPUTE STATISTICS") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`DS`", + "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tableName`") + ) } } } @@ -692,16 +695,26 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto sql(s"INSERT INTO TABLE $tableName PARTITION (ds='2010-01-01') SELECT * FROM src") - assertAnalysisException( - s"ANALYZE TABLE $tableName PARTITION (hour=20) COMPUTE STATISTICS", - "hour is not a valid partition column in table " + - s"`$SESSION_CATALOG_NAME`.`default`.`${tableName.toLowerCase(Locale.ROOT)}`" + checkError( + exception = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $tableName PARTITION (hour=20) COMPUTE STATISTICS") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`hour`", + "tableName" -> + s"`$SESSION_CATALOG_NAME`.`default`.`${tableName.toLowerCase(Locale.ROOT)}`") ) - assertAnalysisException( - s"ANALYZE TABLE $tableName PARTITION (hour) COMPUTE STATISTICS", - "hour is not a valid partition column in table " + - s"`$SESSION_CATALOG_NAME`.`default`.`${tableName.toLowerCase(Locale.ROOT)}`" + checkError( + exception = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $tableName PARTITION (hour) COMPUTE STATISTICS") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`hour`", + "tableName" -> + s"`$SESSION_CATALOG_NAME`.`default`.`${tableName.toLowerCase(Locale.ROOT)}`") ) intercept[NoSuchPartitionException] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 4efc159a3ed44..880d7bdc82243 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -676,10 +676,10 @@ class HiveDDLSuite exception = intercept[AnalysisException] { sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')") }, - condition = "_LEGACY_ERROR_TEMP_1231", + condition = "PARTITIONS_NOT_FOUND", parameters = Map( - "key" -> "unknownCol", - "tblName" -> s"`$SESSION_CATALOG_NAME`.`default`.`exttable_with_partitions`") + "partitionList" -> "`unknownCol`", + "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`exttable_with_partitions`") ) sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 9b50e8f05fcae..de6af30e663d2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -39,8 +39,8 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { catalog: String, namespace: String, table: String): (String, Map[String, String]) = { - ("_LEGACY_ERROR_TEMP_1231", - Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`")) + ("PARTITIONS_NOT_FOUND", + Map("partitionList" -> "`id`", "tableName" -> s"`$catalog`.`$namespace`.`$table`")) } protected override def extendedPartExpectedResult: String = From afb4f822470a6576ab40047ee01b30d76cc4304f Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Wed, 8 Jan 2025 17:23:01 +0900 Subject: [PATCH 357/438] [SPARK-50752][PYTHON][SQL] Introduce configs for tuning Python UDF without Arrow ### What changes were proposed in this pull request? This PR proposes to introduce new configs for tuning Python UDF without Arrow. There are two major configurations for tuning in UDF: * batch size on serde of input/output (executor <-> python worker) * buffer size on writing to channel (executor -> python worker) The first one is hard-coded (100), and the second one is picked from more general config (network buffer), which effectively does not give a proper chance to tune. This PR enables users to tune the above twos, via * batch size: `park.sql.execution.python.udf.maxRecordsPerBatch` * buffer size: `spark.sql.execution.python.udf.buffer.size` ### Why are the changes needed? Explained in above. ### Does this PR introduce _any_ user-facing change? Yes, users will be able to tune for Python UDF without Arrow, with new configuration. ### How was this patch tested? New UTs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49397 from HeartSaVioR/SPARK-50752. Authored-by: Jungtaek Lim Signed-off-by: Hyukjin Kwon --- .../spark/api/python/PythonRunner.scala | 4 +++ python/pyspark/sql/tests/test_udf.py | 27 +++++++++++++++++++ python/pyspark/worker.py | 3 ++- .../apache/spark/sql/internal/SQLConf.scala | 18 +++++++++++++ .../python/BatchEvalPythonExec.scala | 11 +++++--- .../python/BatchEvalPythonUDTFExec.scala | 9 +++++-- .../execution/python/PythonUDFRunner.scala | 2 ++ 7 files changed, 68 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index a34cce980ae44..e3d10574419b3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -129,6 +129,8 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( protected val pythonExec: String = funcs.head.funcs.head.pythonExec protected val pythonVer: String = funcs.head.funcs.head.pythonVer + protected val batchSizeForPythonUDF: Int = 100 + // WARN: Both configurations, 'spark.python.daemon.module' and 'spark.python.worker.module' are // for very advanced users and they are experimental. This should be considered // as expert-only option, and shouldn't be used before knowing what it means exactly. @@ -212,6 +214,8 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (faultHandlerEnabled) { envVars.put("PYTHON_FAULTHANDLER_DIR", BasePythonRunner.faultHandlerLogDir.toString) } + // allow the user to set the batch size for the BatchedSerializer on UDFs + envVars.put("PYTHON_UDF_BATCH_SIZE", batchSizeForPythonUDF.toString) envVars.put("SPARK_JOB_ARTIFACT_UUID", jobArtifactUUID.getOrElse("default")) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 8193913892376..67d243cd29244 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -1229,6 +1229,33 @@ def setUpClass(cls): super(BaseUDFTestsMixin, cls).setUpClass() cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "false") + # We cannot check whether the batch size is effective or not. We just run the query with + # various batch size and see whether the query runs successfully, and the output is + # consistent across different batch sizes. + def test_udf_with_various_batch_size(self): + self.spark.catalog.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + for batch_size in [1, 33, 1000, 2000]: + with self.sql_conf({"spark.sql.execution.python.udf.maxRecordsPerBatch": batch_size}): + df = self.spark.range(1000).selectExpr("twoArgs('test', id) AS ret").orderBy("ret") + rets = [x["ret"] for x in df.collect()] + self.assertEqual(rets, list(range(4, 1004))) + + # We cannot check whether the buffer size is effective or not. We just run the query with + # various buffer size and see whether the query runs successfully, and the output is + # consistent across different batch sizes. + def test_udf_with_various_buffer_size(self): + self.spark.catalog.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + for batch_size in [1, 33, 10000]: + with self.sql_conf({"spark.sql.execution.python.udf.buffer.size": batch_size}): + df = ( + self.spark.range(1000) + .repartition(1) + .selectExpr("twoArgs('test', id) AS ret") + .orderBy("ret") + ) + rets = [x["ret"] for x in df.collect()] + self.assertEqual(rets, list(range(4, 1004))) + class UDFInitializationTests(unittest.TestCase): def tearDown(self): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 712f71d3861ae..e799498cdd80b 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1569,7 +1569,8 @@ def read_udfs(pickleSer, infile, eval_type): arrow_cast, ) else: - ser = BatchedSerializer(CPickleSerializer(), 100) + batch_size = int(os.environ.get("PYTHON_UDF_BATCH_SIZE", "100")) + ser = BatchedSerializer(CPickleSerializer(), batch_size) is_profiling = read_bool(infile) if is_profiling: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7bc4051b45d4c..875e9543c4720 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3377,6 +3377,24 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PYTHON_UDF_MAX_RECORDS_PER_BATCH = + buildConf("spark.sql.execution.python.udf.maxRecordsPerBatch") + .doc("When using Python UDFs, limit the maximum number of records that can be batched " + + "for serialization/deserialization.") + .version("4.0.0") + .intConf + .checkValue(_ > 0, "The value of spark.sql.execution.python.udf.maxRecordsPerBatch " + + "must be positive.") + .createWithDefault(100) + + val PYTHON_UDF_BUFFER_SIZE = + buildConf("spark.sql.execution.python.udf.buffer.size") + .doc( + s"Same as `${BUFFER_SIZE.key}` but only applies to Python UDF executions. If it is not " + + s"set, the fallback is `${BUFFER_SIZE.key}`.") + .version("4.0.0") + .fallbackConf(BUFFER_SIZE) + val PANDAS_UDF_BUFFER_SIZE = buildConf("spark.sql.execution.pandas.udf.buffer.size") .doc( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index e6958392cad48..28318a319b088 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.EvalPythonExec.ArgumentMetadata +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StructField, StructType} /** @@ -39,10 +40,12 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute] private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) override protected def evaluatorFactory: EvalPythonEvaluatorFactory = { + val batchSize = conf.getConf(SQLConf.PYTHON_UDF_MAX_RECORDS_PER_BATCH) new BatchEvalPythonEvaluatorFactory( child.output, udfs, output, + batchSize, pythonMetrics, jobArtifactUUID, conf.pythonUDFProfiler) @@ -56,6 +59,7 @@ class BatchEvalPythonEvaluatorFactory( childOutput: Seq[Attribute], udfs: Seq[PythonUDF], output: Seq[Attribute], + batchSize: Int, pythonMetrics: Map[String, SQLMetric], jobArtifactUUID: Option[String], profiler: Option[String]) @@ -70,7 +74,7 @@ class BatchEvalPythonEvaluatorFactory( EvaluatePython.registerPicklers() // register pickler for Row // Input iterator to Python. - val inputIterator = BatchEvalPythonExec.getInputIterator(iter, schema) + val inputIterator = BatchEvalPythonExec.getInputIterator(iter, schema, batchSize) // Output iterator for results from Python. val outputIterator = @@ -107,7 +111,8 @@ class BatchEvalPythonEvaluatorFactory( object BatchEvalPythonExec { def getInputIterator( iter: Iterator[InternalRow], - schema: StructType): Iterator[Array[Byte]] = { + schema: StructType, + batchSize: Int): Iterator[Array[Byte]] = { val dataTypes = schema.map(_.dataType) val needConversion = dataTypes.exists(EvaluatePython.needConversionInPython) @@ -140,6 +145,6 @@ object BatchEvalPythonExec { } fields } - }.grouped(100).map(x => pickle.dumps(x.toArray)) + }.grouped(batchSize).map(x => pickle.dumps(x.toArray)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala index 9eebd4ea7e79c..c0dcb77817420 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala @@ -23,8 +23,9 @@ import scala.jdk.CollectionConverters._ import net.razorvine.pickle.Unpickler -import org.apache.spark.{JobArtifactSet, TaskContext} +import org.apache.spark.{JobArtifactSet, SparkEnv, TaskContext} import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType, PythonWorkerUtils} +import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.GenericArrayData @@ -63,7 +64,8 @@ case class BatchEvalPythonUDTFExec( EvaluatePython.registerPicklers() // register pickler for Row // Input iterator to Python. - val inputIterator = BatchEvalPythonExec.getInputIterator(iter, schema) + // For Python UDTF, we don't have a separate configuration for the batch size yet. + val inputIterator = BatchEvalPythonExec.getInputIterator(iter, schema, 100) // Output iterator for results from Python. val outputIterator = @@ -101,6 +103,9 @@ class PythonUDTFRunner( Seq((ChainedPythonFunctions(Seq(udtf.func)), udtf.resultId.id)), PythonEvalType.SQL_TABLE_UDF, Array(argMetas.map(_.offset)), pythonMetrics, jobArtifactUUID) { + // Overriding here to NOT use the same value of UDF config in UDTF. + override val bufferSize: Int = SparkEnv.get.conf.get(BUFFER_SIZE) + override protected def writeUDF(dataOut: DataOutputStream): Unit = { PythonUDTFRunner.writeUDTF(dataOut, udtf, argMetas) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala index 92310aa755db8..167e1fd8b0f01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala @@ -46,6 +46,8 @@ abstract class BasePythonUDFRunner( override val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled + override val bufferSize: Int = SQLConf.get.getConf(SQLConf.PYTHON_UDF_BUFFER_SIZE) + protected def writeUDF(dataOut: DataOutputStream): Unit protected override def newWriter( From ceb7583ed394989b8d1f14350bce58a2d291acd8 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 8 Jan 2025 17:40:08 +0900 Subject: [PATCH 358/438] Revert "[SPARK-50756][SQL] Use error class for exceptions in SparkConf.validateSettings" This reverts commit cfb2e4054f77c80afe197ab85c51e12e79d5f821. --- .../main/resources/error/error-conditions.json | 12 ------------ .../scala/org/apache/spark/SparkConf.scala | 18 ++++++------------ 2 files changed, 6 insertions(+), 24 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index a8ff0809be735..f7ca1eae0ef29 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2393,12 +2393,6 @@ ], "sqlState" : "42604" }, - "INVALID_EXECUTOR_HEARTBEAT_INTERVAL" : { - "message" : [ - "The value of =ms must be greater than the value of =ms." - ], - "sqlState" : "42616" - }, "INVALID_EXECUTOR_MEMORY" : { "message" : [ "Executor memory must be at least .", @@ -3699,12 +3693,6 @@ ], "sqlState" : "07501" }, - "NETWORK_AUTH_MUST_BE_ENABLED" : { - "message" : [ - " must be enabled when enabling encryption." - ], - "sqlState" : "42616" - }, "NONEXISTENT_FIELD_NAME_IN_LIST" : { "message" : [ "Field(s) do(es) not exist. Available fields: " diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index aab636e8d8125..ae6ef1ee55608 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -607,23 +607,17 @@ class SparkConf(loadDefaults: Boolean) } val encryptionEnabled = get(NETWORK_CRYPTO_ENABLED) || get(SASL_ENCRYPTION_ENABLED) - SparkException.require( - !encryptionEnabled || get(NETWORK_AUTH_ENABLED), - "NETWORK_AUTH_MUST_BE_ENABLED", - Map("networkAuthEnabledConf" -> NETWORK_AUTH_ENABLED.key)) + require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), + s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") val executorTimeoutThresholdMs = get(NETWORK_TIMEOUT) * 1000 val executorHeartbeatIntervalMs = get(EXECUTOR_HEARTBEAT_INTERVAL) + val networkTimeout = NETWORK_TIMEOUT.key // If spark.executor.heartbeatInterval bigger than spark.network.timeout, // it will almost always cause ExecutorLostFailure. See SPARK-22754. - SparkException.require( - executorTimeoutThresholdMs > executorHeartbeatIntervalMs, - "INVALID_EXECUTOR_HEARTBEAT_INTERVAL", - Map( - "networkTimeoutKey" -> NETWORK_TIMEOUT.key, - "networkTimeoutValue" -> executorTimeoutThresholdMs.toString, - "executorHeartbeatIntervalKey" -> EXECUTOR_HEARTBEAT_INTERVAL.key, - "executorHeartbeatIntervalValue" -> executorHeartbeatIntervalMs.toString)) + require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The value of " + + s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be greater than the value of " + + s"${EXECUTOR_HEARTBEAT_INTERVAL.key}=${executorHeartbeatIntervalMs}ms.") } /** From dabf227959698884d32b60c99136a090e598a89c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 8 Jan 2025 22:49:26 +0800 Subject: [PATCH 359/438] [SPARK-50766][TESTS] Regenerate benchmark results ### What changes were proposed in this pull request? This PR aims to regenerate benchmark results as a new baseline before cutting branch-4.0. ### Why are the changes needed? To make the result up-to-date and make it easy to validate the regression. - `SubExprEliminationBenchmark` benchmark result is not included because it's broken currently. - https://github.com/apache/spark/pull/48466#issuecomment-2575780075 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49409 from dongjoon-hyun/bm_20250107. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- .../AvroReadBenchmark-jdk21-results.txt | 76 +- .../benchmarks/AvroReadBenchmark-results.txt | 76 +- .../AvroWriteBenchmark-jdk21-results.txt | 72 +- .../benchmarks/AvroWriteBenchmark-results.txt | 72 +- .../ChecksumBenchmark-jdk21-results.txt | 6 +- core/benchmarks/ChecksumBenchmark-results.txt | 6 +- .../CoalescedRDDBenchmark-jdk21-results.txt | 62 +- .../CoalescedRDDBenchmark-results.txt | 62 +- .../KryoBenchmark-jdk21-results.txt | 38 +- core/benchmarks/KryoBenchmark-results.txt | 38 +- .../KryoIteratorBenchmark-jdk21-results.txt | 38 +- .../KryoIteratorBenchmark-results.txt | 38 +- .../KryoSerializerBenchmark-jdk21-results.txt | 6 +- .../KryoSerializerBenchmark-results.txt | 6 +- .../benchmarks/LZFBenchmark-jdk21-results.txt | 12 +- core/benchmarks/LZFBenchmark-results.txt | 12 +- ...StatusesConvertBenchmark-jdk21-results.txt | 8 +- .../MapStatusesConvertBenchmark-results.txt | 8 +- ...tatusesSerDeserBenchmark-jdk21-results.txt | 38 +- .../MapStatusesSerDeserBenchmark-results.txt | 38 +- ...rsistenceEngineBenchmark-jdk21-results.txt | 18 +- .../PersistenceEngineBenchmark-results.txt | 18 +- ...PropertiesCloneBenchmark-jdk21-results.txt | 30 +- .../PropertiesCloneBenchmark-results.txt | 30 +- .../XORShiftRandomBenchmark-jdk21-results.txt | 28 +- .../XORShiftRandomBenchmark-results.txt | 28 +- .../ZStandardBenchmark-jdk21-results.txt | 56 +- .../benchmarks/ZStandardBenchmark-results.txt | 56 +- .../BLASBenchmark-jdk21-results.txt | 182 ++--- .../benchmarks/BLASBenchmark-results.txt | 182 ++--- ...DTSerializationBenchmark-jdk21-results.txt | 6 +- .../UDTSerializationBenchmark-results.txt | 6 +- ...alendarIntervalBenchmark-jdk21-results.txt | 4 +- .../CalendarIntervalBenchmark-results.txt | 4 +- .../EnumTypeSetBenchmark-jdk21-results.txt | 90 +-- .../EnumTypeSetBenchmark-results.txt | 90 +-- .../EscapePathBenchmark-jdk21-results.txt | 12 +- .../EscapePathBenchmark-results.txt | 12 +- ...enericArrayDataBenchmark-jdk21-results.txt | 12 +- .../GenericArrayDataBenchmark-results.txt | 12 +- .../HashBenchmark-jdk21-results.txt | 50 +- .../benchmarks/HashBenchmark-results.txt | 50 +- .../HashByteArrayBenchmark-jdk21-results.txt | 72 +- .../HashByteArrayBenchmark-results.txt | 72 +- .../benchmarks/HexBenchmark-jdk21-results.txt | 10 +- .../benchmarks/HexBenchmark-results.txt | 10 +- ...mparableWrapperBenchmark-jdk21-results.txt | 6 +- ...lRowComparableWrapperBenchmark-results.txt | 6 +- ...nsafeProjectionBenchmark-jdk21-results.txt | 10 +- .../UnsafeProjectionBenchmark-results.txt | 10 +- .../AggregateBenchmark-jdk21-results.txt | 108 +-- .../benchmarks/AggregateBenchmark-results.txt | 108 +-- ...nsiIntervalSortBenchmark-jdk21-results.txt | 24 +- .../AnsiIntervalSortBenchmark-results.txt | 24 +- .../Base64Benchmark-jdk21-results.txt | 48 +- .../benchmarks/Base64Benchmark-results.txt | 48 +- .../BloomFilterBenchmark-jdk21-results.txt | 104 +-- .../BloomFilterBenchmark-results.txt | 104 +-- ...DataSourceWriteBenchmark-jdk21-results.txt | 60 +- ...uiltInDataSourceWriteBenchmark-results.txt | 60 +- .../ByteArrayBenchmark-jdk21-results.txt | 18 +- .../benchmarks/ByteArrayBenchmark-results.txt | 18 +- .../benchmarks/CSVBenchmark-jdk21-results.txt | 88 +- sql/core/benchmarks/CSVBenchmark-results.txt | 88 +- .../CharVarcharBenchmark-jdk21-results.txt | 112 +-- .../CharVarcharBenchmark-results.txt | 112 +-- .../CollationBenchmark-jdk21-results.txt | 116 +-- .../benchmarks/CollationBenchmark-results.txt | 116 +-- ...llationNonASCIIBenchmark-jdk21-results.txt | 116 +-- .../CollationNonASCIIBenchmark-results.txt | 116 +-- .../ColumnarBatchBenchmark-jdk21-results.txt | 46 +- .../ColumnarBatchBenchmark-results.txt | 46 +- ...mpressionSchemeBenchmark-jdk21-results.txt | 136 ++-- .../CompressionSchemeBenchmark-results.txt | 136 ++-- ...antColumnVectorBenchmark-jdk21-results.txt | 260 +++--- .../ConstantColumnVectorBenchmark-results.txt | 260 +++--- .../DataSourceReadBenchmark-jdk21-results.txt | 588 +++++++------- .../DataSourceReadBenchmark-results.txt | 588 +++++++------- .../DatasetBenchmark-jdk21-results.txt | 42 +- .../benchmarks/DatasetBenchmark-results.txt | 42 +- .../DateTimeBenchmark-jdk21-results.txt | 372 ++++----- .../benchmarks/DateTimeBenchmark-results.txt | 372 ++++----- .../DateTimeRebaseBenchmark-jdk21-results.txt | 206 ++--- .../DateTimeRebaseBenchmark-results.txt | 206 ++--- .../EncodeBenchmark-jdk21-results.txt | 8 +- .../benchmarks/EncodeBenchmark-results.txt | 8 +- ...yUnsafeRowArrayBenchmark-jdk21-results.txt | 30 +- ...endOnlyUnsafeRowArrayBenchmark-results.txt | 30 +- .../ExtractBenchmark-jdk21-results.txt | 160 ++-- .../benchmarks/ExtractBenchmark-results.txt | 160 ++-- .../FilterPushdownBenchmark-jdk21-results.txt | 760 +++++++++--------- .../FilterPushdownBenchmark-results.txt | 760 +++++++++--------- .../GenerateExecBenchmark-jdk21-results.txt | 6 +- .../GenerateExecBenchmark-results.txt | 6 +- ...RelationMetricsBenchmark-jdk21-results.txt | 4 +- ...HashedRelationMetricsBenchmark-results.txt | 4 +- .../InExpressionBenchmark-jdk21-results.txt | 630 +++++++-------- .../InExpressionBenchmark-results.txt | 630 +++++++-------- ...nMemoryColumnarBenchmark-jdk21-results.txt | 6 +- .../InMemoryColumnarBenchmark-results.txt | 6 +- ...namicPartitionsBenchmark-jdk21-results.txt | 8 +- ...WithDynamicPartitionsBenchmark-results.txt | 8 +- .../IntervalBenchmark-jdk21-results.txt | 64 +- .../benchmarks/IntervalBenchmark-results.txt | 64 +- .../JoinBenchmark-jdk21-results.txt | 66 +- sql/core/benchmarks/JoinBenchmark-results.txt | 66 +- .../JsonBenchmark-jdk21-results.txt | 128 +-- sql/core/benchmarks/JsonBenchmark-results.txt | 128 +-- .../MakeDateTimeBenchmark-jdk21-results.txt | 28 +- .../MakeDateTimeBenchmark-results.txt | 28 +- .../MetadataStructBenchmark-jdk21-results.txt | 58 +- .../MetadataStructBenchmark-results.txt | 58 +- ...ricsAggregationBenchmark-jdk21-results.txt | 14 +- .../MetricsAggregationBenchmark-results.txt | 14 +- .../MiscBenchmark-jdk21-results.txt | 80 +- sql/core/benchmarks/MiscBenchmark-results.txt | 80 +- ...edSchemaPruningBenchmark-jdk21-results.txt | 48 +- ...rcNestedSchemaPruningBenchmark-results.txt | 48 +- ...edSchemaPruningBenchmark-jdk21-results.txt | 48 +- ...V2NestedSchemaPruningBenchmark-results.txt | 48 +- ...edicatePushDownBenchmark-jdk21-results.txt | 18 +- ...stedPredicatePushDownBenchmark-results.txt | 18 +- ...edSchemaPruningBenchmark-jdk21-results.txt | 48 +- ...etNestedSchemaPruningBenchmark-results.txt | 48 +- .../PrimitiveArrayBenchmark-jdk21-results.txt | 6 +- .../PrimitiveArrayBenchmark-results.txt | 6 +- .../RangeBenchmark-jdk21-results.txt | 12 +- .../benchmarks/RangeBenchmark-results.txt | 12 +- .../SetOperationsBenchmark-jdk21-results.txt | 10 +- .../SetOperationsBenchmark-results.txt | 10 +- .../SortBenchmark-jdk21-results.txt | 14 +- sql/core/benchmarks/SortBenchmark-results.txt | 14 +- ...BasicOperationsBenchmark-jdk21-results.txt | 120 +-- ...eStoreBasicOperationsBenchmark-results.txt | 120 +-- ...StringFunctionsBenchmark-jdk21-results.txt | 4 +- .../StringFunctionsBenchmark-results.txt | 4 +- .../TPCDSQueryBenchmark-jdk21-results.txt | 540 ++++++------- .../TPCDSQueryBenchmark-results.txt | 540 ++++++------- ...deredAndProjectBenchmark-jdk21-results.txt | 6 +- ...TakeOrderedAndProjectBenchmark-results.txt | 6 +- .../TopKBenchmark-jdk21-results.txt | 26 +- sql/core/benchmarks/TopKBenchmark-results.txt | 26 +- .../benchmarks/UDFBenchmark-jdk21-results.txt | 42 +- sql/core/benchmarks/UDFBenchmark-results.txt | 42 +- ...UnsafeArrayDataBenchmark-jdk21-results.txt | 24 +- .../UnsafeArrayDataBenchmark-results.txt | 24 +- .../UpdateFieldsBenchmark-jdk21-results.txt | 16 +- .../UpdateFieldsBenchmark-results.txt | 14 +- .../V2FunctionBenchmark-jdk21-results.txt | 56 +- .../V2FunctionBenchmark-results.txt | 56 +- .../WideSchemaBenchmark-jdk21-results.txt | 172 ++-- .../WideSchemaBenchmark-results.txt | 172 ++-- .../WideTableBenchmark-jdk21-results.txt | 16 +- .../benchmarks/WideTableBenchmark-results.txt | 16 +- ...IntoHiveTableBenchmark-hive2.3-results.txt | 14 +- ...veTableBenchmark-jdk21-hive2.3-results.txt | 14 +- ...shAggregateExecBenchmark-jdk21-results.txt | 36 +- ...jectHashAggregateExecBenchmark-results.txt | 36 +- .../OrcReadBenchmark-jdk21-results.txt | 188 ++--- .../benchmarks/OrcReadBenchmark-results.txt | 188 ++--- 160 files changed, 6873 insertions(+), 6925 deletions(-) diff --git a/connector/avro/benchmarks/AvroReadBenchmark-jdk21-results.txt b/connector/avro/benchmarks/AvroReadBenchmark-jdk21-results.txt index 0f4579f5da24f..c41782457cd9e 100644 --- a/connector/avro/benchmarks/AvroReadBenchmark-jdk21-results.txt +++ b/connector/avro/benchmarks/AvroReadBenchmark-jdk21-results.txt @@ -2,140 +2,140 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2061 2066 7 7.6 131.0 1.0X +Sum 1971 1989 26 8.0 125.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2032 2033 2 7.7 129.2 1.0X +Sum 1958 2014 80 8.0 124.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1992 2005 19 7.9 126.6 1.0X +Sum 1956 1987 44 8.0 124.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1992 2017 35 7.9 126.6 1.0X +Sum 1953 1962 12 8.1 124.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1981 1981 0 7.9 125.9 1.0X +Sum 1948 1950 3 8.1 123.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1981 1984 4 7.9 126.0 1.0X +Sum 1933 1938 6 8.1 122.9 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 3863 3867 5 2.7 368.4 1.0X +Sum of columns 3570 3574 6 2.9 340.4 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 2129 2143 20 7.4 135.4 1.0X -Partition column 1984 1986 2 7.9 126.1 1.1X -Both columns 2209 2231 31 7.1 140.4 1.0X +Data column 2062 2083 30 7.6 131.1 1.0X +Partition column 1869 1873 5 8.4 118.9 1.1X +Both columns 2057 2093 51 7.6 130.8 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 2511 2564 75 4.2 239.5 1.0X +Sum of string length 2010 2012 3 5.2 191.7 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3303 3317 20 3.2 315.0 1.0X +Sum of string length 3082 3094 16 3.4 293.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 2221 2252 44 4.7 211.8 1.0X +Sum of string length 2220 2245 36 4.7 211.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 1170 1174 6 9.0 111.6 1.0X +Sum of string length 1152 1159 10 9.1 109.9 1.0X ================================================================================================ Select All From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Wide Column Scan from 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select of all columns 19095 19150 78 0.0 38190.4 1.0X +Select of all columns 20941 20946 6 0.0 41882.8 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 3188 3192 4 0.3 3040.7 1.0X +Sum of single column 3425 3440 21 0.3 3266.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 6229 6254 35 0.2 5940.6 1.0X +Sum of single column 6740 6770 43 0.2 6427.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 9279 9318 56 0.1 8848.8 1.0X +Sum of single column 9988 10056 96 0.1 9525.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 5538 5544 7 0.2 5537.5 1.0X -pushdown disabled 5546 5571 24 0.2 5546.5 1.0X -w/ filters 2312 2324 18 0.4 2312.4 2.4X +w/o filters 5802 5844 44 0.2 5801.7 1.0X +pushdown disabled 5544 5616 97 0.2 5543.9 1.0X +w/ filters 2605 2609 5 0.4 2605.4 2.2X diff --git a/connector/avro/benchmarks/AvroReadBenchmark-results.txt b/connector/avro/benchmarks/AvroReadBenchmark-results.txt index db6193e67ac39..117cb0b05e8a3 100644 --- a/connector/avro/benchmarks/AvroReadBenchmark-results.txt +++ b/connector/avro/benchmarks/AvroReadBenchmark-results.txt @@ -2,140 +2,140 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1959 1993 47 8.0 124.6 1.0X +Sum 1945 1948 4 8.1 123.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1929 1949 28 8.2 122.6 1.0X +Sum 1941 1965 34 8.1 123.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1965 1974 13 8.0 124.9 1.0X +Sum 1910 1921 15 8.2 121.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1904 1918 20 8.3 121.0 1.0X +Sum 1923 1927 6 8.2 122.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1916 1934 26 8.2 121.8 1.0X +Sum 1893 1898 7 8.3 120.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 1911 1917 8 8.2 121.5 1.0X +Sum 1890 1894 6 8.3 120.2 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 3426 3450 34 3.1 326.8 1.0X +Sum of columns 3614 3616 2 2.9 344.7 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 1871 1888 24 8.4 118.9 1.0X -Partition column 1713 1720 9 9.2 108.9 1.1X -Both columns 1962 1970 12 8.0 124.7 1.0X +Data column 2106 2108 2 7.5 133.9 1.0X +Partition column 1862 1864 3 8.4 118.4 1.1X +Both columns 2359 2382 32 6.7 150.0 0.9X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 2042 2055 18 5.1 194.7 1.0X +Sum of string length 2147 2151 6 4.9 204.7 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3089 3109 28 3.4 294.6 1.0X +Sum of string length 3410 3421 16 3.1 325.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 2103 2104 2 5.0 200.5 1.0X +Sum of string length 2133 2157 34 4.9 203.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 1073 1079 10 9.8 102.3 1.0X +Sum of string length 1107 1110 4 9.5 105.6 1.0X ================================================================================================ Select All From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Wide Column Scan from 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select of all columns 18300 18346 64 0.0 36600.5 1.0X +Select of all columns 19318 19384 94 0.0 38635.4 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 3144 3148 5 0.3 2998.8 1.0X +Sum of single column 3159 3183 34 0.3 3012.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 6216 6229 19 0.2 5927.8 1.0X +Sum of single column 6352 6387 49 0.2 6058.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 9286 9585 423 0.1 8855.6 1.0X +Sum of single column 9512 9539 39 0.1 9070.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 5308 5326 23 0.2 5307.5 1.0X -pushdown disabled 5253 5288 33 0.2 5252.7 1.0X -w/ filters 2036 2061 24 0.5 2036.3 2.6X +w/o filters 5474 5481 12 0.2 5474.4 1.0X +pushdown disabled 5453 5490 44 0.2 5452.7 1.0X +w/ filters 2210 2223 18 0.5 2209.7 2.5X diff --git a/connector/avro/benchmarks/AvroWriteBenchmark-jdk21-results.txt b/connector/avro/benchmarks/AvroWriteBenchmark-jdk21-results.txt index a071bc767cfaa..b8c0d3b95e360 100644 --- a/connector/avro/benchmarks/AvroWriteBenchmark-jdk21-results.txt +++ b/connector/avro/benchmarks/AvroWriteBenchmark-jdk21-results.txt @@ -1,56 +1,56 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1566 1588 30 10.0 99.6 1.0X -Output Single Double Column 1655 1668 18 9.5 105.3 0.9X -Output Int and String Column 3493 3496 5 4.5 222.1 0.4X -Output Partitions 3062 3112 71 5.1 194.7 0.5X -Output Buckets 3937 3952 20 4.0 250.3 0.4X +Output Single Int Column 1562 1586 33 10.1 99.3 1.0X +Output Single Double Column 1658 1695 52 9.5 105.4 0.9X +Output Int and String Column 3516 3524 11 4.5 223.6 0.4X +Output Partitions 2936 3033 138 5.4 186.6 0.5X +Output Buckets 3856 3882 36 4.1 245.1 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro compression with different codec: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -BZIP2: 115765 115975 297 0.0 1157649.1 1.0X -DEFLATE: 6345 6370 35 0.0 63448.5 18.2X -UNCOMPRESSED: 5183 5184 1 0.0 51827.4 22.3X -SNAPPY: 4611 4614 3 0.0 46112.5 25.1X -XZ: 54096 57854 5315 0.0 540956.3 2.1X -ZSTANDARD: 4877 4888 15 0.0 48770.9 23.7X +BZIP2: 117457 117471 19 0.0 1174572.2 1.0X +DEFLATE: 6340 6364 34 0.0 63404.0 18.5X +UNCOMPRESSED: 4990 4998 12 0.0 49898.9 23.5X +SNAPPY: 4561 4564 4 0.0 45610.2 25.8X +XZ: 43883 49072 7337 0.0 438832.8 2.7X +ZSTANDARD: 4774 4777 4 0.0 47741.1 24.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro deflate with different levels: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -DEFLATE: deflate.level=1 4807 4847 57 0.0 48065.8 1.0X -DEFLATE: deflate.level=3 4803 4809 7 0.0 48033.5 1.0X -DEFLATE: deflate.level=5 6373 6389 22 0.0 63728.7 0.8X -DEFLATE: deflate.level=7 6427 6460 47 0.0 64266.6 0.7X -DEFLATE: deflate.level=9 6628 6634 10 0.0 66277.2 0.7X +DEFLATE: deflate.level=1 4752 4764 17 0.0 47519.5 1.0X +DEFLATE: deflate.level=3 4682 4687 8 0.0 46819.6 1.0X +DEFLATE: deflate.level=5 6382 6392 13 0.0 63820.4 0.7X +DEFLATE: deflate.level=7 6477 6532 77 0.0 64774.8 0.7X +DEFLATE: deflate.level=9 6773 6783 15 0.0 67729.6 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro xz with different levels: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -XZ: xz.level=1 11974 12000 37 0.0 119736.9 1.0X -XZ: xz.level=3 21671 21813 201 0.0 216709.0 0.6X -XZ: xz.level=5 47055 47335 397 0.0 470545.5 0.3X -XZ: xz.level=7 74766 75069 428 0.0 747658.3 0.2X -XZ: xz.level=9 146478 146490 16 0.0 1464783.7 0.1X +XZ: xz.level=1 11571 11577 9 0.0 115710.1 1.0X +XZ: xz.level=3 21469 21642 245 0.0 214687.1 0.5X +XZ: xz.level=5 40907 40912 7 0.0 409072.6 0.3X +XZ: xz.level=7 60545 61371 1167 0.0 605453.1 0.2X +XZ: xz.level=9 136882 137479 845 0.0 1368819.9 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro zstandard with different levels: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------- -ZSTANDARD: zstandard.level=1 4760 4797 53 0.0 47598.3 1.0X -ZSTANDARD: zstandard.level=1, zstandard.bufferPool.enabled=true 4666 4696 43 0.0 46656.0 1.0X -ZSTANDARD: zstandard.level=3 4845 4869 33 0.0 48452.8 1.0X -ZSTANDARD: zstandard.level=3, zstandard.bufferPool.enabled=true 4790 4801 16 0.0 47896.5 1.0X -ZSTANDARD: zstandard.level=5 5125 5164 55 0.0 51248.6 0.9X -ZSTANDARD: zstandard.level=5, zstandard.bufferPool.enabled=true 4912 4928 22 0.0 49122.5 1.0X -ZSTANDARD: zstandard.level=7 5319 5333 19 0.0 53192.1 0.9X -ZSTANDARD: zstandard.level=7, zstandard.bufferPool.enabled=true 5250 5284 48 0.0 52501.2 0.9X -ZSTANDARD: zstandard.level=9 6087 6087 0 0.0 60869.7 0.8X -ZSTANDARD: zstandard.level=9, zstandard.bufferPool.enabled=true 6219 6234 21 0.0 62191.3 0.8X +ZSTANDARD: zstandard.level=1 4722 4763 58 0.0 47221.7 1.0X +ZSTANDARD: zstandard.level=1, zstandard.bufferPool.enabled=true 4734 5114 536 0.0 47341.8 1.0X +ZSTANDARD: zstandard.level=3 4816 4837 29 0.0 48162.0 1.0X +ZSTANDARD: zstandard.level=3, zstandard.bufferPool.enabled=true 4741 4766 35 0.0 47414.8 1.0X +ZSTANDARD: zstandard.level=5 5054 5155 143 0.0 50536.2 0.9X +ZSTANDARD: zstandard.level=5, zstandard.bufferPool.enabled=true 4869 4874 7 0.0 48690.8 1.0X +ZSTANDARD: zstandard.level=7 5325 5350 35 0.0 53251.2 0.9X +ZSTANDARD: zstandard.level=7, zstandard.bufferPool.enabled=true 5283 5308 35 0.0 52828.8 0.9X +ZSTANDARD: zstandard.level=9 6092 6116 35 0.0 60917.5 0.8X +ZSTANDARD: zstandard.level=9, zstandard.bufferPool.enabled=true 5925 5935 15 0.0 59246.3 0.8X diff --git a/connector/avro/benchmarks/AvroWriteBenchmark-results.txt b/connector/avro/benchmarks/AvroWriteBenchmark-results.txt index 1a605c0ea0e90..03fea3f0379f4 100644 --- a/connector/avro/benchmarks/AvroWriteBenchmark-results.txt +++ b/connector/avro/benchmarks/AvroWriteBenchmark-results.txt @@ -1,56 +1,56 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1544 1567 34 10.2 98.1 1.0X -Output Single Double Column 1635 1647 17 9.6 104.0 0.9X -Output Int and String Column 3324 3334 15 4.7 211.3 0.5X -Output Partitions 2961 3047 122 5.3 188.2 0.5X -Output Buckets 3776 3778 3 4.2 240.1 0.4X +Output Single Int Column 1562 1564 3 10.1 99.3 1.0X +Output Single Double Column 1658 1677 27 9.5 105.4 0.9X +Output Int and String Column 3417 3456 55 4.6 217.2 0.5X +Output Partitions 2923 3064 199 5.4 185.8 0.5X +Output Buckets 3769 3772 4 4.2 239.6 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro compression with different codec: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -BZIP2: 130388 131379 1402 0.0 1303881.3 1.0X -DEFLATE: 6523 6538 21 0.0 65227.6 20.0X -UNCOMPRESSED: 5394 5425 43 0.0 53944.9 24.2X -SNAPPY: 4813 4816 3 0.0 48134.6 27.1X -XZ: 54364 54382 26 0.0 543640.7 2.4X -ZSTANDARD: 4864 4873 13 0.0 48635.9 26.8X +BZIP2: 132067 132334 377 0.0 1320668.2 1.0X +DEFLATE: 6456 6466 14 0.0 64562.5 20.5X +UNCOMPRESSED: 5188 5189 2 0.0 51879.6 25.5X +SNAPPY: 4678 4679 2 0.0 46777.8 28.2X +XZ: 42468 42597 183 0.0 424677.8 3.1X +ZSTANDARD: 4796 4801 7 0.0 47963.8 27.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro deflate with different levels: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -DEFLATE: deflate.level=1 4909 4916 9 0.0 49091.1 1.0X -DEFLATE: deflate.level=3 4874 4903 42 0.0 48735.8 1.0X -DEFLATE: deflate.level=5 6460 6473 19 0.0 64601.7 0.8X -DEFLATE: deflate.level=7 6450 6482 46 0.0 64497.5 0.8X -DEFLATE: deflate.level=9 6875 6878 5 0.0 68745.4 0.7X +DEFLATE: deflate.level=1 4736 4751 21 0.0 47356.5 1.0X +DEFLATE: deflate.level=3 4795 4797 4 0.0 47945.1 1.0X +DEFLATE: deflate.level=5 6489 6492 4 0.0 64885.9 0.7X +DEFLATE: deflate.level=7 6464 6484 29 0.0 64640.7 0.7X +DEFLATE: deflate.level=9 6740 6761 30 0.0 67404.6 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro xz with different levels: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -XZ: xz.level=1 12210 12226 22 0.0 122101.7 1.0X -XZ: xz.level=3 22235 22235 0 0.0 222346.3 0.5X -XZ: xz.level=5 47597 47659 88 0.0 475969.7 0.3X -XZ: xz.level=7 69231 69482 356 0.0 692308.3 0.2X -XZ: xz.level=9 147042 148998 2766 0.0 1470415.9 0.1X +XZ: xz.level=1 12053 12062 13 0.0 120526.1 1.0X +XZ: xz.level=3 22766 22771 7 0.0 227656.6 0.5X +XZ: xz.level=5 40993 42080 1538 0.0 409927.7 0.3X +XZ: xz.level=7 64226 64623 562 0.0 642261.7 0.2X +XZ: xz.level=9 143378 145508 3013 0.0 1433775.6 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Avro zstandard with different levels: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------- -ZSTANDARD: zstandard.level=1 4750 4817 94 0.0 47504.2 1.0X -ZSTANDARD: zstandard.level=1, zstandard.bufferPool.enabled=true 4753 4802 69 0.0 47532.8 1.0X -ZSTANDARD: zstandard.level=3 4920 4924 6 0.0 49198.5 1.0X -ZSTANDARD: zstandard.level=3, zstandard.bufferPool.enabled=true 4792 4799 9 0.0 47921.8 1.0X -ZSTANDARD: zstandard.level=5 5240 5276 51 0.0 52404.0 0.9X -ZSTANDARD: zstandard.level=5, zstandard.bufferPool.enabled=true 5072 5101 41 0.0 50722.5 0.9X -ZSTANDARD: zstandard.level=7 5542 5591 69 0.0 55416.5 0.9X -ZSTANDARD: zstandard.level=7, zstandard.bufferPool.enabled=true 5605 5617 17 0.0 56050.4 0.8X -ZSTANDARD: zstandard.level=9 6311 6403 130 0.0 63109.5 0.8X -ZSTANDARD: zstandard.level=9, zstandard.bufferPool.enabled=true 6324 6331 10 0.0 63236.4 0.8X +ZSTANDARD: zstandard.level=1 4816 4828 16 0.0 48164.5 1.0X +ZSTANDARD: zstandard.level=1, zstandard.bufferPool.enabled=true 4669 4875 292 0.0 46692.3 1.0X +ZSTANDARD: zstandard.level=3 4849 4883 48 0.0 48492.9 1.0X +ZSTANDARD: zstandard.level=3, zstandard.bufferPool.enabled=true 4793 4796 4 0.0 47929.4 1.0X +ZSTANDARD: zstandard.level=5 5098 5145 66 0.0 50982.7 0.9X +ZSTANDARD: zstandard.level=5, zstandard.bufferPool.enabled=true 5011 5024 19 0.0 50106.6 1.0X +ZSTANDARD: zstandard.level=7 5480 5502 31 0.0 54796.9 0.9X +ZSTANDARD: zstandard.level=7, zstandard.bufferPool.enabled=true 5459 5469 14 0.0 54591.1 0.9X +ZSTANDARD: zstandard.level=9 6319 6343 34 0.0 63188.9 0.8X +ZSTANDARD: zstandard.level=9, zstandard.bufferPool.enabled=true 6146 6171 35 0.0 61460.8 0.8X diff --git a/core/benchmarks/ChecksumBenchmark-jdk21-results.txt b/core/benchmarks/ChecksumBenchmark-jdk21-results.txt index 8230f15aa659c..9e20379abe1f5 100644 --- a/core/benchmarks/ChecksumBenchmark-jdk21-results.txt +++ b/core/benchmarks/ChecksumBenchmark-jdk21-results.txt @@ -6,8 +6,8 @@ OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Checksum Algorithms: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Adler32 11116 11123 7 0.0 10855585.4 1.0X -CRC32 2774 2777 4 0.0 2709448.1 4.0X -CRC32C 2083 2148 65 0.0 2034177.5 5.3X +Adler32 11109 11110 1 0.0 10848227.5 1.0X +CRC32 2740 2748 7 0.0 2676147.3 4.1X +CRC32C 1824 1837 22 0.0 1781283.4 6.1X diff --git a/core/benchmarks/ChecksumBenchmark-results.txt b/core/benchmarks/ChecksumBenchmark-results.txt index bf4e6182edfd0..5422cabf4b2b7 100644 --- a/core/benchmarks/ChecksumBenchmark-results.txt +++ b/core/benchmarks/ChecksumBenchmark-results.txt @@ -6,8 +6,8 @@ OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Checksum Algorithms: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Adler32 11112 11130 20 0.0 10851949.2 1.0X -CRC32 2765 2767 2 0.0 2699749.0 4.0X -CRC32C 2101 2159 54 0.0 2051565.3 5.3X +Adler32 11113 11117 4 0.0 10852521.2 1.0X +CRC32 2765 2766 1 0.0 2699768.2 4.0X +CRC32C 2003 2033 45 0.0 1955654.6 5.5X diff --git a/core/benchmarks/CoalescedRDDBenchmark-jdk21-results.txt b/core/benchmarks/CoalescedRDDBenchmark-jdk21-results.txt index 1daac7b710bbf..07e8f05a3d185 100644 --- a/core/benchmarks/CoalescedRDDBenchmark-jdk21-results.txt +++ b/core/benchmarks/CoalescedRDDBenchmark-jdk21-results.txt @@ -2,39 +2,39 @@ Coalesced RDD , large scale ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Coalesced RDD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Coalesce Num Partitions: 100 Num Hosts: 1 359 371 12 0.3 3586.9 1.0X -Coalesce Num Partitions: 100 Num Hosts: 5 181 190 14 0.6 1812.1 2.0X -Coalesce Num Partitions: 100 Num Hosts: 10 178 185 6 0.6 1779.9 2.0X -Coalesce Num Partitions: 100 Num Hosts: 20 153 156 4 0.7 1531.2 2.3X -Coalesce Num Partitions: 100 Num Hosts: 40 148 149 1 0.7 1479.1 2.4X -Coalesce Num Partitions: 100 Num Hosts: 80 166 170 5 0.6 1657.8 2.2X -Coalesce Num Partitions: 500 Num Hosts: 1 1054 1064 14 0.1 10543.7 0.3X -Coalesce Num Partitions: 500 Num Hosts: 5 331 339 13 0.3 3311.1 1.1X -Coalesce Num Partitions: 500 Num Hosts: 10 230 235 8 0.4 2295.7 1.6X -Coalesce Num Partitions: 500 Num Hosts: 20 218 220 1 0.5 2182.0 1.6X -Coalesce Num Partitions: 500 Num Hosts: 40 161 164 2 0.6 1614.8 2.2X -Coalesce Num Partitions: 500 Num Hosts: 80 137 142 7 0.7 1371.6 2.6X -Coalesce Num Partitions: 1000 Num Hosts: 1 1926 1929 3 0.1 19264.6 0.2X -Coalesce Num Partitions: 1000 Num Hosts: 5 501 507 10 0.2 5011.1 0.7X -Coalesce Num Partitions: 1000 Num Hosts: 10 327 331 4 0.3 3268.5 1.1X -Coalesce Num Partitions: 1000 Num Hosts: 20 256 264 8 0.4 2556.1 1.4X -Coalesce Num Partitions: 1000 Num Hosts: 40 185 191 7 0.5 1853.2 1.9X -Coalesce Num Partitions: 1000 Num Hosts: 80 160 166 5 0.6 1603.5 2.2X -Coalesce Num Partitions: 5000 Num Hosts: 1 8672 9054 615 0.0 86716.9 0.0X -Coalesce Num Partitions: 5000 Num Hosts: 5 2016 2020 6 0.0 20159.9 0.2X -Coalesce Num Partitions: 5000 Num Hosts: 10 1084 1096 10 0.1 10844.7 0.3X -Coalesce Num Partitions: 5000 Num Hosts: 20 625 636 11 0.2 6245.6 0.6X -Coalesce Num Partitions: 5000 Num Hosts: 40 418 425 6 0.2 4182.3 0.9X -Coalesce Num Partitions: 5000 Num Hosts: 80 270 276 8 0.4 2704.6 1.3X -Coalesce Num Partitions: 10000 Num Hosts: 1 16208 16391 226 0.0 162076.8 0.0X -Coalesce Num Partitions: 10000 Num Hosts: 5 3930 3949 23 0.0 39300.4 0.1X -Coalesce Num Partitions: 10000 Num Hosts: 10 2021 2031 11 0.0 20213.1 0.2X -Coalesce Num Partitions: 10000 Num Hosts: 20 1114 1115 1 0.1 11139.0 0.3X -Coalesce Num Partitions: 10000 Num Hosts: 40 628 639 17 0.2 6275.3 0.6X -Coalesce Num Partitions: 10000 Num Hosts: 80 402 408 10 0.2 4016.4 0.9X +Coalesce Num Partitions: 100 Num Hosts: 1 268 309 37 0.4 2678.2 1.0X +Coalesce Num Partitions: 100 Num Hosts: 5 124 130 5 0.8 1244.1 2.2X +Coalesce Num Partitions: 100 Num Hosts: 10 107 115 7 0.9 1068.7 2.5X +Coalesce Num Partitions: 100 Num Hosts: 20 104 112 6 1.0 1044.6 2.6X +Coalesce Num Partitions: 100 Num Hosts: 40 106 114 10 0.9 1062.8 2.5X +Coalesce Num Partitions: 100 Num Hosts: 80 101 106 5 1.0 1009.3 2.7X +Coalesce Num Partitions: 500 Num Hosts: 1 885 915 34 0.1 8854.5 0.3X +Coalesce Num Partitions: 500 Num Hosts: 5 263 268 5 0.4 2630.1 1.0X +Coalesce Num Partitions: 500 Num Hosts: 10 181 184 3 0.6 1806.0 1.5X +Coalesce Num Partitions: 500 Num Hosts: 20 139 144 8 0.7 1387.0 1.9X +Coalesce Num Partitions: 500 Num Hosts: 40 116 118 2 0.9 1163.7 2.3X +Coalesce Num Partitions: 500 Num Hosts: 80 108 115 7 0.9 1078.8 2.5X +Coalesce Num Partitions: 1000 Num Hosts: 1 1683 1735 79 0.1 16828.2 0.2X +Coalesce Num Partitions: 1000 Num Hosts: 5 446 449 2 0.2 4461.4 0.6X +Coalesce Num Partitions: 1000 Num Hosts: 10 256 262 7 0.4 2562.3 1.0X +Coalesce Num Partitions: 1000 Num Hosts: 20 182 189 5 0.5 1824.3 1.5X +Coalesce Num Partitions: 1000 Num Hosts: 40 141 145 3 0.7 1413.2 1.9X +Coalesce Num Partitions: 1000 Num Hosts: 80 120 126 8 0.8 1203.3 2.2X +Coalesce Num Partitions: 5000 Num Hosts: 1 7913 8247 291 0.0 79127.6 0.0X +Coalesce Num Partitions: 5000 Num Hosts: 5 1818 1846 24 0.1 18177.8 0.1X +Coalesce Num Partitions: 5000 Num Hosts: 10 990 992 2 0.1 9902.6 0.3X +Coalesce Num Partitions: 5000 Num Hosts: 20 543 545 1 0.2 5432.5 0.5X +Coalesce Num Partitions: 5000 Num Hosts: 40 327 337 11 0.3 3272.6 0.8X +Coalesce Num Partitions: 5000 Num Hosts: 80 211 218 6 0.5 2112.1 1.3X +Coalesce Num Partitions: 10000 Num Hosts: 1 14709 15246 580 0.0 147087.6 0.0X +Coalesce Num Partitions: 10000 Num Hosts: 5 3485 3511 27 0.0 34849.5 0.1X +Coalesce Num Partitions: 10000 Num Hosts: 10 1777 1801 22 0.1 17773.8 0.2X +Coalesce Num Partitions: 10000 Num Hosts: 20 952 953 1 0.1 9517.2 0.3X +Coalesce Num Partitions: 10000 Num Hosts: 40 523 533 9 0.2 5229.6 0.5X +Coalesce Num Partitions: 10000 Num Hosts: 80 316 319 5 0.3 3158.4 0.8X diff --git a/core/benchmarks/CoalescedRDDBenchmark-results.txt b/core/benchmarks/CoalescedRDDBenchmark-results.txt index d370e6956116d..520cb5661a276 100644 --- a/core/benchmarks/CoalescedRDDBenchmark-results.txt +++ b/core/benchmarks/CoalescedRDDBenchmark-results.txt @@ -2,39 +2,39 @@ Coalesced RDD , large scale ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Coalesced RDD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Coalesce Num Partitions: 100 Num Hosts: 1 126 145 24 0.8 1257.7 1.0X -Coalesce Num Partitions: 100 Num Hosts: 5 104 105 1 1.0 1039.8 1.2X -Coalesce Num Partitions: 100 Num Hosts: 10 87 89 2 1.1 873.7 1.4X -Coalesce Num Partitions: 100 Num Hosts: 20 89 90 1 1.1 893.4 1.4X -Coalesce Num Partitions: 100 Num Hosts: 40 88 91 6 1.1 880.4 1.4X -Coalesce Num Partitions: 100 Num Hosts: 80 88 94 10 1.1 875.9 1.4X -Coalesce Num Partitions: 500 Num Hosts: 1 308 314 8 0.3 3078.0 0.4X -Coalesce Num Partitions: 500 Num Hosts: 5 133 136 2 0.7 1334.6 0.9X -Coalesce Num Partitions: 500 Num Hosts: 10 112 114 4 0.9 1118.8 1.1X -Coalesce Num Partitions: 500 Num Hosts: 20 100 111 16 1.0 1004.0 1.3X -Coalesce Num Partitions: 500 Num Hosts: 40 100 106 6 1.0 999.7 1.3X -Coalesce Num Partitions: 500 Num Hosts: 80 95 100 6 1.0 954.0 1.3X -Coalesce Num Partitions: 1000 Num Hosts: 1 531 539 14 0.2 5311.0 0.2X -Coalesce Num Partitions: 1000 Num Hosts: 5 197 201 4 0.5 1970.3 0.6X -Coalesce Num Partitions: 1000 Num Hosts: 10 139 141 2 0.7 1392.4 0.9X -Coalesce Num Partitions: 1000 Num Hosts: 20 114 115 1 0.9 1137.9 1.1X -Coalesce Num Partitions: 1000 Num Hosts: 40 105 108 3 0.9 1054.9 1.2X -Coalesce Num Partitions: 1000 Num Hosts: 80 105 109 4 1.0 1047.4 1.2X -Coalesce Num Partitions: 5000 Num Hosts: 1 2336 2354 16 0.0 23362.8 0.1X -Coalesce Num Partitions: 5000 Num Hosts: 5 680 684 4 0.1 6798.7 0.2X -Coalesce Num Partitions: 5000 Num Hosts: 10 381 390 8 0.3 3810.5 0.3X -Coalesce Num Partitions: 5000 Num Hosts: 20 253 255 2 0.4 2529.6 0.5X -Coalesce Num Partitions: 5000 Num Hosts: 40 171 174 3 0.6 1706.1 0.7X -Coalesce Num Partitions: 5000 Num Hosts: 80 137 139 3 0.7 1365.5 0.9X -Coalesce Num Partitions: 10000 Num Hosts: 1 4220 4253 30 0.0 42203.4 0.0X -Coalesce Num Partitions: 10000 Num Hosts: 5 1377 1394 16 0.1 13769.0 0.1X -Coalesce Num Partitions: 10000 Num Hosts: 10 704 717 12 0.1 7036.4 0.2X -Coalesce Num Partitions: 10000 Num Hosts: 20 420 422 1 0.2 4201.7 0.3X -Coalesce Num Partitions: 10000 Num Hosts: 40 267 271 4 0.4 2669.3 0.5X -Coalesce Num Partitions: 10000 Num Hosts: 80 184 191 6 0.5 1842.2 0.7X +Coalesce Num Partitions: 100 Num Hosts: 1 128 134 9 0.8 1278.9 1.0X +Coalesce Num Partitions: 100 Num Hosts: 5 102 103 2 1.0 1016.5 1.3X +Coalesce Num Partitions: 100 Num Hosts: 10 86 87 1 1.2 860.3 1.5X +Coalesce Num Partitions: 100 Num Hosts: 20 87 92 7 1.1 872.0 1.5X +Coalesce Num Partitions: 100 Num Hosts: 40 83 86 3 1.2 829.7 1.5X +Coalesce Num Partitions: 100 Num Hosts: 80 83 84 1 1.2 832.5 1.5X +Coalesce Num Partitions: 500 Num Hosts: 1 306 306 1 0.3 3055.9 0.4X +Coalesce Num Partitions: 500 Num Hosts: 5 128 130 2 0.8 1277.2 1.0X +Coalesce Num Partitions: 500 Num Hosts: 10 106 110 6 0.9 1061.9 1.2X +Coalesce Num Partitions: 500 Num Hosts: 20 95 96 1 1.1 950.4 1.3X +Coalesce Num Partitions: 500 Num Hosts: 40 92 94 4 1.1 918.5 1.4X +Coalesce Num Partitions: 500 Num Hosts: 80 87 88 1 1.1 871.4 1.5X +Coalesce Num Partitions: 1000 Num Hosts: 1 523 529 9 0.2 5229.5 0.2X +Coalesce Num Partitions: 1000 Num Hosts: 5 185 189 3 0.5 1853.1 0.7X +Coalesce Num Partitions: 1000 Num Hosts: 10 128 131 3 0.8 1278.9 1.0X +Coalesce Num Partitions: 1000 Num Hosts: 20 106 108 3 0.9 1057.8 1.2X +Coalesce Num Partitions: 1000 Num Hosts: 40 97 97 1 1.0 968.2 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 80 93 98 8 1.1 931.4 1.4X +Coalesce Num Partitions: 5000 Num Hosts: 1 2321 2328 11 0.0 23205.2 0.1X +Coalesce Num Partitions: 5000 Num Hosts: 5 674 680 5 0.1 6741.0 0.2X +Coalesce Num Partitions: 5000 Num Hosts: 10 374 378 7 0.3 3738.4 0.3X +Coalesce Num Partitions: 5000 Num Hosts: 20 232 238 6 0.4 2316.6 0.6X +Coalesce Num Partitions: 5000 Num Hosts: 40 163 166 3 0.6 1630.1 0.8X +Coalesce Num Partitions: 5000 Num Hosts: 80 127 129 2 0.8 1274.8 1.0X +Coalesce Num Partitions: 10000 Num Hosts: 1 4228 4243 18 0.0 42280.5 0.0X +Coalesce Num Partitions: 10000 Num Hosts: 5 1387 1400 13 0.1 13870.3 0.1X +Coalesce Num Partitions: 10000 Num Hosts: 10 711 714 4 0.1 7105.0 0.2X +Coalesce Num Partitions: 10000 Num Hosts: 20 401 408 7 0.2 4010.4 0.3X +Coalesce Num Partitions: 10000 Num Hosts: 40 251 253 2 0.4 2513.0 0.5X +Coalesce Num Partitions: 10000 Num Hosts: 80 175 182 8 0.6 1754.8 0.7X diff --git a/core/benchmarks/KryoBenchmark-jdk21-results.txt b/core/benchmarks/KryoBenchmark-jdk21-results.txt index aee420e8ca26a..4cc1e4dd2ba76 100644 --- a/core/benchmarks/KryoBenchmark-jdk21-results.txt +++ b/core/benchmarks/KryoBenchmark-jdk21-results.txt @@ -2,27 +2,27 @@ Benchmark Kryo Unsafe vs safe Serialization ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark Kryo Unsafe vs safe Serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -basicTypes: Int with unsafe:true 174 176 1 5.7 174.3 1.0X -basicTypes: Long with unsafe:true 178 184 5 5.6 178.1 1.0X -basicTypes: Float with unsafe:true 185 187 1 5.4 185.3 0.9X -basicTypes: Double with unsafe:true 187 189 1 5.3 187.0 0.9X -Array: Int with unsafe:true 1 1 0 752.1 1.3 131.1X -Array: Long with unsafe:true 2 2 0 490.6 2.0 85.5X -Array: Float with unsafe:true 1 1 0 757.7 1.3 132.1X -Array: Double with unsafe:true 2 2 0 483.9 2.1 84.4X -Map of string->Double with unsafe:true 26 26 2 38.5 26.0 6.7X -basicTypes: Int with unsafe:false 206 207 1 4.9 205.7 0.8X -basicTypes: Long with unsafe:false 222 223 1 4.5 221.7 0.8X -basicTypes: Float with unsafe:false 217 218 1 4.6 216.5 0.8X -basicTypes: Double with unsafe:false 217 218 2 4.6 216.6 0.8X -Array: Int with unsafe:false 13 13 0 79.5 12.6 13.9X -Array: Long with unsafe:false 21 22 0 46.6 21.4 8.1X -Array: Float with unsafe:false 6 6 0 167.8 6.0 29.3X -Array: Double with unsafe:false 16 16 0 64.2 15.6 11.2X -Map of string->Double with unsafe:false 28 28 1 36.3 27.5 6.3X +basicTypes: Int with unsafe:true 167 168 1 6.0 167.0 1.0X +basicTypes: Long with unsafe:true 174 178 2 5.7 174.1 1.0X +basicTypes: Float with unsafe:true 203 204 1 4.9 202.9 0.8X +basicTypes: Double with unsafe:true 206 207 1 4.9 206.1 0.8X +Array: Int with unsafe:true 1 1 0 768.6 1.3 128.4X +Array: Long with unsafe:true 2 2 0 502.0 2.0 83.9X +Array: Float with unsafe:true 1 1 0 773.6 1.3 129.2X +Array: Double with unsafe:true 2 2 0 492.6 2.0 82.3X +Map of string->Double with unsafe:true 27 27 1 37.5 26.6 6.3X +basicTypes: Int with unsafe:false 198 199 1 5.1 197.9 0.8X +basicTypes: Long with unsafe:false 217 219 2 4.6 216.8 0.8X +basicTypes: Float with unsafe:false 201 203 2 5.0 201.0 0.8X +basicTypes: Double with unsafe:false 202 204 1 5.0 201.9 0.8X +Array: Int with unsafe:false 13 13 0 79.7 12.5 13.3X +Array: Long with unsafe:false 20 21 0 49.1 20.4 8.2X +Array: Float with unsafe:false 7 8 0 134.4 7.4 22.5X +Array: Double with unsafe:false 11 12 0 87.2 11.5 14.6X +Map of string->Double with unsafe:false 28 28 1 36.3 27.5 6.1X diff --git a/core/benchmarks/KryoBenchmark-results.txt b/core/benchmarks/KryoBenchmark-results.txt index ca80b13a5346d..6c46724fbd2e8 100644 --- a/core/benchmarks/KryoBenchmark-results.txt +++ b/core/benchmarks/KryoBenchmark-results.txt @@ -2,27 +2,27 @@ Benchmark Kryo Unsafe vs safe Serialization ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark Kryo Unsafe vs safe Serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -basicTypes: Int with unsafe:true 171 173 2 5.9 170.8 1.0X -basicTypes: Long with unsafe:true 190 193 3 5.3 189.6 0.9X -basicTypes: Float with unsafe:true 186 188 2 5.4 186.1 0.9X -basicTypes: Double with unsafe:true 189 190 1 5.3 188.5 0.9X -Array: Int with unsafe:true 1 2 0 720.0 1.4 123.0X -Array: Long with unsafe:true 2 3 0 462.4 2.2 79.0X -Array: Float with unsafe:true 1 2 0 719.2 1.4 122.9X -Array: Double with unsafe:true 2 3 0 459.8 2.2 78.5X -Map of string->Double with unsafe:true 27 28 1 37.2 26.9 6.3X -basicTypes: Int with unsafe:false 219 220 1 4.6 218.7 0.8X -basicTypes: Long with unsafe:false 242 244 2 4.1 242.1 0.7X -basicTypes: Float with unsafe:false 215 220 10 4.7 214.8 0.8X -basicTypes: Double with unsafe:false 222 224 2 4.5 221.7 0.8X -Array: Int with unsafe:false 15 15 0 66.9 15.0 11.4X -Array: Long with unsafe:false 22 22 0 45.9 21.8 7.8X -Array: Float with unsafe:false 6 6 1 170.1 5.9 29.1X -Array: Double with unsafe:false 10 10 0 103.0 9.7 17.6X -Map of string->Double with unsafe:false 31 32 2 32.4 30.9 5.5X +basicTypes: Int with unsafe:true 167 169 1 6.0 167.5 1.0X +basicTypes: Long with unsafe:true 189 195 3 5.3 188.7 0.9X +basicTypes: Float with unsafe:true 188 192 5 5.3 187.8 0.9X +basicTypes: Double with unsafe:true 190 192 3 5.3 189.7 0.9X +Array: Int with unsafe:true 1 1 0 734.5 1.4 123.0X +Array: Long with unsafe:true 2 2 0 478.3 2.1 80.1X +Array: Float with unsafe:true 1 1 0 736.7 1.4 123.4X +Array: Double with unsafe:true 2 2 0 475.5 2.1 79.6X +Map of string->Double with unsafe:true 27 27 0 37.5 26.7 6.3X +basicTypes: Int with unsafe:false 210 211 2 4.8 210.0 0.8X +basicTypes: Long with unsafe:false 224 225 1 4.5 224.4 0.7X +basicTypes: Float with unsafe:false 203 204 1 4.9 203.4 0.8X +basicTypes: Double with unsafe:false 210 212 1 4.8 210.0 0.8X +Array: Int with unsafe:false 15 15 0 68.2 14.7 11.4X +Array: Long with unsafe:false 20 21 0 49.2 20.3 8.2X +Array: Float with unsafe:false 6 6 0 167.7 6.0 28.1X +Array: Double with unsafe:false 10 10 2 99.3 10.1 16.6X +Map of string->Double with unsafe:false 28 29 1 35.1 28.4 5.9X diff --git a/core/benchmarks/KryoIteratorBenchmark-jdk21-results.txt b/core/benchmarks/KryoIteratorBenchmark-jdk21-results.txt index e3922382068dd..36124a13c29d1 100644 --- a/core/benchmarks/KryoIteratorBenchmark-jdk21-results.txt +++ b/core/benchmarks/KryoIteratorBenchmark-jdk21-results.txt @@ -2,27 +2,27 @@ Benchmark of kryo asIterator on deserialization stream ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark of kryo asIterator on deserialization stream: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------------- -Colletion of int with 1 elements, useIterator: true 6 6 0 1.7 590.7 1.0X -Colletion of int with 10 elements, useIterator: true 14 15 1 0.7 1431.1 0.4X -Colletion of int with 100 elements, useIterator: true 94 96 1 0.1 9429.2 0.1X -Colletion of string with 1 elements, useIterator: true 8 8 0 1.3 760.8 0.8X -Colletion of string with 10 elements, useIterator: true 22 23 0 0.4 2246.1 0.3X -Colletion of string with 100 elements, useIterator: true 167 167 1 0.1 16659.4 0.0X -Colletion of Array[int] with 1 elements, useIterator: true 7 8 0 1.4 735.4 0.8X -Colletion of Array[int] with 10 elements, useIterator: true 20 20 1 0.5 1976.5 0.3X -Colletion of Array[int] with 100 elements, useIterator: true 154 154 1 0.1 15356.7 0.0X -Colletion of int with 1 elements, useIterator: false 6 7 0 1.6 623.6 0.9X -Colletion of int with 10 elements, useIterator: false 13 14 0 0.7 1334.3 0.4X -Colletion of int with 100 elements, useIterator: false 82 82 0 0.1 8164.4 0.1X -Colletion of string with 1 elements, useIterator: false 7 8 0 1.4 727.0 0.8X -Colletion of string with 10 elements, useIterator: false 22 22 0 0.5 2166.5 0.3X -Colletion of string with 100 elements, useIterator: false 159 160 0 0.1 15925.0 0.0X -Colletion of Array[int] with 1 elements, useIterator: false 7 7 0 1.4 712.1 0.8X -Colletion of Array[int] with 10 elements, useIterator: false 19 20 0 0.5 1932.3 0.3X -Colletion of Array[int] with 100 elements, useIterator: false 142 143 1 0.1 14220.2 0.0X +Colletion of int with 1 elements, useIterator: true 6 6 0 1.6 625.1 1.0X +Colletion of int with 10 elements, useIterator: true 15 15 0 0.7 1466.8 0.4X +Colletion of int with 100 elements, useIterator: true 95 96 1 0.1 9536.2 0.1X +Colletion of string with 1 elements, useIterator: true 8 8 0 1.3 771.9 0.8X +Colletion of string with 10 elements, useIterator: true 23 23 0 0.4 2260.7 0.3X +Colletion of string with 100 elements, useIterator: true 162 162 0 0.1 16179.1 0.0X +Colletion of Array[int] with 1 elements, useIterator: true 7 8 1 1.4 730.4 0.9X +Colletion of Array[int] with 10 elements, useIterator: true 20 20 1 0.5 1966.8 0.3X +Colletion of Array[int] with 100 elements, useIterator: true 146 147 1 0.1 14593.4 0.0X +Colletion of int with 1 elements, useIterator: false 6 7 0 1.6 636.6 1.0X +Colletion of int with 10 elements, useIterator: false 14 14 0 0.7 1366.4 0.5X +Colletion of int with 100 elements, useIterator: false 84 85 1 0.1 8439.6 0.1X +Colletion of string with 1 elements, useIterator: false 7 7 0 1.4 725.5 0.9X +Colletion of string with 10 elements, useIterator: false 21 22 0 0.5 2115.1 0.3X +Colletion of string with 100 elements, useIterator: false 173 174 3 0.1 17316.2 0.0X +Colletion of Array[int] with 1 elements, useIterator: false 7 7 0 1.4 698.9 0.9X +Colletion of Array[int] with 10 elements, useIterator: false 19 19 0 0.5 1894.5 0.3X +Colletion of Array[int] with 100 elements, useIterator: false 141 142 1 0.1 14108.1 0.0X diff --git a/core/benchmarks/KryoIteratorBenchmark-results.txt b/core/benchmarks/KryoIteratorBenchmark-results.txt index 77452144ac01d..6c3496909c6b7 100644 --- a/core/benchmarks/KryoIteratorBenchmark-results.txt +++ b/core/benchmarks/KryoIteratorBenchmark-results.txt @@ -2,27 +2,27 @@ Benchmark of kryo asIterator on deserialization stream ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark of kryo asIterator on deserialization stream: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------------- -Colletion of int with 1 elements, useIterator: true 6 6 0 1.6 621.6 1.0X -Colletion of int with 10 elements, useIterator: true 14 14 0 0.7 1422.5 0.4X -Colletion of int with 100 elements, useIterator: true 90 92 1 0.1 9030.9 0.1X -Colletion of string with 1 elements, useIterator: true 7 8 0 1.4 726.3 0.9X -Colletion of string with 10 elements, useIterator: true 23 23 1 0.4 2251.0 0.3X -Colletion of string with 100 elements, useIterator: true 172 172 0 0.1 17183.3 0.0X -Colletion of Array[int] with 1 elements, useIterator: true 7 7 0 1.4 718.6 0.9X -Colletion of Array[int] with 10 elements, useIterator: true 21 21 1 0.5 2078.4 0.3X -Colletion of Array[int] with 100 elements, useIterator: true 162 162 0 0.1 16189.6 0.0X -Colletion of int with 1 elements, useIterator: false 6 6 0 1.7 599.6 1.0X -Colletion of int with 10 elements, useIterator: false 13 13 0 0.8 1313.8 0.5X -Colletion of int with 100 elements, useIterator: false 81 82 0 0.1 8132.1 0.1X -Colletion of string with 1 elements, useIterator: false 7 7 0 1.4 705.6 0.9X -Colletion of string with 10 elements, useIterator: false 22 23 0 0.4 2240.1 0.3X -Colletion of string with 100 elements, useIterator: false 170 170 0 0.1 16995.3 0.0X -Colletion of Array[int] with 1 elements, useIterator: false 7 7 0 1.5 675.7 0.9X -Colletion of Array[int] with 10 elements, useIterator: false 18 19 0 0.5 1842.7 0.3X -Colletion of Array[int] with 100 elements, useIterator: false 138 139 0 0.1 13801.7 0.0X +Colletion of int with 1 elements, useIterator: true 6 7 0 1.5 646.6 1.0X +Colletion of int with 10 elements, useIterator: true 14 14 0 0.7 1354.6 0.5X +Colletion of int with 100 elements, useIterator: true 82 82 0 0.1 8169.3 0.1X +Colletion of string with 1 elements, useIterator: true 8 8 0 1.3 777.8 0.8X +Colletion of string with 10 elements, useIterator: true 22 23 1 0.4 2237.2 0.3X +Colletion of string with 100 elements, useIterator: true 161 161 1 0.1 16071.4 0.0X +Colletion of Array[int] with 1 elements, useIterator: true 7 8 0 1.4 726.7 0.9X +Colletion of Array[int] with 10 elements, useIterator: true 20 20 0 0.5 1984.8 0.3X +Colletion of Array[int] with 100 elements, useIterator: true 151 151 0 0.1 15059.2 0.0X +Colletion of int with 1 elements, useIterator: false 6 6 0 1.6 609.0 1.1X +Colletion of int with 10 elements, useIterator: false 13 14 0 0.8 1322.7 0.5X +Colletion of int with 100 elements, useIterator: false 81 82 1 0.1 8138.4 0.1X +Colletion of string with 1 elements, useIterator: false 7 8 0 1.4 732.7 0.9X +Colletion of string with 10 elements, useIterator: false 23 23 0 0.4 2254.4 0.3X +Colletion of string with 100 elements, useIterator: false 171 173 4 0.1 17050.7 0.0X +Colletion of Array[int] with 1 elements, useIterator: false 7 7 0 1.4 705.9 0.9X +Colletion of Array[int] with 10 elements, useIterator: false 20 20 0 0.5 1974.9 0.3X +Colletion of Array[int] with 100 elements, useIterator: false 147 148 1 0.1 14730.9 0.0X diff --git a/core/benchmarks/KryoSerializerBenchmark-jdk21-results.txt b/core/benchmarks/KryoSerializerBenchmark-jdk21-results.txt index c00cd9152b278..14649cd560327 100644 --- a/core/benchmarks/KryoSerializerBenchmark-jdk21-results.txt +++ b/core/benchmarks/KryoSerializerBenchmark-jdk21-results.txt @@ -2,11 +2,11 @@ Benchmark KryoPool vs old"pool of 1" implementation ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark KryoPool vs old"pool of 1" implementation: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -KryoPool:true 4166 5737 1977 0.0 8331992.4 1.0X -KryoPool:false 6201 7778 1281 0.0 12402118.8 0.7X +KryoPool:true 3445 5067 1740 0.0 6889852.4 1.0X +KryoPool:false 5594 7457 1439 0.0 11188845.8 0.6X diff --git a/core/benchmarks/KryoSerializerBenchmark-results.txt b/core/benchmarks/KryoSerializerBenchmark-results.txt index a86338957cc37..c08bbfebe993d 100644 --- a/core/benchmarks/KryoSerializerBenchmark-results.txt +++ b/core/benchmarks/KryoSerializerBenchmark-results.txt @@ -2,11 +2,11 @@ Benchmark KryoPool vs old"pool of 1" implementation ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark KryoPool vs old"pool of 1" implementation: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -KryoPool:true 3973 5797 1879 0.0 7945107.3 1.0X -KryoPool:false 6041 7623 1484 0.0 12082153.5 0.7X +KryoPool:true 3409 5129 1620 0.0 6817249.8 1.0X +KryoPool:false 5506 7416 1256 0.0 11011835.6 0.6X diff --git a/core/benchmarks/LZFBenchmark-jdk21-results.txt b/core/benchmarks/LZFBenchmark-jdk21-results.txt index 7104879c5c753..1f39e58139e65 100644 --- a/core/benchmarks/LZFBenchmark-jdk21-results.txt +++ b/core/benchmarks/LZFBenchmark-jdk21-results.txt @@ -2,18 +2,18 @@ Benchmark LZFCompressionCodec ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Compress small objects: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Compression 256000000 int values in parallel 599 601 4 427.4 2.3 1.0X -Compression 256000000 int values single-threaded 608 615 7 420.9 2.4 1.0X +Compression 256000000 int values in parallel 605 611 5 423.4 2.4 1.0X +Compression 256000000 int values single-threaded 612 619 5 418.5 2.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Compress large objects: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Compression 1024 array values in 1 threads 37 45 5 0.0 35857.3 1.0X -Compression 1024 array values single-threaded 31 31 0 0.0 30334.5 1.2X +Compression 1024 array values in 1 threads 44 48 3 0.0 43323.6 1.0X +Compression 1024 array values single-threaded 32 32 0 0.0 30772.9 1.4X diff --git a/core/benchmarks/LZFBenchmark-results.txt b/core/benchmarks/LZFBenchmark-results.txt index 142d3aad2f1ba..92d8ba52412df 100644 --- a/core/benchmarks/LZFBenchmark-results.txt +++ b/core/benchmarks/LZFBenchmark-results.txt @@ -2,18 +2,18 @@ Benchmark LZFCompressionCodec ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Compress small objects: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Compression 256000000 int values in parallel 601 610 9 426.3 2.3 1.0X -Compression 256000000 int values single-threaded 610 619 7 419.3 2.4 1.0X +Compression 256000000 int values in parallel 598 608 7 428.2 2.3 1.0X +Compression 256000000 int values single-threaded 615 623 6 416.1 2.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Compress large objects: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Compression 1024 array values in 1 threads 35 44 5 0.0 34512.1 1.0X -Compression 1024 array values single-threaded 31 32 1 0.0 30396.7 1.1X +Compression 1024 array values in 1 threads 39 46 5 0.0 37798.8 1.0X +Compression 1024 array values single-threaded 31 32 0 0.0 29960.9 1.3X diff --git a/core/benchmarks/MapStatusesConvertBenchmark-jdk21-results.txt b/core/benchmarks/MapStatusesConvertBenchmark-jdk21-results.txt index 7c1b1eb4ac803..123a40fad3e62 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.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-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: 50000 Fetch partitions:500 696 705 13 0.0 696430567.0 1.0X -Num Maps: 50000 Fetch partitions:1000 1620 1628 7 0.0 1620094001.0 0.4X -Num Maps: 50000 Fetch partitions:1500 2507 2522 13 0.0 2507485825.0 0.3X +Num Maps: 50000 Fetch partitions:500 716 730 14 0.0 715747604.0 1.0X +Num Maps: 50000 Fetch partitions:1000 1592 1619 29 0.0 1591519021.0 0.4X +Num Maps: 50000 Fetch partitions:1500 2500 2507 7 0.0 2499934291.0 0.3X diff --git a/core/benchmarks/MapStatusesConvertBenchmark-results.txt b/core/benchmarks/MapStatusesConvertBenchmark-results.txt index 4ca2e502b9404..ef390204bb0af 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.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-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: 50000 Fetch partitions:500 775 782 8 0.0 774584162.0 1.0X -Num Maps: 50000 Fetch partitions:1000 1605 1634 29 0.0 1604801022.0 0.5X -Num Maps: 50000 Fetch partitions:1500 2568 2585 22 0.0 2568404459.0 0.3X +Num Maps: 50000 Fetch partitions:500 612 614 3 0.0 611543498.0 1.0X +Num Maps: 50000 Fetch partitions:1000 1389 1398 15 0.0 1388971632.0 0.4X +Num Maps: 50000 Fetch partitions:1500 2178 2222 39 0.0 2177711722.0 0.3X diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk21-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk21-results.txt index 1ffe7594c22cd..708bf8f770d40 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk21-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk21-results.txt @@ -1,64 +1,64 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 81 85 4 2.5 406.5 1.0X -Deserialization 147 155 9 1.4 734.2 0.6X +Serialization 84 87 3 2.4 422.4 1.0X +Deserialization 143 151 6 1.4 712.6 0.6X Compressed Serialized MapStatus sizes: 426.0 B Compressed Serialized Broadcast MapStatus sizes: 2.5 MiB -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Serialization 73 75 3 2.7 365.9 1.0X -Deserialization 146 153 10 1.4 732.1 0.5X +Serialization 83 85 2 2.4 414.3 1.0X +Deserialization 141 145 7 1.4 703.0 0.6X Compressed Serialized MapStatus sizes: 2.5 MiB Compressed Serialized Broadcast MapStatus sizes: 0.0 B -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Serialization 152 157 6 1.3 759.6 1.0X -Deserialization 162 166 4 1.2 811.8 0.9X +Serialization 154 160 9 1.3 770.3 1.0X +Deserialization 158 164 10 1.3 788.4 1.0X Compressed Serialized MapStatus sizes: 442.0 B Compressed Serialized Broadcast MapStatus sizes: 13.6 MiB -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Serialization 141 142 1 1.4 703.4 1.0X -Deserialization 161 165 5 1.2 807.0 0.9X +Serialization 145 146 1 1.4 724.1 1.0X +Deserialization 158 162 7 1.3 790.2 0.9X Compressed Serialized MapStatus sizes: 13.6 MiB Compressed Serialized Broadcast MapStatus sizes: 0.0 B -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Serialization 686 714 31 0.3 3431.6 1.0X -Deserialization 329 346 18 0.6 1645.6 2.1X +Serialization 693 722 42 0.3 3463.5 1.0X +Deserialization 330 357 25 0.6 1648.3 2.1X -Compressed Serialized MapStatus sizes: 569.0 B +Compressed Serialized MapStatus sizes: 568.0 B Compressed Serialized Broadcast MapStatus sizes: 122.3 MiB -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Serialization 560 562 3 0.4 2797.7 1.0X -Deserialization 317 334 20 0.6 1587.0 1.8X +Serialization 569 573 3 0.4 2845.7 1.0X +Deserialization 330 350 17 0.6 1647.8 1.7X Compressed Serialized MapStatus sizes: 122.3 MiB Compressed Serialized Broadcast MapStatus sizes: 0.0 B diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt index edd6207a12f8b..6e69a91cbafdb 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt @@ -1,64 +1,64 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Serialization 93 99 8 2.1 466.1 1.0X -Deserialization 140 151 12 1.4 698.8 0.7X +Serialization 88 92 3 2.3 442.0 1.0X +Deserialization 138 143 10 1.5 688.2 0.6X Compressed Serialized MapStatus sizes: 426.0 B Compressed Serialized Broadcast MapStatus sizes: 2.5 MiB -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Serialization 82 83 1 2.4 409.0 1.0X -Deserialization 139 142 8 1.4 692.8 0.6X +Serialization 77 79 1 2.6 386.9 1.0X +Deserialization 137 140 5 1.5 685.8 0.6X Compressed Serialized MapStatus sizes: 2.5 MiB Compressed Serialized Broadcast MapStatus sizes: 0.0 B -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Serialization 161 168 7 1.2 802.6 1.0X -Deserialization 155 169 13 1.3 777.3 1.0X +Serialization 159 160 1 1.3 793.9 1.0X +Deserialization 154 160 9 1.3 770.9 1.0X Compressed Serialized MapStatus sizes: 442.0 B Compressed Serialized Broadcast MapStatus sizes: 13.6 MiB -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Serialization 145 147 1 1.4 726.2 1.0X -Deserialization 155 160 10 1.3 772.7 0.9X +Serialization 146 147 1 1.4 730.8 1.0X +Deserialization 154 157 3 1.3 772.4 0.9X Compressed Serialized MapStatus sizes: 13.6 MiB Compressed Serialized Broadcast MapStatus sizes: 0.0 B -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Serialization 693 714 23 0.3 3465.9 1.0X -Deserialization 326 351 13 0.6 1628.7 2.1X +Serialization 697 702 9 0.3 3483.3 1.0X +Deserialization 317 323 7 0.6 1583.0 2.2X -Compressed Serialized MapStatus sizes: 568.0 B +Compressed Serialized MapStatus sizes: 569.0 B Compressed Serialized Broadcast MapStatus sizes: 122.3 MiB -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Serialization 564 576 13 0.4 2817.6 1.0X -Deserialization 339 354 9 0.6 1694.4 1.7X +Serialization 568 577 7 0.4 2842.2 1.0X +Deserialization 308 316 7 0.6 1540.4 1.8X Compressed Serialized MapStatus sizes: 122.3 MiB Compressed Serialized Broadcast MapStatus sizes: 0.0 B diff --git a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt index 7262ea63a6ef9..c91af8730b49c 100644 --- a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt +++ b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt @@ -2,17 +2,17 @@ PersistenceEngineBenchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 1000 Workers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -ZooKeeperPersistenceEngine with JavaSerializer 5620 5811 236 0.0 5619587.2 1.0X -FileSystemPersistenceEngine with JavaSerializer 2876 2924 42 0.0 2876068.8 2.0X -FileSystemPersistenceEngine with JavaSerializer (lz4) 825 829 4 0.0 824880.6 6.8X -FileSystemPersistenceEngine with JavaSerializer (lzf) 742 774 39 0.0 742492.3 7.6X -FileSystemPersistenceEngine with JavaSerializer (snappy) 785 832 42 0.0 784738.0 7.2X -FileSystemPersistenceEngine with JavaSerializer (zstd) 966 982 14 0.0 965925.8 5.8X -RocksDBPersistenceEngine with JavaSerializer 299 301 2 0.0 299470.1 18.8X -BlackHolePersistenceEngine 0 0 0 6.0 166.6 33740.5X +ZooKeeperPersistenceEngine with JavaSerializer 7133 7390 257 0.0 7132665.6 1.0X +FileSystemPersistenceEngine with JavaSerializer 2449 2470 22 0.0 2448714.2 2.9X +FileSystemPersistenceEngine with JavaSerializer (lz4) 784 805 19 0.0 783603.9 9.1X +FileSystemPersistenceEngine with JavaSerializer (lzf) 719 763 52 0.0 719310.0 9.9X +FileSystemPersistenceEngine with JavaSerializer (snappy) 731 765 42 0.0 731346.6 9.8X +FileSystemPersistenceEngine with JavaSerializer (zstd) 920 971 52 0.0 919508.6 7.8X +RocksDBPersistenceEngine with JavaSerializer 283 284 1 0.0 282641.0 25.2X +BlackHolePersistenceEngine 0 0 0 6.0 167.4 42612.8X diff --git a/core/benchmarks/PersistenceEngineBenchmark-results.txt b/core/benchmarks/PersistenceEngineBenchmark-results.txt index c373d88842d2e..14ca05abad071 100644 --- a/core/benchmarks/PersistenceEngineBenchmark-results.txt +++ b/core/benchmarks/PersistenceEngineBenchmark-results.txt @@ -2,17 +2,17 @@ PersistenceEngineBenchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 1000 Workers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -ZooKeeperPersistenceEngine with JavaSerializer 6146 6314 215 0.0 6146007.1 1.0X -FileSystemPersistenceEngine with JavaSerializer 2944 2957 17 0.0 2944099.7 2.1X -FileSystemPersistenceEngine with JavaSerializer (lz4) 827 869 37 0.0 827379.0 7.4X -FileSystemPersistenceEngine with JavaSerializer (lzf) 799 826 25 0.0 799318.1 7.7X -FileSystemPersistenceEngine with JavaSerializer (snappy) 775 805 50 0.0 774802.8 7.9X -FileSystemPersistenceEngine with JavaSerializer (zstd) 972 1002 28 0.0 971773.9 6.3X -RocksDBPersistenceEngine with JavaSerializer 310 312 3 0.0 310401.2 19.8X -BlackHolePersistenceEngine 0 0 0 6.0 165.7 37097.7X +ZooKeeperPersistenceEngine with JavaSerializer 6582 6738 184 0.0 6581975.7 1.0X +FileSystemPersistenceEngine with JavaSerializer 2493 2507 12 0.0 2492854.1 2.6X +FileSystemPersistenceEngine with JavaSerializer (lz4) 784 827 40 0.0 783848.3 8.4X +FileSystemPersistenceEngine with JavaSerializer (lzf) 755 774 17 0.0 755155.3 8.7X +FileSystemPersistenceEngine with JavaSerializer (snappy) 739 786 49 0.0 739163.8 8.9X +FileSystemPersistenceEngine with JavaSerializer (zstd) 956 988 33 0.0 955958.8 6.9X +RocksDBPersistenceEngine with JavaSerializer 290 295 7 0.0 289554.4 22.7X +BlackHolePersistenceEngine 0 0 0 6.2 161.8 40674.2X diff --git a/core/benchmarks/PropertiesCloneBenchmark-jdk21-results.txt b/core/benchmarks/PropertiesCloneBenchmark-jdk21-results.txt index ccae104413f6a..cfff77298896a 100644 --- a/core/benchmarks/PropertiesCloneBenchmark-jdk21-results.txt +++ b/core/benchmarks/PropertiesCloneBenchmark-jdk21-results.txt @@ -2,39 +2,39 @@ Properties Cloning ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Empty Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.3 3296.0 1.0X -Utils.cloneProperties 0 0 0 34.5 29.0 113.7X +SerializationUtils.clone 0 0 0 0.3 3146.0 1.0X +Utils.cloneProperties 0 0 0 11.2 89.0 35.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor System Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.0 156331.0 1.0X -Utils.cloneProperties 0 0 0 0.4 2595.0 60.2X +SerializationUtils.clone 0 0 0 0.0 158717.0 1.0X +Utils.cloneProperties 0 0 0 0.2 4819.0 32.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Small Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.0 255154.0 1.0X -Utils.cloneProperties 0 0 0 0.4 2624.0 97.2X +SerializationUtils.clone 0 0 0 0.0 241783.0 1.0X +Utils.cloneProperties 0 0 0 0.2 6051.0 40.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Medium Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 1 1 0 0.0 887239.0 1.0X -Utils.cloneProperties 0 0 0 0.1 14908.0 59.5X +SerializationUtils.clone 1 1 0 0.0 853297.0 1.0X +Utils.cloneProperties 0 0 0 0.0 30927.0 27.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Large Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 2 2 0 0.0 1655264.0 1.0X -Utils.cloneProperties 0 0 0 0.0 30837.0 53.7X +SerializationUtils.clone 2 2 0 0.0 1598481.0 1.0X +Utils.cloneProperties 0 0 0 0.0 63448.0 25.2X diff --git a/core/benchmarks/PropertiesCloneBenchmark-results.txt b/core/benchmarks/PropertiesCloneBenchmark-results.txt index f6c6c8781dc25..cceb3e8710dee 100644 --- a/core/benchmarks/PropertiesCloneBenchmark-results.txt +++ b/core/benchmarks/PropertiesCloneBenchmark-results.txt @@ -2,39 +2,39 @@ Properties Cloning ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Empty Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.3 3466.0 1.0X -Utils.cloneProperties 0 0 0 34.5 29.0 119.5X +SerializationUtils.clone 0 0 0 0.3 3186.0 1.0X +Utils.cloneProperties 0 0 0 11.1 90.0 35.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor System Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.0 156422.0 1.0X -Utils.cloneProperties 0 0 0 0.4 2685.0 58.3X +SerializationUtils.clone 0 0 0 0.0 175435.0 1.0X +Utils.cloneProperties 0 0 0 0.2 4247.0 41.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Small Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 0 0 0 0.0 277017.0 1.0X -Utils.cloneProperties 0 0 0 0.3 3666.0 75.6X +SerializationUtils.clone 0 0 0 0.0 255744.0 1.0X +Utils.cloneProperties 0 0 0 0.1 7273.0 35.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Medium Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 1 1 0 0.0 920141.0 1.0X -Utils.cloneProperties 0 0 0 0.0 20097.0 45.8X +SerializationUtils.clone 1 1 0 0.0 863683.0 1.0X +Utils.cloneProperties 0 0 0 0.0 36508.0 23.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Large Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SerializationUtils.clone 2 2 0 0.0 1714798.0 1.0X -Utils.cloneProperties 0 0 0 0.0 40385.0 42.5X +SerializationUtils.clone 2 2 0 0.0 1612893.0 1.0X +Utils.cloneProperties 0 0 0 0.0 73617.0 21.9X diff --git a/core/benchmarks/XORShiftRandomBenchmark-jdk21-results.txt b/core/benchmarks/XORShiftRandomBenchmark-jdk21-results.txt index 9f2baa5d9bf80..4b892b7ea2c85 100644 --- a/core/benchmarks/XORShiftRandomBenchmark-jdk21-results.txt +++ b/core/benchmarks/XORShiftRandomBenchmark-jdk21-results.txt @@ -2,43 +2,43 @@ Pseudo random ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextInt: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 451 451 0 222.0 4.5 1.0X -XORShiftRandom 185 185 0 539.4 1.9 2.4X +java.util.Random 453 453 0 220.7 4.5 1.0X +XORShiftRandom 186 186 0 536.6 1.9 2.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextLong: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 897 897 0 111.5 9.0 1.0X -XORShiftRandom 371 371 0 269.5 3.7 2.4X +java.util.Random 900 901 1 111.1 9.0 1.0X +XORShiftRandom 373 373 1 268.1 3.7 2.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextDouble: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 914 914 0 109.4 9.1 1.0X -XORShiftRandom 371 371 1 269.5 3.7 2.5X +java.util.Random 905 905 0 110.5 9.0 1.0X +XORShiftRandom 373 373 0 268.2 3.7 2.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextGaussian: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 3381 3384 3 29.6 33.8 1.0X -XORShiftRandom 2480 2498 29 40.3 24.8 1.4X +java.util.Random 3412 3427 13 29.3 34.1 1.0X +XORShiftRandom 2469 2472 4 40.5 24.7 1.4X ================================================================================================ hash seed ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash seed: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -XORShiftRandom.hashSeed 1 1 0 12973.9 0.1 1.0X +XORShiftRandom.hashSeed 1 1 0 12522.5 0.1 1.0X diff --git a/core/benchmarks/XORShiftRandomBenchmark-results.txt b/core/benchmarks/XORShiftRandomBenchmark-results.txt index de5f7c04fddfc..c45a3c66afafa 100644 --- a/core/benchmarks/XORShiftRandomBenchmark-results.txt +++ b/core/benchmarks/XORShiftRandomBenchmark-results.txt @@ -2,43 +2,43 @@ Pseudo random ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextInt: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 439 439 0 227.7 4.4 1.0X -XORShiftRandom 185 185 0 539.5 1.9 2.4X +java.util.Random 441 441 0 226.6 4.4 1.0X +XORShiftRandom 186 186 0 536.7 1.9 2.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextLong: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 911 912 1 109.8 9.1 1.0X -XORShiftRandom 371 371 1 269.7 3.7 2.5X +java.util.Random 918 918 1 109.0 9.2 1.0X +XORShiftRandom 373 373 0 268.2 3.7 2.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextDouble: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 904 904 0 110.6 9.0 1.0X -XORShiftRandom 371 371 0 269.7 3.7 2.4X +java.util.Random 904 905 1 110.7 9.0 1.0X +XORShiftRandom 373 374 1 268.2 3.7 2.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor nextGaussian: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java.util.Random 3997 3998 1 25.0 40.0 1.0X -XORShiftRandom 2926 2929 2 34.2 29.3 1.4X +java.util.Random 3590 3600 10 27.9 35.9 1.0X +XORShiftRandom 2941 2942 1 34.0 29.4 1.2X ================================================================================================ hash seed ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash seed: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -XORShiftRandom.hashSeed 2 2 0 6487.7 0.2 1.0X +XORShiftRandom.hashSeed 2 2 0 6458.4 0.2 1.0X diff --git a/core/benchmarks/ZStandardBenchmark-jdk21-results.txt b/core/benchmarks/ZStandardBenchmark-jdk21-results.txt index f6bd681451d5e..b2a325942cd88 100644 --- a/core/benchmarks/ZStandardBenchmark-jdk21-results.txt +++ b/core/benchmarks/ZStandardBenchmark-jdk21-results.txt @@ -2,48 +2,48 @@ Benchmark ZStandardCompressionCodec ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark ZStandardCompressionCodec: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Compression 10000 times at level 1 without buffer pool 659 676 16 0.0 65860.7 1.0X -Compression 10000 times at level 2 without buffer pool 721 723 2 0.0 72135.5 0.9X -Compression 10000 times at level 3 without buffer pool 815 816 1 0.0 81500.6 0.8X -Compression 10000 times at level 1 with buffer pool 608 609 0 0.0 60846.6 1.1X -Compression 10000 times at level 2 with buffer pool 645 647 3 0.0 64476.3 1.0X -Compression 10000 times at level 3 with buffer pool 746 746 1 0.0 74584.0 0.9X +Compression 10000 times at level 1 without buffer pool 656 668 13 0.0 65555.1 1.0X +Compression 10000 times at level 2 without buffer pool 711 713 2 0.0 71147.9 0.9X +Compression 10000 times at level 3 without buffer pool 827 830 2 0.0 82718.7 0.8X +Compression 10000 times at level 1 with buffer pool 598 599 2 0.0 59789.9 1.1X +Compression 10000 times at level 2 with buffer pool 628 630 2 0.0 62774.0 1.0X +Compression 10000 times at level 3 with buffer pool 735 736 1 0.0 73517.1 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark ZStandardCompressionCodec: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------ -Decompression 10000 times from level 1 without buffer pool 828 829 1 0.0 82822.6 1.0X -Decompression 10000 times from level 2 without buffer pool 829 829 1 0.0 82900.7 1.0X -Decompression 10000 times from level 3 without buffer pool 828 833 8 0.0 82784.4 1.0X -Decompression 10000 times from level 1 with buffer pool 758 760 2 0.0 75756.5 1.1X -Decompression 10000 times from level 2 with buffer pool 758 758 1 0.0 75772.3 1.1X -Decompression 10000 times from level 3 with buffer pool 759 759 0 0.0 75852.7 1.1X +Decompression 10000 times from level 1 without buffer pool 823 824 1 0.0 82271.8 1.0X +Decompression 10000 times from level 2 without buffer pool 823 825 2 0.0 82313.0 1.0X +Decompression 10000 times from level 3 without buffer pool 825 832 10 0.0 82532.5 1.0X +Decompression 10000 times from level 1 with buffer pool 756 757 2 0.0 75593.4 1.1X +Decompression 10000 times from level 2 with buffer pool 757 759 2 0.0 75728.2 1.1X +Decompression 10000 times from level 3 with buffer pool 760 760 0 0.0 75986.2 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parallel Compression at level 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parallel Compression with 0 workers 58 59 1 0.0 452489.9 1.0X -Parallel Compression with 1 workers 42 45 4 0.0 330066.0 1.4X -Parallel Compression with 2 workers 40 42 1 0.0 312560.3 1.4X -Parallel Compression with 4 workers 40 42 2 0.0 308802.7 1.5X -Parallel Compression with 8 workers 41 45 3 0.0 321331.3 1.4X -Parallel Compression with 16 workers 44 45 1 0.0 343311.5 1.3X +Parallel Compression with 0 workers 58 60 4 0.0 456002.7 1.0X +Parallel Compression with 1 workers 43 45 3 0.0 332797.0 1.4X +Parallel Compression with 2 workers 41 42 1 0.0 317101.0 1.4X +Parallel Compression with 4 workers 39 41 1 0.0 306350.9 1.5X +Parallel Compression with 8 workers 42 44 1 0.0 326335.3 1.4X +Parallel Compression with 16 workers 46 47 1 0.0 356789.8 1.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parallel Compression at level 9: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parallel Compression with 0 workers 158 160 2 0.0 1234257.6 1.0X -Parallel Compression with 1 workers 193 194 1 0.0 1507686.4 0.8X -Parallel Compression with 2 workers 113 127 11 0.0 881068.0 1.4X -Parallel Compression with 4 workers 109 111 2 0.0 849241.3 1.5X -Parallel Compression with 8 workers 111 115 3 0.0 869455.2 1.4X -Parallel Compression with 16 workers 113 116 2 0.0 881832.5 1.4X +Parallel Compression with 0 workers 158 160 1 0.0 1237762.1 1.0X +Parallel Compression with 1 workers 189 190 3 0.0 1473899.5 0.8X +Parallel Compression with 2 workers 112 120 9 0.0 874992.3 1.4X +Parallel Compression with 4 workers 108 112 3 0.0 846156.6 1.5X +Parallel Compression with 8 workers 113 117 3 0.0 886576.8 1.4X +Parallel Compression with 16 workers 113 116 2 0.0 881278.0 1.4X diff --git a/core/benchmarks/ZStandardBenchmark-results.txt b/core/benchmarks/ZStandardBenchmark-results.txt index 136f0333590cc..0cd02cc48963a 100644 --- a/core/benchmarks/ZStandardBenchmark-results.txt +++ b/core/benchmarks/ZStandardBenchmark-results.txt @@ -2,48 +2,48 @@ Benchmark ZStandardCompressionCodec ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark ZStandardCompressionCodec: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Compression 10000 times at level 1 without buffer pool 257 259 2 0.0 25704.2 1.0X -Compression 10000 times at level 2 without buffer pool 674 676 2 0.0 67396.3 0.4X -Compression 10000 times at level 3 without buffer pool 775 787 11 0.0 77497.9 0.3X -Compression 10000 times at level 1 with buffer pool 573 574 0 0.0 57347.3 0.4X -Compression 10000 times at level 2 with buffer pool 602 603 2 0.0 60162.8 0.4X -Compression 10000 times at level 3 with buffer pool 722 725 3 0.0 72247.3 0.4X +Compression 10000 times at level 1 without buffer pool 263 405 194 0.0 26293.9 1.0X +Compression 10000 times at level 2 without buffer pool 693 694 1 0.0 69337.7 0.4X +Compression 10000 times at level 3 without buffer pool 805 809 4 0.0 80511.1 0.3X +Compression 10000 times at level 1 with buffer pool 576 577 2 0.0 57572.6 0.5X +Compression 10000 times at level 2 with buffer pool 611 612 1 0.0 61149.9 0.4X +Compression 10000 times at level 3 with buffer pool 730 731 1 0.0 73001.9 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark ZStandardCompressionCodec: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------ -Decompression 10000 times from level 1 without buffer pool 176 177 1 0.1 17641.2 1.0X -Decompression 10000 times from level 2 without buffer pool 176 178 1 0.1 17628.9 1.0X -Decompression 10000 times from level 3 without buffer pool 175 176 0 0.1 17506.1 1.0X -Decompression 10000 times from level 1 with buffer pool 151 152 1 0.1 15051.5 1.2X -Decompression 10000 times from level 2 with buffer pool 150 151 1 0.1 14998.0 1.2X -Decompression 10000 times from level 3 with buffer pool 150 151 0 0.1 15019.4 1.2X +Decompression 10000 times from level 1 without buffer pool 616 616 1 0.0 61555.7 1.0X +Decompression 10000 times from level 2 without buffer pool 617 618 1 0.0 61746.1 1.0X +Decompression 10000 times from level 3 without buffer pool 614 615 1 0.0 61402.4 1.0X +Decompression 10000 times from level 1 with buffer pool 541 542 1 0.0 54078.9 1.1X +Decompression 10000 times from level 2 with buffer pool 541 542 1 0.0 54094.5 1.1X +Decompression 10000 times from level 3 with buffer pool 540 541 1 0.0 54049.5 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parallel Compression at level 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parallel Compression with 0 workers 57 57 0 0.0 444425.2 1.0X -Parallel Compression with 1 workers 42 44 3 0.0 325107.6 1.4X -Parallel Compression with 2 workers 38 39 2 0.0 294840.0 1.5X -Parallel Compression with 4 workers 36 37 1 0.0 282143.1 1.6X -Parallel Compression with 8 workers 39 40 1 0.0 303793.6 1.5X -Parallel Compression with 16 workers 41 43 1 0.0 324165.5 1.4X +Parallel Compression with 0 workers 57 58 1 0.0 442501.6 1.0X +Parallel Compression with 1 workers 42 44 3 0.0 325787.4 1.4X +Parallel Compression with 2 workers 38 40 2 0.0 295047.1 1.5X +Parallel Compression with 4 workers 37 38 1 0.0 285755.4 1.5X +Parallel Compression with 8 workers 39 40 1 0.0 301689.5 1.5X +Parallel Compression with 16 workers 42 44 1 0.0 327951.9 1.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.8.0-1014-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parallel Compression at level 9: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parallel Compression with 0 workers 156 158 1 0.0 1220298.8 1.0X -Parallel Compression with 1 workers 188 189 1 0.0 1467911.4 0.8X -Parallel Compression with 2 workers 111 118 7 0.0 866985.2 1.4X -Parallel Compression with 4 workers 106 109 2 0.0 827592.1 1.5X -Parallel Compression with 8 workers 114 116 2 0.0 888419.5 1.4X -Parallel Compression with 16 workers 111 115 2 0.0 868463.5 1.4X +Parallel Compression with 0 workers 155 158 1 0.0 1213931.6 1.0X +Parallel Compression with 1 workers 189 191 2 0.0 1475730.7 0.8X +Parallel Compression with 2 workers 112 117 5 0.0 878455.7 1.4X +Parallel Compression with 4 workers 107 110 3 0.0 834762.2 1.5X +Parallel Compression with 8 workers 113 116 2 0.0 886435.5 1.4X +Parallel Compression with 16 workers 110 115 3 0.0 859182.0 1.4X diff --git a/mllib-local/benchmarks/BLASBenchmark-jdk21-results.txt b/mllib-local/benchmarks/BLASBenchmark-jdk21-results.txt index 97f88e2fe2de4..b2ec64828b395 100644 --- a/mllib-local/benchmarks/BLASBenchmark-jdk21-results.txt +++ b/mllib-local/benchmarks/BLASBenchmark-jdk21-results.txt @@ -2,337 +2,311 @@ daxpy ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor daxpy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 143 149 3 700.3 1.4 1.0X -java 126 146 8 791.3 1.3 1.1X -native 142 149 4 705.8 1.4 1.0X +f2j 148 155 4 676.7 1.5 1.0X +java 146 153 7 684.9 1.5 1.0X ================================================================================================ saxpy ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor saxpy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 75 82 6 1339.1 0.7 1.0X -java 68 71 2 1471.4 0.7 1.1X -native 76 84 6 1321.9 0.8 1.0X +f2j 79 85 4 1270.6 0.8 1.0X +java 69 73 2 1447.1 0.7 1.1X ================================================================================================ dcopy ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dcopy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 139 145 3 718.2 1.4 1.0X -java 127 143 9 786.7 1.3 1.1X -native 126 145 8 792.5 1.3 1.1X +f2j 131 151 10 766.2 1.3 1.0X +java 132 150 10 757.4 1.3 1.0X ================================================================================================ scopy ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scopy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 73 80 4 1370.0 0.7 1.0X -java 69 72 2 1450.9 0.7 1.1X -native 73 80 4 1374.3 0.7 1.0X +f2j 73 83 7 1363.1 0.7 1.0X +java 72 75 2 1394.6 0.7 1.0X ================================================================================================ ddot ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor ddot: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 95 95 0 1052.1 1.0 1.0X -java 45 47 1 2236.4 0.4 2.1X -native 95 95 0 1053.4 0.9 1.0X +f2j 96 96 0 1044.5 1.0 1.0X +java 47 51 3 2147.8 0.5 2.1X ================================================================================================ sdot ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sdot: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 93 93 0 1074.4 0.9 1.0X -java 23 23 1 4444.2 0.2 4.1X -native 93 93 0 1075.1 0.9 1.0X +f2j 94 94 0 1067.9 0.9 1.0X +java 23 25 1 4320.8 0.2 4.0X ================================================================================================ dnrm2 ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dnrm2: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 142 143 0 701.9 1.4 1.0X -java 33 33 0 3066.4 0.3 4.4X -native 94 95 1 1060.7 0.9 1.5X +f2j 143 144 1 698.6 1.4 1.0X +java 34 35 1 2981.8 0.3 4.3X ================================================================================================ snrm2 ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor snrm2: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 113 114 1 885.5 1.1 1.0X -java 16 16 0 6158.1 0.2 7.0X -native 93 93 0 1073.9 0.9 1.2X +f2j 125 125 2 801.4 1.2 1.0X +java 16 17 1 6092.6 0.2 7.6X ================================================================================================ dscal ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dscal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 132 137 2 755.3 1.3 1.0X -java 120 125 2 830.5 1.2 1.1X -native 128 133 3 779.8 1.3 1.0X +f2j 141 150 7 710.1 1.4 1.0X +java 127 135 3 784.5 1.3 1.1X ================================================================================================ sscal ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sscal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 78 86 7 1288.3 0.8 1.0X -java 57 63 4 1756.3 0.6 1.4X -native 69 77 7 1445.6 0.7 1.1X +f2j 79 88 8 1264.4 0.8 1.0X +java 58 67 5 1711.0 0.6 1.4X ================================================================================================ dgemv[N] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemv[N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 67 68 1 1491.1 0.7 1.0X -java 22 23 1 4447.1 0.2 3.0X -native 44 46 1 2264.6 0.4 1.5X +f2j 100 100 1 1003.2 1.0 1.0X +java 23 25 1 4266.2 0.2 4.3X ================================================================================================ dgemv[T] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemv[T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 93 93 0 1072.3 0.9 1.0X -java 23 23 1 4439.7 0.2 4.1X -native 93 93 0 1073.1 0.9 1.0X +f2j 94 94 1 1065.2 0.9 1.0X +java 23 24 1 4374.5 0.2 4.1X ================================================================================================ sgemv[N] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemv[N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 95 95 0 1053.6 0.9 1.0X -java 11 11 0 9024.3 0.1 8.6X -native 34 35 1 2939.9 0.3 2.8X +f2j 96 96 0 1042.4 1.0 1.0X +java 12 12 1 8626.4 0.1 8.3X ================================================================================================ sgemv[T] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemv[T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 93 93 0 1078.0 0.9 1.0X -java 11 12 0 8986.9 0.1 8.3X -native 93 93 0 1079.0 0.9 1.0X +f2j 93 94 1 1071.4 0.9 1.0X +java 11 12 1 8768.3 0.1 8.2X ================================================================================================ dger ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dger: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 137 141 2 731.0 1.4 1.0X -java 120 123 2 836.1 1.2 1.1X -native 134 139 3 743.8 1.3 1.0X +f2j 139 144 2 717.0 1.4 1.0X +java 121 126 3 828.1 1.2 1.2X ================================================================================================ dspmv[U] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dspmv[U]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 93 93 0 538.1 1.9 1.0X -java 11 12 0 4370.3 0.2 8.1X -native 47 47 0 1066.7 0.9 2.0X +f2j 92 93 2 541.7 1.8 1.0X +java 12 12 1 4276.6 0.2 7.9X ================================================================================================ dspr[U] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dspr[U]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 87 95 7 574.0 1.7 1.0X -java 87 95 7 575.3 1.7 1.0X -native 62 70 6 812.0 1.2 1.4X +f2j 93 96 2 536.8 1.9 1.0X +java 88 96 7 567.6 1.8 1.1X ================================================================================================ dsyr[U] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dsyr[U]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 142 145 2 353.3 2.8 1.0X -java 142 145 2 352.9 2.8 1.0X -native 117 124 3 427.3 2.3 1.2X +f2j 136 141 2 366.7 2.7 1.0X +java 138 143 4 361.4 2.8 1.0X ================================================================================================ dgemm[N,N] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[N,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 661 662 1 1512.5 0.7 1.0X -java 63 68 4 15787.8 0.1 10.4X -native 631 633 2 1583.8 0.6 1.0X +f2j 665 667 1 1503.9 0.7 1.0X +java 63 69 4 15804.0 0.1 10.5X ================================================================================================ dgemm[N,T] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[N,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 669 670 2 1495.3 0.7 1.0X -java 64 70 3 15673.5 0.1 10.5X -native 375 377 5 2665.8 0.4 1.8X +f2j 667 669 2 1499.4 0.7 1.0X +java 64 70 4 15724.9 0.1 10.5X ================================================================================================ dgemm[T,N] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[T,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 900 901 1 1111.3 0.9 1.0X -java 63 68 4 15822.8 0.1 14.2X -native 903 904 1 1107.9 0.9 1.0X +f2j 911 913 2 1097.3 0.9 1.0X +java 63 69 4 15900.2 0.1 14.5X ================================================================================================ dgemm[T,T] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[T,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 941 943 3 1062.6 0.9 1.0X -java 63 69 5 15771.6 0.1 14.8X -native 915 916 1 1092.7 0.9 1.0X +f2j 950 953 5 1053.1 0.9 1.0X +java 63 69 4 15828.3 0.1 15.0X ================================================================================================ sgemm[N,N] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[N,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 650 651 1 1537.8 0.7 1.0X -java 40 41 1 24986.7 0.0 16.2X -native 372 372 1 2691.6 0.4 1.8X +f2j 653 655 3 1530.4 0.7 1.0X +java 40 41 1 25035.2 0.0 16.4X ================================================================================================ sgemm[N,T] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[N,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 651 652 1 1536.7 0.7 1.0X -java 41 41 1 24643.9 0.0 16.0X -native 372 373 1 2688.8 0.4 1.7X +f2j 655 657 3 1526.9 0.7 1.0X +java 40 41 1 24749.3 0.0 16.2X ================================================================================================ sgemm[T,N] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[T,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 901 902 0 1109.8 0.9 1.0X -java 40 41 1 25107.2 0.0 22.6X -native 918 919 1 1089.3 0.9 1.0X +f2j 906 907 2 1104.0 0.9 1.0X +java 40 41 1 25083.9 0.0 22.7X ================================================================================================ sgemm[T,T] ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[T,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 942 944 2 1061.1 0.9 1.0X -java 40 41 1 24888.3 0.0 23.5X -native 914 915 0 1093.7 0.9 1.0X +f2j 946 949 4 1056.7 0.9 1.0X +java 40 41 1 24924.3 0.0 23.6X diff --git a/mllib-local/benchmarks/BLASBenchmark-results.txt b/mllib-local/benchmarks/BLASBenchmark-results.txt index db92355b7a3c0..cb9a670c990f4 100644 --- a/mllib-local/benchmarks/BLASBenchmark-results.txt +++ b/mllib-local/benchmarks/BLASBenchmark-results.txt @@ -2,337 +2,311 @@ daxpy ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor daxpy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 150 158 4 667.3 1.5 1.0X -java 142 147 3 703.2 1.4 1.1X -native 150 158 4 668.3 1.5 1.0X +f2j 136 141 6 733.6 1.4 1.0X +java 128 131 1 782.3 1.3 1.1X ================================================================================================ saxpy ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor saxpy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 85 89 2 1173.5 0.9 1.0X -java 71 74 2 1409.0 0.7 1.2X -native 86 89 2 1158.6 0.9 1.0X +f2j 78 80 1 1278.9 0.8 1.0X +java 65 67 2 1540.9 0.6 1.2X ================================================================================================ dcopy ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dcopy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 143 151 4 698.9 1.4 1.0X -java 142 150 4 705.2 1.4 1.0X -native 143 148 3 697.2 1.4 1.0X +f2j 131 136 3 766.0 1.3 1.0X +java 136 141 3 734.3 1.4 1.0X ================================================================================================ scopy ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scopy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 82 85 2 1215.8 0.8 1.0X -java 72 75 2 1398.0 0.7 1.1X -native 80 83 2 1250.7 0.8 1.0X +f2j 77 81 1 1300.7 0.8 1.0X +java 69 72 1 1439.3 0.7 1.1X ================================================================================================ ddot ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor ddot: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 95 95 0 1051.8 1.0 1.0X -java 44 46 2 2279.3 0.4 2.2X -native 95 95 0 1057.0 0.9 1.0X +f2j 96 96 0 1043.9 1.0 1.0X +java 44 46 1 2251.5 0.4 2.2X ================================================================================================ sdot ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sdot: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 93 93 0 1074.0 0.9 1.0X -java 21 22 1 4768.4 0.2 4.4X -native 93 93 1 1075.7 0.9 1.0X +f2j 94 94 2 1066.8 0.9 1.0X +java 22 23 0 4546.9 0.2 4.3X ================================================================================================ dnrm2 ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dnrm2: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 143 143 0 699.8 1.4 1.0X -java 32 33 1 3105.2 0.3 4.4X -native 94 95 1 1061.0 0.9 1.5X +f2j 144 144 1 695.9 1.4 1.0X +java 32 33 0 3089.0 0.3 4.4X ================================================================================================ snrm2 ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor snrm2: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 120 120 1 834.9 1.2 1.0X -java 16 16 0 6220.1 0.2 7.5X -native 93 93 2 1074.9 0.9 1.3X +f2j 121 121 1 828.5 1.2 1.0X +java 16 16 0 6186.1 0.2 7.5X ================================================================================================ dscal ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dscal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 142 147 2 704.2 1.4 1.0X -java 130 134 2 772.1 1.3 1.1X -native 135 142 3 740.7 1.4 1.1X +f2j 125 130 6 799.4 1.3 1.0X +java 120 123 2 832.6 1.2 1.0X ================================================================================================ sscal ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sscal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 80 82 1 1253.8 0.8 1.0X -java 64 68 1 1554.0 0.6 1.2X -native 80 83 2 1256.6 0.8 1.0X +f2j 73 75 1 1372.9 0.7 1.0X +java 54 59 4 1858.4 0.5 1.4X ================================================================================================ dgemv[N] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemv[N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 96 96 0 1043.0 1.0 1.0X -java 22 23 1 4563.6 0.2 4.4X -native 45 47 1 2229.3 0.4 2.1X +f2j 96 97 2 1036.6 1.0 1.0X +java 23 23 1 4407.0 0.2 4.3X ================================================================================================ dgemv[T] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemv[T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 94 94 0 1065.7 0.9 1.0X -java 22 24 1 4467.6 0.2 4.2X -native 93 93 0 1073.1 0.9 1.0X +f2j 95 95 0 1056.5 0.9 1.0X +java 23 23 0 4436.2 0.2 4.2X ================================================================================================ sgemv[N] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemv[N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 95 96 0 1050.3 1.0 1.0X -java 11 12 0 8901.1 0.1 8.5X -native 34 35 1 2956.0 0.3 2.8X +f2j 96 96 2 1042.7 1.0 1.0X +java 11 12 0 9009.9 0.1 8.6X ================================================================================================ sgemv[T] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemv[T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 93 93 0 1077.7 0.9 1.0X -java 11 12 0 8874.5 0.1 8.2X -native 93 93 0 1079.6 0.9 1.0X +f2j 93 94 0 1070.0 0.9 1.0X +java 11 12 0 8956.4 0.1 8.4X ================================================================================================ dger ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dger: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 150 154 2 666.9 1.5 1.0X -java 125 130 2 801.5 1.2 1.2X -native 143 149 3 698.6 1.4 1.0X +f2j 133 136 1 750.3 1.3 1.0X +java 114 116 2 878.5 1.1 1.2X ================================================================================================ dspmv[U] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dspmv[U]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 99 100 0 503.9 2.0 1.0X -java 11 12 0 4411.4 0.2 8.8X -native 47 47 0 1067.1 0.9 2.1X +f2j 100 101 1 498.5 2.0 1.0X +java 11 12 1 4354.1 0.2 8.7X ================================================================================================ dspr[U] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dspr[U]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 96 99 2 520.9 1.9 1.0X -java 97 98 1 517.4 1.9 1.0X -native 73 77 2 681.8 1.5 1.3X +f2j 89 91 1 561.5 1.8 1.0X +java 89 91 1 559.8 1.8 1.0X ================================================================================================ dsyr[U] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dsyr[U]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 144 149 2 347.0 2.9 1.0X -java 144 148 2 346.2 2.9 1.0X -native 121 126 2 413.6 2.4 1.2X +f2j 130 133 3 384.7 2.6 1.0X +java 129 132 3 386.4 2.6 1.0X ================================================================================================ dgemm[N,N] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[N,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 665 666 2 1503.6 0.7 1.0X -java 65 71 3 15448.3 0.1 10.3X -native 630 632 2 1586.6 0.6 1.1X +f2j 670 673 4 1493.6 0.7 1.0X +java 72 73 1 13968.9 0.1 9.4X ================================================================================================ dgemm[N,T] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[N,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 672 674 2 1487.3 0.7 1.0X -java 65 71 3 15366.3 0.1 10.3X -native 376 377 1 2661.0 0.4 1.8X +f2j 675 678 4 1482.3 0.7 1.0X +java 72 73 1 13923.4 0.1 9.4X ================================================================================================ dgemm[T,N] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[T,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 921 921 1 1086.1 0.9 1.0X -java 64 70 3 15574.7 0.1 14.3X -native 901 902 1 1109.5 0.9 1.0X +f2j 927 929 4 1078.8 0.9 1.0X +java 71 72 1 14079.1 0.1 13.1X ================================================================================================ dgemm[T,T] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dgemm[T,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 950 952 5 1052.4 1.0 1.0X -java 71 72 1 14034.5 0.1 13.3X -native 914 914 1 1094.4 0.9 1.0X +f2j 960 964 6 1041.4 1.0 1.0X +java 71 73 1 13994.1 0.1 13.4X ================================================================================================ sgemm[N,N] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[N,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 648 649 2 1543.9 0.6 1.0X -java 41 42 1 24403.6 0.0 15.8X -native 371 371 1 2699.0 0.4 1.7X +f2j 653 655 3 1531.0 0.7 1.0X +java 41 41 1 24509.8 0.0 16.0X ================================================================================================ sgemm[N,T] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[N,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 652 653 2 1533.0 0.7 1.0X -java 42 43 1 24056.0 0.0 15.7X -native 371 372 5 2697.5 0.4 1.8X +f2j 658 663 4 1520.1 0.7 1.0X +java 41 43 1 24199.3 0.0 15.9X ================================================================================================ sgemm[T,N] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[T,N]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 900 901 2 1111.2 0.9 1.0X -java 40 41 1 24704.4 0.0 22.2X -native 917 919 2 1090.4 0.9 1.0X +f2j 907 908 2 1103.0 0.9 1.0X +java 40 42 1 24844.4 0.0 22.5X ================================================================================================ sgemm[T,T] ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sgemm[T,T]: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -f2j 948 950 1 1054.4 0.9 1.0X -java 41 42 1 24366.6 0.0 23.1X -native 913 915 1 1094.8 0.9 1.0X +f2j 955 958 5 1047.1 1.0 1.0X +java 41 42 1 24509.7 0.0 23.4X diff --git a/mllib/benchmarks/UDTSerializationBenchmark-jdk21-results.txt b/mllib/benchmarks/UDTSerializationBenchmark-jdk21-results.txt index 00de1f2d104f9..f3a4fbcb8fc2a 100644 --- a/mllib/benchmarks/UDTSerializationBenchmark-jdk21-results.txt +++ b/mllib/benchmarks/UDTSerializationBenchmark-jdk21-results.txt @@ -2,11 +2,11 @@ VectorUDT de/serialization ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor VectorUDT de/serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -serialize 92 103 6 0.0 92038.9 1.0X -deserialize 69 74 3 0.0 69046.7 1.3X +serialize 102 107 2 0.0 101911.3 1.0X +deserialize 75 78 1 0.0 74761.1 1.4X diff --git a/mllib/benchmarks/UDTSerializationBenchmark-results.txt b/mllib/benchmarks/UDTSerializationBenchmark-results.txt index 05004e6f74f3c..e77536e502c3b 100644 --- a/mllib/benchmarks/UDTSerializationBenchmark-results.txt +++ b/mllib/benchmarks/UDTSerializationBenchmark-results.txt @@ -2,11 +2,11 @@ VectorUDT de/serialization ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor VectorUDT de/serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -serialize 92 109 9 0.0 91694.5 1.0X -deserialize 69 71 1 0.0 69297.4 1.3X +serialize 95 97 1 0.0 94889.7 1.0X +deserialize 70 74 3 0.0 69517.2 1.4X diff --git a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt index 290568730a22c..24bc5a5efcaae 100644 --- a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt @@ -2,10 +2,10 @@ CalendarInterval ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor CalendarInterval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Call setInterval & getInterval 1028 1028 0 130.5 7.7 1.0X +Call setInterval & getInterval 1040 1051 16 129.1 7.7 1.0X diff --git a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt index 526008a3fced1..a5bd7ce02cc8f 100644 --- a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt +++ b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt @@ -2,10 +2,10 @@ CalendarInterval ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor CalendarInterval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Call setInterval & getInterval 1018 1021 4 131.8 7.6 1.0X +Call setInterval & getInterval 1030 1030 0 130.3 7.7 1.0X diff --git a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt index b5635dcb20d33..8b9dd199d9df8 100644 --- a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt @@ -1,105 +1,105 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 1 1 0 1390.0 0.7 1.0X -Use EnumSet 2 2 0 441.2 2.3 0.3X +Use HashSet 3 3 0 291.9 3.4 1.0X +Use EnumSet 4 4 0 227.7 4.4 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 2 2 0 485.8 2.1 1.0X -Use EnumSet 2 2 0 544.4 1.8 1.1X +Use HashSet 7 8 1 138.0 7.2 1.0X +Use EnumSet 5 5 0 185.8 5.4 1.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 2 2 0 493.1 2.0 1.0X -Use EnumSet 2 2 0 575.2 1.7 1.2X +Use HashSet 14 14 0 71.9 13.9 1.0X +Use EnumSet 5 5 0 186.1 5.4 2.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 9 9 0 107.2 9.3 1.0X -Use EnumSet 2 2 0 534.9 1.9 5.0X +Use HashSet 11 11 1 91.4 10.9 1.0X +Use EnumSet 5 5 0 186.1 5.4 2.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 10 10 0 98.5 10.1 1.0X -Use EnumSet 2 2 0 534.9 1.9 5.4X +Use HashSet 12 13 0 80.5 12.4 1.0X +Use EnumSet 5 5 0 188.4 5.3 2.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 0 0 0 408.8 2.4 1.0X -Use EnumSet 1 1 0 136.6 7.3 0.3X +Use HashSet 0 0 0 397.1 2.5 1.0X +Use EnumSet 0 0 0 291.5 3.4 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 1 1 0 102.5 9.8 1.0X -Use EnumSet 0 0 0 291.4 3.4 2.8X +Use HashSet 2 2 0 49.9 20.0 1.0X +Use EnumSet 0 0 0 291.2 3.4 5.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 7 7 0 14.6 68.6 1.0X -Use EnumSet 1 1 0 132.3 7.6 9.1X +Use HashSet 6 6 0 16.0 62.4 1.0X +Use EnumSet 1 1 0 132.4 7.6 8.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 34 35 0 2.9 342.4 1.0X -Use EnumSet 1 1 0 150.1 6.7 51.4X +Use HashSet 29 30 1 3.4 292.1 1.0X +Use EnumSet 1 1 0 150.4 6.7 43.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 63 63 1 1.6 627.2 1.0X -Use EnumSet 1 1 0 138.3 7.2 86.8X +Use HashSet 56 56 1 1.8 557.0 1.0X +Use EnumSet 1 1 0 138.6 7.2 77.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 4 4 0 227.4 4.4 1.0X -Use EnumSet 5 5 0 187.2 5.3 0.8X +Use HashSet 4 4 0 265.6 3.8 1.0X +Use EnumSet 5 5 0 196.1 5.1 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 14 14 0 70.4 14.2 1.0X -Use EnumSet 7 7 0 150.5 6.6 2.1X +Use HashSet 9 10 0 110.6 9.0 1.0X +Use EnumSet 6 6 0 160.0 6.3 1.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 27 28 0 36.6 27.3 1.0X -Use EnumSet 7 7 0 151.3 6.6 4.1X +Use HashSet 22 22 0 45.4 22.0 1.0X +Use EnumSet 6 6 0 163.3 6.1 3.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 54 55 1 18.4 54.4 1.0X -Use EnumSet 7 7 0 147.6 6.8 8.0X +Use HashSet 49 49 0 20.4 49.1 1.0X +Use EnumSet 6 6 0 158.7 6.3 7.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Use HashSet 84 85 0 11.9 83.9 1.0X -Use EnumSet 7 7 0 137.2 7.3 11.5X +Use HashSet 76 77 1 13.1 76.2 1.0X +Use EnumSet 6 7 0 159.0 6.3 12.1X diff --git a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt index 1794f82b64b11..4b1c3dce2b115 100644 --- a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt +++ b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt @@ -1,105 +1,105 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 1 1 0 1441.2 0.7 1.0X -Use EnumSet 2 2 0 563.7 1.8 0.4X +Use HashSet 4 4 0 279.7 3.6 1.0X +Use EnumSet 4 4 0 225.9 4.4 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 2 2 0 445.8 2.2 1.0X -Use EnumSet 2 2 0 554.4 1.8 1.2X +Use HashSet 9 9 1 110.9 9.0 1.0X +Use EnumSet 5 5 0 185.6 5.4 1.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 2 2 0 547.8 1.8 1.0X -Use EnumSet 2 2 0 561.3 1.8 1.0X +Use HashSet 14 14 2 74.0 13.5 1.0X +Use EnumSet 5 5 0 185.6 5.4 2.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 8 8 0 121.9 8.2 1.0X -Use EnumSet 2 2 0 545.1 1.8 4.5X +Use HashSet 14 14 1 71.1 14.1 1.0X +Use EnumSet 5 5 0 185.7 5.4 2.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 9 9 0 110.1 9.1 1.0X -Use EnumSet 2 2 0 545.0 1.8 5.0X +Use HashSet 15 15 0 68.2 14.7 1.0X +Use EnumSet 5 5 0 185.7 5.4 2.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 0 0 0 409.8 2.4 1.0X -Use EnumSet 1 1 0 127.6 7.8 0.3X +Use HashSet 0 0 0 407.9 2.5 1.0X +Use EnumSet 0 0 0 225.2 4.4 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 2 2 0 60.0 16.7 1.0X -Use EnumSet 1 1 0 145.0 6.9 2.4X +Use HashSet 2 2 0 48.3 20.7 1.0X +Use EnumSet 1 1 0 87.6 11.4 1.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 10 10 0 10.2 97.7 1.0X -Use EnumSet 1 1 0 137.8 7.3 13.5X +Use HashSet 10 11 1 9.6 103.8 1.0X +Use EnumSet 1 1 0 103.2 9.7 10.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 33 33 1 3.1 327.8 1.0X -Use EnumSet 1 1 0 137.9 7.3 45.2X +Use HashSet 40 40 0 2.5 395.0 1.0X +Use EnumSet 1 1 0 99.3 10.1 39.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 60 60 0 1.7 596.5 1.0X -Use EnumSet 1 1 0 131.7 7.6 78.6X +Use HashSet 64 64 1 1.6 639.0 1.0X +Use EnumSet 1 1 0 108.8 9.2 69.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 4 4 0 250.2 4.0 1.0X -Use EnumSet 5 5 0 190.5 5.2 0.8X +Use HashSet 5 5 0 215.6 4.6 1.0X +Use EnumSet 5 5 0 194.3 5.1 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 13 14 0 74.9 13.3 1.0X -Use EnumSet 7 7 0 148.9 6.7 2.0X +Use HashSet 10 11 1 98.4 10.2 1.0X +Use EnumSet 6 6 0 159.7 6.3 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 32 33 1 31.6 31.7 1.0X -Use EnumSet 7 7 0 150.4 6.7 4.8X +Use HashSet 25 26 1 40.4 24.7 1.0X +Use EnumSet 6 6 0 158.8 6.3 3.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Use HashSet 55 58 9 18.2 55.1 1.0X -Use EnumSet 7 7 0 146.6 6.8 8.1X +Use HashSet 54 55 1 18.4 54.3 1.0X +Use EnumSet 6 7 0 155.6 6.4 8.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test create and contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Use HashSet 82 83 1 12.1 82.3 1.0X -Use EnumSet 7 7 0 145.2 6.9 12.0X +Use HashSet 80 82 1 12.4 80.3 1.0X +Use EnumSet 6 7 0 156.7 6.4 12.6X diff --git a/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt index 73f125fc87862..fa12bcbaa3c38 100644 --- a/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt @@ -2,23 +2,23 @@ Escape ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Escape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Legacy 7441 7453 11 0.1 7440.7 1.0X -New 768 770 1 1.3 768.3 9.7X +Legacy 9203 9215 8 0.1 9203.3 1.0X +New 813 816 2 1.2 813.1 11.3X ================================================================================================ Unescape ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Unescape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Legacy 4446 4454 5 0.2 4446.4 1.0X -New 605 611 3 1.7 605.1 7.3X +Legacy 4679 4687 5 0.2 4678.5 1.0X +New 590 595 5 1.7 589.7 7.9X diff --git a/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt b/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt index 87f5177d28715..dcdef85ea89d8 100644 --- a/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt +++ b/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt @@ -2,23 +2,23 @@ Escape ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Escape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Legacy 6851 6867 7 0.1 6850.8 1.0X -New 741 755 38 1.3 741.0 9.2X +Legacy 8620 8633 11 0.1 8620.5 1.0X +New 779 786 4 1.3 779.3 11.1X ================================================================================================ Unescape ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Unescape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Legacy 5810 5823 15 0.2 5809.8 1.0X -New 597 602 5 1.7 596.6 9.7X +Legacy 5714 5728 8 0.2 5714.0 1.0X +New 593 597 3 1.7 592.5 9.6X diff --git a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt index b09cc75270118..edf44bac9a395 100644 --- a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt @@ -1,10 +1,10 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor constructor: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -arrayOfAny 3 3 0 3243.9 0.3 1.0X -arrayOfAnyAsObject 3 3 0 3243.9 0.3 1.0X -arrayOfAnyAsSeq 225 230 6 44.5 22.5 0.0X -arrayOfInt 273 278 5 36.7 27.3 0.0X -arrayOfIntAsObject 274 278 3 36.5 27.4 0.0X +arrayOfAny 6 6 0 1611.9 0.6 1.0X +arrayOfAnyAsObject 6 6 0 1611.9 0.6 1.0X +arrayOfAnyAsSeq 175 175 1 57.3 17.5 0.0X +arrayOfInt 271 272 0 36.8 27.1 0.0X +arrayOfIntAsObject 250 251 1 40.0 25.0 0.0X diff --git a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt index 56d0a136c2933..fae20f2b0ac35 100644 --- a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt +++ b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt @@ -1,10 +1,10 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor constructor: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -arrayOfAny 6 6 0 1619.9 0.6 1.0X -arrayOfAnyAsObject 6 6 0 1619.9 0.6 1.0X -arrayOfAnyAsSeq 157 158 1 63.5 15.7 0.0X -arrayOfInt 252 254 4 39.6 25.2 0.0X -arrayOfIntAsObject 252 253 2 39.6 25.2 0.0X +arrayOfAny 6 6 0 1611.8 0.6 1.0X +arrayOfAnyAsObject 6 6 0 1611.9 0.6 1.0X +arrayOfAnyAsSeq 157 157 2 63.8 15.7 0.0X +arrayOfInt 253 254 0 39.5 25.3 0.0X +arrayOfIntAsObject 253 254 1 39.5 25.3 0.0X diff --git a/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt index d246505fc26ca..829099b78e422 100644 --- a/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt @@ -2,69 +2,69 @@ single ints ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For single ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 2151 2154 4 249.6 4.0 1.0X -codegen version 3580 3597 23 149.9 6.7 0.6X -codegen version 64-bit 3385 3408 33 158.6 6.3 0.6X -codegen HiveHash version 2884 2886 2 186.1 5.4 0.7X +interpreted version 2089 2090 2 257.0 3.9 1.0X +codegen version 3541 3544 4 151.6 6.6 0.6X +codegen version 64-bit 3238 3269 44 165.8 6.0 0.6X +codegen HiveHash version 2563 2568 8 209.5 4.8 0.8X ================================================================================================ single longs ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For single longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 2710 2711 1 198.1 5.0 1.0X -codegen version 5082 5083 0 105.6 9.5 0.5X -codegen version 64-bit 3962 3964 3 135.5 7.4 0.7X -codegen HiveHash version 3309 3310 1 162.2 6.2 0.8X +interpreted version 2718 2719 1 197.5 5.1 1.0X +codegen version 4520 4525 8 118.8 8.4 0.6X +codegen version 64-bit 3863 3874 15 139.0 7.2 0.7X +codegen HiveHash version 3158 3161 4 170.0 5.9 0.9X ================================================================================================ normal ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For normal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 1425 1425 1 1.5 679.5 1.0X -codegen version 1648 1648 1 1.3 785.8 0.9X -codegen version 64-bit 725 726 2 2.9 345.7 2.0X -codegen HiveHash version 3675 3677 3 0.6 1752.2 0.4X +interpreted version 1335 1341 8 1.6 636.6 1.0X +codegen version 1803 1803 0 1.2 859.9 0.7X +codegen version 64-bit 735 735 0 2.9 350.3 1.8X +codegen HiveHash version 3635 3639 6 0.6 1733.2 0.4X ================================================================================================ array ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 1018 1019 1 0.1 7768.0 1.0X -codegen version 3632 3633 2 0.0 27706.7 0.3X -codegen version 64-bit 2340 2342 3 0.1 17849.7 0.4X -codegen HiveHash version 750 751 1 0.2 5721.5 1.4X +interpreted version 958 959 2 0.1 7308.1 1.0X +codegen version 3436 3441 7 0.0 26216.6 0.3X +codegen version 64-bit 2352 2353 1 0.1 17945.7 0.4X +codegen HiveHash version 685 689 5 0.2 5227.3 1.4X ================================================================================================ map ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 0 0 0 97.3 10.3 1.0X -codegen version 249 249 1 0.0 60732.6 0.0X -codegen version 64-bit 169 170 1 0.0 41356.6 0.0X -codegen HiveHash version 27 28 0 0.1 6709.4 0.0X +interpreted version 0 0 0 90.4 11.1 1.0X +codegen version 271 271 0 0.0 66159.1 0.0X +codegen version 64-bit 185 185 0 0.0 45145.5 0.0X +codegen HiveHash version 30 30 0 0.1 7378.6 0.0X diff --git a/sql/catalyst/benchmarks/HashBenchmark-results.txt b/sql/catalyst/benchmarks/HashBenchmark-results.txt index 571a8a1d82881..30934fe57cd42 100644 --- a/sql/catalyst/benchmarks/HashBenchmark-results.txt +++ b/sql/catalyst/benchmarks/HashBenchmark-results.txt @@ -2,69 +2,69 @@ single ints ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For single ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 2276 2276 0 235.9 4.2 1.0X -codegen version 3664 3669 7 146.5 6.8 0.6X -codegen version 64-bit 3478 3483 6 154.3 6.5 0.7X -codegen HiveHash version 3008 3010 3 178.5 5.6 0.8X +interpreted version 2157 2161 6 248.9 4.0 1.0X +codegen version 3655 3660 6 146.9 6.8 0.6X +codegen version 64-bit 3509 3510 1 153.0 6.5 0.6X +codegen HiveHash version 2857 2859 3 187.9 5.3 0.8X ================================================================================================ single longs ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For single longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 3006 3007 2 178.6 5.6 1.0X -codegen version 5317 5322 7 101.0 9.9 0.6X -codegen version 64-bit 3761 3765 6 142.8 7.0 0.8X -codegen HiveHash version 3401 3429 41 157.9 6.3 0.9X +interpreted version 3009 3011 4 178.4 5.6 1.0X +codegen version 5332 5336 6 100.7 9.9 0.6X +codegen version 64-bit 3997 3999 2 134.3 7.4 0.8X +codegen HiveHash version 3310 3310 1 162.2 6.2 0.9X ================================================================================================ normal ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For normal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 2549 2549 0 0.8 1215.4 1.0X -codegen version 2291 2292 1 0.9 1092.5 1.1X -codegen version 64-bit 724 726 2 2.9 345.4 3.5X -codegen HiveHash version 3719 3726 10 0.6 1773.2 0.7X +interpreted version 2557 2557 0 0.8 1219.1 1.0X +codegen version 2217 2218 1 0.9 1057.3 1.2X +codegen version 64-bit 703 704 1 3.0 335.3 3.6X +codegen HiveHash version 3734 3741 10 0.6 1780.7 0.7X ================================================================================================ array ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 990 992 4 0.1 7549.7 1.0X -codegen version 3619 3619 0 0.0 27611.8 0.3X -codegen version 64-bit 2385 2386 0 0.1 18199.3 0.4X -codegen HiveHash version 727 727 0 0.2 5543.0 1.4X +interpreted version 951 954 3 0.1 7252.8 1.0X +codegen version 3450 3452 3 0.0 26319.1 0.3X +codegen version 64-bit 2296 2296 0 0.1 17516.4 0.4X +codegen HiveHash version 703 704 1 0.2 5360.9 1.4X ================================================================================================ map ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash For map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -interpreted version 0 0 0 101.8 9.8 1.0X -codegen version 240 240 0 0.0 58478.2 0.0X -codegen version 64-bit 169 170 0 0.0 41373.6 0.0X -codegen HiveHash version 29 29 0 0.1 7006.4 0.0X +interpreted version 0 0 0 97.4 10.3 1.0X +codegen version 223 223 0 0.0 54377.2 0.0X +codegen version 64-bit 152 152 0 0.0 37102.0 0.0X +codegen HiveHash version 26 26 0 0.2 6290.5 0.0X diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt index 4e4d54c6da6fe..e1cfa115e2730 100644 --- a/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt @@ -2,76 +2,76 @@ Benchmark for MurMurHash 3 and xxHash64 ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 8: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 10 10 0 205.6 4.9 1.0X -xxHash 64-bit 11 11 0 190.0 5.3 0.9X -HiveHasher 14 14 0 152.3 6.6 0.7X +Murmur3_x86_32 10 10 0 205.7 4.9 1.0X +xxHash 64-bit 10 10 0 200.6 5.0 1.0X +HiveHasher 14 14 1 151.9 6.6 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 16: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 14 15 0 145.8 6.9 1.0X -xxHash 64-bit 13 13 0 161.0 6.2 1.1X -HiveHasher 23 23 1 92.4 10.8 0.6X +Murmur3_x86_32 14 15 0 145.5 6.9 1.0X +xxHash 64-bit 12 13 2 168.2 5.9 1.2X +HiveHasher 23 23 1 90.9 11.0 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 24: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 20 20 0 104.9 9.5 1.0X -xxHash 64-bit 15 15 0 139.7 7.2 1.3X -HiveHasher 34 34 0 61.9 16.1 0.6X +Murmur3_x86_32 20 20 0 104.0 9.6 1.0X +xxHash 64-bit 15 15 0 143.1 7.0 1.4X +HiveHasher 34 34 0 62.5 16.0 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 31: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 31 31 0 68.5 14.6 1.0X -xxHash 64-bit 26 26 0 80.0 12.5 1.2X -HiveHasher 45 45 1 46.9 21.3 0.7X +Murmur3_x86_32 32 32 0 65.9 15.2 1.0X +xxHash 64-bit 27 28 0 76.3 13.1 1.2X +HiveHasher 44 44 0 48.1 20.8 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 95: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 68 70 3 30.7 32.6 1.0X -xxHash 64-bit 57 57 0 36.8 27.2 1.2X -HiveHasher 158 158 0 13.3 75.5 0.4X +Murmur3_x86_32 70 71 0 29.8 33.6 1.0X +xxHash 64-bit 58 58 0 36.4 27.5 1.2X +HiveHasher 157 157 0 13.4 74.8 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 287: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 208 210 3 10.1 99.2 1.0X -xxHash 64-bit 102 103 0 20.5 48.8 2.0X -HiveHasher 531 532 0 3.9 253.4 0.4X +Murmur3_x86_32 198 198 0 10.6 94.5 1.0X +xxHash 64-bit 102 102 0 20.6 48.6 1.9X +HiveHasher 533 533 0 3.9 254.0 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 1055: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 711 718 12 3.0 338.9 1.0X -xxHash 64-bit 296 298 4 7.1 141.0 2.4X -HiveHasher 2031 2032 2 1.0 968.4 0.4X +Murmur3_x86_32 709 717 13 3.0 338.3 1.0X +xxHash 64-bit 293 294 1 7.2 139.8 2.4X +HiveHasher 2042 2043 1 1.0 973.9 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 2079: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 1398 1401 5 1.5 666.7 1.0X -xxHash 64-bit 552 553 1 3.8 263.3 2.5X -HiveHasher 4026 4026 0 0.5 1919.5 0.3X +Murmur3_x86_32 1388 1388 1 1.5 661.7 1.0X +xxHash 64-bit 550 550 1 3.8 262.2 2.5X +HiveHasher 4050 4052 3 0.5 1931.1 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 8223: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 5707 5709 3 0.4 2721.3 1.0X -xxHash 64-bit 2074 2074 1 1.0 988.7 2.8X -HiveHasher 15993 15993 0 0.1 7626.2 0.4X +Murmur3_x86_32 5726 5727 1 0.4 2730.6 1.0X +xxHash 64-bit 2068 2069 2 1.0 986.1 2.8X +HiveHasher 16089 16098 12 0.1 7671.8 0.4X diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt index 236b9e5b404d4..d49fd90cd3707 100644 --- a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt +++ b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt @@ -2,76 +2,76 @@ Benchmark for MurMurHash 3 and xxHash64 ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 8: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 11 11 0 184.1 5.4 1.0X -xxHash 64-bit 10 10 0 214.5 4.7 1.2X -HiveHasher 14 14 0 150.9 6.6 0.8X +Murmur3_x86_32 11 12 0 183.2 5.5 1.0X +xxHash 64-bit 10 10 0 213.3 4.7 1.2X +HiveHasher 14 14 0 149.8 6.7 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 16: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 17 17 0 123.5 8.1 1.0X -xxHash 64-bit 12 12 0 176.1 5.7 1.4X -HiveHasher 22 23 0 93.3 10.7 0.8X +Murmur3_x86_32 17 17 0 123.2 8.1 1.0X +xxHash 64-bit 12 12 0 175.5 5.7 1.4X +HiveHasher 23 23 0 92.6 10.8 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 24: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 23 24 0 89.4 11.2 1.0X -xxHash 64-bit 14 14 0 145.9 6.9 1.6X -HiveHasher 33 33 0 63.2 15.8 0.7X +Murmur3_x86_32 24 24 0 89.0 11.2 1.0X +xxHash 64-bit 14 15 0 145.4 6.9 1.6X +HiveHasher 33 33 0 62.9 15.9 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 31: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 36 36 0 58.7 17.0 1.0X -xxHash 64-bit 27 28 0 76.4 13.1 1.3X -HiveHasher 42 44 5 49.4 20.2 0.8X +Murmur3_x86_32 36 36 0 59.1 16.9 1.0X +xxHash 64-bit 28 28 0 76.0 13.2 1.3X +HiveHasher 43 44 4 49.2 20.3 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 95: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 87 87 0 24.2 41.3 1.0X -xxHash 64-bit 61 62 0 34.1 29.3 1.4X -HiveHasher 158 158 0 13.3 75.2 0.5X +Murmur3_x86_32 87 87 0 24.0 41.6 1.0X +xxHash 64-bit 64 64 0 32.9 30.4 1.4X +HiveHasher 159 159 0 13.2 75.6 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 287: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 244 244 0 8.6 116.3 1.0X -xxHash 64-bit 117 117 1 18.0 55.6 2.1X -HiveHasher 531 531 0 3.9 253.4 0.5X +Murmur3_x86_32 244 245 1 8.6 116.4 1.0X +xxHash 64-bit 123 123 0 17.1 58.5 2.0X +HiveHasher 534 535 0 3.9 254.8 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 1055: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 873 873 0 2.4 416.1 1.0X -xxHash 64-bit 387 388 1 5.4 184.6 2.3X -HiveHasher 2032 2032 0 1.0 968.7 0.4X +Murmur3_x86_32 878 879 0 2.4 418.9 1.0X +xxHash 64-bit 400 401 1 5.2 190.9 2.2X +HiveHasher 2045 2045 0 1.0 974.9 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 2079: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 1704 1704 0 1.2 812.5 1.0X -xxHash 64-bit 762 763 1 2.8 363.2 2.2X -HiveHasher 4024 4024 0 0.5 1918.7 0.4X +Murmur3_x86_32 1715 1715 0 1.2 817.9 1.0X +xxHash 64-bit 782 782 0 2.7 372.9 2.2X +HiveHasher 4050 4066 22 0.5 1931.3 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Hash byte arrays with length 8223: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Murmur3_x86_32 6702 6703 1 0.3 3195.8 1.0X -xxHash 64-bit 2999 3003 6 0.7 1429.8 2.2X -HiveHasher 15981 15981 1 0.1 7620.1 0.4X +Murmur3_x86_32 6744 6747 3 0.3 3216.0 1.0X +xxHash 64-bit 3043 3044 1 0.7 1451.2 2.2X +HiveHasher 16085 16085 0 0.1 7669.8 0.4X diff --git a/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt index c1b127d9e7884..88eee350370b9 100644 --- a/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt @@ -2,13 +2,13 @@ UnHex Comparison ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Cardinality 1000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Common Codecs 4444 4451 11 0.2 4444.1 1.0X -Java 5500 5533 41 0.2 5500.5 0.8X -Spark 3466 3472 6 0.3 3466.0 1.3X -Spark Binary 2625 2627 2 0.4 2625.3 1.7X +Common Codecs 4912 4952 35 0.2 4912.5 1.0X +Java 5772 5781 14 0.2 5772.1 0.9X +Spark 3482 3488 10 0.3 3482.0 1.4X +Spark Binary 2638 2639 0 0.4 2638.3 1.9X diff --git a/sql/catalyst/benchmarks/HexBenchmark-results.txt b/sql/catalyst/benchmarks/HexBenchmark-results.txt index c544346c34d33..adc459ceb8c7c 100644 --- a/sql/catalyst/benchmarks/HexBenchmark-results.txt +++ b/sql/catalyst/benchmarks/HexBenchmark-results.txt @@ -2,13 +2,13 @@ UnHex Comparison ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Cardinality 1000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Common Codecs 4794 4800 6 0.2 4793.6 1.0X -Java 4247 4262 16 0.2 4247.2 1.1X -Spark 3957 3963 8 0.3 3957.5 1.2X -Spark Binary 2743 2745 2 0.4 2743.4 1.7X +Common Codecs 4900 4906 5 0.2 4900.0 1.0X +Java 4133 4143 10 0.2 4133.2 1.2X +Spark 3987 3988 1 0.3 3986.6 1.2X +Spark Binary 2762 2766 3 0.4 2761.6 1.8X diff --git a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt index 1cdf1d8e42753..e852e1f715ba4 100644 --- a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt @@ -1,7 +1,7 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor internal row comparable wrapper: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -toSet 100 102 3 2.0 500.5 1.0X -mergePartitions 183 185 2 1.1 913.5 0.5X +toSet 100 102 3 2.0 501.6 1.0X +mergePartitions 180 182 2 1.1 900.1 0.6X diff --git a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt index b920e5255016e..705c53b53bb3a 100644 --- a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt +++ b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt @@ -1,7 +1,7 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor internal row comparable wrapper: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -toSet 113 115 3 1.8 566.2 1.0X -mergePartitions 206 208 2 1.0 1030.5 0.5X +toSet 114 116 2 1.8 570.9 1.0X +mergePartitions 208 209 1 1.0 1040.6 0.5X diff --git a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt index 384cce30b67aa..c58763bed6876 100644 --- a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt +++ b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt @@ -2,13 +2,13 @@ unsafe projection ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor unsafe projection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -single long 1327 1327 0 202.4 4.9 1.0X -single nullable long 2362 2377 22 113.6 8.8 0.6X -7 primitive types 7062 7064 2 38.0 26.3 0.2X -7 nullable primitive types 10610 10625 21 25.3 39.5 0.1X +single long 1330 1330 0 201.8 5.0 1.0X +single nullable long 2375 2389 20 113.0 8.8 0.6X +7 primitive types 7116 7120 6 37.7 26.5 0.2X +7 nullable primitive types 10688 10694 8 25.1 39.8 0.1X diff --git a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt index 60c49d2917eb5..06fb444c3e730 100644 --- a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt +++ b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt @@ -2,13 +2,13 @@ unsafe projection ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor unsafe projection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -single long 1363 1363 1 197.0 5.1 1.0X -single nullable long 2454 2456 2 109.4 9.1 0.6X -7 primitive types 6944 6946 2 38.7 25.9 0.2X -7 nullable primitive types 10300 10314 19 26.1 38.4 0.1X +single long 1380 1382 3 194.5 5.1 1.0X +single nullable long 2449 2450 2 109.6 9.1 0.6X +7 primitive types 7002 7003 2 38.3 26.1 0.2X +7 nullable primitive types 10355 10370 20 25.9 38.6 0.1X diff --git a/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt b/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt index 50a31e7e73bb1..0c14099f23b73 100644 --- a/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt @@ -2,147 +2,147 @@ aggregate without grouping ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 35098 35975 1240 59.8 16.7 1.0X -agg w/o group wholestage on 2835 2844 9 739.9 1.4 12.4X +agg w/o group wholestage off 35412 35848 616 59.2 16.9 1.0X +agg w/o group wholestage on 2849 2853 5 736.1 1.4 12.4X ================================================================================================ stat functions ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -stddev wholestage off 4102 4138 51 25.6 39.1 1.0X -stddev wholestage on 974 983 6 107.6 9.3 4.2X +stddev wholestage off 4533 4585 74 23.1 43.2 1.0X +stddev wholestage on 987 991 4 106.3 9.4 4.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -kurtosis wholestage off 21188 21367 253 4.9 202.1 1.0X -kurtosis wholestage on 992 993 2 105.7 9.5 21.4X +kurtosis wholestage off 21221 21417 277 4.9 202.4 1.0X +kurtosis wholestage on 995 1000 4 105.4 9.5 21.3X ================================================================================================ aggregate with linear keys ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 6757 6835 110 12.4 80.5 1.0X -codegen = T, hashmap = F 3850 4003 160 21.8 45.9 1.8X -codegen = T, row-based hashmap = T 1222 1238 15 68.6 14.6 5.5X -codegen = T, vectorized hashmap = T 804 814 9 104.3 9.6 8.4X +codegen = F 6703 6748 64 12.5 79.9 1.0X +codegen = T, hashmap = F 3746 3777 36 22.4 44.7 1.8X +codegen = T, row-based hashmap = T 1226 1233 5 68.4 14.6 5.5X +codegen = T, vectorized hashmap = T 812 820 12 103.4 9.7 8.3X ================================================================================================ aggregate with randomized keys ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 7331 7374 60 11.4 87.4 1.0X -codegen = T, hashmap = F 4664 4687 24 18.0 55.6 1.6X -codegen = T, row-based hashmap = T 1620 1627 7 51.8 19.3 4.5X -codegen = T, vectorized hashmap = T 1113 1171 72 75.4 13.3 6.6X +codegen = F 7454 7484 42 11.3 88.9 1.0X +codegen = T, hashmap = F 4485 4585 87 18.7 53.5 1.7X +codegen = T, row-based hashmap = T 1676 1685 9 50.1 20.0 4.4X +codegen = T, vectorized hashmap = T 1061 1110 85 79.0 12.7 7.0X ================================================================================================ aggregate with string key ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 2485 2510 35 8.4 118.5 1.0X -codegen = T, hashmap = F 1519 1529 10 13.8 72.4 1.6X -codegen = T, row-based hashmap = T 994 1010 16 21.1 47.4 2.5X -codegen = T, vectorized hashmap = T 804 815 11 26.1 38.3 3.1X +codegen = F 2296 2326 43 9.1 109.5 1.0X +codegen = T, hashmap = F 1499 1512 12 14.0 71.5 1.5X +codegen = T, row-based hashmap = T 996 1013 21 21.1 47.5 2.3X +codegen = T, vectorized hashmap = T 798 801 2 26.3 38.1 2.9X ================================================================================================ aggregate with decimal key ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 2018 2050 46 10.4 96.2 1.0X -codegen = T, hashmap = F 1305 1318 18 16.1 62.2 1.5X -codegen = T, row-based hashmap = T 499 505 6 42.0 23.8 4.0X -codegen = T, vectorized hashmap = T 313 317 4 67.0 14.9 6.4X +codegen = F 2042 2096 76 10.3 97.4 1.0X +codegen = T, hashmap = F 1362 1374 18 15.4 64.9 1.5X +codegen = T, row-based hashmap = T 479 501 17 43.8 22.8 4.3X +codegen = T, vectorized hashmap = T 312 326 13 67.3 14.9 6.6X ================================================================================================ aggregate with multiple key types ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 4453 4457 6 4.7 212.3 1.0X -codegen = T, hashmap = F 2320 2333 20 9.0 110.6 1.9X -codegen = T, row-based hashmap = T 1821 1826 6 11.5 86.8 2.4X -codegen = T, vectorized hashmap = T 1600 1652 74 13.1 76.3 2.8X +codegen = F 4229 4241 16 5.0 201.7 1.0X +codegen = T, hashmap = F 2316 2320 6 9.1 110.4 1.8X +codegen = T, row-based hashmap = T 1819 1827 11 11.5 86.8 2.3X +codegen = T, vectorized hashmap = T 1518 1519 1 13.8 72.4 2.8X ================================================================================================ max function bytecode size of wholestagecodegen ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 358 384 22 1.8 545.8 1.0X -codegen = T, hugeMethodLimit = 10000 134 160 24 4.9 204.1 2.7X -codegen = T, hugeMethodLimit = 1500 129 145 16 5.1 196.1 2.8X +codegen = F 368 389 23 1.8 561.7 1.0X +codegen = T, hugeMethodLimit = 10000 143 162 16 4.6 218.9 2.6X +codegen = T, hugeMethodLimit = 1500 140 154 12 4.7 214.1 2.6X ================================================================================================ cube ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cube wholestage off 1962 1973 16 2.7 374.2 1.0X -cube wholestage on 1054 1075 24 5.0 201.0 1.9X +cube wholestage off 2051 2074 33 2.6 391.2 1.0X +cube wholestage on 1065 1078 10 4.9 203.1 1.9X ================================================================================================ hash and BytesToBytesMap ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeRowhash 146 147 3 143.7 7.0 1.0X -murmur3 hash 53 54 1 392.4 2.5 2.7X -fast hash 24 24 0 887.7 1.1 6.2X -arrayEqual 136 136 0 153.9 6.5 1.1X -Java HashMap (Long) 62 72 8 338.3 3.0 2.4X -Java HashMap (two ints) 85 88 2 245.8 4.1 1.7X -Java HashMap (UnsafeRow) 492 495 2 42.6 23.5 0.3X -LongToUnsafeRowMap (opt=false) 350 354 3 59.9 16.7 0.4X -LongToUnsafeRowMap (opt=true) 79 82 5 263.9 3.8 1.8X -BytesToBytesMap (off Heap) 459 471 12 45.7 21.9 0.3X -BytesToBytesMap (on Heap) 466 468 2 45.0 22.2 0.3X -Aggregate HashMap 30 30 2 697.8 1.4 4.9X +UnsafeRowhash 146 147 1 143.2 7.0 1.0X +murmur3 hash 54 55 4 390.7 2.6 2.7X +fast hash 24 24 0 883.4 1.1 6.2X +arrayEqual 137 137 0 153.2 6.5 1.1X +Java HashMap (Long) 61 67 7 344.8 2.9 2.4X +Java HashMap (two ints) 76 79 2 275.1 3.6 1.9X +Java HashMap (UnsafeRow) 531 533 2 39.5 25.3 0.3X +LongToUnsafeRowMap (opt=false) 346 349 4 60.6 16.5 0.4X +LongToUnsafeRowMap (opt=true) 80 80 1 262.5 3.8 1.8X +BytesToBytesMap (off Heap) 440 443 4 47.6 21.0 0.3X +BytesToBytesMap (on Heap) 456 459 4 46.0 21.7 0.3X +Aggregate HashMap 30 31 0 689.1 1.5 4.8X diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt index f1118da89122d..73ca9abe75a6b 100644 --- a/sql/core/benchmarks/AggregateBenchmark-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -2,147 +2,147 @@ aggregate without grouping ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 37435 38685 1769 56.0 17.9 1.0X -agg w/o group wholestage on 3364 3369 3 623.4 1.6 11.1X +agg w/o group wholestage off 39769 40576 1142 52.7 19.0 1.0X +agg w/o group wholestage on 3381 3388 13 620.3 1.6 11.8X ================================================================================================ stat functions ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -stddev wholestage off 4461 4505 63 23.5 42.5 1.0X -stddev wholestage on 976 980 3 107.5 9.3 4.6X +stddev wholestage off 4522 4526 7 23.2 43.1 1.0X +stddev wholestage on 987 992 7 106.2 9.4 4.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -kurtosis wholestage off 20698 20799 143 5.1 197.4 1.0X -kurtosis wholestage on 990 992 2 105.9 9.4 20.9X +kurtosis wholestage off 21070 21074 5 5.0 200.9 1.0X +kurtosis wholestage on 994 998 4 105.5 9.5 21.2X ================================================================================================ aggregate with linear keys ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 6646 6677 43 12.6 79.2 1.0X -codegen = T, hashmap = F 4024 4116 118 20.8 48.0 1.7X -codegen = T, row-based hashmap = T 1240 1255 13 67.7 14.8 5.4X -codegen = T, vectorized hashmap = T 816 838 14 102.8 9.7 8.1X +codegen = F 7074 7120 64 11.9 84.3 1.0X +codegen = T, hashmap = F 3968 4028 63 21.1 47.3 1.8X +codegen = T, row-based hashmap = T 1211 1214 3 69.3 14.4 5.8X +codegen = T, vectorized hashmap = T 827 846 15 101.5 9.9 8.6X ================================================================================================ aggregate with randomized keys ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 7348 7361 18 11.4 87.6 1.0X -codegen = T, hashmap = F 4766 4799 30 17.6 56.8 1.5X -codegen = T, row-based hashmap = T 1712 1734 23 49.0 20.4 4.3X -codegen = T, vectorized hashmap = T 1052 1057 5 79.7 12.5 7.0X +codegen = F 7673 7686 18 10.9 91.5 1.0X +codegen = T, hashmap = F 4857 4875 20 17.3 57.9 1.6X +codegen = T, row-based hashmap = T 1702 1710 14 49.3 20.3 4.5X +codegen = T, vectorized hashmap = T 1077 1115 48 77.9 12.8 7.1X ================================================================================================ aggregate with string key ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 2303 2306 3 9.1 109.8 1.0X -codegen = T, hashmap = F 1467 1472 7 14.3 70.0 1.6X -codegen = T, row-based hashmap = T 989 998 9 21.2 47.2 2.3X -codegen = T, vectorized hashmap = T 794 799 4 26.4 37.9 2.9X +codegen = F 2478 2506 40 8.5 118.2 1.0X +codegen = T, hashmap = F 1510 1517 10 13.9 72.0 1.6X +codegen = T, row-based hashmap = T 1008 1019 19 20.8 48.1 2.5X +codegen = T, vectorized hashmap = T 779 790 8 26.9 37.1 3.2X ================================================================================================ aggregate with decimal key ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 2026 2039 18 10.4 96.6 1.0X -codegen = T, hashmap = F 1349 1352 4 15.5 64.3 1.5X -codegen = T, row-based hashmap = T 437 445 6 47.9 20.9 4.6X -codegen = T, vectorized hashmap = T 316 322 5 66.4 15.1 6.4X +codegen = F 2118 2182 92 9.9 101.0 1.0X +codegen = T, hashmap = F 1307 1334 38 16.0 62.3 1.6X +codegen = T, row-based hashmap = T 433 436 2 48.4 20.7 4.9X +codegen = T, vectorized hashmap = T 320 327 7 65.6 15.2 6.6X ================================================================================================ aggregate with multiple key types ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 4168 4201 47 5.0 198.7 1.0X -codegen = T, hashmap = F 2412 2418 8 8.7 115.0 1.7X -codegen = T, row-based hashmap = T 1661 1663 2 12.6 79.2 2.5X -codegen = T, vectorized hashmap = T 1606 1610 5 13.1 76.6 2.6X +codegen = F 4200 4204 7 5.0 200.3 1.0X +codegen = T, hashmap = F 2346 2384 53 8.9 111.9 1.8X +codegen = T, row-based hashmap = T 1869 1875 9 11.2 89.1 2.2X +codegen = T, vectorized hashmap = T 1608 1612 6 13.0 76.7 2.6X ================================================================================================ max function bytecode size of wholestagecodegen ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 385 401 10 1.7 587.4 1.0X -codegen = T, hugeMethodLimit = 10000 141 157 12 4.7 214.5 2.7X -codegen = T, hugeMethodLimit = 1500 127 141 11 5.2 193.9 3.0X +codegen = F 394 430 27 1.7 600.5 1.0X +codegen = T, hugeMethodLimit = 10000 140 156 15 4.7 213.5 2.8X +codegen = T, hugeMethodLimit = 1500 136 144 6 4.8 207.1 2.9X ================================================================================================ cube ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cube wholestage off 1943 1948 7 2.7 370.6 1.0X -cube wholestage on 1110 1130 17 4.7 211.7 1.8X +cube wholestage off 1971 2004 47 2.7 376.0 1.0X +cube wholestage on 1129 1158 25 4.6 215.3 1.7X ================================================================================================ hash and BytesToBytesMap ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeRowhash 203 204 2 103.4 9.7 1.0X -murmur3 hash 68 69 1 308.6 3.2 3.0X -fast hash 71 71 0 296.3 3.4 2.9X -arrayEqual 144 145 1 145.7 6.9 1.4X -Java HashMap (Long) 66 69 4 318.6 3.1 3.1X -Java HashMap (two ints) 80 84 10 263.7 3.8 2.5X -Java HashMap (UnsafeRow) 532 536 3 39.4 25.4 0.4X -LongToUnsafeRowMap (opt=false) 335 337 1 62.6 16.0 0.6X -LongToUnsafeRowMap (opt=true) 78 78 1 269.7 3.7 2.6X -BytesToBytesMap (off Heap) 484 487 3 43.4 23.1 0.4X -BytesToBytesMap (on Heap) 484 491 5 43.4 23.1 0.4X -Aggregate HashMap 30 31 1 690.1 1.4 6.7X +UnsafeRowhash 204 204 1 102.9 9.7 1.0X +murmur3 hash 69 70 0 301.7 3.3 2.9X +fast hash 71 72 1 294.3 3.4 2.9X +arrayEqual 144 145 1 145.1 6.9 1.4X +Java HashMap (Long) 66 69 5 319.8 3.1 3.1X +Java HashMap (two ints) 85 87 2 247.5 4.0 2.4X +Java HashMap (UnsafeRow) 547 554 12 38.3 26.1 0.4X +LongToUnsafeRowMap (opt=false) 347 348 1 60.4 16.6 0.6X +LongToUnsafeRowMap (opt=true) 74 74 1 285.3 3.5 2.8X +BytesToBytesMap (off Heap) 487 490 2 43.0 23.2 0.4X +BytesToBytesMap (on Heap) 511 514 2 41.0 24.4 0.4X +Aggregate HashMap 30 30 0 703.2 1.4 6.8X diff --git a/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt b/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt index dd6aabd2695fd..5bbbe6c90a83d 100644 --- a/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt @@ -1,28 +1,28 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor year month interval one column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -year month interval one column enable radix 23157 23546 354 4.3 231.6 1.0X -year month interval one column disable radix 33035 33049 14 3.0 330.3 0.7X +year month interval one column enable radix 23762 24352 802 4.2 237.6 1.0X +year month interval one column disable radix 33034 33062 30 3.0 330.3 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor year month interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -year month interval two columns enable radix 33726 33825 89 3.0 337.3 1.0X -year month interval two columns disable radix 33759 34063 472 3.0 337.6 1.0X +year month interval two columns enable radix 33122 33540 609 3.0 331.2 1.0X +year month interval two columns disable radix 34128 34610 427 2.9 341.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor day time interval one columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -day time interval one columns enable radix 23123 23169 61 4.3 231.2 1.0X -day time interval one columns disable radix 34121 34201 96 2.9 341.2 0.7X +day time interval one columns enable radix 21937 22337 373 4.6 219.4 1.0X +day time interval one columns disable radix 32194 32530 557 3.1 321.9 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor day time interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -day time interval two columns enable radix 35022 35250 212 2.9 350.2 1.0X -day time interval two columns disable radix 35240 35498 224 2.8 352.4 1.0X +day time interval two columns enable radix 33037 33149 97 3.0 330.4 1.0X +day time interval two columns disable radix 32944 33036 103 3.0 329.4 1.0X diff --git a/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt b/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt index e8aadd025df2d..67acf452919e7 100644 --- a/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt +++ b/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt @@ -1,28 +1,28 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor year month interval one column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -year month interval one column enable radix 22561 22685 121 4.4 225.6 1.0X -year month interval one column disable radix 32247 32353 132 3.1 322.5 0.7X +year month interval one column enable radix 22822 22918 117 4.4 228.2 1.0X +year month interval one column disable radix 32739 33177 697 3.1 327.4 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor year month interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -year month interval two columns enable radix 33236 33446 207 3.0 332.4 1.0X -year month interval two columns disable radix 34800 34873 63 2.9 348.0 1.0X +year month interval two columns enable radix 33341 33730 549 3.0 333.4 1.0X +year month interval two columns disable radix 33557 33961 565 3.0 335.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor day time interval one columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -day time interval one columns enable radix 21978 22020 66 4.5 219.8 1.0X -day time interval one columns disable radix 33183 33211 38 3.0 331.8 0.7X +day time interval one columns enable radix 21059 21193 133 4.7 210.6 1.0X +day time interval one columns disable radix 32196 32724 660 3.1 322.0 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor day time interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -day time interval two columns enable radix 34526 34670 185 2.9 345.3 1.0X -day time interval two columns disable radix 35632 35826 191 2.8 356.3 1.0X +day time interval two columns enable radix 33887 34787 793 3.0 338.9 1.0X +day time interval two columns disable radix 35163 35274 181 2.8 351.6 1.0X diff --git a/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt b/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt index 52092328fd576..ab6e5283dba89 100644 --- a/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt +++ b/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt @@ -1,56 +1,56 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 1974 2002 47 10.1 98.7 1.0X -apache 10784 10862 90 1.9 539.2 0.2X +java 2144 2180 49 9.3 107.2 1.0X +apache 11251 11315 62 1.8 562.6 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 2431 2445 12 8.2 121.6 1.0X -apache 12049 12094 41 1.7 602.5 0.2X +java 2446 2450 4 8.2 122.3 1.0X +apache 12449 12535 76 1.6 622.4 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 2857 2867 14 7.0 142.8 1.0X -apache 13281 13344 56 1.5 664.0 0.2X +java 2873 2875 3 7.0 143.7 1.0X +apache 13571 13613 41 1.5 678.5 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 2943 2973 50 6.8 147.1 1.0X -apache 14384 14421 32 1.4 719.2 0.2X +java 2910 2918 9 6.9 145.5 1.0X +apache 14577 14593 25 1.4 728.9 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 3435 3439 4 5.8 171.7 1.0X -apache 12572 12615 40 1.6 628.6 0.3X +java 3169 3171 2 6.3 158.5 1.0X +apache 12500 12611 100 1.6 625.0 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 4040 4052 11 5.0 202.0 1.0X -apache 14274 14363 120 1.4 713.7 0.3X +java 4313 4314 2 4.6 215.7 1.0X +apache 14491 14571 70 1.4 724.5 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 4756 4767 17 4.2 237.8 1.0X -apache 16291 16304 20 1.2 814.6 0.3X +java 4932 4951 24 4.1 246.6 1.0X +apache 15728 15782 49 1.3 786.4 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 5200 5209 8 3.8 260.0 1.0X -apache 17434 17540 101 1.1 871.7 0.3X +java 5328 5330 2 3.8 266.4 1.0X +apache 17182 17223 36 1.2 859.1 0.3X diff --git a/sql/core/benchmarks/Base64Benchmark-results.txt b/sql/core/benchmarks/Base64Benchmark-results.txt index 3e8d7e2727c34..12f54feeed1d5 100644 --- a/sql/core/benchmarks/Base64Benchmark-results.txt +++ b/sql/core/benchmarks/Base64Benchmark-results.txt @@ -1,56 +1,56 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 2287 2338 50 8.7 114.3 1.0X -apache 10870 10993 126 1.8 543.5 0.2X +java 2097 2144 64 9.5 104.9 1.0X +apache 11350 11380 29 1.8 567.5 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 2930 2945 21 6.8 146.5 1.0X -apache 12069 12172 108 1.7 603.4 0.2X +java 2624 2631 11 7.6 131.2 1.0X +apache 12395 12421 40 1.6 619.7 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 3403 3410 7 5.9 170.1 1.0X -apache 13236 13327 139 1.5 661.8 0.3X +java 3196 3212 14 6.3 159.8 1.0X +apache 13591 13708 167 1.5 679.6 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 3914 3947 29 5.1 195.7 1.0X -apache 14411 14441 27 1.4 720.6 0.3X +java 3665 3679 12 5.5 183.3 1.0X +apache 14907 14947 42 1.3 745.4 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 3572 3580 9 5.6 178.6 1.0X -apache 12652 12656 7 1.6 632.6 0.3X +java 3319 3362 37 6.0 166.0 1.0X +apache 12471 12532 64 1.6 623.5 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 4918 4919 2 4.1 245.9 1.0X -apache 14579 14601 20 1.4 728.9 0.3X +java 4068 4068 0 4.9 203.4 1.0X +apache 14651 14708 53 1.4 732.5 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 5939 5949 10 3.4 296.9 1.0X -apache 16626 16675 58 1.2 831.3 0.4X +java 5456 5463 12 3.7 272.8 1.0X +apache 16405 16460 69 1.2 820.2 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor decode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -java 6666 6672 5 3.0 333.3 1.0X -apache 18901 18922 24 1.1 945.0 0.4X +java 6111 6119 8 3.3 305.5 1.0X +apache 17824 17959 117 1.1 891.2 0.3X diff --git a/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt index 5cf56352fa761..2d1b73ac4e241 100644 --- a/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt @@ -2,195 +2,195 @@ ORC Write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter 8070 8132 88 12.4 80.7 1.0X -With bloom filter 10025 10082 81 10.0 100.2 0.8X +Without bloom filter 7949 7971 31 12.6 79.5 1.0X +With bloom filter 9864 9897 47 10.1 98.6 0.8X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 2097152 882 890 7 113.4 8.8 1.0X -With bloom filter, blocksize: 2097152 567 577 10 176.4 5.7 1.6X +Without bloom filter, blocksize: 2097152 895 941 40 111.7 9.0 1.0X +With bloom filter, blocksize: 2097152 838 876 34 119.3 8.4 1.1X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 4194304 810 836 22 123.4 8.1 1.0X -With bloom filter, blocksize: 4194304 550 568 22 181.8 5.5 1.5X +Without bloom filter, blocksize: 4194304 1345 1348 4 74.3 13.5 1.0X +With bloom filter, blocksize: 4194304 835 842 6 119.7 8.4 1.6X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 6291456 823 836 11 121.5 8.2 1.0X -With bloom filter, blocksize: 6291456 540 563 17 185.3 5.4 1.5X +Without bloom filter, blocksize: 6291456 1344 1351 11 74.4 13.4 1.0X +With bloom filter, blocksize: 6291456 814 827 12 122.8 8.1 1.7X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 8388608 797 821 21 125.5 8.0 1.0X -With bloom filter, blocksize: 8388608 533 553 23 187.5 5.3 1.5X +Without bloom filter, blocksize: 8388608 1352 1363 16 74.0 13.5 1.0X +With bloom filter, blocksize: 8388608 812 819 11 123.1 8.1 1.7X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 12582912 859 876 15 116.4 8.6 1.0X -With bloom filter, blocksize: 12582912 545 576 22 183.4 5.5 1.6X +Without bloom filter, blocksize: 12582912 1347 1356 13 74.3 13.5 1.0X +With bloom filter, blocksize: 12582912 816 830 20 122.6 8.2 1.7X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 16777216 810 841 26 123.4 8.1 1.0X -With bloom filter, blocksize: 16777216 554 575 15 180.5 5.5 1.5X +Without bloom filter, blocksize: 16777216 1322 1322 0 75.7 13.2 1.0X +With bloom filter, blocksize: 16777216 793 803 10 126.1 7.9 1.7X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 33554432 845 852 7 118.4 8.4 1.0X -With bloom filter, blocksize: 33554432 545 564 16 183.4 5.5 1.5X +Without bloom filter, blocksize: 33554432 1315 1324 12 76.0 13.2 1.0X +With bloom filter, blocksize: 33554432 790 810 32 126.6 7.9 1.7X ================================================================================================ Parquet Write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Without bloom filter 12141 12156 21 8.2 121.4 1.0X -With bloom filter 21175 21296 172 4.7 211.7 0.6X -With adaptive bloom filter & 3 candidates 20846 20897 71 4.8 208.5 0.6X -With adaptive bloom filter & 5 candidates 20731 20989 365 4.8 207.3 0.6X -With adaptive bloom filter & 9 candidates 23208 23264 79 4.3 232.1 0.5X -With adaptive bloom filter & 15 candidates 23293 23349 78 4.3 232.9 0.5X +Without bloom filter 10145 10239 133 9.9 101.4 1.0X +With bloom filter 21381 21403 32 4.7 213.8 0.5X +With adaptive bloom filter & 3 candidates 21248 21357 154 4.7 212.5 0.5X +With adaptive bloom filter & 5 candidates 21353 21396 60 4.7 213.5 0.5X +With adaptive bloom filter & 9 candidates 21141 21175 48 4.7 211.4 0.5X +With adaptive bloom filter & 15 candidates 21062 21121 83 4.7 210.6 0.5X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 2097152 451 502 37 221.9 4.5 1.0X -With bloom filter, blocksize: 2097152 174 186 12 573.8 1.7 2.6X +Without bloom filter, blocksize: 2097152 426 455 27 234.8 4.3 1.0X +With bloom filter, blocksize: 2097152 182 188 6 550.1 1.8 2.3X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 4194304 404 409 4 247.6 4.0 1.0X -With bloom filter, blocksize: 4194304 139 150 7 719.2 1.4 2.9X +Without bloom filter, blocksize: 4194304 406 416 9 246.5 4.1 1.0X +With bloom filter, blocksize: 4194304 129 135 4 772.6 1.3 3.1X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 6291456 416 423 7 240.5 4.2 1.0X -With bloom filter, blocksize: 6291456 141 152 10 709.9 1.4 3.0X +Without bloom filter, blocksize: 6291456 405 409 3 247.0 4.0 1.0X +With bloom filter, blocksize: 6291456 133 142 7 749.6 1.3 3.0X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 8388608 419 432 10 238.6 4.2 1.0X -With bloom filter, blocksize: 8388608 210 223 7 476.2 2.1 2.0X +Without bloom filter, blocksize: 8388608 413 423 9 242.3 4.1 1.0X +With bloom filter, blocksize: 8388608 162 169 5 616.2 1.6 2.5X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 12582912 422 430 9 236.8 4.2 1.0X -With bloom filter, blocksize: 12582912 325 330 4 307.2 3.3 1.3X +Without bloom filter, blocksize: 12582912 419 436 24 238.5 4.2 1.0X +With bloom filter, blocksize: 12582912 345 355 7 289.7 3.5 1.2X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 16777216 420 436 22 238.3 4.2 1.0X -With bloom filter, blocksize: 16777216 398 428 29 251.2 4.0 1.1X +Without bloom filter, blocksize: 16777216 455 469 12 219.8 4.5 1.0X +With bloom filter, blocksize: 16777216 353 425 44 283.5 3.5 1.3X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 33554432 428 439 9 233.5 4.3 1.0X -With bloom filter, blocksize: 33554432 430 441 15 232.4 4.3 1.0X +Without bloom filter, blocksize: 33554432 448 459 7 223.4 4.5 1.0X +With bloom filter, blocksize: 33554432 419 429 8 238.5 4.2 1.1X diff --git a/sql/core/benchmarks/BloomFilterBenchmark-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-results.txt index 286df98479f97..f01ad4e47f807 100644 --- a/sql/core/benchmarks/BloomFilterBenchmark-results.txt +++ b/sql/core/benchmarks/BloomFilterBenchmark-results.txt @@ -2,195 +2,195 @@ ORC Write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter 8021 8137 165 12.5 80.2 1.0X -With bloom filter 10132 10186 76 9.9 101.3 0.8X +Without bloom filter 7507 7623 165 13.3 75.1 1.0X +With bloom filter 9512 9543 45 10.5 95.1 0.8X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 2097152 876 940 61 114.2 8.8 1.0X -With bloom filter, blocksize: 2097152 588 618 21 169.9 5.9 1.5X +Without bloom filter, blocksize: 2097152 867 880 11 115.3 8.7 1.0X +With bloom filter, blocksize: 2097152 604 641 38 165.5 6.0 1.4X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 4194304 837 839 2 119.4 8.4 1.0X -With bloom filter, blocksize: 4194304 579 601 34 172.7 5.8 1.4X +Without bloom filter, blocksize: 4194304 823 839 21 121.5 8.2 1.0X +With bloom filter, blocksize: 4194304 558 564 8 179.1 5.6 1.5X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 6291456 787 797 9 127.0 7.9 1.0X -With bloom filter, blocksize: 6291456 532 548 12 188.1 5.3 1.5X +Without bloom filter, blocksize: 6291456 810 813 3 123.5 8.1 1.0X +With bloom filter, blocksize: 6291456 524 550 20 190.7 5.2 1.5X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 8388608 796 799 4 125.7 8.0 1.0X -With bloom filter, blocksize: 8388608 534 548 10 187.1 5.3 1.5X +Without bloom filter, blocksize: 8388608 798 803 7 125.4 8.0 1.0X +With bloom filter, blocksize: 8388608 560 604 32 178.7 5.6 1.4X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 12582912 836 839 3 119.7 8.4 1.0X -With bloom filter, blocksize: 12582912 517 544 19 193.4 5.2 1.6X +Without bloom filter, blocksize: 12582912 837 843 8 119.5 8.4 1.0X +With bloom filter, blocksize: 12582912 537 548 10 186.1 5.4 1.6X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 16777216 793 796 4 126.1 7.9 1.0X -With bloom filter, blocksize: 16777216 570 574 5 175.3 5.7 1.4X +Without bloom filter, blocksize: 16777216 804 806 4 124.5 8.0 1.0X +With bloom filter, blocksize: 16777216 514 526 9 194.6 5.1 1.6X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 33554432 784 794 12 127.5 7.8 1.0X -With bloom filter, blocksize: 33554432 565 587 27 177.1 5.6 1.4X +Without bloom filter, blocksize: 33554432 801 807 7 124.8 8.0 1.0X +With bloom filter, blocksize: 33554432 520 551 28 192.5 5.2 1.5X ================================================================================================ Parquet Write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Without bloom filter 11173 11180 11 9.0 111.7 1.0X -With bloom filter 19387 19485 138 5.2 193.9 0.6X -With adaptive bloom filter & 3 candidates 19252 19395 202 5.2 192.5 0.6X -With adaptive bloom filter & 5 candidates 19204 19337 188 5.2 192.0 0.6X -With adaptive bloom filter & 9 candidates 19267 19380 160 5.2 192.7 0.6X -With adaptive bloom filter & 15 candidates 19144 19184 57 5.2 191.4 0.6X +Without bloom filter 10073 10250 250 9.9 100.7 1.0X +With bloom filter 13981 14127 206 7.2 139.8 0.7X +With adaptive bloom filter & 3 candidates 13992 14059 94 7.1 139.9 0.7X +With adaptive bloom filter & 5 candidates 14691 14804 160 6.8 146.9 0.7X +With adaptive bloom filter & 9 candidates 14634 14805 242 6.8 146.3 0.7X +With adaptive bloom filter & 15 candidates 14698 14727 41 6.8 147.0 0.7X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 2097152 447 476 24 223.6 4.5 1.0X -With bloom filter, blocksize: 2097152 177 185 5 565.6 1.8 2.5X +Without bloom filter, blocksize: 2097152 438 457 17 228.1 4.4 1.0X +With bloom filter, blocksize: 2097152 173 182 9 576.9 1.7 2.5X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 4194304 424 440 14 236.0 4.2 1.0X -With bloom filter, blocksize: 4194304 127 135 7 790.4 1.3 3.3X +Without bloom filter, blocksize: 4194304 416 423 7 240.4 4.2 1.0X +With bloom filter, blocksize: 4194304 116 124 7 864.6 1.2 3.6X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 6291456 423 439 16 236.2 4.2 1.0X -With bloom filter, blocksize: 6291456 130 139 9 768.6 1.3 3.3X +Without bloom filter, blocksize: 6291456 412 421 13 242.9 4.1 1.0X +With bloom filter, blocksize: 6291456 145 153 6 687.4 1.5 2.8X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter, blocksize: 8388608 426 435 7 235.0 4.3 1.0X -With bloom filter, blocksize: 8388608 204 214 6 489.3 2.0 2.1X +Without bloom filter, blocksize: 8388608 417 423 5 240.0 4.2 1.0X +With bloom filter, blocksize: 8388608 158 164 5 634.4 1.6 2.6X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 12582912 426 447 23 234.5 4.3 1.0X -With bloom filter, blocksize: 12582912 295 306 8 339.2 2.9 1.4X +Without bloom filter, blocksize: 12582912 413 414 2 242.4 4.1 1.0X +With bloom filter, blocksize: 12582912 312 318 5 320.5 3.1 1.3X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 16777216 427 441 9 234.0 4.3 1.0X -With bloom filter, blocksize: 16777216 372 392 12 268.5 3.7 1.1X +Without bloom filter, blocksize: 16777216 418 424 4 239.1 4.2 1.0X +With bloom filter, blocksize: 16777216 368 417 62 271.5 3.7 1.1X ================================================================================================ Parquet Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Without bloom filter, blocksize: 33554432 508 524 14 197.0 5.1 1.0X -With bloom filter, blocksize: 33554432 439 463 31 227.7 4.4 1.2X +Without bloom filter, blocksize: 33554432 485 503 21 206.3 4.8 1.0X +With bloom filter, blocksize: 33554432 429 457 41 233.0 4.3 1.1X diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt index ac33c0edbcd24..3bb4debe8d59f 100644 --- a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt @@ -2,69 +2,69 @@ Parquet writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet(PARQUET_1_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1630 1688 82 9.7 103.6 1.0X -Output Single Double Column 1848 1854 10 8.5 117.5 0.9X -Output Int and String Column 4604 4635 44 3.4 292.7 0.4X -Output Partitions 3399 3432 46 4.6 216.1 0.5X -Output Buckets 4919 4925 9 3.2 312.7 0.3X +Output Single Int Column 1793 1815 30 8.8 114.0 1.0X +Output Single Double Column 1935 2005 99 8.1 123.0 0.9X +Output Int and String Column 4319 4531 299 3.6 274.6 0.4X +Output Partitions 3240 3261 29 4.9 206.0 0.6X +Output Buckets 4415 4427 17 3.6 280.7 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet(PARQUET_2_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1917 1930 19 8.2 121.9 1.0X -Output Single Double Column 1739 1765 35 9.0 110.6 1.1X -Output Int and String Column 5231 5240 13 3.0 332.6 0.4X -Output Partitions 3531 3537 9 4.5 224.5 0.5X -Output Buckets 4815 4816 1 3.3 306.1 0.4X +Output Single Int Column 2013 2013 1 7.8 128.0 1.0X +Output Single Double Column 1892 1899 9 8.3 120.3 1.1X +Output Int and String Column 5133 5137 6 3.1 326.3 0.4X +Output Partitions 3584 3586 3 4.4 227.8 0.6X +Output Buckets 4639 4641 4 3.4 294.9 0.4X ================================================================================================ ORC writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor ORC writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1145 1150 8 13.7 72.8 1.0X -Output Single Double Column 1775 1788 18 8.9 112.8 0.6X -Output Int and String Column 4092 4104 17 3.8 260.2 0.3X -Output Partitions 2516 2532 22 6.3 160.0 0.5X -Output Buckets 3555 3574 26 4.4 226.0 0.3X +Output Single Int Column 1036 1039 4 15.2 65.9 1.0X +Output Single Double Column 1709 1719 14 9.2 108.6 0.6X +Output Int and String Column 3780 3818 54 4.2 240.3 0.3X +Output Partitions 2598 2600 3 6.1 165.2 0.4X +Output Buckets 3551 3566 21 4.4 225.8 0.3X ================================================================================================ JSON writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1710 1721 15 9.2 108.7 1.0X -Output Single Double Column 2405 2421 22 6.5 152.9 0.7X -Output Int and String Column 4262 4274 18 3.7 271.0 0.4X -Output Partitions 3190 3211 30 4.9 202.8 0.5X -Output Buckets 4134 4160 36 3.8 262.8 0.4X +Output Single Int Column 1586 1590 6 9.9 100.9 1.0X +Output Single Double Column 2260 2270 13 7.0 143.7 0.7X +Output Int and String Column 4163 4179 23 3.8 264.7 0.4X +Output Partitions 3109 3131 31 5.1 197.7 0.5X +Output Buckets 4002 4011 12 3.9 254.5 0.4X ================================================================================================ CSV writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor CSV writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 3871 3895 34 4.1 246.1 1.0X -Output Single Double Column 4379 4382 4 3.6 278.4 0.9X -Output Int and String Column 6820 6835 21 2.3 433.6 0.6X -Output Partitions 5555 5573 26 2.8 353.2 0.7X -Output Buckets 6679 6696 24 2.4 424.6 0.6X +Output Single Int Column 3556 3577 28 4.4 226.1 1.0X +Output Single Double Column 4048 4062 21 3.9 257.3 0.9X +Output Int and String Column 6714 6719 7 2.3 426.9 0.5X +Output Partitions 5340 5353 19 2.9 339.5 0.7X +Output Buckets 6447 6466 26 2.4 409.9 0.6X diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt index 56c83e0940856..5c3d4bad772a9 100644 --- a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt +++ b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt @@ -2,69 +2,69 @@ Parquet writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet(PARQUET_1_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1736 1765 40 9.1 110.4 1.0X -Output Single Double Column 1840 1879 56 8.6 117.0 0.9X -Output Int and String Column 4395 4435 57 3.6 279.4 0.4X -Output Partitions 3279 3373 132 4.8 208.5 0.5X -Output Buckets 4598 4602 6 3.4 292.3 0.4X +Output Single Int Column 1738 1772 48 9.1 110.5 1.0X +Output Single Double Column 1821 1838 25 8.6 115.8 1.0X +Output Int and String Column 4749 4776 39 3.3 301.9 0.4X +Output Partitions 3238 3272 48 4.9 205.9 0.5X +Output Buckets 4428 4461 46 3.6 281.5 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet(PARQUET_2_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1790 1801 15 8.8 113.8 1.0X -Output Single Double Column 1857 1868 17 8.5 118.0 1.0X -Output Int and String Column 4717 4735 26 3.3 299.9 0.4X -Output Partitions 3187 3212 35 4.9 202.6 0.6X -Output Buckets 4353 4358 6 3.6 276.8 0.4X +Output Single Int Column 1961 1992 44 8.0 124.7 1.0X +Output Single Double Column 1876 1878 2 8.4 119.3 1.0X +Output Int and String Column 4831 4837 8 3.3 307.2 0.4X +Output Partitions 3430 3431 2 4.6 218.1 0.6X +Output Buckets 4226 4231 6 3.7 268.7 0.5X ================================================================================================ ORC writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor ORC writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 941 953 20 16.7 59.8 1.0X -Output Single Double Column 1563 1569 8 10.1 99.4 0.6X -Output Int and String Column 3838 3868 43 4.1 244.0 0.2X -Output Partitions 2514 2542 40 6.3 159.8 0.4X -Output Buckets 3554 3555 2 4.4 225.9 0.3X +Output Single Int Column 1076 1084 12 14.6 68.4 1.0X +Output Single Double Column 1808 1811 4 8.7 114.9 0.6X +Output Int and String Column 3981 4036 78 4.0 253.1 0.3X +Output Partitions 2561 2602 59 6.1 162.8 0.4X +Output Buckets 3521 3557 51 4.5 223.9 0.3X ================================================================================================ JSON writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 1606 1613 10 9.8 102.1 1.0X -Output Single Double Column 2245 2257 17 7.0 142.7 0.7X -Output Int and String Column 3818 3837 26 4.1 242.7 0.4X -Output Partitions 3154 3181 38 5.0 200.5 0.5X -Output Buckets 4123 4132 12 3.8 262.1 0.4X +Output Single Int Column 1624 1643 27 9.7 103.2 1.0X +Output Single Double Column 2272 2275 4 6.9 144.5 0.7X +Output Int and String Column 3996 4048 73 3.9 254.1 0.4X +Output Partitions 3045 3054 13 5.2 193.6 0.5X +Output Buckets 3899 3903 6 4.0 247.9 0.4X ================================================================================================ CSV writer benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor CSV writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Output Single Int Column 3260 3286 36 4.8 207.3 1.0X -Output Single Double Column 4065 4076 15 3.9 258.4 0.8X -Output Int and String Column 6295 6310 21 2.5 400.2 0.5X -Output Partitions 5151 5177 37 3.1 327.5 0.6X -Output Buckets 6173 6209 51 2.5 392.5 0.5X +Output Single Int Column 3445 3446 2 4.6 219.0 1.0X +Output Single Double Column 3674 3717 61 4.3 233.6 0.9X +Output Int and String Column 6085 6090 7 2.6 386.9 0.6X +Output Partitions 5107 5123 22 3.1 324.7 0.7X +Output Buckets 6098 6123 35 2.6 387.7 0.6X diff --git a/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt b/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt index c650aa1efbb32..9320ff01601c4 100644 --- a/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt @@ -2,26 +2,26 @@ byte array comparisons ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Byte Array compareTo: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -2-7 byte 254 257 1 257.8 3.9 1.0X -8-16 byte 409 437 37 160.3 6.2 0.6X -16-32 byte 415 416 1 158.0 6.3 0.6X -512-1024 byte 540 542 1 121.3 8.2 0.5X -512 byte slow 1524 1553 23 43.0 23.3 0.2X -2-7 byte 313 314 1 209.5 4.8 0.8X +2-7 byte 256 259 2 256.2 3.9 1.0X +8-16 byte 412 438 34 159.1 6.3 0.6X +16-32 byte 409 411 3 160.1 6.2 0.6X +512-1024 byte 544 546 1 120.6 8.3 0.5X +512 byte slow 1543 1570 23 42.5 23.5 0.2X +2-7 byte 315 316 1 208.2 4.8 0.8X ================================================================================================ byte array equals ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Byte Array equals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Byte Array equals 538 541 8 297.6 3.4 1.0X +Byte Array equals 548 551 1 291.9 3.4 1.0X diff --git a/sql/core/benchmarks/ByteArrayBenchmark-results.txt b/sql/core/benchmarks/ByteArrayBenchmark-results.txt index 723af23b06a3f..d76d86ce54c31 100644 --- a/sql/core/benchmarks/ByteArrayBenchmark-results.txt +++ b/sql/core/benchmarks/ByteArrayBenchmark-results.txt @@ -2,26 +2,26 @@ byte array comparisons ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Byte Array compareTo: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -2-7 byte 258 259 1 254.2 3.9 1.0X -8-16 byte 392 402 11 167.4 6.0 0.7X -16-32 byte 396 398 1 165.4 6.0 0.7X -512-1024 byte 519 523 2 126.4 7.9 0.5X -512 byte slow 3255 3273 13 20.1 49.7 0.1X -2-7 byte 249 250 1 263.0 3.8 1.0X +2-7 byte 259 264 3 253.3 3.9 1.0X +8-16 byte 427 462 23 153.5 6.5 0.6X +16-32 byte 485 486 1 135.3 7.4 0.5X +512-1024 byte 609 612 3 107.7 9.3 0.4X +512 byte slow 1485 1518 30 44.1 22.7 0.2X +2-7 byte 299 301 1 218.8 4.6 0.9X ================================================================================================ byte array equals ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Byte Array equals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Byte Array equals 555 556 1 288.3 3.5 1.0X +Byte Array equals 550 552 3 290.8 3.4 1.0X diff --git a/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt index cc0b3cdaffd11..8ef55135b58f9 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt @@ -2,76 +2,76 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 25656 25710 55 0.0 513115.4 1.0X +One quoted string 24592 24650 50 0.0 491842.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 59317 59851 631 0.0 59316.9 1.0X -Select 100 columns 22419 22524 133 0.0 22419.0 2.6X -Select one column 18736 18821 95 0.1 18736.0 3.2X -count() 4289 4377 88 0.2 4289.5 13.8X -Select 100 columns, one bad input field 27081 27108 26 0.0 27080.9 2.2X -Select 100 columns, corrupt record field 30668 30949 319 0.0 30668.3 1.9X +Select 1000 columns 58745 59095 571 0.0 58745.1 1.0X +Select 100 columns 21111 21163 55 0.0 21111.4 2.8X +Select one column 17328 17405 112 0.1 17328.0 3.4X +count() 3655 4076 698 0.3 3654.7 16.1X +Select 100 columns, one bad input field 25285 25302 16 0.0 25284.7 2.3X +Select 100 columns, corrupt record field 28990 29050 52 0.0 28989.8 2.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 10795 10819 21 0.9 1079.5 1.0X -Select 1 column + count() 7409 7416 8 1.3 740.9 1.5X -count() 1712 1714 1 5.8 171.2 6.3X +Select 10 columns + count() 11213 11266 46 0.9 1121.3 1.0X +Select 1 column + count() 7751 7770 18 1.3 775.1 1.4X +count() 1657 1659 2 6.0 165.7 6.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 859 861 2 11.6 85.9 1.0X -to_csv(timestamp) 6073 6115 62 1.6 607.3 0.1X -write timestamps to files 6478 6487 7 1.5 647.8 0.1X -Create a dataset of dates 974 981 11 10.3 97.4 0.9X -to_csv(date) 4516 4523 9 2.2 451.6 0.2X -write dates to files 4714 4723 9 2.1 471.4 0.2X +Create a dataset of timestamps 877 888 17 11.4 87.7 1.0X +to_csv(timestamp) 5444 5471 34 1.8 544.4 0.2X +write timestamps to files 6094 6122 40 1.6 609.4 0.1X +Create a dataset of dates 1067 1076 14 9.4 106.7 0.8X +to_csv(date) 4115 4127 21 2.4 411.5 0.2X +write dates to files 4389 4456 59 2.3 438.9 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -read timestamp text from files 1167 1177 11 8.6 116.7 1.0X -read timestamps from files 9490 9517 29 1.1 949.0 0.1X -infer timestamps from files 19176 19254 112 0.5 1917.6 0.1X -read date text from files 1133 1149 23 8.8 113.3 1.0X -read date from files 8327 8344 30 1.2 832.7 0.1X -infer date from files 17583 17672 77 0.6 1758.3 0.1X -timestamp strings 1310 1318 7 7.6 131.0 0.9X -parse timestamps from Dataset[String] 11767 11853 85 0.8 1176.7 0.1X -infer timestamps from Dataset[String] 21178 21486 268 0.5 2117.8 0.1X -date strings 1602 1610 8 6.2 160.2 0.7X -parse dates from Dataset[String] 10041 10114 112 1.0 1004.1 0.1X -from_csv(timestamp) 10377 10493 115 1.0 1037.7 0.1X -from_csv(date) 9618 9622 3 1.0 961.8 0.1X -infer error timestamps from Dataset[String] with default format 11925 11968 40 0.8 1192.5 0.1X -infer error timestamps from Dataset[String] with user-provided format 11724 11807 72 0.9 1172.4 0.1X -infer error timestamps from Dataset[String] with legacy format 11781 11879 86 0.8 1178.1 0.1X +read timestamp text from files 1210 1214 4 8.3 121.0 1.0X +read timestamps from files 12528 12534 9 0.8 1252.8 0.1X +infer timestamps from files 24564 24614 48 0.4 2456.4 0.0X +read date text from files 1120 1125 6 8.9 112.0 1.1X +read date from files 11502 11540 35 0.9 1150.2 0.1X +infer date from files 23415 23704 263 0.4 2341.5 0.1X +timestamp strings 1205 1208 3 8.3 120.5 1.0X +parse timestamps from Dataset[String] 13589 13639 48 0.7 1358.9 0.1X +infer timestamps from Dataset[String] 25468 25568 115 0.4 2546.8 0.0X +date strings 1561 1565 4 6.4 156.1 0.8X +parse dates from Dataset[String] 12235 12255 25 0.8 1223.5 0.1X +from_csv(timestamp) 11514 11596 71 0.9 1151.4 0.1X +from_csv(date) 10604 10621 26 0.9 1060.4 0.1X +infer error timestamps from Dataset[String] with default format 14746 14761 14 0.7 1474.6 0.1X +infer error timestamps from Dataset[String] with user-provided format 14743 14787 56 0.7 1474.3 0.1X +infer error timestamps from Dataset[String] with legacy format 14690 14731 56 0.7 1469.0 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 4681 4704 32 0.0 46811.8 1.0X -pushdown disabled 4660 4679 28 0.0 46601.3 1.0X -w/ filters 762 778 16 0.1 7623.6 6.1X +w/o filters 4813 4844 53 0.0 48133.6 1.0X +pushdown disabled 4615 4624 16 0.0 46145.3 1.0X +w/ filters 804 806 3 0.1 8035.4 6.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Read as Intervals 781 785 7 0.4 2602.2 1.0X -Read Raw Strings 291 294 3 1.0 969.3 2.7X +Read as Intervals 815 816 1 0.4 2716.7 1.0X +Read Raw Strings 331 337 6 0.9 1104.2 2.5X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index 5626bbfb08fbd..e96f233c77793 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -2,76 +2,76 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 25766 25929 155 0.0 515313.0 1.0X +One quoted string 24513 24556 44 0.0 490253.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 51465 51922 628 0.0 51465.3 1.0X -Select 100 columns 21796 21940 152 0.0 21796.0 2.4X -Select one column 18651 18703 52 0.1 18651.2 2.8X -count() 3342 3448 103 0.3 3341.9 15.4X -Select 100 columns, one bad input field 27416 27481 60 0.0 27416.2 1.9X -Select 100 columns, corrupt record field 30540 30699 138 0.0 30539.8 1.7X +Select 1000 columns 55661 56102 621 0.0 55661.4 1.0X +Select 100 columns 21761 22015 246 0.0 21761.5 2.6X +Select one column 18450 18504 72 0.1 18449.8 3.0X +count() 3329 3412 72 0.3 3329.0 16.7X +Select 100 columns, one bad input field 27253 27287 48 0.0 27252.5 2.0X +Select 100 columns, corrupt record field 30624 30679 90 0.0 30624.5 1.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 9495 9525 26 1.1 949.5 1.0X -Select 1 column + count() 6922 6961 52 1.4 692.2 1.4X -count() 1742 1752 9 5.7 174.2 5.5X +Select 10 columns + count() 9849 9871 24 1.0 984.9 1.0X +Select 1 column + count() 7065 7075 15 1.4 706.5 1.4X +count() 1738 1741 4 5.8 173.8 5.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 912 958 65 11.0 91.2 1.0X -to_csv(timestamp) 7089 7112 31 1.4 708.9 0.1X -write timestamps to files 7242 7267 22 1.4 724.2 0.1X -Create a dataset of dates 1157 1185 38 8.6 115.7 0.8X -to_csv(date) 5034 5080 65 2.0 503.4 0.2X -write dates to files 5089 5107 29 2.0 508.9 0.2X +Create a dataset of timestamps 821 825 5 12.2 82.1 1.0X +to_csv(timestamp) 6711 6729 20 1.5 671.1 0.1X +write timestamps to files 6843 6858 25 1.5 684.3 0.1X +Create a dataset of dates 939 942 3 10.6 93.9 0.9X +to_csv(date) 4684 4697 21 2.1 468.4 0.2X +write dates to files 4479 4495 13 2.2 447.9 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -read timestamp text from files 1228 1233 4 8.1 122.8 1.0X -read timestamps from files 10598 10626 30 0.9 1059.8 0.1X -infer timestamps from files 21159 21181 19 0.5 2115.9 0.1X -read date text from files 1148 1151 3 8.7 114.8 1.1X -read date from files 10147 10180 35 1.0 1014.7 0.1X -infer date from files 21078 21110 47 0.5 2107.8 0.1X -timestamp strings 1354 1366 21 7.4 135.4 0.9X -parse timestamps from Dataset[String] 12127 12153 23 0.8 1212.7 0.1X -infer timestamps from Dataset[String] 22539 22566 27 0.4 2253.9 0.1X -date strings 1857 1862 5 5.4 185.7 0.7X -parse dates from Dataset[String] 11906 11931 30 0.8 1190.6 0.1X -from_csv(timestamp) 10716 10744 37 0.9 1071.6 0.1X -from_csv(date) 11123 11140 15 0.9 1112.3 0.1X -infer error timestamps from Dataset[String] with default format 12274 12281 9 0.8 1227.4 0.1X -infer error timestamps from Dataset[String] with user-provided format 12281 12304 26 0.8 1228.1 0.1X -infer error timestamps from Dataset[String] with legacy format 12300 12307 9 0.8 1230.0 0.1X +read timestamp text from files 1190 1197 6 8.4 119.0 1.0X +read timestamps from files 10627 10667 52 0.9 1062.7 0.1X +infer timestamps from files 21086 21135 69 0.5 2108.6 0.1X +read date text from files 1081 1084 5 9.2 108.1 1.1X +read date from files 10254 10265 12 1.0 1025.4 0.1X +infer date from files 20908 20924 18 0.5 2090.8 0.1X +timestamp strings 1173 1175 3 8.5 117.3 1.0X +parse timestamps from Dataset[String] 12413 12473 57 0.8 1241.3 0.1X +infer timestamps from Dataset[String] 22801 22829 42 0.4 2280.1 0.1X +date strings 1653 1657 4 6.1 165.3 0.7X +parse dates from Dataset[String] 12033 12057 25 0.8 1203.3 0.1X +from_csv(timestamp) 10339 10355 18 1.0 1033.9 0.1X +from_csv(date) 10554 10561 11 0.9 1055.4 0.1X +infer error timestamps from Dataset[String] with default format 12871 12878 10 0.8 1287.1 0.1X +infer error timestamps from Dataset[String] with user-provided format 12845 12867 33 0.8 1284.5 0.1X +infer error timestamps from Dataset[String] with legacy format 12872 12904 37 0.8 1287.2 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 4058 4061 2 0.0 40583.1 1.0X -pushdown disabled 4092 4099 10 0.0 40924.1 1.0X -w/ filters 699 705 8 0.1 6990.7 5.8X +w/o filters 4281 4288 7 0.0 42807.9 1.0X +pushdown disabled 4059 4067 10 0.0 40590.4 1.1X +w/ filters 764 775 13 0.1 7640.9 5.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Read as Intervals 737 742 9 0.4 2456.8 1.0X -Read Raw Strings 294 300 10 1.0 979.5 2.5X +Read as Intervals 709 716 8 0.4 2364.5 1.0X +Read Raw Strings 295 298 3 1.0 984.4 2.4X diff --git a/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt b/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt index 47781a2cc6e1f..2093ce53a24d3 100644 --- a/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt @@ -2,121 +2,121 @@ Char Varchar Write Side Perf w/o Tailing Spaces ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 5 6905 7223 322 5.8 172.6 1.0X -write char with length 5 10769 10842 66 3.7 269.2 0.6X -write varchar with length 5 7615 7654 35 5.3 190.4 0.9X +write string with length 5 7118 7215 151 5.6 178.0 1.0X +write char with length 5 12527 12590 97 3.2 313.2 0.6X +write varchar with length 5 7976 8046 82 5.0 199.4 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 10 3624 3637 21 5.5 181.2 1.0X -write char with length 10 6455 6488 33 3.1 322.7 0.6X -write varchar with length 10 3802 3861 79 5.3 190.1 1.0X +write string with length 10 3745 3751 6 5.3 187.2 1.0X +write char with length 10 6606 6702 83 3.0 330.3 0.6X +write varchar with length 10 3782 3810 25 5.3 189.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 20 1770 1784 17 5.6 177.0 1.0X -write char with length 20 4741 4751 13 2.1 474.1 0.4X -write varchar with length 20 1921 1926 6 5.2 192.1 0.9X +write string with length 20 1774 1777 5 5.6 177.4 1.0X +write char with length 20 4778 4829 46 2.1 477.8 0.4X +write varchar with length 20 2047 2048 1 4.9 204.7 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 40 945 955 13 5.3 189.0 1.0X -write char with length 40 3669 3697 26 1.4 733.7 0.3X -write varchar with length 40 1024 1029 5 4.9 204.7 0.9X +write string with length 40 931 950 17 5.4 186.3 1.0X +write char with length 40 3709 3713 3 1.3 741.9 0.3X +write varchar with length 40 1065 1068 2 4.7 213.0 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 60 648 671 29 5.1 194.5 1.0X -write char with length 60 3258 3278 17 1.0 977.5 0.2X -write varchar with length 60 726 738 12 4.6 217.8 0.9X +write string with length 60 662 675 14 5.0 198.5 1.0X +write char with length 60 3355 3360 5 1.0 1006.5 0.2X +write varchar with length 60 680 691 11 4.9 204.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 80 522 526 6 4.8 208.7 1.0X -write char with length 80 3151 3173 31 0.8 1260.3 0.2X -write varchar with length 80 555 564 8 4.5 222.2 0.9X +write string with length 80 500 505 5 5.0 200.1 1.0X +write char with length 80 3147 3173 22 0.8 1258.9 0.2X +write varchar with length 80 541 549 9 4.6 216.3 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 100 423 450 26 4.7 211.3 1.0X -write char with length 100 3057 3067 14 0.7 1528.3 0.1X -write varchar with length 100 472 478 6 4.2 235.9 0.9X +write string with length 100 403 417 13 5.0 201.6 1.0X +write char with length 100 3107 3113 10 0.6 1553.5 0.1X +write varchar with length 100 440 447 8 4.5 219.8 0.9X ================================================================================================ Char Varchar Write Side Perf w/ Tailing Spaces ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 5 10664 10748 114 3.8 266.6 1.0X -write char with length 5 13099 13173 91 3.1 327.5 0.8X -write varchar with length 5 12595 12606 10 3.2 314.9 0.8X +write string with length 5 10431 10499 102 3.8 260.8 1.0X +write char with length 5 12396 12404 9 3.2 309.9 0.8X +write varchar with length 5 12858 12889 49 3.1 321.5 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 10 5412 5423 14 3.7 270.6 1.0X -write char with length 10 8402 8405 5 2.4 420.1 0.6X -write varchar with length 10 8000 8031 31 2.5 400.0 0.7X +write string with length 10 5537 5556 18 3.6 276.8 1.0X +write char with length 10 8103 8104 2 2.5 405.1 0.7X +write varchar with length 10 8414 8427 12 2.4 420.7 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 20 3326 3331 5 3.0 332.6 1.0X -write char with length 20 5576 5586 9 1.8 557.6 0.6X -write varchar with length 20 5699 5708 9 1.8 569.9 0.6X +write string with length 20 3560 3566 7 2.8 356.0 1.0X +write char with length 20 5738 5741 3 1.7 573.8 0.6X +write varchar with length 20 5787 5803 22 1.7 578.7 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 40 2210 2214 4 2.3 441.9 1.0X -write char with length 40 4306 4306 0 1.2 861.2 0.5X -write varchar with length 40 4509 4524 16 1.1 901.8 0.5X +write string with length 40 2352 2364 11 2.1 470.4 1.0X +write char with length 40 4431 4449 27 1.1 886.2 0.5X +write varchar with length 40 4461 4473 11 1.1 892.2 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 60 1894 1901 6 1.8 568.2 1.0X -write char with length 60 4088 4093 5 0.8 1226.4 0.5X -write varchar with length 60 3982 3987 5 0.8 1194.5 0.5X +write string with length 60 1923 1928 4 1.7 577.0 1.0X +write char with length 60 4005 4013 12 0.8 1201.6 0.5X +write varchar with length 60 4028 4033 5 0.8 1208.3 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 80 1785 1793 6 1.4 714.2 1.0X -write char with length 80 3937 3952 14 0.6 1574.6 0.5X -write varchar with length 80 3942 3959 24 0.6 1576.9 0.5X +write string with length 80 1969 1986 15 1.3 787.5 1.0X +write char with length 80 3999 4009 16 0.6 1599.5 0.5X +write varchar with length 80 4030 4040 9 0.6 1612.1 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 100 1629 1640 10 1.2 814.6 1.0X -write char with length 100 3686 3693 12 0.5 1842.9 0.4X -write varchar with length 100 3905 3921 15 0.5 1952.6 0.4X +write string with length 100 1616 1626 15 1.2 808.2 1.0X +write char with length 100 3851 3852 1 0.5 1925.7 0.4X +write varchar with length 100 3841 3858 18 0.5 1920.7 0.4X diff --git a/sql/core/benchmarks/CharVarcharBenchmark-results.txt b/sql/core/benchmarks/CharVarcharBenchmark-results.txt index 03a64c6904e9c..8b8a7fe89d469 100644 --- a/sql/core/benchmarks/CharVarcharBenchmark-results.txt +++ b/sql/core/benchmarks/CharVarcharBenchmark-results.txt @@ -2,121 +2,121 @@ Char Varchar Write Side Perf w/o Tailing Spaces ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 5 6760 7092 292 5.9 169.0 1.0X -write char with length 5 9848 9929 87 4.1 246.2 0.7X -write varchar with length 5 7633 7676 37 5.2 190.8 0.9X +write string with length 5 6706 6744 33 6.0 167.7 1.0X +write char with length 5 10593 10612 31 3.8 264.8 0.6X +write varchar with length 5 8187 8238 84 4.9 204.7 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 10 3611 3629 27 5.5 180.5 1.0X -write char with length 10 6130 6165 50 3.3 306.5 0.6X -write varchar with length 10 3742 3772 26 5.3 187.1 1.0X +write string with length 10 3968 3985 21 5.0 198.4 1.0X +write char with length 10 6729 6805 67 3.0 336.5 0.6X +write varchar with length 10 3987 4047 101 5.0 199.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 20 1775 1786 11 5.6 177.5 1.0X -write char with length 20 4560 4562 2 2.2 456.0 0.4X -write varchar with length 20 1923 1933 8 5.2 192.3 0.9X +write string with length 20 1968 1985 18 5.1 196.8 1.0X +write char with length 20 4800 4832 36 2.1 480.0 0.4X +write varchar with length 20 2016 2025 13 5.0 201.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 40 935 958 30 5.3 187.1 1.0X -write char with length 40 3475 3480 4 1.4 695.0 0.3X -write varchar with length 40 1019 1038 18 4.9 203.9 0.9X +write string with length 40 1042 1044 2 4.8 208.4 1.0X +write char with length 40 3794 3804 13 1.3 758.9 0.3X +write varchar with length 40 1124 1129 7 4.4 224.8 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 60 663 666 4 5.0 199.0 1.0X -write char with length 60 3240 3248 10 1.0 972.1 0.2X -write varchar with length 60 711 715 5 4.7 213.2 0.9X +write string with length 60 690 696 6 4.8 207.1 1.0X +write char with length 60 3430 3443 18 1.0 1029.0 0.2X +write varchar with length 60 759 770 9 4.4 227.8 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 80 533 544 10 4.7 213.2 1.0X -write char with length 80 3024 3028 5 0.8 1209.6 0.2X -write varchar with length 80 560 561 1 4.5 223.9 1.0X +write string with length 80 560 569 11 4.5 223.8 1.0X +write char with length 80 3212 3222 11 0.8 1284.8 0.2X +write varchar with length 80 607 612 6 4.1 242.7 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 100 460 464 4 4.3 230.0 1.0X -write char with length 100 2973 2975 1 0.7 1486.7 0.2X -write varchar with length 100 483 486 3 4.1 241.4 1.0X +write string with length 100 437 444 11 4.6 218.4 1.0X +write char with length 100 3106 3109 5 0.6 1552.8 0.1X +write varchar with length 100 483 494 13 4.1 241.5 0.9X ================================================================================================ Char Varchar Write Side Perf w/ Tailing Spaces ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 5 8798 8827 25 4.5 219.9 1.0X -write char with length 5 11984 11999 19 3.3 299.6 0.7X -write varchar with length 5 12379 12401 20 3.2 309.5 0.7X +write string with length 5 9537 9571 32 4.2 238.4 1.0X +write char with length 5 12811 12868 70 3.1 320.3 0.7X +write varchar with length 5 12857 12877 24 3.1 321.4 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 10 5290 5307 19 3.8 264.5 1.0X -write char with length 10 7536 7538 2 2.7 376.8 0.7X -write varchar with length 10 7489 7519 39 2.7 374.5 0.7X +write string with length 10 5556 5559 3 3.6 277.8 1.0X +write char with length 10 8349 8354 6 2.4 417.5 0.7X +write varchar with length 10 7827 7844 23 2.6 391.4 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 20 3213 3218 6 3.1 321.3 1.0X -write char with length 20 5570 5578 7 1.8 557.0 0.6X -write varchar with length 20 5245 5261 15 1.9 524.5 0.6X +write string with length 20 3488 3499 10 2.9 348.8 1.0X +write char with length 20 5847 5854 7 1.7 584.7 0.6X +write varchar with length 20 5408 5409 1 1.8 540.8 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 40 2121 2125 7 2.4 424.2 1.0X -write char with length 40 4399 4419 17 1.1 879.8 0.5X -write varchar with length 40 4118 4124 5 1.2 823.7 0.5X +write string with length 40 2430 2434 6 2.1 486.1 1.0X +write char with length 40 4492 4494 3 1.1 898.4 0.5X +write varchar with length 40 4131 4140 8 1.2 826.1 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 60 1884 1891 6 1.8 565.1 1.0X -write char with length 60 3939 3941 4 0.8 1181.6 0.5X -write varchar with length 60 3584 3591 5 0.9 1075.3 0.5X +write string with length 60 1900 1906 5 1.8 570.0 1.0X +write char with length 60 4153 4155 3 0.8 1245.8 0.5X +write varchar with length 60 3659 3660 2 0.9 1097.7 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 80 1838 1842 3 1.4 735.4 1.0X -write char with length 80 3823 3835 13 0.7 1529.1 0.5X -write varchar with length 80 3454 3456 2 0.7 1381.8 0.5X +write string with length 80 1816 1821 5 1.4 726.5 1.0X +write char with length 80 4030 4050 18 0.6 1611.9 0.5X +write varchar with length 80 3744 3758 12 0.7 1497.6 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -write string with length 100 1635 1643 10 1.2 817.3 1.0X -write char with length 100 3697 3704 6 0.5 1848.3 0.4X -write varchar with length 100 3355 3375 18 0.6 1677.3 0.5X +write string with length 100 1674 1687 14 1.2 836.9 1.0X +write char with length 100 3922 3927 7 0.5 1961.0 0.4X +write varchar with length 100 3503 3505 3 0.6 1751.4 0.5X diff --git a/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt b/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt index 88db9ebfa1e34..8a1599b3cfe42 100644 --- a/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt @@ -1,88 +1,88 @@ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1193 1194 1 0.1 11929.0 1.0X -UTF8_LCASE 2717 2721 6 0.0 27168.5 2.3X -UNICODE 17991 17993 2 0.0 179913.6 15.1X -UNICODE_CI 17837 17842 7 0.0 178369.9 15.0X +UTF8_BINARY 1360 1360 1 0.1 13597.4 1.0X +UTF8_LCASE 2411 2417 9 0.0 24106.7 1.8X +UNICODE 16945 16969 34 0.0 169452.6 12.5X +UNICODE_CI 16645 16671 36 0.0 166452.8 12.2X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1523 1523 0 0.1 15233.9 1.0X -UTF8_LCASE 2441 2441 0 0.0 24407.9 1.6X -UNICODE 17875 17884 13 0.0 178749.6 11.7X -UNICODE_CI 17701 17703 2 0.0 177013.8 11.6X +UTF8_BINARY 1751 1753 2 0.1 17513.9 1.0X +UTF8_LCASE 2571 2573 3 0.0 25712.7 1.5X +UNICODE 16594 16625 44 0.0 165935.1 9.5X +UNICODE_CI 16422 16423 3 0.0 164215.1 9.4X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2660 2666 9 0.0 26601.1 1.0X -UTF8_LCASE 5013 5016 3 0.0 50134.0 1.9X -UNICODE 75622 75623 1 0.0 756217.3 28.4X -UNICODE_CI 63036 63042 9 0.0 630360.9 23.7X +UTF8_BINARY 2817 2824 9 0.0 28170.1 1.0X +UTF8_LCASE 5427 5428 1 0.0 54268.5 1.9X +UNICODE 70045 70096 72 0.0 700450.7 24.9X +UNICODE_CI 56364 56433 97 0.0 563641.8 20.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2121 2122 0 0.0 21214.2 1.0X -UTF8_LCASE 27635 27636 1 0.0 276347.7 13.0X -UNICODE 523746 524012 376 0.0 5237460.5 246.9X -UNICODE_CI 520134 520227 131 0.0 5201343.3 245.2X +UTF8_BINARY 1644 1645 1 0.1 16440.3 1.0X +UTF8_LCASE 14804 14846 59 0.0 148037.2 9.0X +UNICODE 308825 309294 663 0.0 3088250.5 187.8X +UNICODE_CI 310637 312537 2688 0.0 3106367.6 188.9X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2767 2769 4 0.0 27666.3 1.0X -UTF8_LCASE 26861 26861 1 0.0 268606.4 9.7X -UNICODE 518540 518815 389 0.0 5185401.3 187.4X -UNICODE_CI 521156 521261 148 0.0 5211559.5 188.4X +UTF8_BINARY 1941 1942 1 0.1 19412.9 1.0X +UTF8_LCASE 10354 10409 78 0.0 103535.9 5.3X +UNICODE 309786 310124 478 0.0 3097864.6 159.6X +UNICODE_CI 313038 313960 1303 0.0 3130382.9 161.3X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2919 2921 3 0.0 29190.2 1.0X -UTF8_LCASE 26862 26862 1 0.0 268618.0 9.2X -UNICODE 504534 504927 556 0.0 5045340.3 172.8X -UNICODE_CI 506542 506565 32 0.0 5065423.0 173.5X +UTF8_BINARY 1958 1961 4 0.1 19579.3 1.0X +UTF8_LCASE 10329 10332 5 0.0 103285.8 5.3X +UNICODE 323944 328005 5743 0.0 3239437.8 165.5X +UNICODE_CI 332646 333139 697 0.0 3326457.7 169.9X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------- -UNICODE 419 425 5 0.2 4189.2 1.0X -UNICODE_CI 416 426 6 0.2 4163.2 1.0X +UNICODE 370 371 1 0.3 3698.1 1.0X +UNICODE_CI 370 370 1 0.3 3696.5 1.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 575 576 0 0.2 5754.0 1.0X -UTF8_LCASE 575 576 1 0.2 5747.8 1.0X -UNICODE 576 576 0 0.2 5761.5 1.0X -UNICODE_CI 576 578 2 0.2 5758.0 1.0X +UTF8_BINARY 592 593 1 0.2 5915.6 1.0X +UTF8_LCASE 593 593 1 0.2 5926.8 1.0X +UNICODE 591 593 1 0.2 5912.9 1.0X +UNICODE_CI 593 594 1 0.2 5934.1 1.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ----------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 159 159 1 0.6 1587.6 1.0X -UTF8_LCASE 159 159 0 0.6 1586.6 1.0X -UNICODE 158 159 1 0.6 1584.9 1.0X -UNICODE_CI 159 160 1 0.6 1586.1 1.0X +UTF8_BINARY 105 109 10 0.9 1054.8 1.0X +UTF8_LCASE 105 106 1 0.9 1053.3 1.0X +UNICODE 105 106 1 0.9 1054.2 1.0X +UNICODE_CI 105 106 0 1.0 1051.1 1.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 397 405 5 0.3 3974.4 1.0X -UTF8_LCASE 401 405 5 0.2 4009.5 1.0X -UNICODE 395 399 3 0.3 3953.9 1.0X -UNICODE_CI 395 400 3 0.3 3952.0 1.0X +UTF8_BINARY 370 371 1 0.3 3698.3 1.0X +UTF8_LCASE 370 371 1 0.3 3697.7 1.0X +UNICODE 369 370 1 0.3 3692.7 1.0X +UNICODE_CI 370 371 1 0.3 3697.2 1.0X diff --git a/sql/core/benchmarks/CollationBenchmark-results.txt b/sql/core/benchmarks/CollationBenchmark-results.txt index 8402a2db6d869..cbd0727ce92e4 100644 --- a/sql/core/benchmarks/CollationBenchmark-results.txt +++ b/sql/core/benchmarks/CollationBenchmark-results.txt @@ -1,88 +1,88 @@ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1223 1224 1 0.1 12231.5 1.0X -UTF8_LCASE 3280 3281 1 0.0 32803.3 2.7X -UNICODE 17207 17207 0 0.0 172065.7 14.1X -UNICODE_CI 16560 16565 7 0.0 165604.3 13.5X +UTF8_BINARY 1380 1381 1 0.1 13801.3 1.0X +UTF8_LCASE 3334 3336 3 0.0 33337.1 2.4X +UNICODE 19004 19005 1 0.0 190039.6 13.8X +UNICODE_CI 18686 18699 18 0.0 186856.4 13.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1656 1657 0 0.1 16564.0 1.0X -UTF8_LCASE 3320 3321 0 0.0 33203.0 2.0X -UNICODE 16392 16393 2 0.0 163921.3 9.9X -UNICODE_CI 16314 16319 6 0.0 163143.3 9.8X +UTF8_BINARY 1739 1739 0 0.1 17392.0 1.0X +UTF8_LCASE 4175 4175 0 0.0 41745.1 2.4X +UNICODE 20212 20220 11 0.0 202124.2 11.6X +UNICODE_CI 20078 20086 11 0.0 200782.6 11.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2812 2813 1 0.0 28119.0 1.0X -UTF8_LCASE 5682 5685 4 0.0 56823.2 2.0X -UNICODE 71678 71685 10 0.0 716777.4 25.5X -UNICODE_CI 60660 60670 15 0.0 606597.4 21.6X +UTF8_BINARY 3112 3115 4 0.0 31119.4 1.0X +UTF8_LCASE 6348 6354 9 0.0 63477.9 2.0X +UNICODE 67421 67436 22 0.0 674208.7 21.7X +UNICODE_CI 54039 54056 24 0.0 540394.5 17.4X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2528 2528 1 0.0 25276.8 1.0X -UTF8_LCASE 28034 28050 24 0.0 280335.5 11.1X -UNICODE 521518 521690 242 0.0 5215184.7 206.3X -UNICODE_CI 508188 508312 176 0.0 5081880.5 201.0X +UTF8_BINARY 1677 1678 1 0.1 16768.6 1.0X +UTF8_LCASE 17476 17480 5 0.0 174760.6 10.4X +UNICODE 324829 324937 153 0.0 3248290.7 193.7X +UNICODE_CI 317534 317742 294 0.0 3175340.9 189.4X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2772 2774 4 0.0 27715.0 1.0X -UTF8_LCASE 27387 27390 4 0.0 273872.8 9.9X -UNICODE 501025 501076 72 0.0 5010249.5 180.8X -UNICODE_CI 506654 506666 16 0.0 5066544.6 182.8X +UTF8_BINARY 2040 2041 1 0.0 20400.9 1.0X +UTF8_LCASE 17099 17100 1 0.0 170991.2 8.4X +UNICODE 314251 314484 330 0.0 3142508.7 154.0X +UNICODE_CI 319313 319690 533 0.0 3193131.6 156.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2886 2888 3 0.0 28858.9 1.0X -UTF8_LCASE 27433 27445 17 0.0 274326.2 9.5X -UNICODE 501068 501186 168 0.0 5010676.2 173.6X -UNICODE_CI 506619 506655 52 0.0 5066185.6 175.6X +UTF8_BINARY 2077 2077 1 0.0 20765.6 1.0X +UTF8_LCASE 16903 16905 2 0.0 169034.4 8.1X +UNICODE 326824 328355 2165 0.0 3268239.0 157.4X +UNICODE_CI 334072 334237 233 0.0 3340722.1 160.9X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------- -UNICODE 407 411 4 0.2 4065.4 1.0X -UNICODE_CI 419 423 3 0.2 4194.1 1.0X +UNICODE 301 301 0 0.3 3006.6 1.0X +UNICODE_CI 300 301 1 0.3 3003.5 1.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 564 565 2 0.2 5639.2 1.0X -UTF8_LCASE 563 563 0 0.2 5629.0 1.0X -UNICODE 563 565 2 0.2 5634.3 1.0X -UNICODE_CI 564 564 0 0.2 5640.9 1.0X +UTF8_BINARY 599 600 1 0.2 5992.2 1.0X +UTF8_LCASE 599 600 1 0.2 5994.4 1.0X +UNICODE 599 600 1 0.2 5985.1 1.0X +UNICODE_CI 597 598 1 0.2 5971.7 1.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ----------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 165 166 1 0.6 1647.3 1.0X -UTF8_LCASE 165 165 1 0.6 1646.7 1.0X -UNICODE 165 165 1 0.6 1646.5 1.0X -UNICODE_CI 165 166 1 0.6 1648.7 1.0X +UTF8_BINARY 184 185 1 0.5 1844.8 1.0X +UTF8_LCASE 185 185 0 0.5 1847.3 1.0X +UNICODE 184 185 1 0.5 1844.9 1.0X +UNICODE_CI 185 185 0 0.5 1845.6 1.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 391 399 6 0.3 3912.1 1.0X -UTF8_LCASE 389 399 7 0.3 3894.2 1.0X -UNICODE 383 391 6 0.3 3828.6 1.0X -UNICODE_CI 383 387 2 0.3 3833.0 1.0X +UTF8_BINARY 324 325 1 0.3 3242.0 1.0X +UTF8_LCASE 325 326 2 0.3 3251.5 1.0X +UNICODE 325 326 1 0.3 3251.9 1.0X +UNICODE_CI 324 326 1 0.3 3242.6 1.0X diff --git a/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt b/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt index 4da64ade11d68..ffdd34f6aaa8c 100644 --- a/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt @@ -1,88 +1,88 @@ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 156 156 0 0.3 3887.8 1.0X -UTF8_LCASE 9717 9729 18 0.0 242914.7 62.5X -UNICODE 5026 5027 2 0.0 125640.1 32.3X -UNICODE_CI 4969 4972 4 0.0 124224.9 32.0X +UTF8_BINARY 171 172 1 0.2 4282.8 1.0X +UTF8_LCASE 7012 7018 9 0.0 175288.2 40.9X +UNICODE 5206 5207 0 0.0 130157.7 30.4X +UNICODE_CI 5220 5220 0 0.0 130499.0 30.5X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 279 279 0 0.1 6969.5 1.0X -UTF8_LCASE 9624 9628 5 0.0 240611.6 34.5X -UNICODE 5243 5244 0 0.0 131080.1 18.8X -UNICODE_CI 5173 5173 0 0.0 129322.8 18.6X +UTF8_BINARY 315 316 1 0.1 7871.4 1.0X +UTF8_LCASE 7036 7038 4 0.0 175888.2 22.3X +UNICODE 5343 5344 1 0.0 133571.1 17.0X +UNICODE_CI 5284 5284 0 0.0 132104.2 16.8X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 383 383 0 0.1 9576.7 1.0X -UTF8_LCASE 4927 4931 6 0.0 123170.3 12.9X -UNICODE 17244 17261 24 0.0 431096.6 45.0X -UNICODE_CI 12968 12970 3 0.0 324194.1 33.9X +UTF8_BINARY 382 383 1 0.1 9557.2 1.0X +UTF8_LCASE 3587 3592 6 0.0 89683.4 9.4X +UNICODE 15310 15322 16 0.0 382753.7 40.0X +UNICODE_CI 12531 12543 17 0.0 313269.1 32.8X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 535 536 2 0.1 13371.6 1.0X -UTF8_LCASE 9479 9480 2 0.0 236964.5 17.7X -UNICODE 93629 93676 66 0.0 2340726.5 175.1X -UNICODE_CI 93222 93309 124 0.0 2330541.2 174.3X +UTF8_BINARY 350 350 0 0.1 8742.0 1.0X +UTF8_LCASE 9013 9020 11 0.0 225317.3 25.8X +UNICODE 58338 58368 43 0.0 1458444.2 166.8X +UNICODE_CI 58821 58928 152 0.0 1470530.7 168.2X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 430 431 1 0.1 10755.8 1.0X -UTF8_LCASE 6550 6551 2 0.0 163753.7 15.2X -UNICODE 87435 87467 45 0.0 2185886.8 203.2X -UNICODE_CI 90113 90255 201 0.0 2252836.0 209.5X +UTF8_BINARY 290 291 1 0.1 7250.9 1.0X +UTF8_LCASE 5323 5325 3 0.0 133079.8 18.4X +UNICODE 57879 57976 138 0.0 1446968.2 199.6X +UNICODE_CI 59098 59188 127 0.0 1477459.9 203.8X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 455 456 2 0.1 11369.5 1.0X -UTF8_LCASE 7108 7115 9 0.0 177705.2 15.6X -UNICODE 101835 101866 43 0.0 2545883.9 223.9X -UNICODE_CI 100962 101026 91 0.0 2524045.2 222.0X +UTF8_BINARY 300 302 1 0.1 7507.7 1.0X +UTF8_LCASE 5310 5318 11 0.0 132754.5 17.7X +UNICODE 64787 64833 65 0.0 1619680.3 215.7X +UNICODE_CI 64384 64419 50 0.0 1609603.5 214.4X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------- -UNICODE 254 255 1 0.2 6346.5 1.0X -UNICODE_CI 254 254 0 0.2 6348.1 1.0X +UNICODE 214 215 1 0.2 5339.5 1.0X +UNICODE_CI 214 215 0 0.2 5355.7 1.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 322 323 1 0.1 8046.3 1.0X -UTF8_LCASE 322 324 2 0.1 8059.0 1.0X -UNICODE 322 323 1 0.1 8050.7 1.0X -UNICODE_CI 322 325 4 0.1 8062.4 1.0X +UTF8_BINARY 318 318 1 0.1 7946.2 1.0X +UTF8_LCASE 318 319 1 0.1 7945.3 1.0X +UNICODE 318 319 1 0.1 7950.9 1.0X +UNICODE_CI 317 318 1 0.1 7931.1 1.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ----------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 119 120 1 0.3 2972.1 1.0X -UTF8_LCASE 119 120 1 0.3 2971.9 1.0X -UNICODE 119 120 1 0.3 2970.3 1.0X -UNICODE_CI 119 120 1 0.3 2968.6 1.0X +UTF8_BINARY 84 85 0 0.5 2101.1 1.0X +UTF8_LCASE 84 85 1 0.5 2097.7 1.0X +UNICODE 84 85 1 0.5 2106.4 1.0X +UNICODE_CI 84 85 1 0.5 2111.5 1.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 254 255 1 0.2 6345.2 1.0X -UTF8_LCASE 254 255 0 0.2 6351.8 1.0X -UNICODE 254 255 0 0.2 6352.9 1.0X -UNICODE_CI 254 254 0 0.2 6341.2 1.0X +UTF8_BINARY 214 215 2 0.2 5342.3 1.0X +UTF8_LCASE 214 215 1 0.2 5348.9 1.0X +UNICODE 214 215 1 0.2 5349.8 1.0X +UNICODE_CI 214 215 0 0.2 5354.4 1.0X diff --git a/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt b/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt index fba59f3893e22..fa21ecbbb6593 100644 --- a/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt +++ b/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt @@ -1,88 +1,88 @@ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 125 126 1 0.3 3128.6 1.0X -UTF8_LCASE 10335 10345 14 0.0 258377.4 82.6X -UNICODE 5604 5610 8 0.0 140110.8 44.8X -UNICODE_CI 5570 5577 9 0.0 139252.7 44.5X +UTF8_BINARY 141 146 3 0.3 3523.3 1.0X +UTF8_LCASE 7725 7753 40 0.0 193120.1 54.8X +UNICODE 5788 5824 51 0.0 144696.8 41.1X +UNICODE_CI 5997 6002 7 0.0 149920.7 42.6X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 293 294 2 0.1 7326.8 1.0X -UTF8_LCASE 10035 10035 1 0.0 250865.2 34.2X -UNICODE 5578 5580 3 0.0 139455.8 19.0X -UNICODE_CI 5539 5541 2 0.0 138483.8 18.9X +UTF8_BINARY 337 346 5 0.1 8433.8 1.0X +UTF8_LCASE 7829 7852 33 0.0 195727.0 23.2X +UNICODE 6096 6116 29 0.0 152404.8 18.1X +UNICODE_CI 6112 6131 26 0.0 152805.7 18.1X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 388 388 0 0.1 9699.6 1.0X -UTF8_LCASE 4965 4967 3 0.0 124121.3 12.8X -UNICODE 15750 15753 5 0.0 393740.9 40.6X -UNICODE_CI 12509 12511 2 0.0 312735.5 32.2X +UTF8_BINARY 452 455 3 0.1 11306.0 1.0X +UTF8_LCASE 3968 3990 32 0.0 99194.0 8.8X +UNICODE 15247 15296 69 0.0 381186.5 33.7X +UNICODE_CI 12374 12397 32 0.0 309347.5 27.4X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 421 422 2 0.1 10512.9 1.0X -UTF8_LCASE 10793 10796 5 0.0 269819.0 25.7X -UNICODE 94324 94330 9 0.0 2358090.9 224.3X -UNICODE_CI 91647 91748 143 0.0 2291174.6 217.9X +UTF8_BINARY 435 446 7 0.1 10881.1 1.0X +UTF8_LCASE 10346 10366 29 0.0 258656.4 23.8X +UNICODE 78521 78598 110 0.0 1963015.5 180.4X +UNICODE_CI 80810 81202 554 0.0 2020241.0 185.7X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 452 453 0 0.1 11307.9 1.0X -UTF8_LCASE 6871 6872 2 0.0 171782.0 15.2X -UNICODE 90881 90924 60 0.0 2272034.5 200.9X -UNICODE_CI 91333 91363 42 0.0 2283331.3 201.9X +UTF8_BINARY 321 324 3 0.1 8021.8 1.0X +UTF8_LCASE 5970 5976 10 0.0 149242.0 18.6X +UNICODE 86151 86522 525 0.0 2153773.0 268.5X +UNICODE_CI 89308 90327 1441 0.0 2232710.9 278.3X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 451 452 2 0.1 11268.1 1.0X -UTF8_LCASE 6685 6686 2 0.0 167120.8 14.8X -UNICODE 99387 99484 138 0.0 2484672.5 220.5X -UNICODE_CI 98525 98597 101 0.0 2463132.9 218.6X +UTF8_BINARY 310 314 3 0.1 7741.5 1.0X +UTF8_LCASE 5707 5711 5 0.0 142683.3 18.4X +UNICODE 91242 95109 5469 0.0 2281057.2 294.7X +UNICODE_CI 91446 92305 1215 0.0 2286138.3 295.3X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------- -UNICODE 231 232 0 0.2 5784.5 1.0X -UNICODE_CI 231 232 1 0.2 5780.4 1.0X +UNICODE 298 300 2 0.1 7454.2 1.0X +UNICODE_CI 300 301 1 0.1 7500.2 1.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 312 314 1 0.1 7811.2 1.0X -UTF8_LCASE 313 314 2 0.1 7822.9 1.0X -UNICODE 313 314 1 0.1 7815.5 1.0X -UNICODE_CI 313 315 4 0.1 7825.7 1.0X +UTF8_BINARY 343 346 3 0.1 8576.7 1.0X +UTF8_LCASE 343 345 2 0.1 8582.7 1.0X +UNICODE 344 348 2 0.1 8607.4 1.0X +UNICODE_CI 340 345 3 0.1 8493.8 1.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ----------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 132 133 0 0.3 3302.0 1.0X -UTF8_LCASE 132 132 0 0.3 3297.5 1.0X -UNICODE 132 133 1 0.3 3296.9 1.0X -UNICODE_CI 132 132 0 0.3 3298.1 1.0X +UTF8_BINARY 130 132 1 0.3 3245.3 1.0X +UTF8_LCASE 129 132 1 0.3 3235.2 1.0X +UNICODE 129 133 1 0.3 3231.9 1.0X +UNICODE_CI 131 133 1 0.3 3274.8 1.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws -Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 231 231 0 0.2 5770.4 1.0X -UTF8_LCASE 231 232 1 0.2 5776.4 1.0X -UNICODE 231 231 0 0.2 5767.5 1.0X -UNICODE_CI 231 232 1 0.2 5770.2 1.0X +UTF8_BINARY 294 296 1 0.1 7348.6 1.0X +UTF8_LCASE 296 299 2 0.1 7390.7 1.0X +UNICODE 298 300 2 0.1 7461.3 1.0X +UNICODE_CI 297 299 2 0.1 7421.1 1.0X diff --git a/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt b/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt index e6d3fa3dfbe5e..40b7cf00b6669 100644 --- a/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt @@ -2,58 +2,58 @@ Int Read/Write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Java Array 122 123 1 2676.8 0.4 1.0X -ByteBuffer Unsafe 194 201 8 1685.2 0.6 0.6X -ByteBuffer API 501 503 2 653.5 1.5 0.2X -DirectByteBuffer 418 419 1 784.8 1.3 0.3X -Unsafe Buffer 154 154 0 2134.6 0.5 0.8X -Column(on heap) 123 123 1 2668.6 0.4 1.0X -Column(off heap) 154 154 1 2134.0 0.5 0.8X -Column(off heap direct) 154 154 1 2128.0 0.5 0.8X -UnsafeRow (on heap) 432 433 2 758.6 1.3 0.3X -UnsafeRow (off heap) 294 295 1 1116.1 0.9 0.4X -Column On Heap Append 336 337 2 976.5 1.0 0.4X +Java Array 123 123 0 2664.8 0.4 1.0X +ByteBuffer Unsafe 188 194 8 1742.2 0.6 0.7X +ByteBuffer API 429 429 1 764.2 1.3 0.3X +DirectByteBuffer 420 421 2 780.9 1.3 0.3X +Unsafe Buffer 154 156 5 2124.5 0.5 0.8X +Column(on heap) 124 124 0 2646.3 0.4 1.0X +Column(off heap) 155 155 0 2117.5 0.5 0.8X +Column(off heap direct) 155 155 0 2115.9 0.5 0.8X +UnsafeRow (on heap) 452 452 0 725.7 1.4 0.3X +UnsafeRow (off heap) 296 297 0 1106.8 0.9 0.4X +Column On Heap Append 312 315 3 1048.8 1.0 0.4X ================================================================================================ Boolean Read/Write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Boolean Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Bitset 430 431 2 780.3 1.3 1.0X -Byte Array 249 250 2 1348.7 0.7 1.7X +Bitset 432 433 1 776.0 1.3 1.0X +Byte Array 250 251 1 1341.9 0.7 1.7X ================================================================================================ String Read/Write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -On Heap 121 122 1 134.9 7.4 1.0X -Off Heap 523 535 9 31.3 31.9 0.2X +On Heap 126 131 9 129.5 7.7 1.0X +Off Heap 475 495 18 34.5 29.0 0.3X ================================================================================================ Array Vector Read ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array Vector Read: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -On Heap Read Size Only 87 88 1 1878.9 0.5 1.0X -Off Heap Read Size Only 425 425 0 385.7 2.6 0.2X -On Heap Read Elements 2464 2467 5 66.5 15.0 0.0X -Off Heap Read Elements 2409 2412 5 68.0 14.7 0.0X +On Heap Read Size Only 86 87 0 1898.0 0.5 1.0X +Off Heap Read Size Only 307 308 1 533.7 1.9 0.3X +On Heap Read Elements 2385 2398 20 68.7 14.6 0.0X +Off Heap Read Elements 2606 2608 3 62.9 15.9 0.0X diff --git a/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt b/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt index ea5edb89dcfe6..96a39bdadeeaf 100644 --- a/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt +++ b/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt @@ -2,58 +2,58 @@ Int Read/Write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Java Array 174 175 1 1883.1 0.5 1.0X -ByteBuffer Unsafe 278 282 6 1177.8 0.8 0.6X -ByteBuffer API 508 509 1 645.6 1.5 0.3X -DirectByteBuffer 468 469 1 700.7 1.4 0.4X -Unsafe Buffer 159 161 1 2057.9 0.5 1.1X -Column(on heap) 170 171 0 1923.5 0.5 1.0X -Column(off heap) 162 162 0 2023.8 0.5 1.1X -Column(off heap direct) 157 158 1 2083.7 0.5 1.1X -UnsafeRow (on heap) 436 436 1 751.7 1.3 0.4X -UnsafeRow (off heap) 314 321 14 1042.5 1.0 0.6X -Column On Heap Append 361 362 1 906.5 1.1 0.5X +Java Array 175 177 5 1871.5 0.5 1.0X +ByteBuffer Unsafe 279 280 1 1174.5 0.9 0.6X +ByteBuffer API 510 511 1 642.1 1.6 0.3X +DirectByteBuffer 470 471 1 697.0 1.4 0.4X +Unsafe Buffer 162 163 1 2020.2 0.5 1.1X +Column(on heap) 171 172 0 1911.5 0.5 1.0X +Column(off heap) 163 163 0 2012.6 0.5 1.1X +Column(off heap direct) 158 158 0 2076.1 0.5 1.1X +UnsafeRow (on heap) 439 439 0 747.1 1.3 0.4X +UnsafeRow (off heap) 315 318 2 1040.4 1.0 0.6X +Column On Heap Append 363 364 1 901.8 1.1 0.5X ================================================================================================ Boolean Read/Write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Boolean Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Bitset 451 452 1 744.8 1.3 1.0X -Byte Array 288 289 2 1163.9 0.9 1.6X +Bitset 454 455 1 739.1 1.4 1.0X +Byte Array 263 264 1 1274.3 0.8 1.7X ================================================================================================ String Read/Write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -On Heap 193 236 33 85.0 11.8 1.0X -Off Heap 424 440 15 38.7 25.9 0.5X +On Heap 139 141 2 118.0 8.5 1.0X +Off Heap 382 391 12 42.9 23.3 0.4X ================================================================================================ Array Vector Read ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array Vector Read: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -On Heap Read Size Only 90 90 1 1826.1 0.5 1.0X -Off Heap Read Size Only 85 85 1 1927.9 0.5 1.1X -On Heap Read Elements 2177 2178 1 75.3 13.3 0.0X -Off Heap Read Elements 2732 2735 4 60.0 16.7 0.0X +On Heap Read Size Only 87 87 0 1883.4 0.5 1.0X +Off Heap Read Size Only 85 86 0 1918.9 0.5 1.0X +On Heap Read Elements 2428 2430 3 67.5 14.8 0.0X +Off Heap Read Elements 2956 2958 3 55.4 18.0 0.0X diff --git a/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt b/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt index 3338d6b4df0eb..588ce854d1858 100644 --- a/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt @@ -2,136 +2,136 @@ Compression Scheme Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor BOOLEAN Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 1 1 0 46950.3 0.0 1.0X -RunLengthEncoding(2.517) 983 989 6 68.2 14.7 0.0X -BooleanBitSet(0.125) 233 234 1 287.8 3.5 0.0X +PassThrough(1.000) 1 1 0 46470.3 0.0 1.0X +RunLengthEncoding(2.515) 1110 1283 245 60.5 16.5 0.0X +BooleanBitSet(0.125) 285 286 1 235.2 4.3 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor BOOLEAN Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 210 211 1 319.3 3.1 1.0X -RunLengthEncoding 598 605 10 112.3 8.9 0.4X -BooleanBitSet 696 699 3 96.5 10.4 0.3X +PassThrough 211 213 1 317.5 3.1 1.0X +RunLengthEncoding 601 601 0 111.7 9.0 0.4X +BooleanBitSet 672 672 0 99.9 10.0 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 3 3 0 23190.9 0.0 1.0X -RunLengthEncoding(1.495) 1229 1229 1 54.6 18.3 0.0X +PassThrough(1.000) 3 3 0 23145.6 0.0 1.0X +RunLengthEncoding(1.489) 1079 1079 0 62.2 16.1 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 667 668 1 100.6 9.9 1.0X -RunLengthEncoding 1030 1032 3 65.2 15.3 0.6X +PassThrough 795 796 2 84.5 11.8 1.0X +RunLengthEncoding 990 991 1 67.8 14.8 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 3 3 0 23427.9 0.0 1.0X -RunLengthEncoding(2.000) 1234 1234 0 54.4 18.4 0.0X +PassThrough(1.000) 3 3 0 23322.1 0.0 1.0X +RunLengthEncoding(2.000) 1116 1117 2 60.1 16.6 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 665 666 1 100.9 9.9 1.0X -RunLengthEncoding 1007 1007 0 66.6 15.0 0.7X +PassThrough 796 811 16 84.3 11.9 1.0X +RunLengthEncoding 956 957 1 70.2 14.2 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 6 6 0 11347.4 0.1 1.0X -RunLengthEncoding(0.997) 1072 1073 1 62.6 16.0 0.0X -DictionaryEncoding(0.500) 378 378 0 177.5 5.6 0.0X -IntDelta(0.250) 139 141 3 481.6 2.1 0.0X +PassThrough(1.000) 6 6 0 11601.5 0.1 1.0X +RunLengthEncoding(1.004) 1011 1011 1 66.4 15.1 0.0X +DictionaryEncoding(0.500) 335 335 0 200.4 5.0 0.0X +IntDelta(0.250) 110 111 0 607.5 1.6 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 644 646 2 104.2 9.6 1.0X -RunLengthEncoding 1181 1182 2 56.8 17.6 0.5X -DictionaryEncoding 521 522 0 128.8 7.8 1.2X -IntDelta 498 499 2 134.7 7.4 1.3X +PassThrough 647 647 0 103.7 9.6 1.0X +RunLengthEncoding 1213 1214 1 55.3 18.1 0.5X +DictionaryEncoding 526 526 0 127.7 7.8 1.2X +IntDelta 501 503 2 133.9 7.5 1.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 6 6 0 11261.2 0.1 1.0X -RunLengthEncoding(1.329) 1128 1129 2 59.5 16.8 0.0X -DictionaryEncoding(0.501) 378 379 2 177.5 5.6 0.0X -IntDelta(0.250) 125 125 0 536.8 1.9 0.0X +PassThrough(1.000) 6 6 0 11080.2 0.1 1.0X +RunLengthEncoding(1.339) 1048 1051 5 64.1 15.6 0.0X +DictionaryEncoding(0.501) 337 339 1 199.0 5.0 0.0X +IntDelta(0.250) 110 111 0 607.5 1.6 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 711 712 1 94.3 10.6 1.0X -RunLengthEncoding 1150 1154 5 58.4 17.1 0.6X -DictionaryEncoding 651 655 4 103.0 9.7 1.1X -IntDelta 520 573 59 129.1 7.7 1.4X +PassThrough 710 712 2 94.5 10.6 1.0X +RunLengthEncoding 1188 1190 3 56.5 17.7 0.6X +DictionaryEncoding 659 663 7 101.9 9.8 1.1X +IntDelta 524 526 3 128.1 7.8 1.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 13 13 0 5052.7 0.2 1.0X -RunLengthEncoding(0.748) 1072 1073 0 62.6 16.0 0.0X -DictionaryEncoding(0.250) 521 521 0 128.8 7.8 0.0X -LongDelta(0.125) 110 110 0 609.1 1.6 0.1X +PassThrough(1.000) 16 19 0 4302.0 0.2 1.0X +RunLengthEncoding(0.760) 1066 1066 0 63.0 15.9 0.0X +DictionaryEncoding(0.250) 404 405 2 166.2 6.0 0.0X +LongDelta(0.125) 111 111 0 605.5 1.7 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 771 774 3 87.1 11.5 1.0X -RunLengthEncoding 1232 1233 1 54.5 18.4 0.6X -DictionaryEncoding 720 724 6 93.2 10.7 1.1X -LongDelta 541 543 3 124.1 8.1 1.4X +PassThrough 774 776 2 86.7 11.5 1.0X +RunLengthEncoding 1240 1241 2 54.1 18.5 0.6X +DictionaryEncoding 714 717 4 93.9 10.6 1.1X +LongDelta 543 545 2 123.6 8.1 1.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 13 13 0 5054.5 0.2 1.0X -RunLengthEncoding(1.007) 1110 1111 1 60.4 16.5 0.0X -DictionaryEncoding(0.251) 533 534 2 126.0 7.9 0.0X -LongDelta(0.125) 111 112 0 605.2 1.7 0.1X +PassThrough(1.000) 18 18 0 3770.3 0.3 1.0X +RunLengthEncoding(1.002) 1095 1098 4 61.3 16.3 0.0X +DictionaryEncoding(0.251) 404 405 2 166.0 6.0 0.0X +LongDelta(0.125) 111 111 0 603.7 1.7 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 769 770 0 87.2 11.5 1.0X -RunLengthEncoding 1234 1236 4 54.4 18.4 0.6X -DictionaryEncoding 721 723 3 93.0 10.7 1.1X -LongDelta 669 672 3 100.2 10.0 1.1X +PassThrough 774 777 4 86.7 11.5 1.0X +RunLengthEncoding 1217 1218 1 55.2 18.1 0.6X +DictionaryEncoding 715 719 6 93.9 10.7 1.1X +LongDelta 671 672 2 100.1 10.0 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor STRING Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 20 20 0 3376.5 0.3 1.0X -RunLengthEncoding(0.892) 2013 2014 1 33.3 30.0 0.0X -DictionaryEncoding(0.167) 1687 1691 6 39.8 25.1 0.0X +PassThrough(1.000) 20 23 4 3349.0 0.3 1.0X +RunLengthEncoding(0.893) 1852 1854 3 36.2 27.6 0.0X +DictionaryEncoding(0.167) 2101 2111 14 31.9 31.3 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor STRING Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 1776 1776 1 37.8 26.5 1.0X -RunLengthEncoding 2518 2518 0 26.7 37.5 0.7X -DictionaryEncoding 2028 2030 4 33.1 30.2 0.9X +PassThrough 1654 1675 31 40.6 24.6 1.0X +RunLengthEncoding 2501 2505 6 26.8 37.3 0.7X +DictionaryEncoding 2028 2030 2 33.1 30.2 0.8X diff --git a/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt b/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt index c56288558bd5f..4b1206ab2e105 100644 --- a/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt +++ b/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt @@ -2,136 +2,136 @@ Compression Scheme Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor BOOLEAN Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 1 1 0 47046.4 0.0 1.0X -RunLengthEncoding(2.514) 882 883 0 76.0 13.1 0.0X -BooleanBitSet(0.125) 234 235 0 286.3 3.5 0.0X +PassThrough(1.000) 2 2 0 43967.6 0.0 1.0X +RunLengthEncoding(2.492) 900 901 1 74.6 13.4 0.0X +BooleanBitSet(0.125) 292 292 0 229.9 4.4 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor BOOLEAN Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 167 168 1 402.1 2.5 1.0X -RunLengthEncoding 532 534 1 126.1 7.9 0.3X -BooleanBitSet 663 665 2 101.2 9.9 0.3X +PassThrough 168 169 1 400.0 2.5 1.0X +RunLengthEncoding 551 555 5 121.7 8.2 0.3X +BooleanBitSet 639 640 1 105.0 9.5 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 3 3 0 23535.9 0.0 1.0X -RunLengthEncoding(1.501) 1218 1219 1 55.1 18.2 0.0X +PassThrough(1.000) 3 3 0 23004.2 0.0 1.0X +RunLengthEncoding(1.488) 1039 1040 1 64.6 15.5 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 710 712 2 94.6 10.6 1.0X -RunLengthEncoding 1043 1055 18 64.4 15.5 0.7X +PassThrough 548 561 9 122.5 8.2 1.0X +RunLengthEncoding 970 972 2 69.2 14.5 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 3 3 0 24666.2 0.0 1.0X -RunLengthEncoding(2.012) 1157 1159 3 58.0 17.2 0.0X +PassThrough(1.000) 3 3 0 23244.9 0.0 1.0X +RunLengthEncoding(2.018) 1070 1070 1 62.7 15.9 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SHORT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 675 675 0 99.5 10.1 1.0X -RunLengthEncoding 1021 1024 4 65.7 15.2 0.7X +PassThrough 543 544 0 123.5 8.1 1.0X +RunLengthEncoding 930 931 2 72.2 13.9 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 6 6 0 11233.4 0.1 1.0X -RunLengthEncoding(1.002) 1012 1021 12 66.3 15.1 0.0X -DictionaryEncoding(0.500) 386 387 1 174.1 5.7 0.0X -IntDelta(0.250) 115 115 1 585.5 1.7 0.1X +PassThrough(1.000) 6 6 0 11412.0 0.1 1.0X +RunLengthEncoding(1.006) 997 1000 3 67.3 14.9 0.0X +DictionaryEncoding(0.500) 374 374 1 179.6 5.6 0.0X +IntDelta(0.250) 110 110 1 609.3 1.6 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 644 647 3 104.3 9.6 1.0X -RunLengthEncoding 1194 1194 0 56.2 17.8 0.5X -DictionaryEncoding 502 504 2 133.7 7.5 1.3X -IntDelta 457 458 1 146.9 6.8 1.4X +PassThrough 626 627 1 107.2 9.3 1.0X +RunLengthEncoding 1041 1042 2 64.5 15.5 0.6X +DictionaryEncoding 524 527 2 128.0 7.8 1.2X +IntDelta 460 460 1 146.0 6.8 1.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 6 6 0 11739.3 0.1 1.0X -RunLengthEncoding(1.336) 1040 1040 1 64.5 15.5 0.0X -DictionaryEncoding(0.501) 387 388 1 173.2 5.8 0.0X -IntDelta(0.250) 115 115 1 585.4 1.7 0.0X +PassThrough(1.000) 6 6 0 11296.0 0.1 1.0X +RunLengthEncoding(1.338) 1018 1018 0 65.9 15.2 0.0X +DictionaryEncoding(0.501) 374 374 0 179.4 5.6 0.0X +IntDelta(0.250) 110 110 0 609.1 1.6 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor INT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 727 729 3 92.3 10.8 1.0X -RunLengthEncoding 1178 1182 5 57.0 17.6 0.6X -DictionaryEncoding 687 690 3 97.7 10.2 1.1X -IntDelta 480 482 2 139.7 7.2 1.5X +PassThrough 689 691 2 97.4 10.3 1.0X +RunLengthEncoding 1093 1094 1 61.4 16.3 0.6X +DictionaryEncoding 543 544 1 123.6 8.1 1.3X +IntDelta 597 599 2 112.5 8.9 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 13 13 0 5037.6 0.2 1.0X -RunLengthEncoding(0.750) 1017 1019 3 66.0 15.2 0.0X -DictionaryEncoding(0.250) 442 443 2 152.0 6.6 0.0X -LongDelta(0.125) 110 110 1 609.8 1.6 0.1X +PassThrough(1.000) 18 18 0 3771.2 0.3 1.0X +RunLengthEncoding(0.756) 1058 1059 2 63.5 15.8 0.0X +DictionaryEncoding(0.250) 441 442 1 152.2 6.6 0.0X +LongDelta(0.125) 111 111 0 604.4 1.7 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 755 758 3 88.8 11.3 1.0X -RunLengthEncoding 1216 1216 0 55.2 18.1 0.6X -DictionaryEncoding 774 774 0 86.8 11.5 1.0X -LongDelta 485 488 2 138.4 7.2 1.6X +PassThrough 713 715 2 94.2 10.6 1.0X +RunLengthEncoding 1192 1192 0 56.3 17.8 0.6X +DictionaryEncoding 686 689 3 97.8 10.2 1.0X +LongDelta 523 526 3 128.4 7.8 1.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 13 13 0 5032.6 0.2 1.0X -RunLengthEncoding(1.003) 1033 1035 3 65.0 15.4 0.0X -DictionaryEncoding(0.251) 444 446 3 151.1 6.6 0.0X -LongDelta(0.125) 147 147 1 457.3 2.2 0.1X +PassThrough(1.000) 13 14 0 4998.3 0.2 1.0X +RunLengthEncoding(1.000) 1073 1076 4 62.5 16.0 0.0X +DictionaryEncoding(0.251) 442 442 0 151.8 6.6 0.0X +LongDelta(0.125) 111 112 1 602.7 1.7 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LONG Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 753 755 2 89.1 11.2 1.0X -RunLengthEncoding 1225 1227 3 54.8 18.3 0.6X -DictionaryEncoding 773 774 0 86.8 11.5 1.0X -LongDelta 672 675 6 99.9 10.0 1.1X +PassThrough 712 714 2 94.3 10.6 1.0X +RunLengthEncoding 1163 1165 2 57.7 17.3 0.6X +DictionaryEncoding 685 686 2 97.9 10.2 1.0X +LongDelta 609 610 2 110.1 9.1 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor STRING Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough(1.000) 20 26 2 3350.9 0.3 1.0X -RunLengthEncoding(0.887) 1812 1813 2 37.0 27.0 0.0X -DictionaryEncoding(0.167) 2262 2263 1 29.7 33.7 0.0X +PassThrough(1.000) 27 27 0 2518.1 0.4 1.0X +RunLengthEncoding(0.892) 1819 1821 2 36.9 27.1 0.0X +DictionaryEncoding(0.167) 2071 2072 1 32.4 30.9 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor STRING Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -PassThrough 1645 1650 7 40.8 24.5 1.0X -RunLengthEncoding 2281 2284 4 29.4 34.0 0.7X -DictionaryEncoding 1845 1847 3 36.4 27.5 0.9X +PassThrough 1448 1471 33 46.3 21.6 1.0X +RunLengthEncoding 2222 2227 6 30.2 33.1 0.7X +DictionaryEncoding 1998 2010 17 33.6 29.8 0.7X diff --git a/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt b/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt index c53ca57d7242e..d3aa5cb8235de 100644 --- a/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt @@ -1,280 +1,280 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 365410.9 0.0 1.0X -OnHeapColumnVector 3342 3368 36 122.6 8.2 0.0X -OffHeapColumnVector 5519 5519 0 74.2 13.5 0.0X +ConstantColumnVector 1 1 0 372657.0 0.0 1.0X +OnHeapColumnVector 2898 2899 1 141.3 7.1 0.0X +OffHeapColumnVector 5566 5569 4 73.6 13.6 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 399028.5 0.0 1.0X -OnHeapColumnVector 4031 4035 6 101.6 9.8 0.0X -OffHeapColumnVector 4792 4796 6 85.5 11.7 0.0X +ConstantColumnVector 1 1 0 423940.2 0.0 1.0X +OnHeapColumnVector 4102 4103 1 99.9 10.0 0.0X +OffHeapColumnVector 4885 4901 22 83.8 11.9 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 399083.0 0.0 1.0X -OnHeapColumnVector 4041 4043 4 101.4 9.9 0.0X -OffHeapColumnVector 4684 4701 25 87.5 11.4 0.0X +ConstantColumnVector 1 1 0 423996.4 0.0 1.0X +OnHeapColumnVector 4284 4291 10 95.6 10.5 0.0X +OffHeapColumnVector 5062 5071 13 80.9 12.4 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 399168.5 0.0 1.0X -OnHeapColumnVector 4762 4762 0 86.0 11.6 0.0X -OffHeapColumnVector 5314 5316 3 77.1 13.0 0.0X +ConstantColumnVector 1 1 0 423912.6 0.0 1.0X +OnHeapColumnVector 4176 4186 14 98.1 10.2 0.0X +OffHeapColumnVector 4728 4736 11 86.6 11.5 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 399059.2 0.0 1.0X -OnHeapColumnVector 8010 8011 0 51.1 19.6 0.0X -OffHeapColumnVector 5170 5183 19 79.2 12.6 0.0X +ConstantColumnVector 1 1 0 423965.7 0.0 1.0X +OnHeapColumnVector 4527 4529 2 90.5 11.1 0.0X +OffHeapColumnVector 5110 5116 10 80.2 12.5 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 399074.8 0.0 1.0X -OnHeapColumnVector 4366 4366 0 93.8 10.7 0.0X -OffHeapColumnVector 4960 4963 4 82.6 12.1 0.0X +ConstantColumnVector 1 1 0 424082.0 0.0 1.0X +OnHeapColumnVector 4366 4372 9 93.8 10.7 0.0X +OffHeapColumnVector 5147 5152 8 79.6 12.6 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 632717.8 0.0 1.0X -OnHeapColumnVector 16 16 0 25522.9 0.0 0.0X -OffHeapColumnVector 65 65 0 6306.1 0.2 0.0X +ConstantColumnVector 1 1 0 664350.5 0.0 1.0X +OnHeapColumnVector 16 16 0 25444.2 0.0 0.0X +OffHeapColumnVector 65 65 0 6275.3 0.2 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 577697.2 0.0 1.0X -OnHeapColumnVector 33 33 0 12488.2 0.1 0.0X -OffHeapColumnVector 66 66 1 6198.2 0.2 0.0X +ConstantColumnVector 1 1 0 632713.9 0.0 1.0X +OnHeapColumnVector 33 34 0 12422.6 0.1 0.0X +OffHeapColumnVector 67 68 1 6094.1 0.2 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 442449.7 0.0 1.0X -OnHeapColumnVector 16 16 0 25047.7 0.0 0.1X -OffHeapColumnVector 127 128 0 3216.3 0.3 0.0X +ConstantColumnVector 1 1 0 457739.0 0.0 1.0X +OnHeapColumnVector 16 16 0 25107.7 0.0 0.1X +OffHeapColumnVector 129 129 0 3177.6 0.3 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 491627.0 0.0 1.0X -OnHeapColumnVector 33 33 0 12493.3 0.1 0.0X -OffHeapColumnVector 129 129 0 3184.4 0.3 0.0X +ConstantColumnVector 1 1 0 530954.4 0.0 1.0X +OnHeapColumnVector 34 34 0 12039.3 0.1 0.0X +OffHeapColumnVector 129 129 0 3168.0 0.3 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 309 310 1 1324.1 0.8 0.0X -OffHeapColumnVector 3767 3768 1 108.7 9.2 0.0X +ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X +OnHeapColumnVector 105 106 1 3884.1 0.3 0.0X +OffHeapColumnVector 6540 6543 4 62.6 16.0 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 4118 4123 7 99.5 10.1 0.0X -OffHeapColumnVector 3746 3755 13 109.3 9.1 0.0X +ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X +OnHeapColumnVector 4074 4075 0 100.5 9.9 0.0X +OffHeapColumnVector 6602 6610 12 62.0 16.1 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 4114 4115 2 99.6 10.0 0.0X -OffHeapColumnVector 3744 3763 27 109.4 9.1 0.0X +ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X +OnHeapColumnVector 4052 4056 6 101.1 9.9 0.0X +OffHeapColumnVector 6534 6537 5 62.7 16.0 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 4107 4122 21 99.7 10.0 0.0X -OffHeapColumnVector 3763 3779 21 108.8 9.2 0.0X +ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X +OnHeapColumnVector 4056 4058 3 101.0 9.9 0.0X +OffHeapColumnVector 6536 6541 7 62.7 16.0 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 4102 4104 4 99.9 10.0 0.0X -OffHeapColumnVector 3820 3824 7 107.2 9.3 0.0X +ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X +OnHeapColumnVector 4046 4053 9 101.2 9.9 0.0X +OffHeapColumnVector 6530 6531 1 62.7 15.9 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 4246 4248 2 96.5 10.4 0.0X -OffHeapColumnVector 3743 3777 48 109.4 9.1 0.0X +ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X +OnHeapColumnVector 4059 4061 3 100.9 9.9 0.0X +OffHeapColumnVector 6537 6538 2 62.7 16.0 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 26549131.4 0.0 1.0X -OnHeapColumnVector 1 1 0 492066.4 0.0 0.0X -OffHeapColumnVector 889 890 2 461.0 2.2 0.0X +ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X +OnHeapColumnVector 1 1 0 474473.3 0.0 0.0X +OffHeapColumnVector 893 894 1 458.6 2.2 0.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1939 1940 2 211.3 4.7 1.0X -OnHeapColumnVector 2075 2089 19 197.4 5.1 0.9X -OffHeapColumnVector 2601 2603 2 157.5 6.3 0.7X +ConstantColumnVector 1953 1955 2 209.7 4.8 1.0X +OnHeapColumnVector 2072 2077 7 197.7 5.1 0.9X +OffHeapColumnVector 2604 2608 5 157.3 6.4 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1846 1848 2 221.9 4.5 1.0X -OnHeapColumnVector 2099 2101 4 195.2 5.1 0.9X -OffHeapColumnVector 2613 2638 35 156.7 6.4 0.7X +ConstantColumnVector 1845 1846 2 222.0 4.5 1.0X +OnHeapColumnVector 2101 2103 4 195.0 5.1 0.9X +OffHeapColumnVector 2613 2615 3 156.8 6.4 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1986 1987 1 206.3 4.8 1.0X -OnHeapColumnVector 2120 2121 1 193.2 5.2 0.9X -OffHeapColumnVector 2753 2753 0 148.8 6.7 0.7X +ConstantColumnVector 1985 1986 1 206.3 4.8 1.0X +OnHeapColumnVector 2120 2123 4 193.2 5.2 0.9X +OffHeapColumnVector 2758 2762 5 148.5 6.7 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1943 1943 0 210.8 4.7 1.0X -OnHeapColumnVector 5899 5903 5 69.4 14.4 0.3X -OffHeapColumnVector 5086 5089 5 80.5 12.4 0.4X +ConstantColumnVector 1948 1952 5 210.2 4.8 1.0X +OnHeapColumnVector 5737 5746 13 71.4 14.0 0.3X +OffHeapColumnVector 8493 8494 2 48.2 20.7 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1943 1943 1 210.8 4.7 1.0X -OnHeapColumnVector 5919 5922 5 69.2 14.5 0.3X -OffHeapColumnVector 5089 5096 10 80.5 12.4 0.4X +ConstantColumnVector 1950 1951 1 210.0 4.8 1.0X +OnHeapColumnVector 5657 5657 1 72.4 13.8 0.3X +OffHeapColumnVector 8500 8502 3 48.2 20.8 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1940 1946 9 211.1 4.7 1.0X -OnHeapColumnVector 5901 5907 8 69.4 14.4 0.3X -OffHeapColumnVector 5132 5142 14 79.8 12.5 0.4X +ConstantColumnVector 1948 1949 1 210.3 4.8 1.0X +OnHeapColumnVector 5765 5765 1 71.0 14.1 0.3X +OffHeapColumnVector 8512 8533 29 48.1 20.8 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1943 1944 2 210.8 4.7 1.0X -OnHeapColumnVector 5913 5914 1 69.3 14.4 0.3X -OffHeapColumnVector 5133 5159 37 79.8 12.5 0.4X +ConstantColumnVector 1949 1950 0 210.1 4.8 1.0X +OnHeapColumnVector 5660 5670 15 72.4 13.8 0.3X +OffHeapColumnVector 8502 8505 4 48.2 20.8 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1945 1949 6 210.6 4.7 1.0X -OnHeapColumnVector 5954 5955 2 68.8 14.5 0.3X -OffHeapColumnVector 5081 5083 3 80.6 12.4 0.4X +ConstantColumnVector 1952 1956 6 209.9 4.8 1.0X +OnHeapColumnVector 5742 5745 3 71.3 14.0 0.3X +OffHeapColumnVector 8555 8574 26 47.9 20.9 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1956 1957 2 209.4 4.8 1.0X -OnHeapColumnVector 5956 5997 58 68.8 14.5 0.3X -OffHeapColumnVector 5076 5077 1 80.7 12.4 0.4X +ConstantColumnVector 1956 1957 0 209.4 4.8 1.0X +OnHeapColumnVector 5657 5661 4 72.4 13.8 0.3X +OffHeapColumnVector 8523 8539 23 48.1 20.8 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 888 888 0 461.4 2.2 1.0X -OnHeapColumnVector 889 890 1 461.0 2.2 1.0X -OffHeapColumnVector 888 889 1 461.3 2.2 1.0X +ConstantColumnVector 892 892 1 459.3 2.2 1.0X +OnHeapColumnVector 1020 1021 1 401.5 2.5 0.9X +OffHeapColumnVector 892 893 1 459.0 2.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 2850 2850 0 143.7 7.0 1.0X -OnHeapColumnVector 2978 2978 1 137.6 7.3 1.0X -OffHeapColumnVector 2977 2978 1 137.6 7.3 1.0X +ConstantColumnVector 2866 2869 4 142.9 7.0 1.0X +OnHeapColumnVector 2993 2994 0 136.8 7.3 1.0X +OffHeapColumnVector 2991 2993 3 137.0 7.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 2867 2872 6 142.9 7.0 1.0X -OnHeapColumnVector 2993 2994 1 136.8 7.3 1.0X -OffHeapColumnVector 2991 2995 5 136.9 7.3 1.0X +ConstantColumnVector 2877 2892 21 142.4 7.0 1.0X +OnHeapColumnVector 3135 3136 3 130.7 7.7 0.9X +OffHeapColumnVector 3012 3013 1 136.0 7.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 3009 3011 3 136.1 7.3 1.0X -OnHeapColumnVector 3137 3139 3 130.6 7.7 1.0X -OffHeapColumnVector 3141 3142 2 130.4 7.7 1.0X +ConstantColumnVector 2381 2381 0 172.1 5.8 1.0X +OnHeapColumnVector 3157 3158 3 129.8 7.7 0.8X +OffHeapColumnVector 3148 3149 1 130.1 7.7 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with StringType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 0 0 0 3321197.8 0.0 0.0X +OnHeapColumnVector 0 0 0 3321413.2 0.0 0.0X OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 0 0 0 3321197.8 0.0 0.0X +OnHeapColumnVector 0 0 0 3321440.2 0.0 0.0X OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 0 0 0 3321197.8 0.0 0.0X +OnHeapColumnVector 0 0 0 3321440.2 0.0 0.0X OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 0 0 0 3321197.8 0.0 0.0X +OnHeapColumnVector 0 0 0 3321440.2 0.0 0.0X OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X -OnHeapColumnVector 0 0 0 3321467.1 0.0 0.0X +OnHeapColumnVector 0 0 0 3321440.2 0.0 0.0X OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X diff --git a/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt b/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt index c381cbab325fc..39aedf6270830 100644 --- a/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt +++ b/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt @@ -1,280 +1,280 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 329000.5 0.0 1.0X -OnHeapColumnVector 2882 2884 3 142.1 7.0 0.0X -OffHeapColumnVector 3380 3382 2 121.2 8.3 0.0X +ConstantColumnVector 1 1 0 324095.0 0.0 1.0X +OnHeapColumnVector 2813 2814 2 145.6 6.9 0.0X +OffHeapColumnVector 3407 3412 7 120.2 8.3 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 387377.7 0.0 1.0X -OnHeapColumnVector 3661 3670 12 111.9 8.9 0.0X -OffHeapColumnVector 4386 4388 3 93.4 10.7 0.0X +ConstantColumnVector 1 1 0 382856.0 0.0 1.0X +OnHeapColumnVector 4041 4044 4 101.4 9.9 0.0X +OffHeapColumnVector 4288 4289 1 95.5 10.5 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 387399.7 0.0 1.0X -OnHeapColumnVector 3915 3918 4 104.6 9.6 0.0X -OffHeapColumnVector 4559 4560 2 89.8 11.1 0.0X +ConstantColumnVector 1 1 0 383128.5 0.0 1.0X +OnHeapColumnVector 4013 4014 2 102.1 9.8 0.0X +OffHeapColumnVector 4353 4355 2 94.1 10.6 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 387538.2 0.0 1.0X -OnHeapColumnVector 3628 3632 6 112.9 8.9 0.0X -OffHeapColumnVector 4489 4490 2 91.2 11.0 0.0X +ConstantColumnVector 1 1 0 383052.9 0.0 1.0X +OnHeapColumnVector 3818 3820 3 107.3 9.3 0.0X +OffHeapColumnVector 4644 4645 1 88.2 11.3 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 387487.6 0.0 1.0X -OnHeapColumnVector 4219 4222 5 97.1 10.3 0.0X -OffHeapColumnVector 4701 4702 2 87.1 11.5 0.0X +ConstantColumnVector 1 1 0 383078.3 0.0 1.0X +OnHeapColumnVector 4128 4139 16 99.2 10.1 0.0X +OffHeapColumnVector 4602 4605 4 89.0 11.2 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1 1 0 387405.9 0.0 1.0X -OnHeapColumnVector 4336 4342 8 94.5 10.6 0.0X -OffHeapColumnVector 4376 4376 0 93.6 10.7 0.0X +ConstantColumnVector 1 1 0 383207.7 0.0 1.0X +OnHeapColumnVector 4274 4280 8 95.8 10.4 0.0X +OffHeapColumnVector 4583 4584 2 89.4 11.2 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 699323.4 0.0 1.0X -OnHeapColumnVector 16 16 0 25587.5 0.0 0.0X -OffHeapColumnVector 65 65 0 6320.1 0.2 0.0X +ConstantColumnVector 1 1 0 699303.1 0.0 1.0X +OnHeapColumnVector 16 16 0 25461.3 0.0 0.0X +OffHeapColumnVector 66 66 1 6242.8 0.2 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 664355.9 0.0 1.0X -OnHeapColumnVector 33 34 0 12331.0 0.1 0.0X -OffHeapColumnVector 67 67 0 6114.9 0.2 0.0X +ConstantColumnVector 1 1 0 664337.6 0.0 1.0X +OnHeapColumnVector 34 34 0 12100.9 0.1 0.0X +OffHeapColumnVector 68 69 0 5986.6 0.2 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 553059.9 0.0 1.0X -OnHeapColumnVector 16 16 0 25179.1 0.0 0.0X -OffHeapColumnVector 127 127 0 3217.6 0.3 0.0X +ConstantColumnVector 1 1 0 553053.1 0.0 1.0X +OnHeapColumnVector 16 16 0 25009.2 0.0 0.0X +OffHeapColumnVector 128 128 0 3191.8 0.3 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1 1 0 510543.7 0.0 1.0X -OnHeapColumnVector 34 34 0 12081.9 0.1 0.0X -OffHeapColumnVector 128 129 0 3191.4 0.3 0.0X +ConstantColumnVector 1 1 0 510537.3 0.0 1.0X +OnHeapColumnVector 34 35 0 11938.3 0.1 0.0X +OffHeapColumnVector 129 130 0 3165.4 0.3 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 1041 1041 1 393.6 2.5 1.0X -OnHeapColumnVector 2191 2191 0 186.9 5.3 0.5X -OffHeapColumnVector 4378 4379 1 93.6 10.7 0.2X +ConstantColumnVector 1051 1051 1 389.8 2.6 1.0X +OnHeapColumnVector 2133 2135 2 192.0 5.2 0.5X +OffHeapColumnVector 4374 4376 2 93.6 10.7 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 826 827 2 496.1 2.0 1.0X -OnHeapColumnVector 4856 4859 4 84.4 11.9 0.2X -OffHeapColumnVector 4645 4667 32 88.2 11.3 0.2X +ConstantColumnVector 836 836 0 490.1 2.0 1.0X +OnHeapColumnVector 4993 4994 0 82.0 12.2 0.2X +OffHeapColumnVector 4488 4489 1 91.3 11.0 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 828 828 1 494.9 2.0 1.0X -OnHeapColumnVector 4917 4918 2 83.3 12.0 0.2X -OffHeapColumnVector 4624 4631 9 88.6 11.3 0.2X +ConstantColumnVector 835 836 2 490.6 2.0 1.0X +OnHeapColumnVector 5030 5032 3 81.4 12.3 0.2X +OffHeapColumnVector 4509 4513 5 90.8 11.0 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 826 828 2 495.9 2.0 1.0X -OnHeapColumnVector 4914 4917 5 83.4 12.0 0.2X -OffHeapColumnVector 4635 4637 3 88.4 11.3 0.2X +ConstantColumnVector 838 840 3 489.1 2.0 1.0X +OnHeapColumnVector 5039 5045 8 81.3 12.3 0.2X +OffHeapColumnVector 4522 4523 2 90.6 11.0 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 827 829 2 495.1 2.0 1.0X -OnHeapColumnVector 4931 4932 1 83.1 12.0 0.2X -OffHeapColumnVector 4642 4644 2 88.2 11.3 0.2X +ConstantColumnVector 833 836 3 491.5 2.0 1.0X +OnHeapColumnVector 5044 5045 0 81.2 12.3 0.2X +OffHeapColumnVector 4500 4502 3 91.0 11.0 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 826 827 1 496.0 2.0 1.0X -OnHeapColumnVector 4908 4921 19 83.5 12.0 0.2X -OffHeapColumnVector 4627 4628 1 88.5 11.3 0.2X +ConstantColumnVector 835 836 1 490.4 2.0 1.0X +OnHeapColumnVector 5040 5042 3 81.3 12.3 0.2X +OffHeapColumnVector 4499 4499 1 91.0 11.0 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 1811 1811 0 226.2 4.4 1.0X -OnHeapColumnVector 2128 2130 4 192.5 5.2 0.9X -OffHeapColumnVector 2340 2343 3 175.0 5.7 0.8X +ConstantColumnVector 2605 2605 1 157.3 6.4 1.0X +OnHeapColumnVector 2723 2724 2 150.4 6.6 1.0X +OffHeapColumnVector 2729 2730 1 150.1 6.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 2657221.1 0.0 1.0X -OnHeapColumnVector 0 0 0 1022070.8 0.0 0.4X -OffHeapColumnVector 691 692 1 592.6 1.7 0.0X +ConstantColumnVector 0 0 0 1476302.0 0.0 1.0X +OnHeapColumnVector 0 0 0 1022060.6 0.0 0.7X +OffHeapColumnVector 767 767 0 534.2 1.9 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 2214485.0 0.0 1.0X -OnHeapColumnVector 0 1 0 949064.3 0.0 0.4X -OffHeapColumnVector 767 769 3 533.8 1.9 0.0X +ConstantColumnVector 0 0 0 1660780.7 0.0 1.0X +OnHeapColumnVector 0 0 0 1022032.6 0.0 0.6X +OffHeapColumnVector 766 767 1 534.8 1.9 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 1022070.8 0.0 1.0X -OnHeapColumnVector 1 1 0 738160.3 0.0 0.7X -OffHeapColumnVector 762 762 0 537.5 1.9 0.0X +ConstantColumnVector 0 0 0 1476307.4 0.0 1.0X +OnHeapColumnVector 0 0 0 1022058.1 0.0 0.7X +OffHeapColumnVector 767 767 0 534.2 1.9 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 1660794.1 0.0 1.0X -OnHeapColumnVector 3784 3785 2 108.3 9.2 0.0X -OffHeapColumnVector 3768 3782 20 108.7 9.2 0.0X +ConstantColumnVector 0 0 0 1021777.6 0.0 1.0X +OnHeapColumnVector 3918 3923 7 104.5 9.6 0.0X +OffHeapColumnVector 3743 3752 12 109.4 9.1 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -ConstantColumnVector 0 0 0 1660794.1 0.0 1.0X -OnHeapColumnVector 3788 3808 28 108.1 9.2 0.0X -OffHeapColumnVector 3680 3687 10 111.3 9.0 0.0X +ConstantColumnVector 0 0 0 857165.6 0.0 1.0X +OnHeapColumnVector 3933 3938 7 104.1 9.6 0.0X +OffHeapColumnVector 3737 3748 16 109.6 9.1 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 1660794.1 0.0 1.0X -OnHeapColumnVector 3804 3807 5 107.7 9.3 0.0X -OffHeapColumnVector 3712 3713 1 110.3 9.1 0.0X +ConstantColumnVector 0 0 0 857165.6 0.0 1.0X +OnHeapColumnVector 3930 3930 1 104.2 9.6 0.0X +OffHeapColumnVector 3736 3736 1 109.6 9.1 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 1660794.1 0.0 1.0X -OnHeapColumnVector 3801 3802 2 107.8 9.3 0.0X -OffHeapColumnVector 3704 3704 1 110.6 9.0 0.0X +ConstantColumnVector 0 0 0 857165.6 0.0 1.0X +OnHeapColumnVector 3922 3923 1 104.4 9.6 0.0X +OffHeapColumnVector 3742 3743 1 109.5 9.1 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 1660794.1 0.0 1.0X -OnHeapColumnVector 3795 3797 2 107.9 9.3 0.0X -OffHeapColumnVector 3703 3715 16 110.6 9.0 0.0X +ConstantColumnVector 0 0 0 857165.6 0.0 1.0X +OnHeapColumnVector 3920 3926 8 104.5 9.6 0.0X +OffHeapColumnVector 3745 3753 12 109.4 9.1 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 1660794.1 0.0 1.0X -OnHeapColumnVector 3794 3797 4 108.0 9.3 0.0X -OffHeapColumnVector 3719 3720 1 110.1 9.1 0.0X +ConstantColumnVector 0 0 0 857183.5 0.0 1.0X +OnHeapColumnVector 3920 3926 9 104.5 9.6 0.0X +OffHeapColumnVector 3723 3725 3 110.0 9.1 0.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 3673 3675 3 111.5 9.0 1.0X -OnHeapColumnVector 2448 2450 3 167.3 6.0 1.5X -OffHeapColumnVector 2585 2585 1 158.5 6.3 1.4X +ConstantColumnVector 3097 3099 3 132.3 7.6 1.0X +OnHeapColumnVector 2732 2733 1 149.9 6.7 1.1X +OffHeapColumnVector 2741 2742 1 149.4 6.7 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 0 0 0 2657221.1 0.0 1.0X -OnHeapColumnVector 651 652 1 629.3 1.6 0.0X -OffHeapColumnVector 691 692 1 592.4 1.7 0.0X +ConstantColumnVector 765 766 1 535.4 1.9 1.0X +OnHeapColumnVector 774 774 1 529.3 1.9 1.0X +OffHeapColumnVector 830 831 2 493.6 2.0 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 887 888 1 461.6 2.2 1.0X -OnHeapColumnVector 764 764 0 535.9 1.9 1.2X -OffHeapColumnVector 762 763 1 537.5 1.9 1.2X +ConstantColumnVector 765 768 3 535.2 1.9 1.0X +OnHeapColumnVector 772 773 1 530.4 1.9 1.0X +OffHeapColumnVector 831 832 1 492.7 2.0 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test write and read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ConstantColumnVector 761 761 0 538.5 1.9 1.0X -OnHeapColumnVector 765 765 1 535.7 1.9 1.0X -OffHeapColumnVector 763 763 1 537.2 1.9 1.0X +ConstantColumnVector 892 893 1 459.2 2.2 1.0X +OnHeapColumnVector 774 775 1 528.9 1.9 1.2X +OffHeapColumnVector 831 831 0 493.0 2.0 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with StringType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X -OnHeapColumnVector 0 0 0 2211973.6 0.0 0.0X +OnHeapColumnVector 0 0 0 2211949.7 0.0 0.0X OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X -OnHeapColumnVector 0 0 0 2211985.5 0.0 0.0X +OnHeapColumnVector 0 0 0 2211949.7 0.0 0.0X OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X -OnHeapColumnVector 0 0 0 2211985.5 0.0 0.0X +OnHeapColumnVector 0 0 0 2211949.7 0.0 0.0X OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X -OnHeapColumnVector 0 0 0 2211985.5 0.0 0.0X +OnHeapColumnVector 0 0 0 2211949.7 0.0 0.0X OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Test isNull with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X -OnHeapColumnVector 0 0 0 2211985.5 0.0 0.0X +OnHeapColumnVector 0 0 0 2211949.7 0.0 0.0X OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt index bdc453db1735d..ea578d9f6d8aa 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt @@ -2,437 +2,437 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 10214 10246 45 1.5 649.4 1.0X -SQL Json 7831 7865 48 2.0 497.9 1.3X -SQL Json with UnsafeRow 8565 8571 8 1.8 544.6 1.2X -SQL Parquet Vectorized: DataPageV1 81 96 11 193.3 5.2 125.6X -SQL Parquet Vectorized: DataPageV2 201 210 8 78.4 12.8 50.9X -SQL Parquet MR: DataPageV1 1794 1818 34 8.8 114.1 5.7X -SQL Parquet MR: DataPageV2 1650 1651 1 9.5 104.9 6.2X -SQL ORC Vectorized 120 132 8 130.5 7.7 84.8X -SQL ORC MR 1447 1453 9 10.9 92.0 7.1X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 10281 10314 46 1.5 653.6 1.0X +SQL Json 7952 8108 220 2.0 505.6 1.3X +SQL Json with UnsafeRow 9090 9092 3 1.7 577.9 1.1X +SQL Parquet Vectorized: DataPageV1 82 94 10 192.5 5.2 125.8X +SQL Parquet Vectorized: DataPageV2 92 99 8 171.9 5.8 112.3X +SQL Parquet MR: DataPageV1 1701 1728 38 9.2 108.2 6.0X +SQL Parquet MR: DataPageV2 1594 1607 19 9.9 101.3 6.5X +SQL ORC Vectorized 137 142 6 114.9 8.7 75.1X +SQL ORC MR 1464 1465 2 10.7 93.1 7.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 84 86 1 187.3 5.3 1.0X -ParquetReader Vectorized: DataPageV2 208 211 4 75.7 13.2 0.4X -ParquetReader Vectorized -> Row: DataPageV1 72 73 1 219.2 4.6 1.2X -ParquetReader Vectorized -> Row: DataPageV2 199 201 4 79.2 12.6 0.4X +ParquetReader Vectorized: DataPageV1 84 86 2 186.8 5.4 1.0X +ParquetReader Vectorized: DataPageV2 100 101 1 157.9 6.3 0.8X +ParquetReader Vectorized -> Row: DataPageV1 73 74 1 216.3 4.6 1.2X +ParquetReader Vectorized -> Row: DataPageV2 90 91 1 175.2 5.7 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 9574 9607 46 1.6 608.7 1.0X -SQL Json 8719 8757 55 1.8 554.3 1.1X -SQL Json with UnsafeRow 9120 9130 13 1.7 579.9 1.0X -SQL Parquet Vectorized: DataPageV1 95 101 5 164.9 6.1 100.4X -SQL Parquet Vectorized: DataPageV2 95 104 8 165.3 6.0 100.6X -SQL Parquet MR: DataPageV1 1927 1938 15 8.2 122.5 5.0X -SQL Parquet MR: DataPageV2 1792 1851 84 8.8 114.0 5.3X -SQL ORC Vectorized 110 118 7 143.1 7.0 87.1X -SQL ORC MR 1579 1582 4 10.0 100.4 6.1X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 9866 9904 53 1.6 627.3 1.0X +SQL Json 9122 9125 5 1.7 579.9 1.1X +SQL Json with UnsafeRow 10109 10124 20 1.6 642.7 1.0X +SQL Parquet Vectorized: DataPageV1 96 104 8 163.5 6.1 102.6X +SQL Parquet Vectorized: DataPageV2 98 111 8 160.7 6.2 100.8X +SQL Parquet MR: DataPageV1 1870 1883 19 8.4 118.9 5.3X +SQL Parquet MR: DataPageV2 1857 1895 54 8.5 118.1 5.3X +SQL ORC Vectorized 139 149 15 113.1 8.8 70.9X +SQL ORC MR 1588 1591 4 9.9 101.0 6.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 80 83 2 196.0 5.1 1.0X -ParquetReader Vectorized: DataPageV2 81 83 1 194.9 5.1 1.0X -ParquetReader Vectorized -> Row: DataPageV1 44 46 2 353.7 2.8 1.8X -ParquetReader Vectorized -> Row: DataPageV2 45 46 1 352.4 2.8 1.8X +ParquetReader Vectorized: DataPageV1 82 84 2 191.5 5.2 1.0X +ParquetReader Vectorized: DataPageV2 85 98 7 184.5 5.4 1.0X +ParquetReader Vectorized -> Row: DataPageV1 46 51 6 341.6 2.9 1.8X +ParquetReader Vectorized -> Row: DataPageV2 46 50 5 339.9 2.9 1.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 10409 10436 39 1.5 661.8 1.0X -SQL Json 8942 8944 4 1.8 568.5 1.2X -SQL Json with UnsafeRow 9693 9697 5 1.6 616.3 1.1X -SQL Parquet Vectorized: DataPageV1 118 134 17 133.8 7.5 88.5X -SQL Parquet Vectorized: DataPageV2 139 152 16 113.5 8.8 75.1X -SQL Parquet MR: DataPageV1 2019 2054 50 7.8 128.4 5.2X -SQL Parquet MR: DataPageV2 2011 2011 0 7.8 127.9 5.2X -SQL ORC Vectorized 140 148 8 112.1 8.9 74.2X -SQL ORC MR 1818 1825 10 8.7 115.6 5.7X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 10575 10586 14 1.5 672.4 1.0X +SQL Json 9463 9503 57 1.7 601.6 1.1X +SQL Json with UnsafeRow 10388 10399 15 1.5 660.5 1.0X +SQL Parquet Vectorized: DataPageV1 118 131 14 133.4 7.5 89.7X +SQL Parquet Vectorized: DataPageV2 140 183 19 112.4 8.9 75.6X +SQL Parquet MR: DataPageV1 2010 2013 4 7.8 127.8 5.3X +SQL Parquet MR: DataPageV2 2018 2038 28 7.8 128.3 5.2X +SQL ORC Vectorized 139 172 28 113.3 8.8 76.2X +SQL ORC MR 1687 1701 20 9.3 107.3 6.3X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 138 146 9 113.8 8.8 1.0X -ParquetReader Vectorized: DataPageV2 169 176 10 93.2 10.7 0.8X -ParquetReader Vectorized -> Row: DataPageV1 134 139 5 117.0 8.5 1.0X -ParquetReader Vectorized -> Row: DataPageV2 183 186 5 86.1 11.6 0.8X +ParquetReader Vectorized: DataPageV1 149 155 5 105.3 9.5 1.0X +ParquetReader Vectorized: DataPageV2 178 184 7 88.2 11.3 0.8X +ParquetReader Vectorized -> Row: DataPageV1 135 140 5 116.9 8.6 1.1X +ParquetReader Vectorized -> Row: DataPageV2 166 176 10 95.0 10.5 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 11428 11435 9 1.4 726.6 1.0X -SQL Json 9048 9074 37 1.7 575.2 1.3X -SQL Json with UnsafeRow 9790 9800 14 1.6 622.4 1.2X -SQL Parquet Vectorized: DataPageV1 97 110 13 162.2 6.2 117.8X -SQL Parquet Vectorized: DataPageV2 176 197 18 89.2 11.2 64.8X -SQL Parquet MR: DataPageV1 1974 1978 6 8.0 125.5 5.8X -SQL Parquet MR: DataPageV2 2028 2031 5 7.8 128.9 5.6X -SQL ORC Vectorized 177 201 27 89.0 11.2 64.6X -SQL ORC MR 2053 2059 9 7.7 130.5 5.6X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 11729 11735 10 1.3 745.7 1.0X +SQL Json 9804 9835 43 1.6 623.3 1.2X +SQL Json with UnsafeRow 10754 10760 9 1.5 683.7 1.1X +SQL Parquet Vectorized: DataPageV1 97 113 14 162.9 6.1 121.5X +SQL Parquet Vectorized: DataPageV2 176 191 12 89.3 11.2 66.6X +SQL Parquet MR: DataPageV1 1949 1973 34 8.1 123.9 6.0X +SQL Parquet MR: DataPageV2 2019 2034 21 7.8 128.4 5.8X +SQL ORC Vectorized 180 190 17 87.6 11.4 65.3X +SQL ORC MR 1692 1707 22 9.3 107.5 6.9X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 158 162 5 99.5 10.0 1.0X -ParquetReader Vectorized: DataPageV2 237 248 18 66.4 15.1 0.7X -ParquetReader Vectorized -> Row: DataPageV1 128 134 7 122.5 8.2 1.2X -ParquetReader Vectorized -> Row: DataPageV2 209 216 6 75.3 13.3 0.8X +ParquetReader Vectorized: DataPageV1 130 138 6 120.9 8.3 1.0X +ParquetReader Vectorized: DataPageV2 214 219 6 73.6 13.6 0.6X +ParquetReader Vectorized -> Row: DataPageV1 129 133 5 122.0 8.2 1.0X +ParquetReader Vectorized -> Row: DataPageV2 225 246 24 69.8 14.3 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 11758 11763 8 1.3 747.6 1.0X -SQL Json 9255 9264 12 1.7 588.4 1.3X -SQL Json with UnsafeRow 9871 9876 6 1.6 627.6 1.2X -SQL Parquet Vectorized: DataPageV1 286 308 13 54.9 18.2 41.1X -SQL Parquet Vectorized: DataPageV2 238 269 14 66.0 15.2 49.3X -SQL Parquet MR: DataPageV1 2493 2494 1 6.3 158.5 4.7X -SQL Parquet MR: DataPageV2 2053 2054 2 7.7 130.5 5.7X -SQL ORC Vectorized 165 174 10 95.5 10.5 71.4X -SQL ORC MR 1821 1822 1 8.6 115.8 6.5X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 11573 11671 139 1.4 735.8 1.0X +SQL Json 9549 9558 12 1.6 607.1 1.2X +SQL Json with UnsafeRow 10532 10532 1 1.5 669.6 1.1X +SQL Parquet Vectorized: DataPageV1 279 300 17 56.3 17.8 41.4X +SQL Parquet Vectorized: DataPageV2 248 272 11 63.5 15.7 46.7X +SQL Parquet MR: DataPageV1 2453 2454 2 6.4 156.0 4.7X +SQL Parquet MR: DataPageV2 1991 1997 8 7.9 126.6 5.8X +SQL ORC Vectorized 166 179 12 94.5 10.6 69.5X +SQL ORC MR 1773 1776 4 8.9 112.7 6.5X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 305 313 10 51.6 19.4 1.0X -ParquetReader Vectorized: DataPageV2 258 270 15 60.8 16.4 1.2X -ParquetReader Vectorized -> Row: DataPageV1 317 319 3 49.6 20.2 1.0X -ParquetReader Vectorized -> Row: DataPageV2 254 268 9 61.9 16.2 1.2X +ParquetReader Vectorized: DataPageV1 306 309 3 51.5 19.4 1.0X +ParquetReader Vectorized: DataPageV2 278 284 6 56.5 17.7 1.1X +ParquetReader Vectorized -> Row: DataPageV1 317 323 6 49.6 20.2 1.0X +ParquetReader Vectorized -> Row: DataPageV2 262 272 9 60.1 16.6 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 11470 11490 28 1.4 729.3 1.0X -SQL Json 10456 10495 56 1.5 664.8 1.1X -SQL Json with UnsafeRow 11508 11514 10 1.4 731.6 1.0X -SQL Parquet Vectorized: DataPageV1 85 101 17 185.0 5.4 134.9X -SQL Parquet Vectorized: DataPageV2 84 96 12 187.7 5.3 136.9X -SQL Parquet MR: DataPageV1 2003 2039 51 7.9 127.3 5.7X -SQL Parquet MR: DataPageV2 1969 1969 1 8.0 125.2 5.8X -SQL ORC Vectorized 239 248 14 65.9 15.2 48.0X -SQL ORC MR 1782 1791 13 8.8 113.3 6.4X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 11778 11795 24 1.3 748.8 1.0X +SQL Json 11267 11356 127 1.4 716.3 1.0X +SQL Json with UnsafeRow 12181 12204 32 1.3 774.5 1.0X +SQL Parquet Vectorized: DataPageV1 84 99 14 187.8 5.3 140.6X +SQL Parquet Vectorized: DataPageV2 83 96 16 189.9 5.3 142.2X +SQL Parquet MR: DataPageV1 2002 2005 4 7.9 127.3 5.9X +SQL Parquet MR: DataPageV2 1943 1971 40 8.1 123.5 6.1X +SQL ORC Vectorized 220 243 21 71.6 14.0 53.6X +SQL ORC MR 1680 1688 11 9.4 106.8 7.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 145 153 12 108.7 9.2 1.0X -ParquetReader Vectorized: DataPageV2 143 149 7 110.0 9.1 1.0X -ParquetReader Vectorized -> Row: DataPageV1 136 143 8 115.2 8.7 1.1X -ParquetReader Vectorized -> Row: DataPageV2 135 141 6 116.3 8.6 1.1X +ParquetReader Vectorized: DataPageV1 135 152 37 116.7 8.6 1.0X +ParquetReader Vectorized: DataPageV2 132 138 6 119.0 8.4 1.0X +ParquetReader Vectorized -> Row: DataPageV1 129 135 5 121.8 8.2 1.0X +ParquetReader Vectorized -> Row: DataPageV2 145 147 2 108.3 9.2 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 11799 11829 43 1.3 750.1 1.0X -SQL Json 11125 11128 3 1.4 707.3 1.1X -SQL Json with UnsafeRow 11800 11815 22 1.3 750.2 1.0X -SQL Parquet Vectorized: DataPageV1 266 288 20 59.1 16.9 44.4X -SQL Parquet Vectorized: DataPageV2 263 286 14 59.7 16.8 44.8X -SQL Parquet MR: DataPageV1 2457 2472 22 6.4 156.2 4.8X -SQL Parquet MR: DataPageV2 2414 2423 13 6.5 153.5 4.9X -SQL ORC Vectorized 576 581 9 27.3 36.6 20.5X -SQL ORC MR 2192 2197 7 7.2 139.4 5.4X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 12383 12385 2 1.3 787.3 1.0X +SQL Json 11720 11726 8 1.3 745.1 1.1X +SQL Json with UnsafeRow 12528 12562 47 1.3 796.5 1.0X +SQL Parquet Vectorized: DataPageV1 279 301 19 56.3 17.8 44.3X +SQL Parquet Vectorized: DataPageV2 267 288 14 58.9 17.0 46.3X +SQL Parquet MR: DataPageV1 2421 2431 14 6.5 154.0 5.1X +SQL Parquet MR: DataPageV2 2354 2382 39 6.7 149.7 5.3X +SQL ORC Vectorized 585 598 16 26.9 37.2 21.2X +SQL ORC MR 2199 2199 0 7.2 139.8 5.6X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 341 346 6 46.2 21.6 1.0X -ParquetReader Vectorized: DataPageV2 351 358 5 44.8 22.3 1.0X -ParquetReader Vectorized -> Row: DataPageV1 324 331 6 48.5 20.6 1.0X -ParquetReader Vectorized -> Row: DataPageV2 323 326 4 48.7 20.5 1.1X +ParquetReader Vectorized: DataPageV1 334 342 8 47.1 21.2 1.0X +ParquetReader Vectorized: DataPageV2 334 338 5 47.1 21.2 1.0X +ParquetReader Vectorized -> Row: DataPageV1 333 336 5 47.2 21.2 1.0X +ParquetReader Vectorized -> Row: DataPageV2 335 338 2 46.9 21.3 1.0X ================================================================================================ SQL Single Numeric Column Scan in Struct ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2295 2333 53 6.9 145.9 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2261 2268 10 7.0 143.8 1.0X -SQL ORC Vectorized (Nested Column Enabled) 128 136 11 122.7 8.2 17.9X -SQL Parquet MR: DataPageV1 2378 2387 13 6.6 151.2 1.0X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2801 2804 5 5.6 178.1 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 103 119 19 152.5 6.6 22.3X -SQL Parquet MR: DataPageV2 2295 2312 25 6.9 145.9 1.0X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2669 2679 14 5.9 169.7 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 104 114 13 150.9 6.6 22.0X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2168 2196 39 7.3 137.9 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2168 2173 7 7.3 137.8 1.0X +SQL ORC Vectorized (Nested Column Enabled) 146 152 12 107.9 9.3 14.9X +SQL Parquet MR: DataPageV1 2344 2367 33 6.7 149.0 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2798 2805 9 5.6 177.9 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 107 126 20 147.6 6.8 20.4X +SQL Parquet MR: DataPageV2 2289 2318 41 6.9 145.5 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2687 2690 5 5.9 170.8 0.8X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 105 117 18 149.6 6.7 20.6X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2265 2302 52 6.9 144.0 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2221 2276 78 7.1 141.2 1.0X -SQL ORC Vectorized (Nested Column Enabled) 261 274 21 60.3 16.6 8.7X -SQL Parquet MR: DataPageV1 2435 2440 6 6.5 154.8 0.9X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2972 2982 15 5.3 188.9 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 110 121 14 143.6 7.0 20.7X -SQL Parquet MR: DataPageV2 2429 2437 12 6.5 154.4 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2882 2884 4 5.5 183.2 0.8X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 142 160 21 110.5 9.0 15.9X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2155 2166 15 7.3 137.0 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2139 2150 17 7.4 136.0 1.0X +SQL ORC Vectorized (Nested Column Enabled) 276 283 12 57.0 17.6 7.8X +SQL Parquet MR: DataPageV1 2477 2489 17 6.4 157.5 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2995 3013 26 5.3 190.4 0.7X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 113 148 19 139.1 7.2 19.1X +SQL Parquet MR: DataPageV2 2394 2401 10 6.6 152.2 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2943 2994 73 5.3 187.1 0.7X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 139 159 21 113.1 8.8 15.5X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2182 2205 32 7.2 138.7 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2192 2223 45 7.2 139.3 1.0X -SQL ORC Vectorized (Nested Column Enabled) 284 293 14 55.4 18.1 7.7X -SQL Parquet MR: DataPageV1 2445 2464 26 6.4 155.4 0.9X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3037 3038 2 5.2 193.1 0.7X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 113 116 2 138.7 7.2 19.2X -SQL Parquet MR: DataPageV2 2437 2448 17 6.5 154.9 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3036 3037 1 5.2 193.0 0.7X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 265 271 5 59.4 16.8 8.2X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2146 2196 72 7.3 136.4 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2099 2111 17 7.5 133.5 1.0X +SQL ORC Vectorized (Nested Column Enabled) 302 322 17 52.1 19.2 7.1X +SQL Parquet MR: DataPageV1 2420 2446 36 6.5 153.9 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2844 2849 6 5.5 180.8 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 106 118 13 148.3 6.7 20.2X +SQL Parquet MR: DataPageV2 2372 2383 14 6.6 150.8 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2871 2880 12 5.5 182.5 0.7X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 267 279 15 58.8 17.0 8.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2185 2193 12 7.2 138.9 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2202 2216 19 7.1 140.0 1.0X -SQL ORC Vectorized (Nested Column Enabled) 283 298 14 55.5 18.0 7.7X -SQL Parquet MR: DataPageV1 2872 2882 14 5.5 182.6 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3376 3392 23 4.7 214.7 0.6X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 320 329 8 49.2 20.3 6.8X -SQL Parquet MR: DataPageV2 2512 2518 9 6.3 159.7 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3009 3010 2 5.2 191.3 0.7X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 291 298 11 54.1 18.5 7.5X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2138 2162 35 7.4 135.9 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2118 2125 10 7.4 134.6 1.0X +SQL ORC Vectorized (Nested Column Enabled) 305 310 4 51.5 19.4 7.0X +SQL Parquet MR: DataPageV1 2786 2802 23 5.6 177.1 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3459 3460 1 4.5 219.9 0.6X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 323 328 5 48.7 20.5 6.6X +SQL Parquet MR: DataPageV2 2403 2419 22 6.5 152.8 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2896 2921 35 5.4 184.1 0.7X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 269 296 17 58.4 17.1 7.9X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2205 2207 4 7.1 140.2 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2238 2243 7 7.0 142.3 1.0X -SQL ORC Vectorized (Nested Column Enabled) 346 374 27 45.5 22.0 6.4X -SQL Parquet MR: DataPageV1 2463 2465 2 6.4 156.6 0.9X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3059 3060 2 5.1 194.5 0.7X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 91 109 17 172.3 5.8 24.2X -SQL Parquet MR: DataPageV2 2419 2446 37 6.5 153.8 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3078 3084 9 5.1 195.7 0.7X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 91 108 16 172.9 5.8 24.2X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2198 2213 21 7.2 139.7 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2184 2219 49 7.2 138.9 1.0X +SQL ORC Vectorized (Nested Column Enabled) 360 374 25 43.7 22.9 6.1X +SQL Parquet MR: DataPageV1 2434 2445 16 6.5 154.7 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3003 3008 7 5.2 191.0 0.7X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 106 115 6 148.1 6.8 20.7X +SQL Parquet MR: DataPageV2 2354 2357 4 6.7 149.7 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2847 2860 17 5.5 181.0 0.8X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 91 103 6 171.9 5.8 24.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2639 2643 6 6.0 167.8 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2676 2677 1 5.9 170.1 1.0X -SQL ORC Vectorized (Nested Column Enabled) 700 703 4 22.5 44.5 3.8X -SQL Parquet MR: DataPageV1 2949 2962 17 5.3 187.5 0.9X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3307 3315 12 4.8 210.2 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 310 324 17 50.7 19.7 8.5X -SQL Parquet MR: DataPageV2 2785 2810 36 5.6 177.0 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3269 3269 1 4.8 207.8 0.8X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 310 324 11 50.7 19.7 8.5X +SQL ORC MR 2598 2614 23 6.1 165.2 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2566 2583 24 6.1 163.1 1.0X +SQL ORC Vectorized (Nested Column Enabled) 713 720 11 22.0 45.4 3.6X +SQL Parquet MR: DataPageV1 2767 2850 119 5.7 175.9 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3267 3280 18 4.8 207.7 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 284 294 16 55.4 18.1 9.1X +SQL Parquet MR: DataPageV2 2713 2727 20 5.8 172.5 1.0X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3235 3237 2 4.9 205.7 0.8X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 281 314 23 55.9 17.9 9.2X ================================================================================================ SQL Nested Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Nested Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 12995 13153 131 0.1 12393.4 1.0X -SQL ORC Vectorized (Nested Column Disabled) 13011 13181 142 0.1 12408.4 1.0X -SQL ORC Vectorized (Nested Column Enabled) 7084 7096 11 0.1 6755.6 1.8X -SQL Parquet MR: DataPageV1 9427 9453 27 0.1 8990.6 1.4X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 9722 9802 39 0.1 9271.2 1.3X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 5931 6030 41 0.2 5656.2 2.2X -SQL Parquet MR: DataPageV2 9704 9744 59 0.1 9254.3 1.3X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 10391 10496 55 0.1 9909.7 1.3X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 5687 5729 23 0.2 5423.2 2.3X +SQL ORC MR 13204 13257 72 0.1 12592.7 1.0X +SQL ORC Vectorized (Nested Column Disabled) 13023 13064 43 0.1 12419.4 1.0X +SQL ORC Vectorized (Nested Column Enabled) 7170 7182 15 0.1 6837.7 1.8X +SQL Parquet MR: DataPageV1 9320 9408 68 0.1 8887.8 1.4X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 9632 9684 27 0.1 9186.0 1.4X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 5954 6007 35 0.2 5678.3 2.2X +SQL Parquet MR: DataPageV2 9823 9976 213 0.1 9368.0 1.3X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 10198 10460 203 0.1 9725.6 1.3X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 5708 5778 39 0.2 5443.9 2.3X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 10241 10290 70 1.0 976.6 1.0X -SQL Json 9827 9840 19 1.1 937.1 1.0X -SQL Parquet Vectorized: DataPageV1 1711 1736 35 6.1 163.2 6.0X -SQL Parquet Vectorized: DataPageV2 1912 1916 6 5.5 182.3 5.4X -SQL Parquet MR: DataPageV1 4027 4028 1 2.6 384.1 2.5X -SQL Parquet MR: DataPageV2 3967 3967 1 2.6 378.3 2.6X -SQL ORC Vectorized 1819 1845 37 5.8 173.5 5.6X -SQL ORC MR 3460 3468 11 3.0 330.0 3.0X +SQL CSV 10885 10952 95 1.0 1038.0 1.0X +SQL Json 10052 10073 30 1.0 958.6 1.1X +SQL Parquet Vectorized: DataPageV1 1759 1768 13 6.0 167.7 6.2X +SQL Parquet Vectorized: DataPageV2 1974 1974 1 5.3 188.2 5.5X +SQL Parquet MR: DataPageV1 3896 3902 9 2.7 371.6 2.8X +SQL Parquet MR: DataPageV2 3869 3895 36 2.7 369.0 2.8X +SQL ORC Vectorized 1823 1848 35 5.8 173.8 6.0X +SQL ORC MR 3507 3524 24 3.0 334.4 3.1X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 5689 5724 49 1.8 542.6 1.0X -SQL Json 6157 6173 22 1.7 587.1 0.9X -SQL Parquet Vectorized: DataPageV1 465 470 4 22.5 44.4 12.2X -SQL Parquet Vectorized: DataPageV2 459 460 1 22.8 43.8 12.4X -SQL Parquet MR: DataPageV1 1551 1558 10 6.8 147.9 3.7X -SQL Parquet MR: DataPageV2 1501 1506 7 7.0 143.2 3.8X -SQL ORC Vectorized 366 369 3 28.7 34.9 15.5X -SQL ORC MR 1703 1740 51 6.2 162.4 3.3X +SQL CSV 5972 5973 2 1.8 569.6 1.0X +SQL Json 6515 6538 32 1.6 621.4 0.9X +SQL Parquet Vectorized: DataPageV1 481 499 18 21.8 45.9 12.4X +SQL Parquet Vectorized: DataPageV2 482 495 16 21.8 46.0 12.4X +SQL Parquet MR: DataPageV1 1603 1624 29 6.5 152.9 3.7X +SQL Parquet MR: DataPageV2 1543 1554 16 6.8 147.2 3.9X +SQL ORC Vectorized 378 383 5 27.7 36.1 15.8X +SQL ORC MR 1747 1750 3 6.0 166.6 3.4X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -Data column - CSV 10920 10985 92 1.4 694.2 1.0X -Data column - Json 9064 9065 2 1.7 576.3 1.2X -Data column - Parquet Vectorized: DataPageV1 117 124 6 134.6 7.4 93.5X -Data column - Parquet Vectorized: DataPageV2 223 239 14 70.5 14.2 49.0X -Data column - Parquet MR: DataPageV1 2287 2295 12 6.9 145.4 4.8X -Data column - Parquet MR: DataPageV2 2302 2305 4 6.8 146.4 4.7X -Data column - ORC Vectorized 179 191 20 87.9 11.4 61.0X -Data column - ORC MR 2135 2161 36 7.4 135.8 5.1X -Partition column - CSV 3806 3806 0 4.1 242.0 2.9X -Partition column - Json 8340 8352 16 1.9 530.3 1.3X -Partition column - Parquet Vectorized: DataPageV1 30 34 6 529.7 1.9 367.7X -Partition column - Parquet Vectorized: DataPageV2 29 34 6 549.7 1.8 381.6X -Partition column - Parquet MR: DataPageV1 1425 1435 14 11.0 90.6 7.7X -Partition column - Parquet MR: DataPageV2 1414 1428 20 11.1 89.9 7.7X -Partition column - ORC Vectorized 30 33 5 525.5 1.9 364.8X -Partition column - ORC MR 1284 1293 13 12.3 81.6 8.5X -Both columns - CSV 11211 11232 30 1.4 712.8 1.0X -Both columns - Json 9167 9184 24 1.7 582.8 1.2X -Both columns - Parquet Vectorized: DataPageV1 153 167 13 102.5 9.8 71.2X -Both columns - Parquet Vectorized: DataPageV2 267 298 31 58.8 17.0 40.8X -Both columns - Parquet MR: DataPageV1 2567 2611 62 6.1 163.2 4.3X -Both columns - Parquet MR: DataPageV2 2647 2659 17 5.9 168.3 4.1X -Both columns - ORC Vectorized 178 200 26 88.3 11.3 61.3X -Both columns - ORC MR 2119 2131 17 7.4 134.7 5.2X +Data column - CSV 11700 11730 43 1.3 743.9 1.0X +Data column - Json 9276 9304 40 1.7 589.8 1.3X +Data column - Parquet Vectorized: DataPageV1 102 131 25 154.9 6.5 115.2X +Data column - Parquet Vectorized: DataPageV2 220 252 37 71.6 14.0 53.3X +Data column - Parquet MR: DataPageV1 2276 2345 97 6.9 144.7 5.1X +Data column - Parquet MR: DataPageV2 2205 2216 15 7.1 140.2 5.3X +Data column - ORC Vectorized 178 189 13 88.4 11.3 65.8X +Data column - ORC MR 1942 1952 14 8.1 123.5 6.0X +Partition column - CSV 3761 3778 24 4.2 239.1 3.1X +Partition column - Json 8482 8581 141 1.9 539.3 1.4X +Partition column - Parquet Vectorized: DataPageV1 30 37 8 528.0 1.9 392.7X +Partition column - Parquet Vectorized: DataPageV2 28 35 7 561.2 1.8 417.5X +Partition column - Parquet MR: DataPageV1 1184 1185 2 13.3 75.3 9.9X +Partition column - Parquet MR: DataPageV2 1179 1228 69 13.3 74.9 9.9X +Partition column - ORC Vectorized 30 33 6 531.9 1.9 395.6X +Partition column - ORC MR 1209 1211 3 13.0 76.9 9.7X +Both columns - CSV 11640 11652 17 1.4 740.0 1.0X +Both columns - Json 9733 9757 34 1.6 618.8 1.2X +Both columns - Parquet Vectorized: DataPageV1 141 162 15 111.2 9.0 82.7X +Both columns - Parquet Vectorized: DataPageV2 269 288 24 58.4 17.1 43.4X +Both columns - Parquet MR: DataPageV1 2487 2500 18 6.3 158.1 4.7X +Both columns - Parquet MR: DataPageV2 2441 2489 68 6.4 155.2 4.8X +Both columns - ORC Vectorized 203 214 16 77.6 12.9 57.8X +Both columns - ORC MR 2001 2006 7 7.9 127.2 5.8X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 7385 7393 11 1.4 704.3 1.0X -SQL Json 8624 8638 20 1.2 822.4 0.9X -SQL Parquet Vectorized: DataPageV1 1123 1130 10 9.3 107.1 6.6X -SQL Parquet Vectorized: DataPageV2 1398 1403 6 7.5 133.3 5.3X -SQL Parquet MR: DataPageV1 3770 3795 35 2.8 359.6 2.0X -SQL Parquet MR: DataPageV2 3738 3769 43 2.8 356.5 2.0X -ParquetReader Vectorized: DataPageV1 753 760 7 13.9 71.8 9.8X -ParquetReader Vectorized: DataPageV2 1084 1095 16 9.7 103.3 6.8X -SQL ORC Vectorized 818 836 23 12.8 78.1 9.0X -SQL ORC MR 2885 2904 27 3.6 275.1 2.6X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 7656 7673 24 1.4 730.1 1.0X +SQL Json 8974 8995 30 1.2 855.8 0.9X +SQL Parquet Vectorized: DataPageV1 1114 1143 41 9.4 106.2 6.9X +SQL Parquet Vectorized: DataPageV2 1477 1501 34 7.1 140.8 5.2X +SQL Parquet MR: DataPageV1 3613 3614 1 2.9 344.6 2.1X +SQL Parquet MR: DataPageV2 3877 3877 0 2.7 369.7 2.0X +ParquetReader Vectorized: DataPageV1 765 773 12 13.7 72.9 10.0X +ParquetReader Vectorized: DataPageV2 1109 1130 30 9.5 105.8 6.9X +SQL ORC Vectorized 841 851 18 12.5 80.2 9.1X +SQL ORC MR 2849 2862 19 3.7 271.7 2.7X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 5899 5900 2 1.8 562.6 1.0X -SQL Json 7189 7199 14 1.5 685.6 0.8X -SQL Parquet Vectorized: DataPageV1 737 756 22 14.2 70.3 8.0X -SQL Parquet Vectorized: DataPageV2 1004 1035 45 10.4 95.7 5.9X -SQL Parquet MR: DataPageV1 2744 2752 12 3.8 261.6 2.2X -SQL Parquet MR: DataPageV2 2917 2923 8 3.6 278.2 2.0X -ParquetReader Vectorized: DataPageV1 719 734 19 14.6 68.6 8.2X -ParquetReader Vectorized: DataPageV2 950 957 12 11.0 90.6 6.2X -SQL ORC Vectorized 986 1002 22 10.6 94.1 6.0X -SQL ORC MR 2840 2866 36 3.7 270.9 2.1X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 5670 5681 16 1.8 540.7 1.0X +SQL Json 7363 7363 1 1.4 702.2 0.8X +SQL Parquet Vectorized: DataPageV1 747 770 35 14.0 71.2 7.6X +SQL Parquet Vectorized: DataPageV2 981 1019 53 10.7 93.6 5.8X +SQL Parquet MR: DataPageV1 2684 2693 13 3.9 256.0 2.1X +SQL Parquet MR: DataPageV2 2820 2830 14 3.7 269.0 2.0X +ParquetReader Vectorized: DataPageV1 697 706 11 15.1 66.4 8.1X +ParquetReader Vectorized: DataPageV2 920 935 20 11.4 87.8 6.2X +SQL ORC Vectorized 976 1000 35 10.7 93.1 5.8X +SQL ORC MR 2670 2690 28 3.9 254.6 2.1X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 3951 3956 7 2.7 376.8 1.0X -SQL Json 4888 4888 1 2.1 466.1 0.8X -SQL Parquet Vectorized: DataPageV1 173 193 11 60.5 16.5 22.8X -SQL Parquet Vectorized: DataPageV2 194 199 3 54.0 18.5 20.3X -SQL Parquet MR: DataPageV1 1666 1672 8 6.3 158.9 2.4X -SQL Parquet MR: DataPageV2 1626 1633 10 6.5 155.0 2.4X -ParquetReader Vectorized: DataPageV1 174 178 5 60.2 16.6 22.7X -ParquetReader Vectorized: DataPageV2 201 203 2 52.1 19.2 19.6X -SQL ORC Vectorized 328 331 4 32.0 31.2 12.1X -SQL ORC MR 1633 1636 3 6.4 155.8 2.4X +SQL CSV 4208 4236 40 2.5 401.3 1.0X +SQL Json 5288 5295 11 2.0 504.3 0.8X +SQL Parquet Vectorized: DataPageV1 165 174 6 63.7 15.7 25.6X +SQL Parquet Vectorized: DataPageV2 194 198 5 54.1 18.5 21.7X +SQL Parquet MR: DataPageV1 1693 1697 5 6.2 161.5 2.5X +SQL Parquet MR: DataPageV2 1668 1686 25 6.3 159.0 2.5X +ParquetReader Vectorized: DataPageV1 155 157 2 67.6 14.8 27.1X +ParquetReader Vectorized: DataPageV2 184 186 2 56.9 17.6 22.8X +SQL ORC Vectorized 327 340 17 32.1 31.2 12.9X +SQL ORC MR 1521 1538 23 6.9 145.1 2.8X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 1259 1261 2 0.8 1201.0 1.0X -SQL Json 1688 1695 9 0.6 1610.1 0.7X -SQL Parquet Vectorized: DataPageV1 24 29 6 43.9 22.8 52.7X -SQL Parquet Vectorized: DataPageV2 32 36 6 32.8 30.5 39.4X -SQL Parquet MR: DataPageV1 169 176 6 6.2 161.2 7.5X -SQL Parquet MR: DataPageV2 157 165 7 6.7 149.6 8.0X -SQL ORC Vectorized 29 35 6 36.2 27.6 43.5X -SQL ORC MR 132 140 6 7.9 126.2 9.5X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 1322 1325 4 0.8 1261.2 1.0X +SQL Json 1758 1767 13 0.6 1676.5 0.8X +SQL Parquet Vectorized: DataPageV1 24 28 6 44.0 22.7 55.5X +SQL Parquet Vectorized: DataPageV2 33 36 6 32.2 31.1 40.6X +SQL Parquet MR: DataPageV1 154 160 5 6.8 147.1 8.6X +SQL Parquet MR: DataPageV2 163 166 2 6.4 155.6 8.1X +SQL ORC Vectorized 28 33 6 37.8 26.5 47.6X +SQL ORC MR 134 141 5 7.8 127.7 9.9X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 50 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 2656 2659 4 0.4 2533.4 1.0X -SQL Json 6186 6199 19 0.2 5899.5 0.4X -SQL Parquet Vectorized: DataPageV1 27 33 7 39.1 25.6 99.1X -SQL Parquet Vectorized: DataPageV2 35 40 6 30.3 33.0 76.9X -SQL Parquet MR: DataPageV1 170 176 6 6.2 162.4 15.6X -SQL Parquet MR: DataPageV2 163 173 10 6.5 155.0 16.3X -SQL ORC Vectorized 33 38 6 32.3 31.0 81.7X -SQL ORC MR 137 145 8 7.7 130.4 19.4X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 2634 2639 7 0.4 2511.9 1.0X +SQL Json 5624 5655 44 0.2 5363.7 0.5X +SQL Parquet Vectorized: DataPageV1 27 33 7 39.4 25.4 99.0X +SQL Parquet Vectorized: DataPageV2 34 41 7 30.5 32.8 76.6X +SQL Parquet MR: DataPageV1 158 167 6 6.6 150.9 16.6X +SQL Parquet MR: DataPageV2 153 159 6 6.9 145.9 17.2X +SQL ORC Vectorized 31 35 6 34.0 29.4 85.3X +SQL ORC MR 131 137 5 8.0 124.6 20.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 4422 4439 25 0.2 4217.1 1.0X -SQL Json 11222 11248 37 0.1 10702.2 0.4X -SQL Parquet Vectorized: DataPageV1 35 41 6 30.2 33.1 127.5X -SQL Parquet Vectorized: DataPageV2 42 46 6 25.0 40.0 105.5X -SQL Parquet MR: DataPageV1 182 191 8 5.8 173.8 24.3X -SQL Parquet MR: DataPageV2 182 185 2 5.8 173.6 24.3X -SQL ORC Vectorized 39 44 5 27.0 37.0 114.0X -SQL ORC MR 148 159 6 7.1 141.2 29.9X +SQL CSV 4252 4309 81 0.2 4054.8 1.0X +SQL Json 10496 10648 215 0.1 10009.3 0.4X +SQL Parquet Vectorized: DataPageV1 35 48 10 30.2 33.1 122.4X +SQL Parquet Vectorized: DataPageV2 42 46 6 25.2 39.6 102.4X +SQL Parquet MR: DataPageV1 167 177 7 6.3 159.5 25.4X +SQL Parquet MR: DataPageV2 177 182 6 5.9 168.4 24.1X +SQL ORC Vectorized 38 44 7 27.9 35.9 113.1X +SQL ORC MR 138 146 8 7.6 131.3 30.9X diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt index d2180ecb771d5..948694d17066c 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt @@ -2,437 +2,437 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 10580 10590 15 1.5 672.6 1.0X -SQL Json 8244 8399 219 1.9 524.2 1.3X -SQL Json with UnsafeRow 9338 9354 22 1.7 593.7 1.1X -SQL Parquet Vectorized: DataPageV1 103 117 7 152.2 6.6 102.4X -SQL Parquet Vectorized: DataPageV2 105 116 8 149.7 6.7 100.7X -SQL Parquet MR: DataPageV1 1871 1932 87 8.4 118.9 5.7X -SQL Parquet MR: DataPageV2 1762 1767 8 8.9 112.0 6.0X -SQL ORC Vectorized 142 151 6 110.8 9.0 74.5X -SQL ORC MR 1697 1702 7 9.3 107.9 6.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 11082 11121 55 1.4 704.6 1.0X +SQL Json 8235 8413 252 1.9 523.6 1.3X +SQL Json with UnsafeRow 9534 9547 17 1.6 606.2 1.2X +SQL Parquet Vectorized: DataPageV1 99 114 9 158.3 6.3 111.6X +SQL Parquet Vectorized: DataPageV2 99 106 5 158.1 6.3 111.4X +SQL Parquet MR: DataPageV1 1781 1787 9 8.8 113.2 6.2X +SQL Parquet MR: DataPageV2 1685 1760 106 9.3 107.1 6.6X +SQL ORC Vectorized 139 145 4 112.9 8.9 79.5X +SQL ORC MR 1447 1449 3 10.9 92.0 7.7X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 94 96 3 167.8 6.0 1.0X -ParquetReader Vectorized: DataPageV2 101 103 1 155.0 6.4 0.9X -ParquetReader Vectorized -> Row: DataPageV1 74 76 2 211.8 4.7 1.3X -ParquetReader Vectorized -> Row: DataPageV2 83 84 2 190.4 5.3 1.1X +ParquetReader Vectorized: DataPageV1 88 90 1 178.8 5.6 1.0X +ParquetReader Vectorized: DataPageV2 95 96 1 165.3 6.0 0.9X +ParquetReader Vectorized -> Row: DataPageV1 73 74 1 214.9 4.7 1.2X +ParquetReader Vectorized -> Row: DataPageV2 81 82 1 193.4 5.2 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 11731 11783 74 1.3 745.8 1.0X -SQL Json 9315 9364 69 1.7 592.2 1.3X -SQL Json with UnsafeRow 10241 10246 7 1.5 651.1 1.1X -SQL Parquet Vectorized: DataPageV1 113 120 7 138.9 7.2 103.6X -SQL Parquet Vectorized: DataPageV2 111 118 6 142.1 7.0 106.0X -SQL Parquet MR: DataPageV1 1992 2010 26 7.9 126.6 5.9X -SQL Parquet MR: DataPageV2 1918 1939 29 8.2 122.0 6.1X -SQL ORC Vectorized 112 120 6 139.9 7.1 104.4X -SQL ORC MR 1643 1647 5 9.6 104.5 7.1X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 10768 10815 67 1.5 684.6 1.0X +SQL Json 9495 9518 33 1.7 603.7 1.1X +SQL Json with UnsafeRow 10257 10262 7 1.5 652.1 1.0X +SQL Parquet Vectorized: DataPageV1 91 100 10 173.0 5.8 118.4X +SQL Parquet Vectorized: DataPageV2 90 99 10 175.0 5.7 119.8X +SQL Parquet MR: DataPageV1 1839 1839 0 8.6 116.9 5.9X +SQL Parquet MR: DataPageV2 1807 1816 13 8.7 114.9 6.0X +SQL ORC Vectorized 114 118 3 138.1 7.2 94.5X +SQL ORC MR 1485 1485 0 10.6 94.4 7.3X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 83 85 2 190.1 5.3 1.0X -ParquetReader Vectorized: DataPageV2 83 84 2 189.8 5.3 1.0X -ParquetReader Vectorized -> Row: DataPageV1 62 63 1 254.3 3.9 1.3X -ParquetReader Vectorized -> Row: DataPageV2 62 64 2 253.5 3.9 1.3X +ParquetReader Vectorized: DataPageV1 68 69 1 232.8 4.3 1.0X +ParquetReader Vectorized: DataPageV2 68 70 2 232.1 4.3 1.0X +ParquetReader Vectorized -> Row: DataPageV1 46 48 2 338.4 3.0 1.5X +ParquetReader Vectorized -> Row: DataPageV2 46 48 2 340.2 2.9 1.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 12442 12467 35 1.3 791.1 1.0X -SQL Json 9536 9578 58 1.6 606.3 1.3X -SQL Json with UnsafeRow 10484 10484 1 1.5 666.5 1.2X -SQL Parquet Vectorized: DataPageV1 110 115 3 142.4 7.0 112.6X -SQL Parquet Vectorized: DataPageV2 139 144 5 112.9 8.9 89.3X -SQL Parquet MR: DataPageV1 2082 2122 57 7.6 132.4 6.0X -SQL Parquet MR: DataPageV2 2050 2071 30 7.7 130.3 6.1X -SQL ORC Vectorized 143 148 4 110.2 9.1 87.2X -SQL ORC MR 1722 1723 1 9.1 109.5 7.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 11107 11122 21 1.4 706.2 1.0X +SQL Json 9895 9916 30 1.6 629.1 1.1X +SQL Json with UnsafeRow 10606 10615 13 1.5 674.3 1.0X +SQL Parquet Vectorized: DataPageV1 100 107 6 157.6 6.3 111.3X +SQL Parquet Vectorized: DataPageV2 129 135 6 122.2 8.2 86.3X +SQL Parquet MR: DataPageV1 1978 1980 3 8.0 125.8 5.6X +SQL Parquet MR: DataPageV2 1877 1894 24 8.4 119.3 5.9X +SQL ORC Vectorized 138 143 4 113.9 8.8 80.5X +SQL ORC MR 1570 1572 2 10.0 99.8 7.1X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 140 144 3 112.3 8.9 1.0X -ParquetReader Vectorized: DataPageV2 168 170 2 93.8 10.7 0.8X -ParquetReader Vectorized -> Row: DataPageV1 138 140 3 114.1 8.8 1.0X -ParquetReader Vectorized -> Row: DataPageV2 166 167 2 95.0 10.5 0.8X +ParquetReader Vectorized: DataPageV1 144 145 2 109.2 9.2 1.0X +ParquetReader Vectorized: DataPageV2 172 174 2 91.4 10.9 0.8X +ParquetReader Vectorized -> Row: DataPageV1 136 138 2 115.4 8.7 1.1X +ParquetReader Vectorized -> Row: DataPageV2 168 170 3 93.7 10.7 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 13427 13451 33 1.2 853.7 1.0X -SQL Json 10000 10014 20 1.6 635.8 1.3X -SQL Json with UnsafeRow 10816 10829 18 1.5 687.7 1.2X -SQL Parquet Vectorized: DataPageV1 121 126 3 130.5 7.7 111.4X -SQL Parquet Vectorized: DataPageV2 197 203 12 79.7 12.5 68.0X -SQL Parquet MR: DataPageV1 2149 2246 137 7.3 136.7 6.2X -SQL Parquet MR: DataPageV2 2058 2072 19 7.6 130.9 6.5X -SQL ORC Vectorized 159 165 6 98.8 10.1 84.3X -SQL ORC MR 1868 1869 1 8.4 118.8 7.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 12198 12209 16 1.3 775.5 1.0X +SQL Json 10381 10390 13 1.5 660.0 1.2X +SQL Json with UnsafeRow 11101 11149 69 1.4 705.8 1.1X +SQL Parquet Vectorized: DataPageV1 106 109 3 147.9 6.8 114.7X +SQL Parquet Vectorized: DataPageV2 181 186 8 87.1 11.5 67.5X +SQL Parquet MR: DataPageV1 2004 2004 0 7.9 127.4 6.1X +SQL Parquet MR: DataPageV2 1962 1976 20 8.0 124.7 6.2X +SQL ORC Vectorized 146 149 3 107.9 9.3 83.7X +SQL ORC MR 1583 1585 3 9.9 100.7 7.7X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 142 144 1 110.5 9.0 1.0X -ParquetReader Vectorized: DataPageV2 215 219 7 73.0 13.7 0.7X -ParquetReader Vectorized -> Row: DataPageV1 141 142 1 111.9 8.9 1.0X -ParquetReader Vectorized -> Row: DataPageV2 212 213 1 74.2 13.5 0.7X +ParquetReader Vectorized: DataPageV1 146 147 1 107.8 9.3 1.0X +ParquetReader Vectorized: DataPageV2 216 217 1 73.0 13.7 0.7X +ParquetReader Vectorized -> Row: DataPageV1 139 142 6 113.1 8.8 1.0X +ParquetReader Vectorized -> Row: DataPageV2 211 214 4 74.4 13.4 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 13182 13188 8 1.2 838.1 1.0X -SQL Json 10134 10141 10 1.6 644.3 1.3X -SQL Json with UnsafeRow 10915 10920 7 1.4 693.9 1.2X -SQL Parquet Vectorized: DataPageV1 281 285 4 55.9 17.9 46.8X -SQL Parquet Vectorized: DataPageV2 176 181 4 89.2 11.2 74.8X -SQL Parquet MR: DataPageV1 2659 2694 49 5.9 169.1 5.0X -SQL Parquet MR: DataPageV2 2191 2194 5 7.2 139.3 6.0X -SQL ORC Vectorized 144 151 4 109.2 9.2 91.5X -SQL ORC MR 1814 1887 103 8.7 115.4 7.3X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 12198 12205 10 1.3 775.5 1.0X +SQL Json 10391 10400 13 1.5 660.6 1.2X +SQL Json with UnsafeRow 11102 11110 12 1.4 705.8 1.1X +SQL Parquet Vectorized: DataPageV1 280 284 3 56.3 17.8 43.6X +SQL Parquet Vectorized: DataPageV2 175 179 4 90.0 11.1 69.8X +SQL Parquet MR: DataPageV1 2379 2432 75 6.6 151.2 5.1X +SQL Parquet MR: DataPageV2 1910 1917 11 8.2 121.4 6.4X +SQL ORC Vectorized 127 132 5 124.2 8.1 96.3X +SQL ORC MR 1701 1717 23 9.2 108.1 7.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 316 317 1 49.8 20.1 1.0X -ParquetReader Vectorized: DataPageV2 214 218 7 73.5 13.6 1.5X -ParquetReader Vectorized -> Row: DataPageV1 338 343 8 46.6 21.5 0.9X -ParquetReader Vectorized -> Row: DataPageV2 234 236 2 67.1 14.9 1.3X +ParquetReader Vectorized: DataPageV1 335 337 2 47.0 21.3 1.0X +ParquetReader Vectorized: DataPageV2 217 231 9 72.4 13.8 1.5X +ParquetReader Vectorized -> Row: DataPageV1 347 353 6 45.4 22.0 1.0X +ParquetReader Vectorized -> Row: DataPageV2 243 248 4 64.6 15.5 1.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 13724 13734 14 1.1 872.5 1.0X -SQL Json 11883 11914 45 1.3 755.5 1.2X -SQL Json with UnsafeRow 12737 12740 4 1.2 809.8 1.1X -SQL Parquet Vectorized: DataPageV1 86 97 10 183.4 5.5 160.0X -SQL Parquet Vectorized: DataPageV2 94 107 8 168.1 5.9 146.7X -SQL Parquet MR: DataPageV1 2291 2295 6 6.9 145.7 6.0X -SQL Parquet MR: DataPageV2 2156 2157 2 7.3 137.1 6.4X -SQL ORC Vectorized 258 270 11 60.9 16.4 53.1X -SQL ORC MR 1903 1908 7 8.3 121.0 7.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 12624 12635 16 1.2 802.6 1.0X +SQL Json 11911 11924 19 1.3 757.3 1.1X +SQL Json with UnsafeRow 12643 12657 21 1.2 803.8 1.0X +SQL Parquet Vectorized: DataPageV1 90 98 8 175.2 5.7 140.6X +SQL Parquet Vectorized: DataPageV2 90 103 10 174.5 5.7 140.0X +SQL Parquet MR: DataPageV1 2018 2022 5 7.8 128.3 6.3X +SQL Parquet MR: DataPageV2 1947 1965 25 8.1 123.8 6.5X +SQL ORC Vectorized 251 268 16 62.6 16.0 50.2X +SQL ORC MR 1729 1732 4 9.1 109.9 7.3X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 141 141 0 111.9 8.9 1.0X -ParquetReader Vectorized: DataPageV2 152 155 4 103.2 9.7 0.9X -ParquetReader Vectorized -> Row: DataPageV1 152 157 4 103.2 9.7 0.9X -ParquetReader Vectorized -> Row: DataPageV2 152 156 6 103.6 9.7 0.9X +ParquetReader Vectorized: DataPageV1 139 142 2 113.0 8.8 1.0X +ParquetReader Vectorized: DataPageV2 156 158 4 101.0 9.9 0.9X +ParquetReader Vectorized -> Row: DataPageV1 151 153 3 104.3 9.6 0.9X +ParquetReader Vectorized -> Row: DataPageV2 150 153 4 104.5 9.6 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 13926 13932 8 1.1 885.4 1.0X -SQL Json 12135 12148 19 1.3 771.5 1.1X -SQL Json with UnsafeRow 12983 13003 29 1.2 825.4 1.1X -SQL Parquet Vectorized: DataPageV1 292 298 7 53.9 18.5 47.7X -SQL Parquet Vectorized: DataPageV2 292 297 4 53.9 18.5 47.7X -SQL Parquet MR: DataPageV1 2769 2775 9 5.7 176.1 5.0X -SQL Parquet MR: DataPageV2 2619 2623 6 6.0 166.5 5.3X -SQL ORC Vectorized 632 649 18 24.9 40.2 22.0X -SQL ORC MR 2386 2405 27 6.6 151.7 5.8X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 12881 13006 176 1.2 819.0 1.0X +SQL Json 12083 12109 37 1.3 768.2 1.1X +SQL Json with UnsafeRow 12697 12731 49 1.2 807.2 1.0X +SQL Parquet Vectorized: DataPageV1 281 286 9 56.0 17.9 45.8X +SQL Parquet Vectorized: DataPageV2 280 287 5 56.2 17.8 46.0X +SQL Parquet MR: DataPageV1 2442 2490 67 6.4 155.3 5.3X +SQL Parquet MR: DataPageV2 2356 2370 19 6.7 149.8 5.5X +SQL ORC Vectorized 639 643 3 24.6 40.7 20.1X +SQL ORC MR 2155 2161 7 7.3 137.0 6.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet Reader Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -ParquetReader Vectorized: DataPageV1 357 360 2 44.0 22.7 1.0X -ParquetReader Vectorized: DataPageV2 356 359 2 44.1 22.7 1.0X -ParquetReader Vectorized -> Row: DataPageV1 365 371 7 43.1 23.2 1.0X -ParquetReader Vectorized -> Row: DataPageV2 367 370 4 42.9 23.3 1.0X +ParquetReader Vectorized: DataPageV1 334 335 2 47.1 21.2 1.0X +ParquetReader Vectorized: DataPageV2 335 339 3 46.9 21.3 1.0X +ParquetReader Vectorized -> Row: DataPageV1 346 353 7 45.4 22.0 1.0X +ParquetReader Vectorized -> Row: DataPageV2 346 351 4 45.4 22.0 1.0X ================================================================================================ SQL Single Numeric Column Scan in Struct ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2238 2269 44 7.0 142.3 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2290 2319 42 6.9 145.6 1.0X -SQL ORC Vectorized (Nested Column Enabled) 129 144 34 121.9 8.2 17.3X -SQL Parquet MR: DataPageV1 2487 2501 20 6.3 158.1 0.9X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3250 3274 35 4.8 206.6 0.7X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 112 118 7 140.2 7.1 19.9X -SQL Parquet MR: DataPageV2 2368 2393 35 6.6 150.5 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3091 3118 37 5.1 196.5 0.7X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 113 118 7 139.2 7.2 19.8X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2062 2069 10 7.6 131.1 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2068 2085 24 7.6 131.5 1.0X +SQL ORC Vectorized (Nested Column Enabled) 119 132 28 132.3 7.6 17.3X +SQL Parquet MR: DataPageV1 2402 2421 27 6.5 152.7 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2910 2913 5 5.4 185.0 0.7X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 102 108 5 153.9 6.5 20.2X +SQL Parquet MR: DataPageV2 2340 2361 29 6.7 148.8 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2765 2774 12 5.7 175.8 0.7X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 100 104 4 157.4 6.4 20.6X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2156 2195 55 7.3 137.0 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2174 2191 24 7.2 138.2 1.0X -SQL ORC Vectorized (Nested Column Enabled) 259 264 4 60.6 16.5 8.3X -SQL Parquet MR: DataPageV1 2617 2631 20 6.0 166.4 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3209 3215 8 4.9 204.0 0.7X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 171 182 14 92.0 10.9 12.6X -SQL Parquet MR: DataPageV2 2463 2498 50 6.4 156.6 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3018 3023 6 5.2 191.9 0.7X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 287 303 14 54.8 18.2 7.5X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2092 2099 10 7.5 133.0 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2093 2110 24 7.5 133.1 1.0X +SQL ORC Vectorized (Nested Column Enabled) 280 286 7 56.2 17.8 7.5X +SQL Parquet MR: DataPageV1 2341 2354 18 6.7 148.9 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2922 2926 6 5.4 185.8 0.7X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 144 150 4 109.3 9.1 14.5X +SQL Parquet MR: DataPageV2 2276 2287 16 6.9 144.7 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2828 2831 4 5.6 179.8 0.7X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 266 283 18 59.2 16.9 7.9X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2301 2367 94 6.8 146.3 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2417 2421 6 6.5 153.7 1.0X -SQL ORC Vectorized (Nested Column Enabled) 282 288 4 55.7 17.9 8.2X -SQL Parquet MR: DataPageV1 2681 2694 18 5.9 170.5 0.9X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3187 3213 36 4.9 202.6 0.7X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 152 159 6 103.8 9.6 15.2X -SQL Parquet MR: DataPageV2 2636 2650 20 6.0 167.6 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3077 3089 17 5.1 195.6 0.7X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 287 304 31 54.7 18.3 8.0X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2149 2172 33 7.3 136.6 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2074 2107 47 7.6 131.8 1.0X +SQL ORC Vectorized (Nested Column Enabled) 274 282 8 57.4 17.4 7.8X +SQL Parquet MR: DataPageV1 2363 2370 10 6.7 150.2 0.9X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2863 2898 49 5.5 182.0 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 120 134 7 130.8 7.6 17.9X +SQL Parquet MR: DataPageV2 2301 2318 23 6.8 146.3 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2729 2763 48 5.8 173.5 0.8X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 267 274 6 58.9 17.0 8.1X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2419 2419 1 6.5 153.8 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2342 2392 71 6.7 148.9 1.0X -SQL ORC Vectorized (Nested Column Enabled) 285 291 4 55.2 18.1 8.5X -SQL Parquet MR: DataPageV1 2915 2931 23 5.4 185.4 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3405 3418 19 4.6 216.5 0.7X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 321 346 15 49.0 20.4 7.5X -SQL Parquet MR: DataPageV2 2554 2570 24 6.2 162.4 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2943 2954 15 5.3 187.1 0.8X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 210 215 3 74.9 13.4 11.5X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2115 2121 8 7.4 134.5 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2103 2130 37 7.5 133.7 1.0X +SQL ORC Vectorized (Nested Column Enabled) 270 305 76 58.2 17.2 7.8X +SQL Parquet MR: DataPageV1 2791 2796 8 5.6 177.4 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3190 3211 29 4.9 202.8 0.7X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 307 315 9 51.2 19.5 6.9X +SQL Parquet MR: DataPageV2 2447 2458 15 6.4 155.6 0.9X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2786 2804 26 5.6 177.1 0.8X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 201 208 10 78.4 12.8 10.5X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2457 2629 243 6.4 156.2 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2469 2481 17 6.4 157.0 1.0X -SQL ORC Vectorized (Nested Column Enabled) 354 368 10 44.4 22.5 6.9X -SQL Parquet MR: DataPageV1 2592 2592 1 6.1 164.8 0.9X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3017 3022 7 5.2 191.8 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 104 123 14 151.0 6.6 23.6X -SQL Parquet MR: DataPageV2 2511 2554 61 6.3 159.6 1.0X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2932 2964 44 5.4 186.4 0.8X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 102 106 7 154.9 6.5 24.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL ORC MR 2200 2240 56 7.1 139.9 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2211 2231 28 7.1 140.5 1.0X +SQL ORC Vectorized (Nested Column Enabled) 356 376 18 44.2 22.6 6.2X +SQL Parquet MR: DataPageV1 2249 2280 43 7.0 143.0 1.0X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2676 2677 2 5.9 170.1 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 93 121 14 169.6 5.9 23.7X +SQL Parquet MR: DataPageV2 2244 2258 19 7.0 142.7 1.0X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2605 2631 37 6.0 165.6 0.8X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 104 121 15 151.7 6.6 21.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 2921 2933 17 5.4 185.7 1.0X -SQL ORC Vectorized (Nested Column Disabled) 2929 2950 30 5.4 186.2 1.0X -SQL ORC Vectorized (Nested Column Enabled) 790 793 4 19.9 50.2 3.7X -SQL Parquet MR: DataPageV1 2944 2952 12 5.3 187.2 1.0X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3441 3485 62 4.6 218.8 0.8X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 300 313 12 52.5 19.0 9.8X -SQL Parquet MR: DataPageV2 2922 2972 71 5.4 185.8 1.0X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3389 3393 7 4.6 215.4 0.9X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 299 306 7 52.6 19.0 9.8X +SQL ORC MR 2579 2580 1 6.1 164.0 1.0X +SQL ORC Vectorized (Nested Column Disabled) 2595 2624 40 6.1 165.0 1.0X +SQL ORC Vectorized (Nested Column Enabled) 748 767 23 21.0 47.5 3.4X +SQL Parquet MR: DataPageV1 2668 2686 26 5.9 169.6 1.0X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3358 3363 7 4.7 213.5 0.8X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 291 302 12 54.0 18.5 8.9X +SQL Parquet MR: DataPageV2 2652 2655 4 5.9 168.6 1.0X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3264 3284 28 4.8 207.5 0.8X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 292 299 7 53.9 18.5 8.8X ================================================================================================ SQL Nested Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Nested Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------- -SQL ORC MR 14325 14522 154 0.1 13661.0 1.0X -SQL ORC Vectorized (Nested Column Disabled) 14107 14392 251 0.1 13453.2 1.0X -SQL ORC Vectorized (Nested Column Enabled) 7445 7470 16 0.1 7099.8 1.9X -SQL Parquet MR: DataPageV1 8992 9032 32 0.1 8575.8 1.6X -SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 9615 9741 77 0.1 9169.2 1.5X -SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 6242 6391 95 0.2 5952.4 2.3X -SQL Parquet MR: DataPageV2 10019 10415 264 0.1 9555.2 1.4X -SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 10273 10371 146 0.1 9796.8 1.4X -SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 5730 5779 33 0.2 5464.9 2.5X +SQL ORC MR 12979 13152 162 0.1 12377.6 1.0X +SQL ORC Vectorized (Nested Column Disabled) 12920 12989 53 0.1 12321.6 1.0X +SQL ORC Vectorized (Nested Column Enabled) 7225 7249 17 0.1 6890.2 1.8X +SQL Parquet MR: DataPageV1 8620 8655 24 0.1 8221.1 1.5X +SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 8972 8983 7 0.1 8556.5 1.4X +SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 5756 5799 31 0.2 5489.8 2.3X +SQL Parquet MR: DataPageV2 9485 9514 18 0.1 9045.5 1.4X +SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 9765 9805 19 0.1 9312.8 1.3X +SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 5567 5600 19 0.2 5309.3 2.3X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 12003 12156 217 0.9 1144.7 1.0X -SQL Json 10706 10711 7 1.0 1021.0 1.1X -SQL Parquet Vectorized: DataPageV1 1800 1807 10 5.8 171.6 6.7X -SQL Parquet Vectorized: DataPageV2 1923 1930 10 5.5 183.4 6.2X -SQL Parquet MR: DataPageV1 4008 4018 14 2.6 382.2 3.0X -SQL Parquet MR: DataPageV2 4075 4082 10 2.6 388.7 2.9X -SQL ORC Vectorized 1903 1925 30 5.5 181.5 6.3X -SQL ORC MR 3934 3949 21 2.7 375.2 3.1X +SQL CSV 11208 11255 67 0.9 1068.9 1.0X +SQL Json 10457 10487 41 1.0 997.3 1.1X +SQL Parquet Vectorized: DataPageV1 1820 1834 20 5.8 173.5 6.2X +SQL Parquet Vectorized: DataPageV2 1917 1918 1 5.5 182.8 5.8X +SQL Parquet MR: DataPageV1 3975 3976 1 2.6 379.1 2.8X +SQL Parquet MR: DataPageV2 3974 3994 28 2.6 379.0 2.8X +SQL ORC Vectorized 1939 1944 7 5.4 184.9 5.8X +SQL ORC MR 3490 3502 17 3.0 332.8 3.2X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 7254 7268 20 1.4 691.8 1.0X -SQL Json 6959 6959 1 1.5 663.6 1.0X -SQL Parquet Vectorized: DataPageV1 477 482 6 22.0 45.5 15.2X -SQL Parquet Vectorized: DataPageV2 475 488 21 22.1 45.3 15.3X -SQL Parquet MR: DataPageV1 1778 1780 3 5.9 169.6 4.1X -SQL Parquet MR: DataPageV2 1723 1726 5 6.1 164.3 4.2X -SQL ORC Vectorized 396 409 22 26.5 37.7 18.3X -SQL ORC MR 1884 1905 30 5.6 179.6 3.9X +SQL CSV 6355 6393 55 1.7 606.0 1.0X +SQL Json 6798 6811 17 1.5 648.4 0.9X +SQL Parquet Vectorized: DataPageV1 517 522 4 20.3 49.3 12.3X +SQL Parquet Vectorized: DataPageV2 511 521 13 20.5 48.7 12.4X +SQL Parquet MR: DataPageV1 1725 1746 30 6.1 164.5 3.7X +SQL Parquet MR: DataPageV2 1631 1650 27 6.4 155.5 3.9X +SQL ORC Vectorized 371 377 4 28.3 35.3 17.1X +SQL ORC MR 1701 1713 17 6.2 162.2 3.7X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -Data column - CSV 13466 13514 67 1.2 856.2 1.0X -Data column - Json 10162 10191 42 1.5 646.1 1.3X -Data column - Parquet Vectorized: DataPageV1 119 134 10 132.4 7.6 113.3X -Data column - Parquet Vectorized: DataPageV2 294 302 13 53.6 18.7 45.9X -Data column - Parquet MR: DataPageV1 2489 2627 195 6.3 158.2 5.4X -Data column - Parquet MR: DataPageV2 2443 2466 33 6.4 155.3 5.5X -Data column - ORC Vectorized 187 200 11 84.0 11.9 71.9X -Data column - ORC MR 2306 2313 9 6.8 146.6 5.8X -Partition column - CSV 3790 3809 27 4.2 241.0 3.6X -Partition column - Json 8570 8579 12 1.8 544.9 1.6X -Partition column - Parquet Vectorized: DataPageV1 35 38 3 444.9 2.2 380.9X -Partition column - Parquet Vectorized: DataPageV2 35 38 3 452.0 2.2 387.0X -Partition column - Parquet MR: DataPageV1 1411 1422 15 11.1 89.7 9.5X -Partition column - Parquet MR: DataPageV2 1396 1435 54 11.3 88.8 9.6X -Partition column - ORC Vectorized 36 39 3 432.0 2.3 369.9X -Partition column - ORC MR 1503 1514 16 10.5 95.6 9.0X -Both columns - CSV 13408 13425 24 1.2 852.5 1.0X -Both columns - Json 10284 10301 24 1.5 653.9 1.3X -Both columns - Parquet Vectorized: DataPageV1 154 182 24 101.8 9.8 87.2X -Both columns - Parquet Vectorized: DataPageV2 341 350 17 46.1 21.7 39.5X -Both columns - Parquet MR: DataPageV1 2465 2490 35 6.4 156.7 5.5X -Both columns - Parquet MR: DataPageV2 2450 2489 55 6.4 155.8 5.5X -Both columns - ORC Vectorized 220 245 19 71.4 14.0 61.1X -Both columns - ORC MR 2333 2334 1 6.7 148.4 5.8X +Data column - CSV 12083 12159 107 1.3 768.2 1.0X +Data column - Json 10115 10122 10 1.6 643.1 1.2X +Data column - Parquet Vectorized: DataPageV1 102 107 4 154.0 6.5 118.3X +Data column - Parquet Vectorized: DataPageV2 237 242 4 66.3 15.1 50.9X +Data column - Parquet MR: DataPageV1 2228 2369 199 7.1 141.7 5.4X +Data column - Parquet MR: DataPageV2 2196 2201 7 7.2 139.6 5.5X +Data column - ORC Vectorized 138 142 4 113.7 8.8 87.4X +Data column - ORC MR 1925 1944 27 8.2 122.4 6.3X +Partition column - CSV 3593 3619 37 4.4 228.4 3.4X +Partition column - Json 8708 8717 13 1.8 553.6 1.4X +Partition column - Parquet Vectorized: DataPageV1 29 32 4 549.4 1.8 422.0X +Partition column - Parquet Vectorized: DataPageV2 28 32 4 554.1 1.8 425.7X +Partition column - Parquet MR: DataPageV1 1173 1183 14 13.4 74.6 10.3X +Partition column - Parquet MR: DataPageV2 1168 1176 11 13.5 74.3 10.3X +Partition column - ORC Vectorized 30 36 6 525.7 1.9 403.8X +Partition column - ORC MR 1210 1211 1 13.0 76.9 10.0X +Both columns - CSV 12007 12141 189 1.3 763.4 1.0X +Both columns - Json 10312 10333 29 1.5 655.6 1.2X +Both columns - Parquet Vectorized: DataPageV1 136 157 21 115.5 8.7 88.7X +Both columns - Parquet Vectorized: DataPageV2 279 310 24 56.4 17.7 43.3X +Both columns - Parquet MR: DataPageV1 2345 2361 23 6.7 149.1 5.2X +Both columns - Parquet MR: DataPageV2 2257 2309 74 7.0 143.5 5.4X +Both columns - ORC Vectorized 183 211 19 85.8 11.6 65.9X +Both columns - ORC MR 2075 2086 15 7.6 131.9 5.8X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 8487 8504 24 1.2 809.4 1.0X -SQL Json 9230 9236 9 1.1 880.3 0.9X -SQL Parquet Vectorized: DataPageV1 1279 1294 20 8.2 122.0 6.6X -SQL Parquet Vectorized: DataPageV2 1327 1382 78 7.9 126.5 6.4X -SQL Parquet MR: DataPageV1 3655 3662 10 2.9 348.6 2.3X -SQL Parquet MR: DataPageV2 3708 3742 47 2.8 353.6 2.3X -ParquetReader Vectorized: DataPageV1 837 838 1 12.5 79.8 10.1X -ParquetReader Vectorized: DataPageV2 898 900 3 11.7 85.7 9.4X -SQL ORC Vectorized 970 1025 77 10.8 92.5 8.7X -SQL ORC MR 3092 3123 44 3.4 294.9 2.7X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 7818 7868 70 1.3 745.6 1.0X +SQL Json 9376 9390 20 1.1 894.1 0.8X +SQL Parquet Vectorized: DataPageV1 1240 1242 3 8.5 118.2 6.3X +SQL Parquet Vectorized: DataPageV2 1301 1302 2 8.1 124.1 6.0X +SQL Parquet MR: DataPageV1 3359 3365 9 3.1 320.3 2.3X +SQL Parquet MR: DataPageV2 3670 3674 6 2.9 350.0 2.1X +ParquetReader Vectorized: DataPageV1 830 834 3 12.6 79.2 9.4X +ParquetReader Vectorized: DataPageV2 896 898 2 11.7 85.4 8.7X +SQL ORC Vectorized 867 887 32 12.1 82.7 9.0X +SQL ORC MR 2905 2907 3 3.6 277.1 2.7X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 6254 6267 18 1.7 596.4 1.0X -SQL Json 7852 7857 8 1.3 748.8 0.8X -SQL Parquet Vectorized: DataPageV1 889 907 17 11.8 84.8 7.0X -SQL Parquet Vectorized: DataPageV2 978 983 8 10.7 93.3 6.4X -SQL Parquet MR: DataPageV1 2939 2948 14 3.6 280.2 2.1X -SQL Parquet MR: DataPageV2 3175 3189 20 3.3 302.8 2.0X -ParquetReader Vectorized: DataPageV1 756 761 7 13.9 72.1 8.3X -ParquetReader Vectorized: DataPageV2 853 858 5 12.3 81.3 7.3X -SQL ORC Vectorized 1024 1027 4 10.2 97.6 6.1X -SQL ORC MR 2930 2933 4 3.6 279.4 2.1X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 6074 6083 13 1.7 579.2 1.0X +SQL Json 7930 7931 2 1.3 756.2 0.8X +SQL Parquet Vectorized: DataPageV1 862 888 22 12.2 82.2 7.0X +SQL Parquet Vectorized: DataPageV2 951 959 9 11.0 90.7 6.4X +SQL Parquet MR: DataPageV1 2636 2703 95 4.0 251.4 2.3X +SQL Parquet MR: DataPageV2 2697 2706 13 3.9 257.2 2.3X +ParquetReader Vectorized: DataPageV1 758 765 10 13.8 72.3 8.0X +ParquetReader Vectorized: DataPageV2 824 826 4 12.7 78.6 7.4X +SQL ORC Vectorized 982 993 10 10.7 93.6 6.2X +SQL ORC MR 2763 2774 15 3.8 263.5 2.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 4333 4340 10 2.4 413.3 1.0X -SQL Json 5440 5448 11 1.9 518.8 0.8X -SQL Parquet Vectorized: DataPageV1 166 173 10 63.1 15.8 26.1X -SQL Parquet Vectorized: DataPageV2 184 187 3 56.9 17.6 23.5X -SQL Parquet MR: DataPageV1 1846 1854 11 5.7 176.0 2.3X -SQL Parquet MR: DataPageV2 1813 1815 2 5.8 172.9 2.4X -ParquetReader Vectorized: DataPageV1 171 174 4 61.2 16.3 25.3X -ParquetReader Vectorized: DataPageV2 190 191 1 55.2 18.1 22.8X -SQL ORC Vectorized 308 310 1 34.0 29.4 14.1X -SQL ORC MR 1700 1707 10 6.2 162.2 2.5X +SQL CSV 4577 4595 25 2.3 436.5 1.0X +SQL Json 5583 5595 16 1.9 532.5 0.8X +SQL Parquet Vectorized: DataPageV1 165 171 7 63.5 15.7 27.7X +SQL Parquet Vectorized: DataPageV2 179 182 2 58.7 17.0 25.6X +SQL Parquet MR: DataPageV1 1643 1652 13 6.4 156.7 2.8X +SQL Parquet MR: DataPageV2 1603 1604 1 6.5 152.9 2.9X +ParquetReader Vectorized: DataPageV1 170 172 4 61.9 16.2 27.0X +ParquetReader Vectorized: DataPageV2 184 185 1 56.9 17.6 24.8X +SQL ORC Vectorized 317 322 4 33.1 30.2 14.4X +SQL ORC MR 1585 1598 18 6.6 151.1 2.9X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 1183 1208 34 0.9 1128.7 1.0X -SQL Json 1836 1837 2 0.6 1750.6 0.6X -SQL Parquet Vectorized: DataPageV1 28 31 3 37.5 26.6 42.4X -SQL Parquet Vectorized: DataPageV2 38 42 5 27.8 35.9 31.4X -SQL Parquet MR: DataPageV1 185 189 3 5.7 176.1 6.4X -SQL Parquet MR: DataPageV2 180 188 10 5.8 171.9 6.6X -SQL ORC Vectorized 33 36 3 31.4 31.8 35.5X -SQL ORC MR 167 175 5 6.3 159.1 7.1X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 1232 1233 0 0.9 1175.3 1.0X +SQL Json 1765 1775 14 0.6 1683.6 0.7X +SQL Parquet Vectorized: DataPageV1 24 27 4 43.1 23.2 50.7X +SQL Parquet Vectorized: DataPageV2 33 36 4 31.8 31.4 37.4X +SQL Parquet MR: DataPageV1 156 160 3 6.7 148.6 7.9X +SQL Parquet MR: DataPageV2 151 156 3 6.9 144.1 8.2X +SQL ORC Vectorized 29 32 4 35.7 28.0 42.0X +SQL ORC MR 124 129 4 8.5 118.1 10.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 50 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 2586 2589 4 0.4 2466.4 1.0X -SQL Json 6706 6852 207 0.2 6395.3 0.4X -SQL Parquet Vectorized: DataPageV1 32 35 3 33.0 30.3 81.5X -SQL Parquet Vectorized: DataPageV2 42 47 6 25.0 40.0 61.7X -SQL Parquet MR: DataPageV1 187 193 4 5.6 178.3 13.8X -SQL Parquet MR: DataPageV2 181 186 5 5.8 172.2 14.3X -SQL ORC Vectorized 38 41 3 27.3 36.7 67.2X -SQL ORC MR 171 178 11 6.1 163.5 15.1X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +SQL CSV 2667 2671 6 0.4 2543.6 1.0X +SQL Json 6256 6274 25 0.2 5966.2 0.4X +SQL Parquet Vectorized: DataPageV1 27 29 4 38.9 25.7 99.0X +SQL Parquet Vectorized: DataPageV2 36 39 4 29.3 34.1 74.5X +SQL Parquet MR: DataPageV1 160 166 5 6.5 152.9 16.6X +SQL Parquet MR: DataPageV2 155 160 4 6.8 147.9 17.2X +SQL ORC Vectorized 33 38 6 32.0 31.3 81.3X +SQL ORC MR 127 131 4 8.2 121.3 21.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 4290 4320 42 0.2 4091.7 1.0X -SQL Json 12544 12642 139 0.1 11963.0 0.3X -SQL Parquet Vectorized: DataPageV1 41 45 5 25.7 38.9 105.2X -SQL Parquet Vectorized: DataPageV2 50 57 9 20.8 48.0 85.3X -SQL Parquet MR: DataPageV1 199 205 4 5.3 189.9 21.5X -SQL Parquet MR: DataPageV2 196 200 2 5.3 187.0 21.9X -SQL ORC Vectorized 46 49 4 22.6 44.2 92.5X -SQL ORC MR 181 185 3 5.8 172.7 23.7X +SQL CSV 4437 4470 46 0.2 4231.7 1.0X +SQL Json 11849 12082 329 0.1 11300.4 0.4X +SQL Parquet Vectorized: DataPageV1 34 38 5 30.8 32.4 130.5X +SQL Parquet Vectorized: DataPageV2 43 47 6 24.5 40.9 103.5X +SQL Parquet MR: DataPageV1 169 174 3 6.2 161.3 26.2X +SQL Parquet MR: DataPageV2 167 172 5 6.3 159.0 26.6X +SQL ORC Vectorized 38 41 4 27.3 36.7 115.4X +SQL ORC MR 134 138 3 7.8 127.5 33.2X diff --git a/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt b/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt index a98af93289208..80f7753f53541 100644 --- a/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt @@ -2,45 +2,45 @@ Dataset Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back map long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 6410 6514 147 15.6 64.1 1.0X -DataFrame 1121 1133 17 89.2 11.2 5.7X -Dataset 1691 1698 10 59.1 16.9 3.8X +RDD 6419 6615 277 15.6 64.2 1.0X +DataFrame 1215 1219 6 82.3 12.2 5.3X +Dataset 1694 1698 7 59.0 16.9 3.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 7313 7329 23 13.7 73.1 1.0X -DataFrame 2721 2764 60 36.7 27.2 2.7X -Dataset 6563 6672 155 15.2 65.6 1.1X +RDD 7768 7769 1 12.9 77.7 1.0X +DataFrame 2860 2869 13 35.0 28.6 2.7X +Dataset 7013 7025 17 14.3 70.1 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back filter Long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 3870 3894 35 25.8 38.7 1.0X -DataFrame 723 733 11 138.3 7.2 5.4X -Dataset 1534 1566 45 65.2 15.3 2.5X +RDD 4387 4430 61 22.8 43.9 1.0X +DataFrame 755 782 28 132.4 7.6 5.8X +Dataset 1571 1581 14 63.7 15.7 2.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 1967 1996 41 50.8 19.7 1.0X -DataFrame 116 126 10 864.5 1.2 17.0X -Dataset 2234 2273 55 44.8 22.3 0.9X +RDD 2090 2092 3 47.9 20.9 1.0X +DataFrame 106 121 8 941.0 1.1 19.7X +Dataset 2335 2438 146 42.8 23.4 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor aggregate: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD sum 1470 1504 48 68.0 14.7 1.0X -DataFrame sum 66 84 13 1506.4 0.7 22.1X -Dataset sum using Aggregator 1929 1944 21 51.8 19.3 0.8X -Dataset complex Aggregator 4979 5163 260 20.1 49.8 0.3X +RDD sum 1419 1424 7 70.5 14.2 1.0X +DataFrame sum 57 70 11 1765.6 0.6 25.1X +Dataset sum using Aggregator 1942 1973 44 51.5 19.4 0.7X +Dataset complex Aggregator 5348 5593 347 18.7 53.5 0.3X diff --git a/sql/core/benchmarks/DatasetBenchmark-results.txt b/sql/core/benchmarks/DatasetBenchmark-results.txt index 78e90cf783593..6cd94dd233f80 100644 --- a/sql/core/benchmarks/DatasetBenchmark-results.txt +++ b/sql/core/benchmarks/DatasetBenchmark-results.txt @@ -2,45 +2,45 @@ Dataset Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back map long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 6908 6969 86 14.5 69.1 1.0X -DataFrame 1286 1300 21 77.8 12.9 5.4X -Dataset 1763 1778 21 56.7 17.6 3.9X +RDD 5780 5868 124 17.3 57.8 1.0X +DataFrame 1150 1157 9 86.9 11.5 5.0X +Dataset 1584 1664 114 63.1 15.8 3.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 8250 8274 34 12.1 82.5 1.0X -DataFrame 2867 2868 2 34.9 28.7 2.9X -Dataset 6939 6971 45 14.4 69.4 1.2X +RDD 6968 7114 205 14.4 69.7 1.0X +DataFrame 2743 2753 15 36.5 27.4 2.5X +Dataset 7436 7456 29 13.4 74.4 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back filter Long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 4265 4343 110 23.4 42.7 1.0X -DataFrame 712 763 45 140.5 7.1 6.0X -Dataset 1722 1732 14 58.1 17.2 2.5X +RDD 4086 4184 139 24.5 40.9 1.0X +DataFrame 663 692 48 150.7 6.6 6.2X +Dataset 1515 1528 18 66.0 15.1 2.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor back-to-back filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD 2250 2275 36 44.4 22.5 1.0X -DataFrame 115 126 10 873.3 1.1 19.7X -Dataset 2441 2459 25 41.0 24.4 0.9X +RDD 2091 2111 29 47.8 20.9 1.0X +DataFrame 112 127 13 892.0 1.1 18.7X +Dataset 2456 2476 29 40.7 24.6 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor aggregate: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -RDD sum 1434 1444 14 69.7 14.3 1.0X -DataFrame sum 67 80 10 1483.2 0.7 21.3X -Dataset sum using Aggregator 2083 2146 90 48.0 20.8 0.7X -Dataset complex Aggregator 5100 5116 23 19.6 51.0 0.3X +RDD sum 1426 1474 68 70.1 14.3 1.0X +DataFrame sum 68 79 11 1478.3 0.7 21.1X +Dataset sum using Aggregator 2117 2130 19 47.2 21.2 0.7X +Dataset complex Aggregator 5121 5237 165 19.5 51.2 0.3X diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt index c230aea8da606..e4b39687c4080 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt @@ -2,460 +2,460 @@ datetime +/- interval ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date + interval(m) 1185 1217 45 8.4 118.5 1.0X -date + interval(m, d) 1166 1188 31 8.6 116.6 1.0X -date + interval(m, d, ms) 3784 3794 13 2.6 378.4 0.3X -date - interval(m) 1098 1101 3 9.1 109.8 1.1X -date - interval(m, d) 1119 1128 13 8.9 111.9 1.1X -date - interval(m, d, ms) 3792 3799 9 2.6 379.2 0.3X -timestamp + interval(m) 1516 1522 8 6.6 151.6 0.8X -timestamp + interval(m, d) 1571 1573 3 6.4 157.1 0.8X -timestamp + interval(m, d, ms) 1716 1717 2 5.8 171.6 0.7X -timestamp - interval(m) 1503 1505 2 6.7 150.3 0.8X -timestamp - interval(m, d) 1557 1558 1 6.4 155.7 0.8X -timestamp - interval(m, d, ms) 1714 1716 3 5.8 171.4 0.7X +date + interval(m) 1127 1142 21 8.9 112.7 1.0X +date + interval(m, d) 1098 1100 3 9.1 109.8 1.0X +date + interval(m, d, ms) 3853 3857 5 2.6 385.3 0.3X +date - interval(m) 1084 1089 7 9.2 108.4 1.0X +date - interval(m, d) 1104 1106 4 9.1 110.4 1.0X +date - interval(m, d, ms) 3908 3946 54 2.6 390.8 0.3X +timestamp + interval(m) 1985 1988 4 5.0 198.5 0.6X +timestamp + interval(m, d) 2020 2026 9 5.0 202.0 0.6X +timestamp + interval(m, d, ms) 2106 2122 22 4.7 210.6 0.5X +timestamp - interval(m) 1935 1938 4 5.2 193.5 0.6X +timestamp - interval(m, d) 1973 1981 11 5.1 197.3 0.6X +timestamp - interval(m, d, ms) 2100 2110 14 4.8 210.0 0.5X ================================================================================================ Extract components ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 200 202 3 49.9 20.0 1.0X -cast to timestamp wholestage on 220 231 9 45.5 22.0 0.9X +cast to timestamp wholestage off 199 202 4 50.3 19.9 1.0X +cast to timestamp wholestage on 219 223 3 45.7 21.9 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 707 709 3 14.2 70.7 1.0X -year of timestamp wholestage on 718 721 4 13.9 71.8 1.0X +year of timestamp wholestage off 695 697 3 14.4 69.5 1.0X +year of timestamp wholestage on 705 713 10 14.2 70.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 743 744 2 13.5 74.3 1.0X -quarter of timestamp wholestage on 747 754 9 13.4 74.7 1.0X +quarter of timestamp wholestage off 752 758 10 13.3 75.2 1.0X +quarter of timestamp wholestage on 790 795 4 12.7 79.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 720 724 5 13.9 72.0 1.0X -month of timestamp wholestage on 729 731 2 13.7 72.9 1.0X +month of timestamp wholestage off 709 716 9 14.1 70.9 1.0X +month of timestamp wholestage on 722 727 5 13.8 72.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1098 1098 0 9.1 109.8 1.0X -weekofyear of timestamp wholestage on 1141 1151 17 8.8 114.1 1.0X +weekofyear of timestamp wholestage off 1113 1113 1 9.0 111.3 1.0X +weekofyear of timestamp wholestage on 1137 1151 22 8.8 113.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 732 737 7 13.7 73.2 1.0X -day of timestamp wholestage on 756 760 3 13.2 75.6 1.0X +day of timestamp wholestage off 724 726 2 13.8 72.4 1.0X +day of timestamp wholestage on 766 779 9 13.1 76.6 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 767 770 4 13.0 76.7 1.0X -dayofyear of timestamp wholestage on 780 785 3 12.8 78.0 1.0X +dayofyear of timestamp wholestage off 751 753 3 13.3 75.1 1.0X +dayofyear of timestamp wholestage on 763 767 4 13.1 76.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 755 763 11 13.2 75.5 1.0X -dayofmonth of timestamp wholestage on 758 764 7 13.2 75.8 1.0X +dayofmonth of timestamp wholestage off 742 744 3 13.5 74.2 1.0X +dayofmonth of timestamp wholestage on 777 782 5 12.9 77.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 886 889 5 11.3 88.6 1.0X -dayofweek of timestamp wholestage on 933 943 9 10.7 93.3 0.9X +dayofweek of timestamp wholestage off 885 887 2 11.3 88.5 1.0X +dayofweek of timestamp wholestage on 974 976 1 10.3 97.4 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 822 822 0 12.2 82.2 1.0X -weekday of timestamp wholestage on 839 845 9 11.9 83.9 1.0X +weekday of timestamp wholestage off 818 821 5 12.2 81.8 1.0X +weekday of timestamp wholestage on 901 911 9 11.1 90.1 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 549 558 13 18.2 54.9 1.0X -hour of timestamp wholestage on 564 567 2 17.7 56.4 1.0X +hour of timestamp wholestage off 545 547 2 18.4 54.5 1.0X +hour of timestamp wholestage on 617 626 9 16.2 61.7 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 547 549 2 18.3 54.7 1.0X -minute of timestamp wholestage on 561 567 4 17.8 56.1 1.0X +minute of timestamp wholestage off 542 550 11 18.4 54.2 1.0X +minute of timestamp wholestage on 556 565 13 18.0 55.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 552 555 5 18.1 55.2 1.0X -second of timestamp wholestage on 561 564 3 17.8 56.1 1.0X +second of timestamp wholestage off 541 546 8 18.5 54.1 1.0X +second of timestamp wholestage on 560 566 4 17.8 56.0 1.0X ================================================================================================ Current date and time ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 179 179 0 55.9 17.9 1.0X -current_date wholestage on 224 228 6 44.7 22.4 0.8X +current_date wholestage off 176 178 3 56.7 17.6 1.0X +current_date wholestage on 216 223 8 46.3 21.6 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 176 181 6 56.7 17.6 1.0X -current_timestamp wholestage on 236 241 5 42.3 23.6 0.7X +current_timestamp wholestage off 182 190 12 55.1 18.2 1.0X +current_timestamp wholestage on 232 240 8 43.1 23.2 0.8X ================================================================================================ Date arithmetic ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 639 641 2 15.6 63.9 1.0X -cast to date wholestage on 717 721 6 14.0 71.7 0.9X +cast to date wholestage off 638 643 7 15.7 63.8 1.0X +cast to date wholestage on 714 720 7 14.0 71.4 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 794 797 3 12.6 79.4 1.0X -last_day wholestage on 817 821 4 12.2 81.7 1.0X +last_day wholestage off 773 777 6 12.9 77.3 1.0X +last_day wholestage on 809 811 1 12.4 80.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 724 728 6 13.8 72.4 1.0X -next_day wholestage on 744 747 3 13.4 74.4 1.0X +next_day wholestage off 715 751 51 14.0 71.5 1.0X +next_day wholestage on 735 739 7 13.6 73.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 676 679 4 14.8 67.6 1.0X -date_add wholestage on 700 704 3 14.3 70.0 1.0X +date_add wholestage off 669 681 16 14.9 66.9 1.0X +date_add wholestage on 684 689 4 14.6 68.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 684 684 1 14.6 68.4 1.0X -date_sub wholestage on 698 701 3 14.3 69.8 1.0X +date_sub wholestage off 670 670 1 14.9 67.0 1.0X +date_sub wholestage on 684 694 14 14.6 68.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 983 984 2 10.2 98.3 1.0X -add_months wholestage on 1069 1074 7 9.4 106.9 0.9X +add_months wholestage off 960 962 3 10.4 96.0 1.0X +add_months wholestage on 1058 1065 5 9.5 105.8 0.9X ================================================================================================ Formatting dates ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 3043 3062 27 3.3 304.3 1.0X -format date wholestage on 3118 3133 14 3.2 311.8 1.0X +format date wholestage off 3175 3213 53 3.1 317.5 1.0X +format date wholestage on 3134 3152 22 3.2 313.4 1.0X ================================================================================================ Formatting timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 2560 2560 0 3.9 256.0 1.0X -from_unixtime wholestage on 2594 2653 37 3.9 259.4 1.0X +from_unixtime wholestage off 2599 2601 3 3.8 259.9 1.0X +from_unixtime wholestage on 2682 2692 10 3.7 268.2 1.0X ================================================================================================ Convert timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 641 642 2 15.6 64.1 1.0X -from_utc_timestamp wholestage on 767 770 3 13.0 76.7 0.8X +from_utc_timestamp wholestage off 623 625 2 16.0 62.3 1.0X +from_utc_timestamp wholestage on 751 753 2 13.3 75.1 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 809 812 5 12.4 80.9 1.0X -to_utc_timestamp wholestage on 882 889 7 11.3 88.2 0.9X +to_utc_timestamp wholestage off 769 772 4 13.0 76.9 1.0X +to_utc_timestamp wholestage on 826 828 1 12.1 82.6 0.9X ================================================================================================ Intervals ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 225 242 24 44.5 22.5 1.0X -cast interval wholestage on 225 226 2 44.5 22.5 1.0X +cast interval wholestage off 227 233 9 44.0 22.7 1.0X +cast interval wholestage on 217 223 8 46.1 21.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1226 1229 3 8.2 122.6 1.0X -datediff wholestage on 1220 1224 3 8.2 122.0 1.0X +datediff wholestage off 1231 1234 4 8.1 123.1 1.0X +datediff wholestage on 1225 1230 8 8.2 122.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 3576 3582 8 2.8 357.6 1.0X -months_between wholestage on 3568 3581 23 2.8 356.8 1.0X +months_between wholestage off 3561 3561 1 2.8 356.1 1.0X +months_between wholestage on 3597 3607 7 2.8 359.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 383 395 18 2.6 382.5 1.0X -window wholestage on 634 657 25 1.6 633.8 0.6X +window wholestage off 389 396 11 2.6 388.6 1.0X +window wholestage on 669 685 15 1.5 668.8 0.6X ================================================================================================ Truncation ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 1862 1863 0 5.4 186.2 1.0X -date_trunc YEAR wholestage on 1867 1875 6 5.4 186.7 1.0X +date_trunc YEAR wholestage off 1902 1902 0 5.3 190.2 1.0X +date_trunc YEAR wholestage on 1909 1915 6 5.2 190.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 1864 1867 4 5.4 186.4 1.0X -date_trunc YYYY wholestage on 1865 1871 4 5.4 186.5 1.0X +date_trunc YYYY wholestage off 1908 1908 1 5.2 190.8 1.0X +date_trunc YYYY wholestage on 1909 1911 1 5.2 190.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 1867 1869 3 5.4 186.7 1.0X -date_trunc YY wholestage on 1867 1874 5 5.4 186.7 1.0X +date_trunc YY wholestage off 1899 1901 2 5.3 189.9 1.0X +date_trunc YY wholestage on 1907 1918 6 5.2 190.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 1897 1904 10 5.3 189.7 1.0X -date_trunc MON wholestage on 1857 1862 5 5.4 185.7 1.0X +date_trunc MON wholestage off 1925 1926 2 5.2 192.5 1.0X +date_trunc MON wholestage on 1887 1899 14 5.3 188.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 1901 1901 1 5.3 190.1 1.0X -date_trunc MONTH wholestage on 1858 1863 4 5.4 185.8 1.0X +date_trunc MONTH wholestage off 1930 1932 2 5.2 193.0 1.0X +date_trunc MONTH wholestage on 1890 1895 6 5.3 189.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 1890 1895 7 5.3 189.0 1.0X -date_trunc MM wholestage on 1858 1861 2 5.4 185.8 1.0X +date_trunc MM wholestage off 1928 1930 2 5.2 192.8 1.0X +date_trunc MM wholestage on 1889 1895 5 5.3 188.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 1232 1234 2 8.1 123.2 1.0X -date_trunc DAY wholestage on 1330 1336 4 7.5 133.0 0.9X +date_trunc DAY wholestage off 1216 1219 5 8.2 121.6 1.0X +date_trunc DAY wholestage on 1183 1187 3 8.5 118.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 1231 1233 4 8.1 123.1 1.0X -date_trunc DD wholestage on 1334 1337 5 7.5 133.4 0.9X +date_trunc DD wholestage off 1216 1217 2 8.2 121.6 1.0X +date_trunc DD wholestage on 1185 1194 17 8.4 118.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 1200 1201 3 8.3 120.0 1.0X -date_trunc HOUR wholestage on 1162 1168 5 8.6 116.2 1.0X +date_trunc HOUR wholestage off 1212 1215 4 8.2 121.2 1.0X +date_trunc HOUR wholestage on 1169 1174 5 8.6 116.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 1206 1209 5 8.3 120.6 1.0X -date_trunc MINUTE wholestage on 1170 1174 3 8.5 117.0 1.0X +date_trunc MINUTE wholestage off 1233 1233 0 8.1 123.3 1.0X +date_trunc MINUTE wholestage on 1199 1204 3 8.3 119.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 289 289 0 34.6 28.9 1.0X -date_trunc SECOND wholestage on 264 271 4 37.8 26.4 1.1X +date_trunc SECOND wholestage off 307 309 3 32.6 30.7 1.0X +date_trunc SECOND wholestage on 281 283 1 35.6 28.1 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 1788 1794 8 5.6 178.8 1.0X -date_trunc WEEK wholestage on 1753 1756 3 5.7 175.3 1.0X +date_trunc WEEK wholestage off 1810 1810 0 5.5 181.0 1.0X +date_trunc WEEK wholestage on 1775 1789 16 5.6 177.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 2320 2323 4 4.3 232.0 1.0X -date_trunc QUARTER wholestage on 2324 2349 54 4.3 232.4 1.0X +date_trunc QUARTER wholestage off 2367 2370 4 4.2 236.7 1.0X +date_trunc QUARTER wholestage on 2414 2419 6 4.1 241.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 849 851 2 11.8 84.9 1.0X -trunc year wholestage on 829 832 3 12.1 82.9 1.0X +trunc year wholestage off 873 876 4 11.5 87.3 1.0X +trunc year wholestage on 847 852 8 11.8 84.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 850 853 4 11.8 85.0 1.0X -trunc yyyy wholestage on 829 843 20 12.1 82.9 1.0X +trunc yyyy wholestage off 870 882 16 11.5 87.0 1.0X +trunc yyyy wholestage on 844 846 3 11.9 84.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 851 851 0 11.8 85.1 1.0X -trunc yy wholestage on 827 831 4 12.1 82.7 1.0X +trunc yy wholestage off 869 870 1 11.5 86.9 1.0X +trunc yy wholestage on 844 850 7 11.9 84.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 815 817 3 12.3 81.5 1.0X -trunc mon wholestage on 809 812 4 12.4 80.9 1.0X +trunc mon wholestage off 836 840 5 12.0 83.6 1.0X +trunc mon wholestage on 810 815 7 12.3 81.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 815 817 4 12.3 81.5 1.0X -trunc month wholestage on 806 809 2 12.4 80.6 1.0X +trunc month wholestage off 833 835 3 12.0 83.3 1.0X +trunc month wholestage on 812 815 2 12.3 81.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 812 813 1 12.3 81.2 1.0X -trunc mm wholestage on 805 810 4 12.4 80.5 1.0X +trunc mm wholestage off 838 840 3 11.9 83.8 1.0X +trunc mm wholestage on 815 818 2 12.3 81.5 1.0X ================================================================================================ Parsing ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 97 99 2 10.3 96.9 1.0X -to timestamp str wholestage on 103 106 3 9.7 102.9 0.9X +to timestamp str wholestage off 106 107 1 9.4 106.2 1.0X +to timestamp str wholestage on 101 102 1 9.9 100.7 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 678 679 2 1.5 677.5 1.0X -to_timestamp wholestage on 676 680 3 1.5 676.1 1.0X +to_timestamp wholestage off 681 683 3 1.5 680.9 1.0X +to_timestamp wholestage on 680 684 3 1.5 680.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 677 682 6 1.5 677.3 1.0X -to_unix_timestamp wholestage on 669 672 2 1.5 669.1 1.0X +to_unix_timestamp wholestage off 662 662 0 1.5 662.1 1.0X +to_unix_timestamp wholestage on 658 661 2 1.5 657.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 133 135 2 7.5 133.1 1.0X -to date str wholestage on 126 131 3 7.9 126.4 1.1X +to date str wholestage off 133 138 7 7.5 133.4 1.0X +to date str wholestage on 129 132 3 7.7 129.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 716 723 10 1.4 716.2 1.0X -to_date wholestage on 690 693 3 1.4 690.4 1.0X +to_date wholestage off 676 679 4 1.5 676.2 1.0X +to_date wholestage on 672 674 2 1.5 671.6 1.0X ================================================================================================ Conversion from/to external types ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Date 282 284 3 17.8 56.3 1.0X -From java.time.LocalDate 265 276 12 18.8 53.1 1.1X -Collect java.sql.Date 1145 1206 97 4.4 229.0 0.2X -Collect java.time.LocalDate 959 1050 100 5.2 191.7 0.3X -From java.sql.Timestamp 229 245 22 21.9 45.7 1.2X -From java.time.Instant 173 176 6 28.9 34.5 1.6X -Collect longs 910 960 73 5.5 182.0 0.3X -Collect java.sql.Timestamp 920 1118 173 5.4 183.9 0.3X -Collect java.time.Instant 877 967 79 5.7 175.3 0.3X -java.sql.Date to Hive string 3960 4078 184 1.3 792.0 0.1X -java.time.LocalDate to Hive string 3039 3117 128 1.6 607.8 0.1X -java.sql.Timestamp to Hive string 6521 6619 162 0.8 1304.1 0.0X -java.time.Instant to Hive string 4252 4346 91 1.2 850.4 0.1X +From java.sql.Date 283 285 2 17.6 56.7 1.0X +From java.time.LocalDate 265 267 1 18.8 53.1 1.1X +Collect java.sql.Date 1215 1255 34 4.1 243.1 0.2X +Collect java.time.LocalDate 1061 1141 113 4.7 212.1 0.3X +From java.sql.Timestamp 232 248 14 21.5 46.4 1.2X +From java.time.Instant 196 203 11 25.5 39.2 1.4X +Collect longs 811 1029 201 6.2 162.2 0.3X +Collect java.sql.Timestamp 904 1113 183 5.5 180.8 0.3X +Collect java.time.Instant 943 1080 143 5.3 188.7 0.3X +java.sql.Date to Hive string 4049 4296 215 1.2 809.8 0.1X +java.time.LocalDate to Hive string 3393 3476 72 1.5 678.6 0.1X +java.sql.Timestamp to Hive string 6599 6712 98 0.8 1319.7 0.0X +java.time.Instant to Hive string 4387 4475 83 1.1 877.3 0.1X diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 176cc1dc361b8..91e954cec7253 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -2,460 +2,460 @@ datetime +/- interval ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date + interval(m) 1253 1259 9 8.0 125.3 1.0X -date + interval(m, d) 1258 1260 2 7.9 125.8 1.0X -date + interval(m, d, ms) 3904 3923 26 2.6 390.4 0.3X -date - interval(m) 1201 1214 18 8.3 120.1 1.0X -date - interval(m, d) 1236 1238 4 8.1 123.6 1.0X -date - interval(m, d, ms) 3983 3987 5 2.5 398.3 0.3X -timestamp + interval(m) 1846 1852 9 5.4 184.6 0.7X -timestamp + interval(m, d) 1919 1932 18 5.2 191.9 0.7X -timestamp + interval(m, d, ms) 2264 2273 12 4.4 226.4 0.6X -timestamp - interval(m) 2025 2027 3 4.9 202.5 0.6X -timestamp - interval(m, d) 2097 2104 10 4.8 209.7 0.6X -timestamp - interval(m, d, ms) 2265 2270 8 4.4 226.5 0.6X +date + interval(m) 1218 1236 26 8.2 121.8 1.0X +date + interval(m, d) 1194 1214 28 8.4 119.4 1.0X +date + interval(m, d, ms) 3975 3982 11 2.5 397.5 0.3X +date - interval(m) 1140 1149 13 8.8 114.0 1.1X +date - interval(m, d) 1180 1180 1 8.5 118.0 1.0X +date - interval(m, d, ms) 4014 4017 4 2.5 401.4 0.3X +timestamp + interval(m) 1801 1819 25 5.6 180.1 0.7X +timestamp + interval(m, d) 1857 1861 5 5.4 185.7 0.7X +timestamp + interval(m, d, ms) 2184 2185 1 4.6 218.4 0.6X +timestamp - interval(m) 1950 1952 3 5.1 195.0 0.6X +timestamp - interval(m, d) 2016 2027 15 5.0 201.6 0.6X +timestamp - interval(m, d, ms) 2173 2174 1 4.6 217.3 0.6X ================================================================================================ Extract components ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 193 195 3 51.9 19.3 1.0X -cast to timestamp wholestage on 213 220 7 47.0 21.3 0.9X +cast to timestamp wholestage off 200 201 1 49.9 20.0 1.0X +cast to timestamp wholestage on 210 220 8 47.5 21.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 828 832 5 12.1 82.8 1.0X -year of timestamp wholestage on 855 865 11 11.7 85.5 1.0X +year of timestamp wholestage off 821 827 8 12.2 82.1 1.0X +year of timestamp wholestage on 825 828 4 12.1 82.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 854 854 0 11.7 85.4 1.0X -quarter of timestamp wholestage on 884 893 9 11.3 88.4 1.0X +quarter of timestamp wholestage off 872 876 5 11.5 87.2 1.0X +quarter of timestamp wholestage on 843 846 4 11.9 84.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 833 834 1 12.0 83.3 1.0X -month of timestamp wholestage on 845 849 4 11.8 84.5 1.0X +month of timestamp wholestage off 820 821 1 12.2 82.0 1.0X +month of timestamp wholestage on 827 829 3 12.1 82.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1210 1214 5 8.3 121.0 1.0X -weekofyear of timestamp wholestage on 1255 1266 12 8.0 125.5 1.0X +weekofyear of timestamp wholestage off 1207 1208 2 8.3 120.7 1.0X +weekofyear of timestamp wholestage on 1221 1224 3 8.2 122.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 828 830 3 12.1 82.8 1.0X -day of timestamp wholestage on 847 854 12 11.8 84.7 1.0X +day of timestamp wholestage off 821 826 7 12.2 82.1 1.0X +day of timestamp wholestage on 823 831 6 12.2 82.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 854 855 2 11.7 85.4 1.0X -dayofyear of timestamp wholestage on 913 921 5 10.9 91.3 0.9X +dayofyear of timestamp wholestage off 871 872 1 11.5 87.1 1.0X +dayofyear of timestamp wholestage on 858 861 4 11.7 85.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 849 854 8 11.8 84.9 1.0X -dayofmonth of timestamp wholestage on 848 859 9 11.8 84.8 1.0X +dayofmonth of timestamp wholestage off 832 834 3 12.0 83.2 1.0X +dayofmonth of timestamp wholestage on 823 826 3 12.1 82.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 984 989 7 10.2 98.4 1.0X -dayofweek of timestamp wholestage on 1026 1038 7 9.7 102.6 1.0X +dayofweek of timestamp wholestage off 969 972 5 10.3 96.9 1.0X +dayofweek of timestamp wholestage on 976 978 2 10.3 97.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 944 945 2 10.6 94.4 1.0X -weekday of timestamp wholestage on 978 985 7 10.2 97.8 1.0X +weekday of timestamp wholestage off 941 943 3 10.6 94.1 1.0X +weekday of timestamp wholestage on 926 930 2 10.8 92.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 601 604 3 16.6 60.1 1.0X -hour of timestamp wholestage on 609 613 4 16.4 60.9 1.0X +hour of timestamp wholestage off 605 610 7 16.5 60.5 1.0X +hour of timestamp wholestage on 610 613 4 16.4 61.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 603 606 5 16.6 60.3 1.0X -minute of timestamp wholestage on 609 622 21 16.4 60.9 1.0X +minute of timestamp wholestage off 600 603 5 16.7 60.0 1.0X +minute of timestamp wholestage on 609 610 2 16.4 60.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 603 604 1 16.6 60.3 1.0X -second of timestamp wholestage on 612 617 5 16.3 61.2 1.0X +second of timestamp wholestage off 604 611 10 16.6 60.4 1.0X +second of timestamp wholestage on 608 610 3 16.5 60.8 1.0X ================================================================================================ Current date and time ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 188 190 2 53.1 18.8 1.0X -current_date wholestage on 213 217 3 47.0 21.3 0.9X +current_date wholestage off 183 184 1 54.6 18.3 1.0X +current_date wholestage on 216 218 3 46.3 21.6 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 185 189 6 54.0 18.5 1.0X -current_timestamp wholestage on 225 228 2 44.4 22.5 0.8X +current_timestamp wholestage off 192 205 19 52.2 19.2 1.0X +current_timestamp wholestage on 220 231 9 45.4 22.0 0.9X ================================================================================================ Date arithmetic ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 754 757 5 13.3 75.4 1.0X -cast to date wholestage on 771 777 6 13.0 77.1 1.0X +cast to date wholestage off 862 863 1 11.6 86.2 1.0X +cast to date wholestage on 876 893 22 11.4 87.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 854 855 1 11.7 85.4 1.0X -last_day wholestage on 868 871 3 11.5 86.8 1.0X +last_day wholestage off 967 970 4 10.3 96.7 1.0X +last_day wholestage on 982 985 3 10.2 98.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 782 783 2 12.8 78.2 1.0X -next_day wholestage on 811 818 9 12.3 81.1 1.0X +next_day wholestage off 888 892 5 11.3 88.8 1.0X +next_day wholestage on 899 901 2 11.1 89.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 735 736 1 13.6 73.5 1.0X -date_add wholestage on 754 759 8 13.3 75.4 1.0X +date_add wholestage off 843 843 1 11.9 84.3 1.0X +date_add wholestage on 875 882 12 11.4 87.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 740 741 0 13.5 74.0 1.0X -date_sub wholestage on 753 757 6 13.3 75.3 1.0X +date_sub wholestage off 842 845 5 11.9 84.2 1.0X +date_sub wholestage on 876 883 6 11.4 87.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1142 1143 1 8.8 114.2 1.0X -add_months wholestage on 1138 1149 14 8.8 113.8 1.0X +add_months wholestage off 1182 1185 4 8.5 118.2 1.0X +add_months wholestage on 1205 1210 5 8.3 120.5 1.0X ================================================================================================ Formatting dates ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 3591 3598 9 2.8 359.1 1.0X -format date wholestage on 3704 3724 25 2.7 370.4 1.0X +format date wholestage off 4003 4012 14 2.5 400.3 1.0X +format date wholestage on 4044 4048 5 2.5 404.4 1.0X ================================================================================================ Formatting timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 3881 3886 7 2.6 388.1 1.0X -from_unixtime wholestage on 3844 4051 117 2.6 384.4 1.0X +from_unixtime wholestage off 4055 4059 5 2.5 405.5 1.0X +from_unixtime wholestage on 4081 4091 8 2.5 408.1 1.0X ================================================================================================ Convert timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 673 677 6 14.9 67.3 1.0X -from_utc_timestamp wholestage on 782 788 6 12.8 78.2 0.9X +from_utc_timestamp wholestage off 675 685 14 14.8 67.5 1.0X +from_utc_timestamp wholestage on 802 810 7 12.5 80.2 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1038 1038 0 9.6 103.8 1.0X -to_utc_timestamp wholestage on 1025 1031 5 9.8 102.5 1.0X +to_utc_timestamp wholestage off 1054 1055 1 9.5 105.4 1.0X +to_utc_timestamp wholestage on 1073 1076 3 9.3 107.3 1.0X ================================================================================================ Intervals ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 250 257 10 40.1 25.0 1.0X -cast interval wholestage on 215 220 6 46.6 21.5 1.2X +cast interval wholestage off 218 221 4 45.8 21.8 1.0X +cast interval wholestage on 217 221 4 46.2 21.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1340 1342 4 7.5 134.0 1.0X -datediff wholestage on 1389 1395 5 7.2 138.9 1.0X +datediff wholestage off 1492 1500 12 6.7 149.2 1.0X +datediff wholestage on 1514 1517 2 6.6 151.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 3444 3450 8 2.9 344.4 1.0X -months_between wholestage on 3439 3453 14 2.9 343.9 1.0X +months_between wholestage off 3634 3641 10 2.8 363.4 1.0X +months_between wholestage on 3577 3583 4 2.8 357.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 427 449 31 2.3 427.2 1.0X -window wholestage on 656 690 23 1.5 655.8 0.7X +window wholestage off 445 445 0 2.2 445.3 1.0X +window wholestage on 645 660 12 1.6 645.0 0.7X ================================================================================================ Truncation ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 1828 1834 8 5.5 182.8 1.0X -date_trunc YEAR wholestage on 1776 1780 6 5.6 177.6 1.0X +date_trunc YEAR wholestage off 1870 1870 0 5.3 187.0 1.0X +date_trunc YEAR wholestage on 1837 1843 8 5.4 183.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 1847 1849 2 5.4 184.7 1.0X -date_trunc YYYY wholestage on 1774 1781 6 5.6 177.4 1.0X +date_trunc YYYY wholestage off 1867 1870 5 5.4 186.7 1.0X +date_trunc YYYY wholestage on 1841 1844 4 5.4 184.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 1843 1844 1 5.4 184.3 1.0X -date_trunc YY wholestage on 1778 1781 2 5.6 177.8 1.0X +date_trunc YY wholestage off 1868 1871 4 5.4 186.8 1.0X +date_trunc YY wholestage on 1838 1842 4 5.4 183.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 1857 1861 6 5.4 185.7 1.0X -date_trunc MON wholestage on 1786 1791 6 5.6 178.6 1.0X +date_trunc MON wholestage off 1961 1964 4 5.1 196.1 1.0X +date_trunc MON wholestage on 1880 1884 3 5.3 188.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 1860 1871 15 5.4 186.0 1.0X -date_trunc MONTH wholestage on 1782 1789 4 5.6 178.2 1.0X +date_trunc MONTH wholestage off 1966 1966 1 5.1 196.6 1.0X +date_trunc MONTH wholestage on 1881 1884 3 5.3 188.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 1860 1868 11 5.4 186.0 1.0X -date_trunc MM wholestage on 1789 1792 4 5.6 178.9 1.0X +date_trunc MM wholestage off 1966 1971 7 5.1 196.6 1.0X +date_trunc MM wholestage on 1881 1885 4 5.3 188.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 1328 1338 14 7.5 132.8 1.0X -date_trunc DAY wholestage on 1281 1286 5 7.8 128.1 1.0X +date_trunc DAY wholestage off 1318 1319 1 7.6 131.8 1.0X +date_trunc DAY wholestage on 1278 1284 5 7.8 127.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 1330 1335 6 7.5 133.0 1.0X -date_trunc DD wholestage on 1277 1280 2 7.8 127.7 1.0X +date_trunc DD wholestage off 1310 1312 2 7.6 131.0 1.0X +date_trunc DD wholestage on 1280 1285 6 7.8 128.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 1342 1347 7 7.5 134.2 1.0X -date_trunc HOUR wholestage on 1281 1285 3 7.8 128.1 1.0X +date_trunc HOUR wholestage off 1325 1328 5 7.5 132.5 1.0X +date_trunc HOUR wholestage on 1288 1294 4 7.8 128.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 1344 1346 2 7.4 134.4 1.0X -date_trunc MINUTE wholestage on 1306 1310 3 7.7 130.6 1.0X +date_trunc MINUTE wholestage off 1335 1339 5 7.5 133.5 1.0X +date_trunc MINUTE wholestage on 1316 1321 4 7.6 131.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 331 334 4 30.2 33.1 1.0X -date_trunc SECOND wholestage on 278 282 6 35.9 27.8 1.2X +date_trunc SECOND wholestage off 317 321 6 31.6 31.7 1.0X +date_trunc SECOND wholestage on 276 279 5 36.3 27.6 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 1742 1746 5 5.7 174.2 1.0X -date_trunc WEEK wholestage on 1688 1692 7 5.9 168.8 1.0X +date_trunc WEEK wholestage off 1812 1816 6 5.5 181.2 1.0X +date_trunc WEEK wholestage on 1764 1768 3 5.7 176.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 2385 2385 1 4.2 238.5 1.0X -date_trunc QUARTER wholestage on 2479 2495 32 4.0 247.9 1.0X +date_trunc QUARTER wholestage off 2664 2666 2 3.8 266.4 1.0X +date_trunc QUARTER wholestage on 2670 2684 16 3.7 267.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 1025 1025 1 9.8 102.5 1.0X -trunc year wholestage on 995 1003 8 10.0 99.5 1.0X +trunc year wholestage off 1123 1123 0 8.9 112.3 1.0X +trunc year wholestage on 1082 1085 3 9.2 108.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 1024 1027 4 9.8 102.4 1.0X -trunc yyyy wholestage on 995 999 4 10.1 99.5 1.0X +trunc yyyy wholestage off 1119 1119 1 8.9 111.9 1.0X +trunc yyyy wholestage on 1081 1092 16 9.3 108.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 1026 1026 0 9.8 102.6 1.0X -trunc yy wholestage on 999 1001 2 10.0 99.9 1.0X +trunc yy wholestage off 1121 1122 2 8.9 112.1 1.0X +trunc yy wholestage on 1083 1088 10 9.2 108.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 998 1000 3 10.0 99.8 1.0X -trunc mon wholestage on 952 953 1 10.5 95.2 1.0X +trunc mon wholestage off 1110 1123 18 9.0 111.0 1.0X +trunc mon wholestage on 1052 1061 12 9.5 105.2 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 999 1000 1 10.0 99.9 1.0X -trunc month wholestage on 951 961 18 10.5 95.1 1.1X +trunc month wholestage off 1102 1105 4 9.1 110.2 1.0X +trunc month wholestage on 1054 1057 2 9.5 105.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 1001 1003 3 10.0 100.1 1.0X -trunc mm wholestage on 951 953 2 10.5 95.1 1.1X +trunc mm wholestage off 1103 1103 0 9.1 110.3 1.0X +trunc mm wholestage on 1056 1067 12 9.5 105.6 1.0X ================================================================================================ Parsing ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 104 113 12 9.6 104.0 1.0X -to timestamp str wholestage on 100 103 3 10.0 99.5 1.0X +to timestamp str wholestage off 97 98 2 10.3 96.7 1.0X +to timestamp str wholestage on 99 102 4 10.1 99.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 760 763 3 1.3 760.4 1.0X -to_timestamp wholestage on 757 766 12 1.3 757.5 1.0X +to_timestamp wholestage off 721 721 1 1.4 720.6 1.0X +to_timestamp wholestage on 724 728 4 1.4 723.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 754 754 0 1.3 753.6 1.0X -to_unix_timestamp wholestage on 742 743 2 1.3 742.0 1.0X +to_unix_timestamp wholestage off 731 731 0 1.4 730.6 1.0X +to_unix_timestamp wholestage on 731 734 4 1.4 731.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 137 141 6 7.3 137.0 1.0X -to date str wholestage on 130 136 3 7.7 130.0 1.1X +to date str wholestage off 133 134 2 7.5 132.8 1.0X +to date str wholestage on 131 134 3 7.6 131.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 655 656 1 1.5 655.0 1.0X -to_date wholestage on 637 642 5 1.6 636.8 1.0X +to_date wholestage off 648 648 0 1.5 647.9 1.0X +to_date wholestage on 640 643 2 1.6 640.0 1.0X ================================================================================================ Conversion from/to external types ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Date 266 269 3 18.8 53.2 1.0X -From java.time.LocalDate 286 294 12 17.5 57.2 0.9X -Collect java.sql.Date 1152 1337 171 4.3 230.5 0.2X -Collect java.time.LocalDate 962 1123 144 5.2 192.4 0.3X -From java.sql.Timestamp 197 202 5 25.3 39.5 1.3X -From java.time.Instant 176 196 33 28.3 35.3 1.5X -Collect longs 847 1023 198 5.9 169.4 0.3X -Collect java.sql.Timestamp 1160 1208 80 4.3 232.0 0.2X -Collect java.time.Instant 1083 1158 78 4.6 216.6 0.2X -java.sql.Date to Hive string 4114 4175 91 1.2 822.8 0.1X -java.time.LocalDate to Hive string 3656 3737 98 1.4 731.2 0.1X -java.sql.Timestamp to Hive string 6474 6727 243 0.8 1294.8 0.0X -java.time.Instant to Hive string 5303 5420 117 0.9 1060.6 0.1X +From java.sql.Date 281 282 2 17.8 56.1 1.0X +From java.time.LocalDate 280 283 4 17.8 56.0 1.0X +Collect java.sql.Date 1328 1427 118 3.8 265.6 0.2X +Collect java.time.LocalDate 984 1125 124 5.1 196.7 0.3X +From java.sql.Timestamp 199 204 6 25.1 39.8 1.4X +From java.time.Instant 181 183 2 27.7 36.1 1.6X +Collect longs 945 998 46 5.3 189.0 0.3X +Collect java.sql.Timestamp 1008 1209 196 5.0 201.6 0.3X +Collect java.time.Instant 822 1017 206 6.1 164.4 0.3X +java.sql.Date to Hive string 3880 4013 125 1.3 775.9 0.1X +java.time.LocalDate to Hive string 3584 3632 45 1.4 716.8 0.1X +java.sql.Timestamp to Hive string 6366 6433 58 0.8 1273.2 0.0X +java.time.Instant to Hive string 5133 5224 113 1.0 1026.7 0.1X diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt index 4b3fc3f4d4a1f..fbe7b36bffa60 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt @@ -2,153 +2,153 @@ Rebasing dates/timestamps in Parquet datasource ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 10901 10901 0 9.2 109.0 1.0X -before 1582, noop 6649 6649 0 15.0 66.5 1.6X -after 1582, rebase EXCEPTION 20020 20020 0 5.0 200.2 0.5X -after 1582, rebase LEGACY 20257 20257 0 4.9 202.6 0.5X -after 1582, rebase CORRECTED 20050 20050 0 5.0 200.5 0.5X -before 1582, rebase LEGACY 16178 16178 0 6.2 161.8 0.7X -before 1582, rebase CORRECTED 16322 16322 0 6.1 163.2 0.7X - -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +after 1582, noop 11144 11144 0 9.0 111.4 1.0X +before 1582, noop 7066 7066 0 14.2 70.7 1.6X +after 1582, rebase EXCEPTION 19440 19440 0 5.1 194.4 0.6X +after 1582, rebase LEGACY 19280 19280 0 5.2 192.8 0.6X +after 1582, rebase CORRECTED 19431 19431 0 5.1 194.3 0.6X +before 1582, rebase LEGACY 15530 15530 0 6.4 155.3 0.7X +before 1582, rebase CORRECTED 15717 15717 0 6.4 157.2 0.7X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase EXCEPTION 11905 11978 106 8.4 119.1 1.0X -after 1582, vec off, rebase LEGACY 12069 12083 17 8.3 120.7 1.0X -after 1582, vec off, rebase CORRECTED 11835 11861 40 8.4 118.4 1.0X -after 1582, vec on, rebase EXCEPTION 2453 2492 38 40.8 24.5 4.9X -after 1582, vec on, rebase LEGACY 2471 2494 22 40.5 24.7 4.8X -after 1582, vec on, rebase CORRECTED 2386 2442 59 41.9 23.9 5.0X -before 1582, vec off, rebase LEGACY 12049 12076 27 8.3 120.5 1.0X -before 1582, vec off, rebase CORRECTED 11948 11999 48 8.4 119.5 1.0X -before 1582, vec on, rebase LEGACY 2787 2810 24 35.9 27.9 4.3X -before 1582, vec on, rebase CORRECTED 2443 2457 14 40.9 24.4 4.9X - -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +after 1582, vec off, rebase EXCEPTION 11704 11746 52 8.5 117.0 1.0X +after 1582, vec off, rebase LEGACY 11525 11573 66 8.7 115.2 1.0X +after 1582, vec off, rebase CORRECTED 11505 11532 25 8.7 115.0 1.0X +after 1582, vec on, rebase EXCEPTION 2347 2370 21 42.6 23.5 5.0X +after 1582, vec on, rebase LEGACY 2450 2453 2 40.8 24.5 4.8X +after 1582, vec on, rebase CORRECTED 2431 2446 17 41.1 24.3 4.8X +before 1582, vec off, rebase LEGACY 11748 11779 39 8.5 117.5 1.0X +before 1582, vec off, rebase CORRECTED 11591 11630 33 8.6 115.9 1.0X +before 1582, vec on, rebase LEGACY 2781 2786 6 36.0 27.8 4.2X +before 1582, vec on, rebase CORRECTED 2420 2425 7 41.3 24.2 4.8X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2429 2429 0 41.2 24.3 1.0X -before 1900, noop 2334 2334 0 42.8 23.3 1.0X -after 1900, rebase EXCEPTION 14358 14358 0 7.0 143.6 0.2X -after 1900, rebase LEGACY 14210 14210 0 7.0 142.1 0.2X -after 1900, rebase CORRECTED 14052 14052 0 7.1 140.5 0.2X -before 1900, rebase LEGACY 16255 16255 0 6.2 162.5 0.1X -before 1900, rebase CORRECTED 14010 14010 0 7.1 140.1 0.2X - -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2518 2518 0 39.7 25.2 1.0X +before 1900, noop 2375 2375 0 42.1 23.8 1.1X +after 1900, rebase EXCEPTION 13654 13654 0 7.3 136.5 0.2X +after 1900, rebase LEGACY 13187 13187 0 7.6 131.9 0.2X +after 1900, rebase CORRECTED 13174 13174 0 7.6 131.7 0.2X +before 1900, rebase LEGACY 15129 15129 0 6.6 151.3 0.2X +before 1900, rebase CORRECTED 13438 13438 0 7.4 134.4 0.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 16891 16906 13 5.9 168.9 1.0X -after 1900, vec off, rebase LEGACY 16737 16761 21 6.0 167.4 1.0X -after 1900, vec off, rebase CORRECTED 16555 16614 54 6.0 165.5 1.0X -after 1900, vec on, rebase EXCEPTION 4046 4094 54 24.7 40.5 4.2X -after 1900, vec on, rebase LEGACY 4057 4067 11 24.6 40.6 4.2X -after 1900, vec on, rebase CORRECTED 4063 4068 6 24.6 40.6 4.2X -before 1900, vec off, rebase LEGACY 18775 18838 56 5.3 187.7 0.9X -before 1900, vec off, rebase CORRECTED 16681 16713 36 6.0 166.8 1.0X -before 1900, vec on, rebase LEGACY 6105 6129 32 16.4 61.0 2.8X -before 1900, vec on, rebase CORRECTED 4046 4067 27 24.7 40.5 4.2X - -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 16824 16877 53 5.9 168.2 1.0X +after 1900, vec off, rebase LEGACY 16304 16337 31 6.1 163.0 1.0X +after 1900, vec off, rebase CORRECTED 16164 16239 76 6.2 161.6 1.0X +after 1900, vec on, rebase EXCEPTION 4041 4045 7 24.7 40.4 4.2X +after 1900, vec on, rebase LEGACY 4015 4039 36 24.9 40.2 4.2X +after 1900, vec on, rebase CORRECTED 4012 4040 27 24.9 40.1 4.2X +before 1900, vec off, rebase LEGACY 18457 18537 71 5.4 184.6 0.9X +before 1900, vec off, rebase CORRECTED 16232 16269 55 6.2 162.3 1.0X +before 1900, vec on, rebase LEGACY 6106 6120 17 16.4 61.1 2.8X +before 1900, vec on, rebase CORRECTED 4018 4053 48 24.9 40.2 4.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2415 2415 0 41.4 24.1 1.0X -before 1900, noop 2438 2438 0 41.0 24.4 1.0X -after 1900, rebase EXCEPTION 13703 13703 0 7.3 137.0 0.2X -after 1900, rebase LEGACY 13404 13404 0 7.5 134.0 0.2X -after 1900, rebase CORRECTED 13538 13538 0 7.4 135.4 0.2X -before 1900, rebase LEGACY 15245 15245 0 6.6 152.5 0.2X -before 1900, rebase CORRECTED 13242 13242 0 7.6 132.4 0.2X - -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2412 2412 0 41.5 24.1 1.0X +before 1900, noop 2413 2413 0 41.4 24.1 1.0X +after 1900, rebase EXCEPTION 11406 11406 0 8.8 114.1 0.2X +after 1900, rebase LEGACY 11249 11249 0 8.9 112.5 0.2X +after 1900, rebase CORRECTED 11318 11318 0 8.8 113.2 0.2X +before 1900, rebase LEGACY 13104 13104 0 7.6 131.0 0.2X +before 1900, rebase CORRECTED 11269 11269 0 8.9 112.7 0.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 14618 14680 54 6.8 146.2 1.0X -after 1900, vec off, rebase LEGACY 14578 14613 35 6.9 145.8 1.0X -after 1900, vec off, rebase CORRECTED 14621 14691 61 6.8 146.2 1.0X -after 1900, vec on, rebase EXCEPTION 3727 3738 16 26.8 37.3 3.9X -after 1900, vec on, rebase LEGACY 3715 3738 22 26.9 37.1 3.9X -after 1900, vec on, rebase CORRECTED 3705 3727 21 27.0 37.1 3.9X -before 1900, vec off, rebase LEGACY 16907 16943 32 5.9 169.1 0.9X -before 1900, vec off, rebase CORRECTED 14704 14711 6 6.8 147.0 1.0X -before 1900, vec on, rebase LEGACY 5588 5610 30 17.9 55.9 2.6X -before 1900, vec on, rebase CORRECTED 3730 3744 15 26.8 37.3 3.9X - -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 15047 15100 59 6.6 150.5 1.0X +after 1900, vec off, rebase LEGACY 14965 15033 59 6.7 149.7 1.0X +after 1900, vec off, rebase CORRECTED 15041 15064 35 6.6 150.4 1.0X +after 1900, vec on, rebase EXCEPTION 3714 3737 21 26.9 37.1 4.1X +after 1900, vec on, rebase LEGACY 3748 3766 26 26.7 37.5 4.0X +after 1900, vec on, rebase CORRECTED 3733 3743 14 26.8 37.3 4.0X +before 1900, vec off, rebase LEGACY 17297 17350 52 5.8 173.0 0.9X +before 1900, vec off, rebase CORRECTED 14977 14993 24 6.7 149.8 1.0X +before 1900, vec on, rebase LEGACY 5709 5720 10 17.5 57.1 2.6X +before 1900, vec on, rebase CORRECTED 3696 3717 19 27.1 37.0 4.1X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2391 2391 0 41.8 23.9 1.0X -before 1900, noop 2431 2431 0 41.1 24.3 1.0X -after 1900, rebase EXCEPTION 11856 11856 0 8.4 118.6 0.2X -after 1900, rebase LEGACY 11077 11077 0 9.0 110.8 0.2X -after 1900, rebase CORRECTED 11292 11292 0 8.9 112.9 0.2X -before 1900, rebase LEGACY 13101 13101 0 7.6 131.0 0.2X -before 1900, rebase CORRECTED 11333 11333 0 8.8 113.3 0.2X - -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2382 2382 0 42.0 23.8 1.0X +before 1900, noop 2414 2414 0 41.4 24.1 1.0X +after 1900, rebase EXCEPTION 11542 11542 0 8.7 115.4 0.2X +after 1900, rebase LEGACY 11074 11074 0 9.0 110.7 0.2X +after 1900, rebase CORRECTED 11275 11275 0 8.9 112.7 0.2X +before 1900, rebase LEGACY 13166 13166 0 7.6 131.7 0.2X +before 1900, rebase CORRECTED 11341 11341 0 8.8 113.4 0.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 14732 14828 88 6.8 147.3 1.0X -after 1900, vec off, rebase LEGACY 15011 15061 58 6.7 150.1 1.0X -after 1900, vec off, rebase CORRECTED 14924 14978 86 6.7 149.2 1.0X -after 1900, vec on, rebase EXCEPTION 4935 4964 37 20.3 49.4 3.0X -after 1900, vec on, rebase LEGACY 4488 4509 19 22.3 44.9 3.3X -after 1900, vec on, rebase CORRECTED 4896 4934 33 20.4 49.0 3.0X -before 1900, vec off, rebase LEGACY 16917 17029 151 5.9 169.2 0.9X -before 1900, vec off, rebase CORRECTED 14924 14953 35 6.7 149.2 1.0X -before 1900, vec on, rebase LEGACY 6010 6023 12 16.6 60.1 2.5X -before 1900, vec on, rebase CORRECTED 4916 4939 25 20.3 49.2 3.0X +after 1900, vec off, rebase EXCEPTION 14250 14288 40 7.0 142.5 1.0X +after 1900, vec off, rebase LEGACY 14235 14315 82 7.0 142.4 1.0X +after 1900, vec off, rebase CORRECTED 14284 14304 25 7.0 142.8 1.0X +after 1900, vec on, rebase EXCEPTION 4925 4941 27 20.3 49.2 2.9X +after 1900, vec on, rebase LEGACY 4489 4499 10 22.3 44.9 3.2X +after 1900, vec on, rebase CORRECTED 4916 4943 33 20.3 49.2 2.9X +before 1900, vec off, rebase LEGACY 16801 16813 11 6.0 168.0 0.8X +before 1900, vec off, rebase CORRECTED 14259 14307 50 7.0 142.6 1.0X +before 1900, vec on, rebase LEGACY 5958 5966 7 16.8 59.6 2.4X +before 1900, vec on, rebase CORRECTED 4900 4920 19 20.4 49.0 2.9X ================================================================================================ Rebasing dates/timestamps in ORC datasource ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 12628 12628 0 7.9 126.3 1.0X -before 1582, noop 6710 6710 0 14.9 67.1 1.9X -after 1582 16956 16956 0 5.9 169.6 0.7X -before 1582 10840 10840 0 9.2 108.4 1.2X +after 1582, noop 10754 10754 0 9.3 107.5 1.0X +before 1582, noop 6783 6783 0 14.7 67.8 1.6X +after 1582 15425 15425 0 6.5 154.2 0.7X +before 1582 10856 10856 0 9.2 108.6 1.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 8521 8619 138 11.7 85.2 1.0X -after 1582, vec on 2414 2421 10 41.4 24.1 3.5X -before 1582, vec off 8621 8693 89 11.6 86.2 1.0X -before 1582, vec on 2555 2562 8 39.1 25.5 3.3X +after 1582, vec off 8437 8530 90 11.9 84.4 1.0X +after 1582, vec on 2419 2430 10 41.3 24.2 3.5X +before 1582, vec off 8505 8526 19 11.8 85.1 1.0X +before 1582, vec on 2557 2566 11 39.1 25.6 3.3X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2290 2290 0 43.7 22.9 1.0X -before 1900, noop 2313 2313 0 43.2 23.1 1.0X -after 1900 10097 10097 0 9.9 101.0 0.2X -before 1900 12223 12223 0 8.2 122.2 0.2X +after 1900, noop 2308 2308 0 43.3 23.1 1.0X +before 1900, noop 2302 2302 0 43.4 23.0 1.0X +after 1900 9526 9526 0 10.5 95.3 0.2X +before 1900 11558 11558 0 8.7 115.6 0.2X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 9908 9927 29 10.1 99.1 1.0X -after 1900, vec on 3868 3881 18 25.9 38.7 2.6X -before 1900, vec off 11663 11671 7 8.6 116.6 0.8X -before 1900, vec on 5396 5408 10 18.5 54.0 1.8X +after 1900, vec off 10757 10772 13 9.3 107.6 1.0X +after 1900, vec on 3892 3899 11 25.7 38.9 2.8X +before 1900, vec off 13141 13195 52 7.6 131.4 0.8X +before 1900, vec on 6226 6301 129 16.1 62.3 1.7X diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt index 980897bc96b9b..eed620cdeced6 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt @@ -2,153 +2,153 @@ Rebasing dates/timestamps in Parquet datasource ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 13229 13229 0 7.6 132.3 1.0X -before 1582, noop 7736 7736 0 12.9 77.4 1.7X -after 1582, rebase EXCEPTION 21925 21925 0 4.6 219.2 0.6X -after 1582, rebase LEGACY 21714 21714 0 4.6 217.1 0.6X -after 1582, rebase CORRECTED 21864 21864 0 4.6 218.6 0.6X -before 1582, rebase LEGACY 16338 16338 0 6.1 163.4 0.8X -before 1582, rebase CORRECTED 16312 16312 0 6.1 163.1 0.8X - -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +after 1582, noop 13169 13169 0 7.6 131.7 1.0X +before 1582, noop 7787 7787 0 12.8 77.9 1.7X +after 1582, rebase EXCEPTION 21399 21399 0 4.7 214.0 0.6X +after 1582, rebase LEGACY 21530 21530 0 4.6 215.3 0.6X +after 1582, rebase CORRECTED 21579 21579 0 4.6 215.8 0.6X +before 1582, rebase LEGACY 16095 16095 0 6.2 160.9 0.8X +before 1582, rebase CORRECTED 16011 16011 0 6.2 160.1 0.8X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase EXCEPTION 11106 11144 39 9.0 111.1 1.0X -after 1582, vec off, rebase LEGACY 11476 11510 49 8.7 114.8 1.0X -after 1582, vec off, rebase CORRECTED 11373 11409 62 8.8 113.7 1.0X -after 1582, vec on, rebase EXCEPTION 2522 2548 31 39.7 25.2 4.4X -after 1582, vec on, rebase LEGACY 2628 2642 15 38.1 26.3 4.2X -after 1582, vec on, rebase CORRECTED 2490 2547 61 40.2 24.9 4.5X -before 1582, vec off, rebase LEGACY 11671 11743 88 8.6 116.7 1.0X -before 1582, vec off, rebase CORRECTED 11350 11379 41 8.8 113.5 1.0X -before 1582, vec on, rebase LEGACY 2855 2857 2 35.0 28.6 3.9X -before 1582, vec on, rebase CORRECTED 2559 2570 16 39.1 25.6 4.3X - -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +after 1582, vec off, rebase EXCEPTION 11598 11638 62 8.6 116.0 1.0X +after 1582, vec off, rebase LEGACY 11716 11730 17 8.5 117.2 1.0X +after 1582, vec off, rebase CORRECTED 11556 11616 52 8.7 115.6 1.0X +after 1582, vec on, rebase EXCEPTION 2528 2537 10 39.6 25.3 4.6X +after 1582, vec on, rebase LEGACY 2564 2569 6 39.0 25.6 4.5X +after 1582, vec on, rebase CORRECTED 2487 2534 44 40.2 24.9 4.7X +before 1582, vec off, rebase LEGACY 11740 11799 56 8.5 117.4 1.0X +before 1582, vec off, rebase CORRECTED 11606 11656 50 8.6 116.1 1.0X +before 1582, vec on, rebase LEGACY 2840 2871 27 35.2 28.4 4.1X +before 1582, vec on, rebase CORRECTED 2401 2429 31 41.6 24.0 4.8X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2239 2239 0 44.7 22.4 1.0X -before 1900, noop 2229 2229 0 44.9 22.3 1.0X -after 1900, rebase EXCEPTION 13150 13150 0 7.6 131.5 0.2X -after 1900, rebase LEGACY 13004 13004 0 7.7 130.0 0.2X -after 1900, rebase CORRECTED 13216 13216 0 7.6 132.2 0.2X -before 1900, rebase LEGACY 14965 14965 0 6.7 149.6 0.1X -before 1900, rebase CORRECTED 13218 13218 0 7.6 132.2 0.2X - -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2304 2304 0 43.4 23.0 1.0X +before 1900, noop 2242 2242 0 44.6 22.4 1.0X +after 1900, rebase EXCEPTION 13198 13198 0 7.6 132.0 0.2X +after 1900, rebase LEGACY 12894 12894 0 7.8 128.9 0.2X +after 1900, rebase CORRECTED 12991 12991 0 7.7 129.9 0.2X +before 1900, rebase LEGACY 14288 14288 0 7.0 142.9 0.2X +before 1900, rebase CORRECTED 12614 12614 0 7.9 126.1 0.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 17738 17763 26 5.6 177.4 1.0X -after 1900, vec off, rebase LEGACY 16932 16966 49 5.9 169.3 1.0X -after 1900, vec off, rebase CORRECTED 16738 16751 12 6.0 167.4 1.1X -after 1900, vec on, rebase EXCEPTION 4046 4072 34 24.7 40.5 4.4X -after 1900, vec on, rebase LEGACY 4081 4090 11 24.5 40.8 4.3X -after 1900, vec on, rebase CORRECTED 4015 4046 32 24.9 40.1 4.4X -before 1900, vec off, rebase LEGACY 19319 19369 43 5.2 193.2 0.9X -before 1900, vec off, rebase CORRECTED 16460 16479 19 6.1 164.6 1.1X -before 1900, vec on, rebase LEGACY 5956 5960 5 16.8 59.6 3.0X -before 1900, vec on, rebase CORRECTED 4036 4086 83 24.8 40.4 4.4X - -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 15238 15263 23 6.6 152.4 1.0X +after 1900, vec off, rebase LEGACY 14777 14793 22 6.8 147.8 1.0X +after 1900, vec off, rebase CORRECTED 14578 14650 107 6.9 145.8 1.0X +after 1900, vec on, rebase EXCEPTION 4051 4103 67 24.7 40.5 3.8X +after 1900, vec on, rebase LEGACY 4097 4123 34 24.4 41.0 3.7X +after 1900, vec on, rebase CORRECTED 4080 4092 16 24.5 40.8 3.7X +before 1900, vec off, rebase LEGACY 17402 17431 26 5.7 174.0 0.9X +before 1900, vec off, rebase CORRECTED 15337 15394 51 6.5 153.4 1.0X +before 1900, vec on, rebase LEGACY 6180 6197 17 16.2 61.8 2.5X +before 1900, vec on, rebase CORRECTED 4082 4094 14 24.5 40.8 3.7X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2262 2262 0 44.2 22.6 1.0X -before 1900, noop 2253 2253 0 44.4 22.5 1.0X -after 1900, rebase EXCEPTION 11901 11901 0 8.4 119.0 0.2X -after 1900, rebase LEGACY 11564 11564 0 8.6 115.6 0.2X -after 1900, rebase CORRECTED 11804 11804 0 8.5 118.0 0.2X -before 1900, rebase LEGACY 13264 13264 0 7.5 132.6 0.2X -before 1900, rebase CORRECTED 11816 11816 0 8.5 118.2 0.2X - -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2285 2285 0 43.8 22.9 1.0X +before 1900, noop 2287 2287 0 43.7 22.9 1.0X +after 1900, rebase EXCEPTION 12295 12295 0 8.1 122.9 0.2X +after 1900, rebase LEGACY 11653 11653 0 8.6 116.5 0.2X +after 1900, rebase CORRECTED 11718 11718 0 8.5 117.2 0.2X +before 1900, rebase LEGACY 13462 13462 0 7.4 134.6 0.2X +before 1900, rebase CORRECTED 11886 11886 0 8.4 118.9 0.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 15095 15151 58 6.6 150.9 1.0X -after 1900, vec off, rebase LEGACY 15243 15292 47 6.6 152.4 1.0X -after 1900, vec off, rebase CORRECTED 15200 15211 11 6.6 152.0 1.0X -after 1900, vec on, rebase EXCEPTION 3758 3772 14 26.6 37.6 4.0X -after 1900, vec on, rebase LEGACY 3775 3811 39 26.5 37.8 4.0X -after 1900, vec on, rebase CORRECTED 3784 3796 16 26.4 37.8 4.0X -before 1900, vec off, rebase LEGACY 17548 17583 41 5.7 175.5 0.9X -before 1900, vec off, rebase CORRECTED 15155 15166 10 6.6 151.6 1.0X -before 1900, vec on, rebase LEGACY 5640 5665 43 17.7 56.4 2.7X -before 1900, vec on, rebase CORRECTED 3803 3856 64 26.3 38.0 4.0X - -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 14824 14877 57 6.7 148.2 1.0X +after 1900, vec off, rebase LEGACY 14876 14899 26 6.7 148.8 1.0X +after 1900, vec off, rebase CORRECTED 14924 14947 24 6.7 149.2 1.0X +after 1900, vec on, rebase EXCEPTION 3813 3817 5 26.2 38.1 3.9X +after 1900, vec on, rebase LEGACY 3829 3855 28 26.1 38.3 3.9X +after 1900, vec on, rebase CORRECTED 3803 3811 11 26.3 38.0 3.9X +before 1900, vec off, rebase LEGACY 17141 17177 53 5.8 171.4 0.9X +before 1900, vec off, rebase CORRECTED 14916 14936 26 6.7 149.2 1.0X +before 1900, vec on, rebase LEGACY 5638 5656 15 17.7 56.4 2.6X +before 1900, vec on, rebase CORRECTED 3792 3820 43 26.4 37.9 3.9X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2253 2253 0 44.4 22.5 1.0X -before 1900, noop 2253 2253 0 44.4 22.5 1.0X -after 1900, rebase EXCEPTION 11471 11471 0 8.7 114.7 0.2X -after 1900, rebase LEGACY 11173 11173 0 8.9 111.7 0.2X -after 1900, rebase CORRECTED 11497 11497 0 8.7 115.0 0.2X -before 1900, rebase LEGACY 13368 13368 0 7.5 133.7 0.2X -before 1900, rebase CORRECTED 11570 11570 0 8.6 115.7 0.2X - -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2283 2283 0 43.8 22.8 1.0X +before 1900, noop 2286 2286 0 43.8 22.9 1.0X +after 1900, rebase EXCEPTION 11040 11040 0 9.1 110.4 0.2X +after 1900, rebase LEGACY 11421 11421 0 8.8 114.2 0.2X +after 1900, rebase CORRECTED 11132 11132 0 9.0 111.3 0.2X +before 1900, rebase LEGACY 13097 13097 0 7.6 131.0 0.2X +before 1900, rebase CORRECTED 11359 11359 0 8.8 113.6 0.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 15315 15363 82 6.5 153.1 1.0X -after 1900, vec off, rebase LEGACY 15462 15506 70 6.5 154.6 1.0X -after 1900, vec off, rebase CORRECTED 15394 15426 38 6.5 153.9 1.0X -after 1900, vec on, rebase EXCEPTION 3910 3941 28 25.6 39.1 3.9X -after 1900, vec on, rebase LEGACY 4503 4516 22 22.2 45.0 3.4X -after 1900, vec on, rebase CORRECTED 3915 3919 5 25.5 39.1 3.9X -before 1900, vec off, rebase LEGACY 17595 17636 46 5.7 176.0 0.9X -before 1900, vec off, rebase CORRECTED 15344 15354 17 6.5 153.4 1.0X -before 1900, vec on, rebase LEGACY 6188 6202 23 16.2 61.9 2.5X -before 1900, vec on, rebase CORRECTED 3890 3899 8 25.7 38.9 3.9X +after 1900, vec off, rebase EXCEPTION 15017 15053 38 6.7 150.2 1.0X +after 1900, vec off, rebase LEGACY 14941 15013 75 6.7 149.4 1.0X +after 1900, vec off, rebase CORRECTED 15057 15070 17 6.6 150.6 1.0X +after 1900, vec on, rebase EXCEPTION 3942 3949 7 25.4 39.4 3.8X +after 1900, vec on, rebase LEGACY 4605 4628 26 21.7 46.1 3.3X +after 1900, vec on, rebase CORRECTED 4002 4027 22 25.0 40.0 3.8X +before 1900, vec off, rebase LEGACY 17121 17169 47 5.8 171.2 0.9X +before 1900, vec off, rebase CORRECTED 15086 15132 42 6.6 150.9 1.0X +before 1900, vec on, rebase LEGACY 6262 6271 10 16.0 62.6 2.4X +before 1900, vec on, rebase CORRECTED 3942 3960 24 25.4 39.4 3.8X ================================================================================================ Rebasing dates/timestamps in ORC datasource ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 13342 13342 0 7.5 133.4 1.0X -before 1582, noop 7875 7875 0 12.7 78.7 1.7X -after 1582 16884 16884 0 5.9 168.8 0.8X -before 1582 11948 11948 0 8.4 119.5 1.1X +after 1582, noop 13322 13322 0 7.5 133.2 1.0X +before 1582, noop 7967 7967 0 12.6 79.7 1.7X +after 1582 17193 17193 0 5.8 171.9 0.8X +before 1582 12729 12729 0 7.9 127.3 1.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 8890 8914 29 11.2 88.9 1.0X -after 1582, vec on 2459 2499 64 40.7 24.6 3.6X -before 1582, vec off 8453 8462 8 11.8 84.5 1.1X -before 1582, vec on 2618 2636 16 38.2 26.2 3.4X +after 1582, vec off 8797 8843 71 11.4 88.0 1.0X +after 1582, vec on 2457 2469 14 40.7 24.6 3.6X +before 1582, vec off 8555 8572 16 11.7 85.5 1.0X +before 1582, vec on 2613 2621 11 38.3 26.1 3.4X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2181 2181 0 45.9 21.8 1.0X -before 1900, noop 2161 2161 0 46.3 21.6 1.0X -after 1900 10953 10953 0 9.1 109.5 0.2X -before 1900 12293 12293 0 8.1 122.9 0.2X +after 1900, noop 2182 2182 0 45.8 21.8 1.0X +before 1900, noop 2169 2169 0 46.1 21.7 1.0X +after 1900 10099 10099 0 9.9 101.0 0.2X +before 1900 12162 12162 0 8.2 121.6 0.2X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 10398 10414 25 9.6 104.0 1.0X -after 1900, vec on 4165 4173 7 24.0 41.7 2.5X -before 1900, vec off 12138 12211 112 8.2 121.4 0.9X -before 1900, vec on 5781 5784 4 17.3 57.8 1.8X +after 1900, vec off 9898 9923 26 10.1 99.0 1.0X +after 1900, vec on 4013 4048 55 24.9 40.1 2.5X +before 1900, vec off 11962 11980 18 8.4 119.6 0.8X +before 1900, vec on 5608 5635 43 17.8 56.1 1.8X diff --git a/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt b/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt index d74eb426cf341..a7aebf3e61025 100644 --- a/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt @@ -1,8 +1,8 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UTF-32 47715 47833 167 0.2 4771.5 1.0X -UTF-16 57379 57408 42 0.2 5737.9 0.8X -UTF-8 2840 2872 45 3.5 284.0 16.8X +UTF-32 64447 64482 50 0.2 6444.7 1.0X +UTF-16 60035 60070 49 0.2 6003.5 1.1X +UTF-8 33512 33524 16 0.3 3351.2 1.9X diff --git a/sql/core/benchmarks/EncodeBenchmark-results.txt b/sql/core/benchmarks/EncodeBenchmark-results.txt index 5fdbbf72d7e77..bd888d90c17de 100644 --- a/sql/core/benchmarks/EncodeBenchmark-results.txt +++ b/sql/core/benchmarks/EncodeBenchmark-results.txt @@ -1,8 +1,8 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UTF-32 29962 30019 81 0.3 2996.2 1.0X -UTF-16 47699 47702 3 0.2 4769.9 0.6X -UTF-8 3112 3154 59 3.2 311.2 9.6X +UTF-32 33442 33457 21 0.3 3344.2 1.0X +UTF-16 50707 50731 35 0.2 5070.7 0.7X +UTF-8 30829 30847 25 0.3 3082.9 1.1X diff --git a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt index 08f3d54f5ae81..3bc77b17102fe 100644 --- a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt @@ -2,44 +2,44 @@ WITHOUT SPILL ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array with 100000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ArrayBuffer 2456 2456 0 41.7 24.0 1.0X -ExternalAppendOnlyUnsafeRowArray 3572 3595 33 28.7 34.9 0.7X +ArrayBuffer 2569 2579 14 39.9 25.1 1.0X +ExternalAppendOnlyUnsafeRowArray 3494 3513 27 29.3 34.1 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ArrayBuffer 5511 5519 11 47.6 21.0 1.0X -ExternalAppendOnlyUnsafeRowArray 12331 12382 73 21.3 47.0 0.4X +ArrayBuffer 5447 5500 75 48.1 20.8 1.0X +ExternalAppendOnlyUnsafeRowArray 11886 11907 29 22.1 45.3 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array with 30000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ArrayBuffer 10731 10759 39 45.8 21.8 1.0X -ExternalAppendOnlyUnsafeRowArray 18516 18568 72 26.5 37.7 0.6X +ArrayBuffer 10664 10664 1 46.1 21.7 1.0X +ExternalAppendOnlyUnsafeRowArray 17290 17397 151 28.4 35.2 0.6X ================================================================================================ WITH SPILL ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Spilling with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeExternalSorter 8284 8328 63 31.6 31.6 1.0X -ExternalAppendOnlyUnsafeRowArray 6615 6624 14 39.6 25.2 1.3X +UnsafeExternalSorter 8436 8440 6 31.1 32.2 1.0X +ExternalAppendOnlyUnsafeRowArray 6686 6713 39 39.2 25.5 1.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Spilling with 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeExternalSorter 5 5 0 32.8 30.5 1.0X -ExternalAppendOnlyUnsafeRowArray 4 4 0 38.5 26.0 1.2X +UnsafeExternalSorter 5 5 0 33.4 29.9 1.0X +ExternalAppendOnlyUnsafeRowArray 4 4 0 39.5 25.3 1.2X diff --git a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt index ca447f9e97dbc..cd6241caf25b0 100644 --- a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt +++ b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt @@ -2,44 +2,44 @@ WITHOUT SPILL ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array with 100000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ArrayBuffer 2496 2499 4 41.0 24.4 1.0X -ExternalAppendOnlyUnsafeRowArray 3495 3513 24 29.3 34.1 0.7X +ArrayBuffer 2453 2458 8 41.7 24.0 1.0X +ExternalAppendOnlyUnsafeRowArray 3401 3413 18 30.1 33.2 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ArrayBuffer 5277 5284 10 49.7 20.1 1.0X -ExternalAppendOnlyUnsafeRowArray 12169 12171 3 21.5 46.4 0.4X +ArrayBuffer 5330 5332 3 49.2 20.3 1.0X +ExternalAppendOnlyUnsafeRowArray 12411 12462 72 21.1 47.3 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Array with 30000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ArrayBuffer 10107 10110 4 48.6 20.6 1.0X -ExternalAppendOnlyUnsafeRowArray 17021 17035 20 28.9 34.6 0.6X +ArrayBuffer 10236 10250 20 48.0 20.8 1.0X +ExternalAppendOnlyUnsafeRowArray 16811 16821 15 29.2 34.2 0.6X ================================================================================================ WITH SPILL ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Spilling with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeExternalSorter 8435 8499 89 31.1 32.2 1.0X -ExternalAppendOnlyUnsafeRowArray 7126 7131 6 36.8 27.2 1.2X +UnsafeExternalSorter 8715 8747 45 30.1 33.2 1.0X +ExternalAppendOnlyUnsafeRowArray 6495 6507 16 40.4 24.8 1.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Spilling with 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeExternalSorter 5 5 0 34.5 29.0 1.0X -ExternalAppendOnlyUnsafeRowArray 4 4 0 36.6 27.3 1.1X +UnsafeExternalSorter 5 5 0 33.5 29.9 1.0X +ExternalAppendOnlyUnsafeRowArray 4 4 0 40.5 24.7 1.2X diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt index 78df1f6557073..9420529bb5166 100644 --- a/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt @@ -1,104 +1,104 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 278 295 28 35.9 27.8 1.0X -YEAR of timestamp 604 616 15 16.6 60.4 0.5X -YEAROFWEEK of timestamp 648 659 10 15.4 64.8 0.4X -QUARTER of timestamp 653 672 30 15.3 65.3 0.4X -MONTH of timestamp 572 581 11 17.5 57.2 0.5X -WEEK of timestamp 865 868 3 11.6 86.5 0.3X -DAY of timestamp 576 583 9 17.4 57.6 0.5X -DAYOFWEEK of timestamp 755 759 7 13.3 75.5 0.4X -DOW of timestamp 751 775 39 13.3 75.1 0.4X -DOW_ISO of timestamp 709 716 6 14.1 70.9 0.4X -DAYOFWEEK_ISO of timestamp 708 709 1 14.1 70.8 0.4X -DOY of timestamp 603 614 18 16.6 60.3 0.5X -HOUR of timestamp 475 479 3 21.1 47.5 0.6X -MINUTE of timestamp 479 479 1 20.9 47.9 0.6X -SECOND of timestamp 533 536 3 18.7 53.3 0.5X +cast to timestamp 260 281 28 38.5 26.0 1.0X +YEAR of timestamp 660 684 27 15.1 66.0 0.4X +YEAROFWEEK of timestamp 621 623 2 16.1 62.1 0.4X +QUARTER of timestamp 635 637 2 15.8 63.5 0.4X +MONTH of timestamp 553 555 2 18.1 55.3 0.5X +WEEK of timestamp 847 882 41 11.8 84.7 0.3X +DAY of timestamp 561 562 1 17.8 56.1 0.5X +DAYOFWEEK of timestamp 739 743 3 13.5 73.9 0.4X +DOW of timestamp 744 744 1 13.4 74.4 0.3X +DOW_ISO of timestamp 670 676 9 14.9 67.0 0.4X +DAYOFWEEK_ISO of timestamp 668 670 2 15.0 66.8 0.4X +DOY of timestamp 596 597 1 16.8 59.6 0.4X +HOUR of timestamp 465 468 3 21.5 46.5 0.6X +MINUTE of timestamp 464 467 2 21.5 46.4 0.6X +SECOND of timestamp 531 537 6 18.8 53.1 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 251 251 1 39.9 25.1 1.0X -YEAR of timestamp 537 539 2 18.6 53.7 0.5X -YEAROFWEEK of timestamp 624 633 8 16.0 62.4 0.4X -QUARTER of timestamp 634 635 1 15.8 63.4 0.4X -MONTH of timestamp 556 564 10 18.0 55.6 0.5X -WEEK of timestamp 854 859 4 11.7 85.4 0.3X -DAY of timestamp 572 579 11 17.5 57.2 0.4X -DAYOFWEEK of timestamp 741 747 5 13.5 74.1 0.3X -DOW of timestamp 741 743 2 13.5 74.1 0.3X -DOW_ISO of timestamp 703 704 1 14.2 70.3 0.4X -DAYOFWEEK_ISO of timestamp 701 701 1 14.3 70.1 0.4X -DOY of timestamp 592 595 3 16.9 59.2 0.4X -HOUR of timestamp 474 476 2 21.1 47.4 0.5X -MINUTE of timestamp 476 479 5 21.0 47.6 0.5X -SECOND of timestamp 528 530 2 18.9 52.8 0.5X +cast to timestamp 234 244 9 42.8 23.4 1.0X +YEAR of timestamp 532 538 6 18.8 53.2 0.4X +YEAROFWEEK of timestamp 602 606 3 16.6 60.2 0.4X +QUARTER of timestamp 618 625 7 16.2 61.8 0.4X +MONTH of timestamp 540 549 10 18.5 54.0 0.4X +WEEK of timestamp 835 837 2 12.0 83.5 0.3X +DAY of timestamp 553 558 6 18.1 55.3 0.4X +DAYOFWEEK of timestamp 732 735 3 13.7 73.2 0.3X +DOW of timestamp 733 736 3 13.6 73.3 0.3X +DOW_ISO of timestamp 664 670 8 15.1 66.4 0.4X +DAYOFWEEK_ISO of timestamp 664 668 6 15.1 66.4 0.4X +DOY of timestamp 591 593 1 16.9 59.1 0.4X +HOUR of timestamp 461 468 7 21.7 46.1 0.5X +MINUTE of timestamp 462 464 2 21.6 46.2 0.5X +SECOND of timestamp 530 530 1 18.9 53.0 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 523 526 4 19.1 52.3 1.0X -YEAR of date 533 535 3 18.8 53.3 1.0X -YEAROFWEEK of date 628 639 17 15.9 62.8 0.8X -QUARTER of date 629 631 3 15.9 62.9 0.8X -MONTH of date 566 577 18 17.7 56.6 0.9X -WEEK of date 859 872 21 11.6 85.9 0.6X -DAY of date 572 590 18 17.5 57.2 0.9X -DAYOFWEEK of date 741 746 9 13.5 74.1 0.7X -DOW of date 740 766 45 13.5 74.0 0.7X -DOW_ISO of date 700 707 10 14.3 70.0 0.7X -DAYOFWEEK_ISO of date 698 703 7 14.3 69.8 0.7X -DOY of date 592 596 5 16.9 59.2 0.9X -HOUR of date 993 1014 24 10.1 99.3 0.5X -MINUTE of date 995 1003 10 10.0 99.5 0.5X -SECOND of date 1058 1058 0 9.5 105.8 0.5X +cast to date 511 514 2 19.6 51.1 1.0X +YEAR of date 526 529 3 19.0 52.6 1.0X +YEAROFWEEK of date 601 607 8 16.6 60.1 0.9X +QUARTER of date 617 627 9 16.2 61.7 0.8X +MONTH of date 537 538 1 18.6 53.7 1.0X +WEEK of date 836 847 14 12.0 83.6 0.6X +DAY of date 551 557 9 18.2 55.1 0.9X +DAYOFWEEK of date 734 742 7 13.6 73.4 0.7X +DOW of date 731 734 4 13.7 73.1 0.7X +DOW_ISO of date 664 667 5 15.1 66.4 0.8X +DAYOFWEEK_ISO of date 661 666 4 15.1 66.1 0.8X +DOY of date 588 593 8 17.0 58.8 0.9X +HOUR of date 985 986 1 10.2 98.5 0.5X +MINUTE of date 980 991 14 10.2 98.0 0.5X +SECOND of date 1035 1043 13 9.7 103.5 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 525 574 83 19.1 52.5 1.0X -YEAR of date 539 540 1 18.6 53.9 1.0X -YEAROFWEEK of date 628 631 5 15.9 62.8 0.8X -QUARTER of date 629 640 15 15.9 62.9 0.8X -MONTH of date 553 555 2 18.1 55.3 0.9X -WEEK of date 850 852 1 11.8 85.0 0.6X -DAY of date 568 574 10 17.6 56.8 0.9X -DAYOFWEEK of date 740 741 1 13.5 74.0 0.7X -DOW of date 739 746 6 13.5 73.9 0.7X -DOW_ISO of date 699 703 4 14.3 69.9 0.8X -DAYOFWEEK_ISO of date 699 700 1 14.3 69.9 0.8X -DOY of date 590 592 3 17.0 59.0 0.9X -HOUR of date 991 992 0 10.1 99.1 0.5X -MINUTE of date 989 990 1 10.1 98.9 0.5X -SECOND of date 1058 1062 5 9.4 105.8 0.5X +cast to date 512 515 3 19.5 51.2 1.0X +YEAR of date 526 534 8 19.0 52.6 1.0X +YEAROFWEEK of date 600 602 2 16.7 60.0 0.9X +QUARTER of date 616 623 11 16.2 61.6 0.8X +MONTH of date 538 543 9 18.6 53.8 1.0X +WEEK of date 837 838 1 12.0 83.7 0.6X +DAY of date 550 553 3 18.2 55.0 0.9X +DAYOFWEEK of date 734 739 5 13.6 73.4 0.7X +DOW of date 733 759 43 13.7 73.3 0.7X +DOW_ISO of date 664 668 3 15.1 66.4 0.8X +DAYOFWEEK_ISO of date 665 666 0 15.0 66.5 0.8X +DOY of date 593 594 1 16.9 59.3 0.9X +HOUR of date 983 986 3 10.2 98.3 0.5X +MINUTE of date 979 981 3 10.2 97.9 0.5X +SECOND of date 1038 1039 1 9.6 103.8 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke extract for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to interval 738 741 2 13.5 73.8 1.0X -YEAR of interval 718 721 4 13.9 71.8 1.0X -MONTH of interval 721 725 3 13.9 72.1 1.0X -DAY of interval 718 722 4 13.9 71.8 1.0X -HOUR of interval 730 733 4 13.7 73.0 1.0X -MINUTE of interval 724 728 3 13.8 72.4 1.0X -SECOND of interval 775 785 13 12.9 77.5 1.0X +cast to interval 723 728 5 13.8 72.3 1.0X +YEAR of interval 717 718 2 13.9 71.7 1.0X +MONTH of interval 720 722 2 13.9 72.0 1.0X +DAY of interval 716 719 2 14.0 71.6 1.0X +HOUR of interval 729 731 2 13.7 72.9 1.0X +MINUTE of interval 725 726 1 13.8 72.5 1.0X +SECOND of interval 769 771 2 13.0 76.9 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to interval 742 745 4 13.5 74.2 1.0X -YEAR of interval 724 725 0 13.8 72.4 1.0X -MONTH of interval 722 724 2 13.9 72.2 1.0X -DAY of interval 728 730 1 13.7 72.8 1.0X -HOUR of interval 731 739 8 13.7 73.1 1.0X -MINUTE of interval 733 740 11 13.6 73.3 1.0X -SECOND of interval 785 800 16 12.7 78.5 0.9X +cast to interval 728 729 1 13.7 72.8 1.0X +YEAR of interval 722 722 1 13.9 72.2 1.0X +MONTH of interval 718 723 5 13.9 71.8 1.0X +DAY of interval 713 718 7 14.0 71.3 1.0X +HOUR of interval 726 727 2 13.8 72.6 1.0X +MINUTE of interval 734 736 3 13.6 73.4 1.0X +SECOND of interval 770 771 2 13.0 77.0 0.9X diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index a60f24142bc60..b472b3fea998b 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,104 +1,104 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 286 311 23 34.9 28.6 1.0X -YEAR of timestamp 792 798 8 12.6 79.2 0.4X -YEAROFWEEK of timestamp 879 901 19 11.4 87.9 0.3X -QUARTER of timestamp 842 849 9 11.9 84.2 0.3X -MONTH of timestamp 799 804 5 12.5 79.9 0.4X -WEEK of timestamp 1104 1107 3 9.1 110.4 0.3X -DAY of timestamp 780 788 9 12.8 78.0 0.4X -DAYOFWEEK of timestamp 967 973 5 10.3 96.7 0.3X -DOW of timestamp 965 970 5 10.4 96.5 0.3X -DOW_ISO of timestamp 1022 1024 4 9.8 102.2 0.3X -DAYOFWEEK_ISO of timestamp 1022 1024 4 9.8 102.2 0.3X -DOY of timestamp 844 855 13 11.8 84.4 0.3X -HOUR of timestamp 558 563 5 17.9 55.8 0.5X -MINUTE of timestamp 564 564 0 17.7 56.4 0.5X -SECOND of timestamp 657 658 1 15.2 65.7 0.4X +cast to timestamp 243 273 33 41.2 24.3 1.0X +YEAR of timestamp 780 785 5 12.8 78.0 0.3X +YEAROFWEEK of timestamp 849 883 36 11.8 84.9 0.3X +QUARTER of timestamp 798 799 0 12.5 79.8 0.3X +MONTH of timestamp 758 762 4 13.2 75.8 0.3X +WEEK of timestamp 1113 1118 6 9.0 111.3 0.2X +DAY of timestamp 752 757 5 13.3 75.2 0.3X +DAYOFWEEK of timestamp 940 945 4 10.6 94.0 0.3X +DOW of timestamp 940 949 14 10.6 94.0 0.3X +DOW_ISO of timestamp 997 1004 11 10.0 99.7 0.2X +DAYOFWEEK_ISO of timestamp 991 995 4 10.1 99.1 0.2X +DOY of timestamp 811 816 6 12.3 81.1 0.3X +HOUR of timestamp 536 543 7 18.7 53.6 0.5X +MINUTE of timestamp 532 541 7 18.8 53.2 0.5X +SECOND of timestamp 636 648 21 15.7 63.6 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 245 248 4 40.8 24.5 1.0X -YEAR of timestamp 785 788 3 12.7 78.5 0.3X -YEAROFWEEK of timestamp 859 859 0 11.6 85.9 0.3X -QUARTER of timestamp 818 819 1 12.2 81.8 0.3X -MONTH of timestamp 776 781 4 12.9 77.6 0.3X -WEEK of timestamp 1099 1109 17 9.1 109.9 0.2X -DAY of timestamp 778 780 2 12.9 77.8 0.3X -DAYOFWEEK of timestamp 964 966 2 10.4 96.4 0.3X -DOW of timestamp 964 966 3 10.4 96.4 0.3X -DOW_ISO of timestamp 1015 1020 5 9.9 101.5 0.2X -DAYOFWEEK_ISO of timestamp 1012 1014 3 9.9 101.2 0.2X -DOY of timestamp 847 850 6 11.8 84.7 0.3X -HOUR of timestamp 560 562 4 17.9 56.0 0.4X -MINUTE of timestamp 560 569 11 17.8 56.0 0.4X -SECOND of timestamp 656 660 6 15.2 65.6 0.4X +cast to timestamp 216 223 10 46.3 21.6 1.0X +YEAR of timestamp 767 770 5 13.0 76.7 0.3X +YEAROFWEEK of timestamp 830 840 14 12.0 83.0 0.3X +QUARTER of timestamp 786 791 4 12.7 78.6 0.3X +MONTH of timestamp 758 761 3 13.2 75.8 0.3X +WEEK of timestamp 1110 1119 8 9.0 111.0 0.2X +DAY of timestamp 759 760 1 13.2 75.9 0.3X +DAYOFWEEK of timestamp 939 942 5 10.7 93.9 0.2X +DOW of timestamp 937 938 1 10.7 93.7 0.2X +DOW_ISO of timestamp 986 987 1 10.1 98.6 0.2X +DAYOFWEEK_ISO of timestamp 985 990 4 10.1 98.5 0.2X +DOY of timestamp 819 824 4 12.2 81.9 0.3X +HOUR of timestamp 531 541 12 18.8 53.1 0.4X +MINUTE of timestamp 528 532 6 19.0 52.8 0.4X +SECOND of timestamp 635 638 5 15.7 63.5 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 727 729 4 13.8 72.7 1.0X -YEAR of date 777 787 10 12.9 77.7 0.9X -YEAROFWEEK of date 852 858 8 11.7 85.2 0.9X -QUARTER of date 813 815 3 12.3 81.3 0.9X -MONTH of date 772 775 4 12.9 77.2 0.9X -WEEK of date 1091 1093 2 9.2 109.1 0.7X -DAY of date 777 778 1 12.9 77.7 0.9X -DAYOFWEEK of date 963 965 3 10.4 96.3 0.8X -DOW of date 960 963 3 10.4 96.0 0.8X -DOW_ISO of date 1017 1018 1 9.8 101.7 0.7X -DAYOFWEEK_ISO of date 1010 1013 2 9.9 101.0 0.7X -DOY of date 840 841 1 11.9 84.0 0.9X -HOUR of date 1288 1295 8 7.8 128.8 0.6X -MINUTE of date 1299 1313 20 7.7 129.9 0.6X -SECOND of date 1383 1393 10 7.2 138.3 0.5X +cast to date 701 710 12 14.3 70.1 1.0X +YEAR of date 766 770 4 13.1 76.6 0.9X +YEAROFWEEK of date 824 828 6 12.1 82.4 0.9X +QUARTER of date 787 790 3 12.7 78.7 0.9X +MONTH of date 756 756 1 13.2 75.6 0.9X +WEEK of date 1112 1113 1 9.0 111.2 0.6X +DAY of date 756 758 3 13.2 75.6 0.9X +DAYOFWEEK of date 940 941 1 10.6 94.0 0.7X +DOW of date 942 944 2 10.6 94.2 0.7X +DOW_ISO of date 986 1001 21 10.1 98.6 0.7X +DAYOFWEEK_ISO of date 984 991 7 10.2 98.4 0.7X +DOY of date 819 827 7 12.2 81.9 0.9X +HOUR of date 1278 1290 10 7.8 127.8 0.5X +MINUTE of date 1290 1293 2 7.8 129.0 0.5X +SECOND of date 1374 1376 3 7.3 137.4 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 731 735 3 13.7 73.1 1.0X -YEAR of date 779 784 7 12.8 77.9 0.9X -YEAROFWEEK of date 851 856 6 11.8 85.1 0.9X -QUARTER of date 816 817 1 12.3 81.6 0.9X -MONTH of date 771 774 4 13.0 77.1 0.9X -WEEK of date 1095 1097 4 9.1 109.5 0.7X -DAY of date 774 777 3 12.9 77.4 0.9X -DAYOFWEEK of date 960 961 2 10.4 96.0 0.8X -DOW of date 959 962 4 10.4 95.9 0.8X -DOW_ISO of date 1009 1011 2 9.9 100.9 0.7X -DAYOFWEEK_ISO of date 1009 1011 2 9.9 100.9 0.7X -DOY of date 843 844 1 11.9 84.3 0.9X -HOUR of date 1289 1290 1 7.8 128.9 0.6X -MINUTE of date 1285 1289 5 7.8 128.5 0.6X -SECOND of date 1390 1395 5 7.2 139.0 0.5X +cast to date 711 722 10 14.1 71.1 1.0X +YEAR of date 758 760 3 13.2 75.8 0.9X +YEAROFWEEK of date 826 830 6 12.1 82.6 0.9X +QUARTER of date 783 785 3 12.8 78.3 0.9X +MONTH of date 755 756 1 13.2 75.5 0.9X +WEEK of date 1102 1115 11 9.1 110.2 0.6X +DAY of date 749 753 3 13.3 74.9 0.9X +DAYOFWEEK of date 940 941 1 10.6 94.0 0.8X +DOW of date 934 936 3 10.7 93.4 0.8X +DOW_ISO of date 988 988 0 10.1 98.8 0.7X +DAYOFWEEK_ISO of date 988 998 18 10.1 98.8 0.7X +DOY of date 812 817 7 12.3 81.2 0.9X +HOUR of date 1274 1281 6 7.8 127.4 0.6X +MINUTE of date 1282 1287 6 7.8 128.2 0.6X +SECOND of date 1382 1384 2 7.2 138.2 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke extract for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to interval 1127 1130 2 8.9 112.7 1.0X -YEAR of interval 1119 1123 5 8.9 111.9 1.0X -MONTH of interval 1117 1118 2 9.0 111.7 1.0X -DAY of interval 1124 1126 2 8.9 112.4 1.0X -HOUR of interval 1119 1120 2 8.9 111.9 1.0X -MINUTE of interval 1119 1122 3 8.9 111.9 1.0X -SECOND of interval 1216 1224 10 8.2 121.6 0.9X +cast to interval 1093 1095 2 9.2 109.3 1.0X +YEAR of interval 1085 1086 2 9.2 108.5 1.0X +MONTH of interval 1075 1075 0 9.3 107.5 1.0X +DAY of interval 1071 1076 5 9.3 107.1 1.0X +HOUR of interval 1075 1082 7 9.3 107.5 1.0X +MINUTE of interval 1113 1122 12 9.0 111.3 1.0X +SECOND of interval 1179 1181 3 8.5 117.9 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to interval 1130 1131 2 8.8 113.0 1.0X -YEAR of interval 1113 1116 2 9.0 111.3 1.0X -MONTH of interval 1122 1122 1 8.9 112.2 1.0X -DAY of interval 1122 1124 4 8.9 112.2 1.0X -HOUR of interval 1119 1121 2 8.9 111.9 1.0X -MINUTE of interval 1118 1125 9 8.9 111.8 1.0X -SECOND of interval 1208 1211 3 8.3 120.8 0.9X +cast to interval 1080 1082 3 9.3 108.0 1.0X +YEAR of interval 1077 1080 3 9.3 107.7 1.0X +MONTH of interval 1080 1081 1 9.3 108.0 1.0X +DAY of interval 1069 1070 2 9.4 106.9 1.0X +HOUR of interval 1073 1074 2 9.3 107.3 1.0X +MINUTE of interval 1122 1125 5 8.9 112.2 1.0X +SECOND of interval 1180 1184 4 8.5 118.0 0.9X diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt index 17ffe9f3fab41..417979cfb62a7 100644 --- a/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt @@ -2,733 +2,733 @@ Pushdown for many distinct value case ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6345 6437 61 2.5 403.4 1.0X -Parquet Vectorized (Pushdown) 341 363 12 46.2 21.7 18.6X -Native ORC Vectorized 5118 5274 131 3.1 325.4 1.2X -Native ORC Vectorized (Pushdown) 318 323 5 49.5 20.2 20.0X +Parquet Vectorized 6457 6500 40 2.4 410.5 1.0X +Parquet Vectorized (Pushdown) 362 383 16 43.4 23.0 17.8X +Native ORC Vectorized 5171 5288 107 3.0 328.8 1.2X +Native ORC Vectorized (Pushdown) 314 323 9 50.1 20.0 20.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6333 6355 22 2.5 402.7 1.0X -Parquet Vectorized (Pushdown) 331 347 9 47.5 21.1 19.1X -Native ORC Vectorized 5259 5281 25 3.0 334.4 1.2X -Native ORC Vectorized (Pushdown) 310 330 19 50.7 19.7 20.4X +Parquet Vectorized 6405 6424 20 2.5 407.2 1.0X +Parquet Vectorized (Pushdown) 314 326 9 50.0 20.0 20.4X +Native ORC Vectorized 5221 5259 39 3.0 331.9 1.2X +Native ORC Vectorized (Pushdown) 299 317 13 52.6 19.0 21.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6378 6405 21 2.5 405.5 1.0X -Parquet Vectorized (Pushdown) 315 324 10 50.0 20.0 20.3X -Native ORC Vectorized 5359 5364 5 2.9 340.7 1.2X -Native ORC Vectorized (Pushdown) 301 308 5 52.2 19.2 21.2X +Parquet Vectorized 6432 6453 22 2.4 408.9 1.0X +Parquet Vectorized (Pushdown) 298 310 9 52.8 18.9 21.6X +Native ORC Vectorized 5377 5388 8 2.9 341.9 1.2X +Native ORC Vectorized (Pushdown) 303 312 7 51.8 19.3 21.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6371 6394 16 2.5 405.1 1.0X -Parquet Vectorized (Pushdown) 310 315 7 50.7 19.7 20.5X -Native ORC Vectorized 5354 5384 25 2.9 340.4 1.2X -Native ORC Vectorized (Pushdown) 291 299 6 54.1 18.5 21.9X +Parquet Vectorized 6433 6478 25 2.4 409.0 1.0X +Parquet Vectorized (Pushdown) 295 302 6 53.4 18.7 21.8X +Native ORC Vectorized 5363 5368 5 2.9 341.0 1.2X +Native ORC Vectorized (Pushdown) 286 294 7 55.0 18.2 22.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6427 6456 19 2.4 408.6 1.0X -Parquet Vectorized (Pushdown) 310 312 1 50.7 19.7 20.7X -Native ORC Vectorized 5240 5253 10 3.0 333.2 1.2X -Native ORC Vectorized (Pushdown) 288 301 11 54.7 18.3 22.3X +Parquet Vectorized 6457 6470 11 2.4 410.5 1.0X +Parquet Vectorized (Pushdown) 293 300 6 53.6 18.7 22.0X +Native ORC Vectorized 5356 5366 8 2.9 340.5 1.2X +Native ORC Vectorized (Pushdown) 288 295 5 54.6 18.3 22.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 13717 13822 93 1.1 872.1 1.0X -Parquet Vectorized (Pushdown) 13817 13833 19 1.1 878.4 1.0X -Native ORC Vectorized 12689 12724 34 1.2 806.7 1.1X -Native ORC Vectorized (Pushdown) 12802 12812 9 1.2 813.9 1.1X +Parquet Vectorized 14274 14374 112 1.1 907.5 1.0X +Parquet Vectorized (Pushdown) 14553 14581 27 1.1 925.2 1.0X +Native ORC Vectorized 13537 13553 20 1.2 860.7 1.1X +Native ORC Vectorized (Pushdown) 13620 13650 40 1.2 865.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6152 6231 51 2.6 391.2 1.0X -Parquet Vectorized (Pushdown) 306 315 9 51.4 19.5 20.1X -Native ORC Vectorized 4694 4761 104 3.4 298.4 1.3X -Native ORC Vectorized (Pushdown) 274 282 9 57.4 17.4 22.5X +Parquet Vectorized 6163 6242 78 2.6 391.9 1.0X +Parquet Vectorized (Pushdown) 277 290 12 56.8 17.6 22.2X +Native ORC Vectorized 4740 4795 56 3.3 301.3 1.3X +Native ORC Vectorized (Pushdown) 281 290 6 56.0 17.8 22.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5996 6015 14 2.6 381.2 1.0X -Parquet Vectorized (Pushdown) 302 311 7 52.2 19.2 19.9X -Native ORC Vectorized 4684 4691 5 3.4 297.8 1.3X -Native ORC Vectorized (Pushdown) 281 290 9 56.0 17.9 21.3X +Parquet Vectorized 6072 6080 11 2.6 386.1 1.0X +Parquet Vectorized (Pushdown) 283 301 24 55.5 18.0 21.4X +Native ORC Vectorized 4715 4731 22 3.3 299.8 1.3X +Native ORC Vectorized (Pushdown) 281 290 10 56.0 17.9 21.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6036 6049 10 2.6 383.8 1.0X -Parquet Vectorized (Pushdown) 296 302 4 53.1 18.8 20.4X -Native ORC Vectorized 4725 4753 22 3.3 300.4 1.3X -Native ORC Vectorized (Pushdown) 276 286 6 56.9 17.6 21.8X +Parquet Vectorized 6104 6135 28 2.6 388.1 1.0X +Parquet Vectorized (Pushdown) 279 288 6 56.3 17.8 21.8X +Native ORC Vectorized 4780 4816 31 3.3 303.9 1.3X +Native ORC Vectorized (Pushdown) 279 297 13 56.4 17.7 21.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6041 6050 6 2.6 384.1 1.0X -Parquet Vectorized (Pushdown) 292 302 7 53.8 18.6 20.7X -Native ORC Vectorized 4711 4747 26 3.3 299.5 1.3X -Native ORC Vectorized (Pushdown) 271 286 8 58.0 17.2 22.3X +Parquet Vectorized 6122 6149 19 2.6 389.2 1.0X +Parquet Vectorized (Pushdown) 283 290 5 55.5 18.0 21.6X +Native ORC Vectorized 4788 4807 15 3.3 304.4 1.3X +Native ORC Vectorized (Pushdown) 274 285 6 57.4 17.4 22.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6046 6067 22 2.6 384.4 1.0X -Parquet Vectorized (Pushdown) 296 302 4 53.1 18.8 20.4X -Native ORC Vectorized 4767 4804 28 3.3 303.1 1.3X -Native ORC Vectorized (Pushdown) 274 286 7 57.4 17.4 22.1X +Parquet Vectorized 6128 6134 6 2.6 389.6 1.0X +Parquet Vectorized (Pushdown) 277 282 3 56.8 17.6 22.1X +Native ORC Vectorized 4819 4831 9 3.3 306.4 1.3X +Native ORC Vectorized (Pushdown) 296 303 7 53.1 18.8 20.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6053 6060 6 2.6 384.9 1.0X -Parquet Vectorized (Pushdown) 296 298 2 53.2 18.8 20.5X -Native ORC Vectorized 4792 4801 8 3.3 304.7 1.3X -Native ORC Vectorized (Pushdown) 273 286 8 57.7 17.3 22.2X +Parquet Vectorized 6143 6158 16 2.6 390.5 1.0X +Parquet Vectorized (Pushdown) 281 289 9 55.9 17.9 21.8X +Native ORC Vectorized 4810 4822 12 3.3 305.8 1.3X +Native ORC Vectorized (Pushdown) 276 280 4 57.1 17.5 22.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6751 6789 38 2.3 429.2 1.0X -Parquet Vectorized (Pushdown) 1591 1607 15 9.9 101.1 4.2X -Native ORC Vectorized 5460 5476 19 2.9 347.1 1.2X -Native ORC Vectorized (Pushdown) 1457 1469 11 10.8 92.7 4.6X +Parquet Vectorized 6791 6806 13 2.3 431.7 1.0X +Parquet Vectorized (Pushdown) 1541 1553 8 10.2 98.0 4.4X +Native ORC Vectorized 5445 5461 15 2.9 346.2 1.2X +Native ORC Vectorized (Pushdown) 1389 1399 11 11.3 88.3 4.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 9467 9480 15 1.7 601.9 1.0X -Parquet Vectorized (Pushdown) 6594 6601 10 2.4 419.2 1.4X -Native ORC Vectorized 8160 8178 19 1.9 518.8 1.2X -Native ORC Vectorized (Pushdown) 5978 5991 14 2.6 380.1 1.6X +Parquet Vectorized 9208 9246 24 1.7 585.4 1.0X +Parquet Vectorized (Pushdown) 6355 6366 12 2.5 404.0 1.4X +Native ORC Vectorized 7986 8006 22 2.0 507.7 1.2X +Native ORC Vectorized (Pushdown) 5817 5836 24 2.7 369.8 1.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 11963 11975 19 1.3 760.6 1.0X -Parquet Vectorized (Pushdown) 11449 11464 16 1.4 727.9 1.0X -Native ORC Vectorized 10773 10783 10 1.5 684.9 1.1X -Native ORC Vectorized (Pushdown) 10394 10409 19 1.5 660.8 1.2X +Parquet Vectorized 11608 11632 22 1.4 738.0 1.0X +Parquet Vectorized (Pushdown) 11058 11081 15 1.4 703.1 1.0X +Native ORC Vectorized 10392 10449 58 1.5 660.7 1.1X +Native ORC Vectorized (Pushdown) 9987 10003 13 1.6 635.0 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 12713 12733 15 1.2 808.3 1.0X -Parquet Vectorized (Pushdown) 12801 12815 14 1.2 813.9 1.0X -Native ORC Vectorized 11367 11387 16 1.4 722.7 1.1X -Native ORC Vectorized (Pushdown) 11474 11480 10 1.4 729.5 1.1X +Parquet Vectorized 12256 12273 18 1.3 779.2 1.0X +Parquet Vectorized (Pushdown) 12325 12363 28 1.3 783.6 1.0X +Native ORC Vectorized 10919 10943 29 1.4 694.2 1.1X +Native ORC Vectorized (Pushdown) 10980 11026 35 1.4 698.1 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 12741 12750 9 1.2 810.1 1.0X -Parquet Vectorized (Pushdown) 12807 12836 31 1.2 814.2 1.0X -Native ORC Vectorized 11501 11506 6 1.4 731.2 1.1X -Native ORC Vectorized (Pushdown) 11585 11594 8 1.4 736.6 1.1X +Parquet Vectorized 12196 12240 38 1.3 775.4 1.0X +Parquet Vectorized (Pushdown) 12243 12306 54 1.3 778.4 1.0X +Native ORC Vectorized 10848 10869 22 1.4 689.7 1.1X +Native ORC Vectorized (Pushdown) 10937 10964 29 1.4 695.4 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 12572 12595 22 1.3 799.3 1.0X -Parquet Vectorized (Pushdown) 12635 12654 28 1.2 803.3 1.0X -Native ORC Vectorized 11466 11493 19 1.4 729.0 1.1X -Native ORC Vectorized (Pushdown) 11548 11558 10 1.4 734.2 1.1X +Parquet Vectorized 12402 12415 12 1.3 788.5 1.0X +Parquet Vectorized (Pushdown) 12413 12427 14 1.3 789.2 1.0X +Native ORC Vectorized 10821 10859 25 1.5 688.0 1.1X +Native ORC Vectorized (Pushdown) 10916 10932 13 1.4 694.0 1.1X ================================================================================================ Pushdown for few distinct value case (use dictionary encoding) ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5530 5572 29 2.8 351.6 1.0X -Parquet Vectorized (Pushdown) 243 256 15 64.7 15.5 22.7X -Native ORC Vectorized 6173 6214 31 2.5 392.5 0.9X -Native ORC Vectorized (Pushdown) 933 935 4 16.9 59.3 5.9X +Parquet Vectorized 5635 5682 35 2.8 358.3 1.0X +Parquet Vectorized (Pushdown) 246 252 6 63.9 15.7 22.9X +Native ORC Vectorized 6232 6241 6 2.5 396.2 0.9X +Native ORC Vectorized (Pushdown) 924 934 13 17.0 58.7 6.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 5521 5537 14 2.8 351.0 1.0X -Parquet Vectorized (Pushdown) 245 257 11 64.2 15.6 22.5X -Native ORC Vectorized 6340 6348 5 2.5 403.1 0.9X -Native ORC Vectorized (Pushdown) 931 935 4 16.9 59.2 5.9X +Parquet Vectorized 5646 5669 23 2.8 359.0 1.0X +Parquet Vectorized (Pushdown) 249 259 6 63.3 15.8 22.7X +Native ORC Vectorized 6380 6408 26 2.5 405.7 0.9X +Native ORC Vectorized (Pushdown) 900 909 9 17.5 57.2 6.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5476 5490 14 2.9 348.2 1.0X -Parquet Vectorized (Pushdown) 288 298 12 54.7 18.3 19.0X -Native ORC Vectorized 6322 6341 11 2.5 401.9 0.9X -Native ORC Vectorized (Pushdown) 964 971 7 16.3 61.3 5.7X +Parquet Vectorized 5569 5581 11 2.8 354.0 1.0X +Parquet Vectorized (Pushdown) 297 303 6 52.9 18.9 18.7X +Native ORC Vectorized 6378 6387 6 2.5 405.5 0.9X +Native ORC Vectorized (Pushdown) 940 959 17 16.7 59.7 5.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 5477 5493 10 2.9 348.2 1.0X -Parquet Vectorized (Pushdown) 286 302 20 55.0 18.2 19.2X -Native ORC Vectorized 6324 6340 17 2.5 402.1 0.9X -Native ORC Vectorized (Pushdown) 966 975 11 16.3 61.4 5.7X +Parquet Vectorized 5560 5569 6 2.8 353.5 1.0X +Parquet Vectorized (Pushdown) 305 309 4 51.6 19.4 18.2X +Native ORC Vectorized 6377 6407 31 2.5 405.4 0.9X +Native ORC Vectorized (Pushdown) 952 959 13 16.5 60.5 5.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5550 5561 9 2.8 352.8 1.0X -Parquet Vectorized (Pushdown) 288 296 8 54.6 18.3 19.3X -Native ORC Vectorized 6438 6452 10 2.4 409.3 0.9X -Native ORC Vectorized (Pushdown) 972 977 5 16.2 61.8 5.7X +Parquet Vectorized 5648 5663 17 2.8 359.1 1.0X +Parquet Vectorized (Pushdown) 288 293 4 54.5 18.3 19.6X +Native ORC Vectorized 6430 6456 20 2.4 408.8 0.9X +Native ORC Vectorized (Pushdown) 968 971 3 16.2 61.6 5.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 13575 13632 39 1.2 863.1 1.0X -Parquet Vectorized (Pushdown) 13578 13607 21 1.2 863.2 1.0X -Native ORC Vectorized 14550 14590 49 1.1 925.0 0.9X -Native ORC Vectorized (Pushdown) 14664 14775 78 1.1 932.3 0.9X +Parquet Vectorized 14383 14409 24 1.1 914.5 1.0X +Parquet Vectorized (Pushdown) 14425 14443 18 1.1 917.1 1.0X +Native ORC Vectorized 15288 15300 11 1.0 972.0 0.9X +Native ORC Vectorized (Pushdown) 15482 15517 60 1.0 984.3 0.9X ================================================================================================ Pushdown benchmark for StringStartsWith ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6543 6644 84 2.4 416.0 1.0X -Parquet Vectorized (Pushdown) 891 922 41 17.7 56.6 7.3X -Native ORC Vectorized 5543 5553 11 2.8 352.4 1.2X -Native ORC Vectorized (Pushdown) 5605 5619 8 2.8 356.4 1.2X +Parquet Vectorized 6744 6776 24 2.3 428.7 1.0X +Parquet Vectorized (Pushdown) 908 916 6 17.3 57.8 7.4X +Native ORC Vectorized 5592 5608 22 2.8 355.5 1.2X +Native ORC Vectorized (Pushdown) 5664 5687 23 2.8 360.1 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6367 6383 16 2.5 404.8 1.0X -Parquet Vectorized (Pushdown) 279 286 6 56.3 17.8 22.8X -Native ORC Vectorized 5367 5377 8 2.9 341.2 1.2X -Native ORC Vectorized (Pushdown) 5436 5463 21 2.9 345.6 1.2X +Parquet Vectorized 6447 6462 16 2.4 409.9 1.0X +Parquet Vectorized (Pushdown) 300 302 2 52.4 19.1 21.5X +Native ORC Vectorized 5416 5426 11 2.9 344.3 1.2X +Native ORC Vectorized (Pushdown) 5508 5521 12 2.9 350.2 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6369 6386 15 2.5 404.9 1.0X -Parquet Vectorized (Pushdown) 277 284 6 56.9 17.6 23.0X -Native ORC Vectorized 5341 5370 28 2.9 339.6 1.2X -Native ORC Vectorized (Pushdown) 5435 5443 10 2.9 345.5 1.2X +Parquet Vectorized 6447 6462 12 2.4 409.9 1.0X +Parquet Vectorized (Pushdown) 281 288 6 56.0 17.8 23.0X +Native ORC Vectorized 5394 5400 7 2.9 342.9 1.2X +Native ORC Vectorized (Pushdown) 5467 5502 29 2.9 347.6 1.2X ================================================================================================ Pushdown benchmark for StringEndsWith ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringEndsWith filter: (value like '%10'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5594 5712 85 2.8 355.6 1.0X -Parquet Vectorized (Pushdown) 384 401 14 41.0 24.4 14.6X -Native ORC Vectorized 6399 6440 49 2.5 406.8 0.9X -Native ORC Vectorized (Pushdown) 6587 6606 15 2.4 418.8 0.8X +Parquet Vectorized 5688 5817 122 2.8 361.7 1.0X +Parquet Vectorized (Pushdown) 368 379 16 42.7 23.4 15.4X +Native ORC Vectorized 6433 6447 10 2.4 409.0 0.9X +Native ORC Vectorized (Pushdown) 6684 6708 21 2.4 424.9 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringEndsWith filter: (value like '%1000'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5518 5545 19 2.9 350.8 1.0X -Parquet Vectorized (Pushdown) 294 324 50 53.5 18.7 18.8X -Native ORC Vectorized 6314 6348 27 2.5 401.5 0.9X -Native ORC Vectorized (Pushdown) 6509 6530 20 2.4 413.8 0.8X +Parquet Vectorized 5563 5576 14 2.8 353.7 1.0X +Parquet Vectorized (Pushdown) 266 272 4 59.1 16.9 20.9X +Native ORC Vectorized 6386 6425 30 2.5 406.0 0.9X +Native ORC Vectorized (Pushdown) 6639 6689 48 2.4 422.1 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringEndsWith filter: (value like '%786432'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5527 5535 7 2.8 351.4 1.0X -Parquet Vectorized (Pushdown) 284 296 9 55.3 18.1 19.4X -Native ORC Vectorized 6290 6301 10 2.5 399.9 0.9X -Native ORC Vectorized (Pushdown) 6552 6565 13 2.4 416.6 0.8X +Parquet Vectorized 5574 5578 4 2.8 354.4 1.0X +Parquet Vectorized (Pushdown) 272 278 4 57.8 17.3 20.5X +Native ORC Vectorized 6333 6412 60 2.5 402.6 0.9X +Native ORC Vectorized (Pushdown) 6604 6667 88 2.4 419.9 0.8X ================================================================================================ Pushdown benchmark for StringContains ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringContains filter: (value like '%10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5855 6077 170 2.7 372.3 1.0X -Parquet Vectorized (Pushdown) 922 952 40 17.1 58.6 6.4X -Native ORC Vectorized 6452 6541 82 2.4 410.2 0.9X -Native ORC Vectorized (Pushdown) 6639 6651 10 2.4 422.1 0.9X +Parquet Vectorized 5827 5939 73 2.7 370.5 1.0X +Parquet Vectorized (Pushdown) 810 829 17 19.4 51.5 7.2X +Native ORC Vectorized 6466 6550 65 2.4 411.1 0.9X +Native ORC Vectorized (Pushdown) 6691 6714 21 2.4 425.4 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringContains filter: (value like '%1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5542 5550 8 2.8 352.3 1.0X -Parquet Vectorized (Pushdown) 296 310 8 53.2 18.8 18.7X -Native ORC Vectorized 6214 6226 11 2.5 395.1 0.9X -Native ORC Vectorized (Pushdown) 6419 6431 20 2.5 408.1 0.9X +Parquet Vectorized 5546 5555 10 2.8 352.6 1.0X +Parquet Vectorized (Pushdown) 268 276 4 58.6 17.1 20.7X +Native ORC Vectorized 6251 6258 7 2.5 397.4 0.9X +Native ORC Vectorized (Pushdown) 6454 6471 13 2.4 410.3 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringContains filter: (value like '%786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 5537 5552 17 2.8 352.0 1.0X -Parquet Vectorized (Pushdown) 297 308 10 53.0 18.9 18.7X -Native ORC Vectorized 6232 6246 15 2.5 396.2 0.9X -Native ORC Vectorized (Pushdown) 6407 6419 14 2.5 407.4 0.9X +Parquet Vectorized 5548 5561 10 2.8 352.7 1.0X +Parquet Vectorized (Pushdown) 268 275 7 58.6 17.1 20.7X +Native ORC Vectorized 6259 6269 8 2.5 397.9 0.9X +Native ORC Vectorized (Pushdown) 6466 6487 20 2.4 411.1 0.9X ================================================================================================ Pushdown benchmark for decimal ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2442 2459 21 6.4 155.3 1.0X -Parquet Vectorized (Pushdown) 77 90 14 203.1 4.9 31.5X -Native ORC Vectorized 3128 3145 24 5.0 198.9 0.8X -Native ORC Vectorized (Pushdown) 57 72 9 273.6 3.7 42.5X +Parquet Vectorized 2436 2443 4 6.5 154.9 1.0X +Parquet Vectorized (Pushdown) 71 74 4 222.9 4.5 34.5X +Native ORC Vectorized 3333 3346 14 4.7 211.9 0.7X +Native ORC Vectorized (Pushdown) 58 61 4 271.6 3.7 42.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3756 3808 31 4.2 238.8 1.0X -Parquet Vectorized (Pushdown) 1912 1937 42 8.2 121.6 2.0X -Native ORC Vectorized 4593 4618 26 3.4 292.0 0.8X -Native ORC Vectorized (Pushdown) 2069 2105 29 7.6 131.5 1.8X +Parquet Vectorized 3845 3881 42 4.1 244.4 1.0X +Parquet Vectorized (Pushdown) 1961 1989 29 8.0 124.7 2.0X +Native ORC Vectorized 4660 4678 17 3.4 296.3 0.8X +Native ORC Vectorized (Pushdown) 2076 2087 11 7.6 132.0 1.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8252 8270 22 1.9 524.6 1.0X -Parquet Vectorized (Pushdown) 7939 7987 28 2.0 504.7 1.0X -Native ORC Vectorized 9304 9335 42 1.7 591.5 0.9X -Native ORC Vectorized (Pushdown) 8912 8946 32 1.8 566.6 0.9X +Parquet Vectorized 8473 8500 17 1.9 538.7 1.0X +Parquet Vectorized (Pushdown) 8212 8248 23 1.9 522.1 1.0X +Native ORC Vectorized 9900 9917 15 1.6 629.4 0.9X +Native ORC Vectorized (Pushdown) 9487 9498 9 1.7 603.2 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9337 9380 67 1.7 593.6 1.0X -Parquet Vectorized (Pushdown) 9347 9376 22 1.7 594.3 1.0X -Native ORC Vectorized 10538 10565 29 1.5 670.0 0.9X -Native ORC Vectorized (Pushdown) 10533 10559 28 1.5 669.7 0.9X +Parquet Vectorized 9464 9502 33 1.7 601.7 1.0X +Parquet Vectorized (Pushdown) 9462 9502 39 1.7 601.6 1.0X +Native ORC Vectorized 10726 10775 35 1.5 681.9 0.9X +Native ORC Vectorized (Pushdown) 10755 10784 24 1.5 683.8 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2598 2612 15 6.1 165.2 1.0X -Parquet Vectorized (Pushdown) 72 83 13 217.1 4.6 35.9X -Native ORC Vectorized 3113 3124 14 5.1 197.9 0.8X -Native ORC Vectorized (Pushdown) 55 64 12 285.4 3.5 47.1X +Parquet Vectorized 2638 2651 15 6.0 167.7 1.0X +Parquet Vectorized (Pushdown) 71 85 18 220.9 4.5 37.0X +Native ORC Vectorized 3330 3344 14 4.7 211.7 0.8X +Native ORC Vectorized (Pushdown) 55 60 5 285.6 3.5 47.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3358 3379 34 4.7 213.5 1.0X -Parquet Vectorized (Pushdown) 1080 1111 28 14.6 68.7 3.1X -Native ORC Vectorized 3874 3884 13 4.1 246.3 0.9X -Native ORC Vectorized (Pushdown) 1111 1137 34 14.2 70.6 3.0X +Parquet Vectorized 3399 3440 49 4.6 216.1 1.0X +Parquet Vectorized (Pushdown) 1064 1076 10 14.8 67.7 3.2X +Native ORC Vectorized 4064 4078 12 3.9 258.4 0.8X +Native ORC Vectorized (Pushdown) 1103 1109 6 14.3 70.2 3.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6310 6332 17 2.5 401.2 1.0X -Parquet Vectorized (Pushdown) 5049 5073 21 3.1 321.0 1.2X -Native ORC Vectorized 6975 6984 7 2.3 443.5 0.9X -Native ORC Vectorized (Pushdown) 5396 5411 14 2.9 343.1 1.2X +Parquet Vectorized 6242 6260 21 2.5 396.9 1.0X +Parquet Vectorized (Pushdown) 4988 5018 32 3.2 317.2 1.3X +Native ORC Vectorized 6949 6963 14 2.3 441.8 0.9X +Native ORC Vectorized (Pushdown) 5318 5332 10 3.0 338.1 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9115 9121 4 1.7 579.5 1.0X -Parquet Vectorized (Pushdown) 8907 8924 18 1.8 566.3 1.0X -Native ORC Vectorized 9981 9994 9 1.6 634.6 0.9X -Native ORC Vectorized (Pushdown) 9656 9675 13 1.6 613.9 0.9X +Parquet Vectorized 9079 9090 14 1.7 577.2 1.0X +Parquet Vectorized (Pushdown) 8825 8842 19 1.8 561.1 1.0X +Native ORC Vectorized 9902 9928 26 1.6 629.5 0.9X +Native ORC Vectorized (Pushdown) 9611 9616 4 1.6 611.0 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3755 3781 23 4.2 238.7 1.0X -Parquet Vectorized (Pushdown) 78 81 2 201.4 5.0 48.1X -Native ORC Vectorized 3131 3155 36 5.0 199.0 1.2X -Native ORC Vectorized (Pushdown) 54 56 4 292.6 3.4 69.8X +Parquet Vectorized 3823 3841 12 4.1 243.1 1.0X +Parquet Vectorized (Pushdown) 80 83 4 196.7 5.1 47.8X +Native ORC Vectorized 3330 3350 18 4.7 211.7 1.1X +Native ORC Vectorized (Pushdown) 55 60 5 287.2 3.5 69.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 4741 4755 18 3.3 301.4 1.0X -Parquet Vectorized (Pushdown) 1415 1417 2 11.1 90.0 3.3X -Native ORC Vectorized 4049 4065 20 3.9 257.4 1.2X -Native ORC Vectorized (Pushdown) 1220 1231 17 12.9 77.6 3.9X +Parquet Vectorized 4750 4807 89 3.3 302.0 1.0X +Parquet Vectorized (Pushdown) 1400 1407 7 11.2 89.0 3.4X +Native ORC Vectorized 4157 4168 12 3.8 264.3 1.1X +Native ORC Vectorized (Pushdown) 1211 1215 3 13.0 77.0 3.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8556 8564 9 1.8 543.9 1.0X -Parquet Vectorized (Pushdown) 6743 6755 7 2.3 428.7 1.3X -Native ORC Vectorized 7513 7524 9 2.1 477.7 1.1X -Native ORC Vectorized (Pushdown) 5906 5914 5 2.7 375.5 1.4X +Parquet Vectorized 8636 8662 19 1.8 549.1 1.0X +Parquet Vectorized (Pushdown) 6754 6787 25 2.3 429.4 1.3X +Native ORC Vectorized 7526 7536 12 2.1 478.5 1.1X +Native ORC Vectorized (Pushdown) 5915 5934 13 2.7 376.0 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 12314 12332 16 1.3 782.9 1.0X -Parquet Vectorized (Pushdown) 11976 11983 6 1.3 761.4 1.0X -Native ORC Vectorized 10898 10916 14 1.4 692.9 1.1X -Native ORC Vectorized (Pushdown) 10605 10636 30 1.5 674.3 1.2X +Parquet Vectorized 12415 12446 27 1.3 789.3 1.0X +Parquet Vectorized (Pushdown) 12049 12076 24 1.3 766.1 1.0X +Native ORC Vectorized 10912 10980 93 1.4 693.7 1.1X +Native ORC Vectorized (Pushdown) 10559 10608 43 1.5 671.4 1.2X ================================================================================================ Pushdown benchmark for InSet -> InFilters ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6282 6318 39 2.5 399.4 1.0X -Parquet Vectorized (Pushdown) 329 370 54 47.8 20.9 19.1X -Native ORC Vectorized 4793 4843 35 3.3 304.7 1.3X -Native ORC Vectorized (Pushdown) 307 321 15 51.2 19.5 20.4X +Parquet Vectorized 6312 6343 25 2.5 401.3 1.0X +Parquet Vectorized (Pushdown) 312 328 11 50.4 19.8 20.2X +Native ORC Vectorized 4774 4861 102 3.3 303.5 1.3X +Native ORC Vectorized (Pushdown) 286 300 15 55.0 18.2 22.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6120 6135 12 2.6 389.1 1.0X -Parquet Vectorized (Pushdown) 294 304 8 53.5 18.7 20.8X -Native ORC Vectorized 4787 4815 38 3.3 304.4 1.3X -Native ORC Vectorized (Pushdown) 285 301 12 55.2 18.1 21.5X +Parquet Vectorized 6158 6190 20 2.6 391.5 1.0X +Parquet Vectorized (Pushdown) 292 302 11 53.9 18.5 21.1X +Native ORC Vectorized 4712 4748 28 3.3 299.6 1.3X +Native ORC Vectorized (Pushdown) 285 299 13 55.2 18.1 21.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6136 6151 20 2.6 390.1 1.0X -Parquet Vectorized (Pushdown) 297 304 9 53.0 18.9 20.7X -Native ORC Vectorized 4787 4802 26 3.3 304.3 1.3X -Native ORC Vectorized (Pushdown) 286 296 7 55.0 18.2 21.4X +Parquet Vectorized 6182 6209 20 2.5 393.0 1.0X +Parquet Vectorized (Pushdown) 288 301 9 54.5 18.3 21.4X +Native ORC Vectorized 4730 4794 51 3.3 300.7 1.3X +Native ORC Vectorized (Pushdown) 295 301 5 53.3 18.8 20.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6143 6155 13 2.6 390.6 1.0X -Parquet Vectorized (Pushdown) 304 307 3 51.7 19.3 20.2X -Native ORC Vectorized 4811 4826 16 3.3 305.9 1.3X -Native ORC Vectorized (Pushdown) 294 301 5 53.5 18.7 20.9X +Parquet Vectorized 6239 6254 20 2.5 396.7 1.0X +Parquet Vectorized (Pushdown) 306 311 4 51.4 19.5 20.4X +Native ORC Vectorized 4747 4811 37 3.3 301.8 1.3X +Native ORC Vectorized (Pushdown) 304 307 3 51.8 19.3 20.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6127 6133 6 2.6 389.5 1.0X -Parquet Vectorized (Pushdown) 307 312 6 51.3 19.5 20.0X -Native ORC Vectorized 4818 4845 35 3.3 306.3 1.3X -Native ORC Vectorized (Pushdown) 298 310 9 52.8 18.9 20.6X +Parquet Vectorized 6212 6253 42 2.5 395.0 1.0X +Parquet Vectorized (Pushdown) 306 317 9 51.5 19.4 20.3X +Native ORC Vectorized 4814 4853 40 3.3 306.1 1.3X +Native ORC Vectorized (Pushdown) 306 314 6 51.3 19.5 20.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6122 6141 22 2.6 389.2 1.0X -Parquet Vectorized (Pushdown) 310 317 7 50.8 19.7 19.8X -Native ORC Vectorized 4813 4835 23 3.3 306.0 1.3X -Native ORC Vectorized (Pushdown) 300 307 6 52.4 19.1 20.4X +Parquet Vectorized 6192 6216 23 2.5 393.7 1.0X +Parquet Vectorized (Pushdown) 303 309 4 51.9 19.3 20.4X +Native ORC Vectorized 4752 4807 46 3.3 302.1 1.3X +Native ORC Vectorized (Pushdown) 307 320 9 51.2 19.5 20.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6326 6331 5 2.5 402.2 1.0X -Parquet Vectorized (Pushdown) 893 901 9 17.6 56.7 7.1X -Native ORC Vectorized 5039 5049 12 3.1 320.3 1.3X -Native ORC Vectorized (Pushdown) 399 402 3 39.4 25.4 15.8X +Parquet Vectorized 6384 6413 29 2.5 405.9 1.0X +Parquet Vectorized (Pushdown) 885 890 5 17.8 56.2 7.2X +Native ORC Vectorized 4935 4972 41 3.2 313.8 1.3X +Native ORC Vectorized (Pushdown) 421 425 2 37.3 26.8 15.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6309 6328 20 2.5 401.1 1.0X -Parquet Vectorized (Pushdown) 3291 3308 16 4.8 209.2 1.9X -Native ORC Vectorized 5019 5032 12 3.1 319.1 1.3X -Native ORC Vectorized (Pushdown) 429 433 3 36.6 27.3 14.7X +Parquet Vectorized 6392 6432 25 2.5 406.4 1.0X +Parquet Vectorized (Pushdown) 3230 3247 12 4.9 205.4 2.0X +Native ORC Vectorized 4940 4974 34 3.2 314.1 1.3X +Native ORC Vectorized (Pushdown) 430 434 4 36.6 27.4 14.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6311 6335 22 2.5 401.3 1.0X -Parquet Vectorized (Pushdown) 5508 5519 9 2.9 350.2 1.1X -Native ORC Vectorized 5020 5036 13 3.1 319.2 1.3X -Native ORC Vectorized (Pushdown) 442 444 2 35.6 28.1 14.3X +Parquet Vectorized 6399 6407 5 2.5 406.8 1.0X +Parquet Vectorized (Pushdown) 5280 5305 32 3.0 335.7 1.2X +Native ORC Vectorized 4913 4920 9 3.2 312.4 1.3X +Native ORC Vectorized (Pushdown) 422 428 5 37.3 26.8 15.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6271 6290 14 2.5 398.7 1.0X -Parquet Vectorized (Pushdown) 872 877 4 18.0 55.4 7.2X -Native ORC Vectorized 4971 4981 9 3.2 316.0 1.3X -Native ORC Vectorized (Pushdown) 497 502 4 31.7 31.6 12.6X +Parquet Vectorized 6387 6415 28 2.5 406.1 1.0X +Parquet Vectorized (Pushdown) 879 884 3 17.9 55.9 7.3X +Native ORC Vectorized 4898 4907 8 3.2 311.4 1.3X +Native ORC Vectorized (Pushdown) 512 514 1 30.7 32.6 12.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6284 6295 8 2.5 399.5 1.0X -Parquet Vectorized (Pushdown) 3320 3340 13 4.7 211.1 1.9X -Native ORC Vectorized 4972 4984 8 3.2 316.1 1.3X -Native ORC Vectorized (Pushdown) 564 567 3 27.9 35.9 11.1X +Parquet Vectorized 6409 6424 15 2.5 407.5 1.0X +Parquet Vectorized (Pushdown) 3279 3297 30 4.8 208.5 2.0X +Native ORC Vectorized 4900 4920 24 3.2 311.5 1.3X +Native ORC Vectorized (Pushdown) 584 592 7 26.9 37.2 11.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6289 6307 12 2.5 399.8 1.0X -Parquet Vectorized (Pushdown) 5740 5750 11 2.7 365.0 1.1X -Native ORC Vectorized 4972 4982 6 3.2 316.1 1.3X -Native ORC Vectorized (Pushdown) 559 567 11 28.1 35.6 11.2X +Parquet Vectorized 6420 6445 26 2.4 408.2 1.0X +Parquet Vectorized (Pushdown) 5734 5745 12 2.7 364.6 1.1X +Native ORC Vectorized 4940 5018 59 3.2 314.0 1.3X +Native ORC Vectorized (Pushdown) 575 581 7 27.4 36.5 11.2X ================================================================================================ Pushdown benchmark for tinyint ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2748 2766 16 5.7 174.7 1.0X -Parquet Vectorized (Pushdown) 107 114 8 146.5 6.8 25.6X -Native ORC Vectorized 2194 2203 9 7.2 139.5 1.3X -Native ORC Vectorized (Pushdown) 112 121 9 140.3 7.1 24.5X +Parquet Vectorized 2841 2865 29 5.5 180.6 1.0X +Parquet Vectorized (Pushdown) 112 122 10 140.6 7.1 25.4X +Native ORC Vectorized 2239 2247 8 7.0 142.4 1.3X +Native ORC Vectorized (Pushdown) 115 130 16 136.9 7.3 24.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3325 3359 48 4.7 211.4 1.0X -Parquet Vectorized (Pushdown) 960 973 13 16.4 61.0 3.5X -Native ORC Vectorized 2691 2705 17 5.8 171.1 1.2X -Native ORC Vectorized (Pushdown) 840 846 3 18.7 53.4 4.0X +Parquet Vectorized 3366 3422 87 4.7 214.0 1.0X +Parquet Vectorized (Pushdown) 987 990 2 15.9 62.7 3.4X +Native ORC Vectorized 2766 2784 10 5.7 175.9 1.2X +Native ORC Vectorized (Pushdown) 876 879 3 18.0 55.7 3.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5896 5906 9 2.7 374.9 1.0X -Parquet Vectorized (Pushdown) 4608 4631 20 3.4 293.0 1.3X -Native ORC Vectorized 5059 5084 21 3.1 321.6 1.2X -Native ORC Vectorized (Pushdown) 4014 4027 8 3.9 255.2 1.5X +Parquet Vectorized 6015 6025 7 2.6 382.4 1.0X +Parquet Vectorized (Pushdown) 4690 4699 9 3.4 298.2 1.3X +Native ORC Vectorized 5138 5194 38 3.1 326.7 1.2X +Native ORC Vectorized (Pushdown) 3988 4002 10 3.9 253.5 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8646 8666 24 1.8 549.7 1.0X -Parquet Vectorized (Pushdown) 8379 8396 11 1.9 532.7 1.0X -Native ORC Vectorized 7526 7539 14 2.1 478.5 1.1X -Native ORC Vectorized (Pushdown) 7319 7342 16 2.1 465.4 1.2X +Parquet Vectorized 8535 8557 21 1.8 542.7 1.0X +Parquet Vectorized (Pushdown) 8308 8326 14 1.9 528.2 1.0X +Native ORC Vectorized 7581 7600 24 2.1 482.0 1.1X +Native ORC Vectorized (Pushdown) 7379 7399 22 2.1 469.1 1.2X ================================================================================================ Pushdown benchmark for Timestamp ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 timestamp stored as INT96 row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3102 3122 23 5.1 197.2 1.0X -Parquet Vectorized (Pushdown) 3096 3104 5 5.1 196.8 1.0X -Native ORC Vectorized 1983 1994 15 7.9 126.1 1.6X -Native ORC Vectorized (Pushdown) 39 44 5 404.7 2.5 79.8X +Parquet Vectorized 3155 3166 9 5.0 200.6 1.0X +Parquet Vectorized (Pushdown) 3169 3174 5 5.0 201.5 1.0X +Native ORC Vectorized 2102 2116 17 7.5 133.7 1.5X +Native ORC Vectorized (Pushdown) 39 44 6 399.9 2.5 80.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% timestamp stored as INT96 rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3791 3820 27 4.1 241.0 1.0X -Parquet Vectorized (Pushdown) 3755 3774 12 4.2 238.8 1.0X -Native ORC Vectorized 2618 2635 18 6.0 166.5 1.4X -Native ORC Vectorized (Pushdown) 860 865 6 18.3 54.7 4.4X +Parquet Vectorized 3827 3848 30 4.1 243.3 1.0X +Parquet Vectorized (Pushdown) 3803 3831 37 4.1 241.8 1.0X +Native ORC Vectorized 2738 2757 20 5.7 174.1 1.4X +Native ORC Vectorized (Pushdown) 879 887 9 17.9 55.9 4.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% timestamp stored as INT96 rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6492 6500 8 2.4 412.8 1.0X -Parquet Vectorized (Pushdown) 6485 6497 12 2.4 412.3 1.0X -Native ORC Vectorized 5272 5286 21 3.0 335.2 1.2X -Native ORC Vectorized (Pushdown) 4245 4253 9 3.7 269.9 1.5X +Parquet Vectorized 6597 6622 19 2.4 419.4 1.0X +Parquet Vectorized (Pushdown) 6618 6639 14 2.4 420.8 1.0X +Native ORC Vectorized 5324 5342 19 3.0 338.5 1.2X +Native ORC Vectorized (Pushdown) 4259 4264 5 3.7 270.8 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% timestamp stored as INT96 rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9353 9362 9 1.7 594.7 1.0X -Parquet Vectorized (Pushdown) 9335 9349 17 1.7 593.5 1.0X -Native ORC Vectorized 7781 7799 17 2.0 494.7 1.2X -Native ORC Vectorized (Pushdown) 7598 7613 9 2.1 483.1 1.2X +Parquet Vectorized 9349 9366 22 1.7 594.4 1.0X +Parquet Vectorized (Pushdown) 9360 9391 21 1.7 595.1 1.0X +Native ORC Vectorized 7882 7909 23 2.0 501.1 1.2X +Native ORC Vectorized (Pushdown) 7666 7676 6 2.1 487.4 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 timestamp stored as TIMESTAMP_MICROS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2578 2590 12 6.1 163.9 1.0X -Parquet Vectorized (Pushdown) 70 77 11 223.4 4.5 36.6X -Native ORC Vectorized 1982 1987 8 7.9 126.0 1.3X -Native ORC Vectorized (Pushdown) 39 43 5 404.6 2.5 66.3X +Parquet Vectorized 2617 2627 7 6.0 166.4 1.0X +Parquet Vectorized (Pushdown) 69 74 9 229.5 4.4 38.2X +Native ORC Vectorized 2092 2097 4 7.5 133.0 1.3X +Native ORC Vectorized (Pushdown) 38 43 5 409.7 2.4 68.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3228 3236 9 4.9 205.2 1.0X -Parquet Vectorized (Pushdown) 983 987 5 16.0 62.5 3.3X -Native ORC Vectorized 2607 2617 6 6.0 165.7 1.2X -Native ORC Vectorized (Pushdown) 859 864 4 18.3 54.6 3.8X +Parquet Vectorized 3278 3287 10 4.8 208.4 1.0X +Parquet Vectorized (Pushdown) 999 1010 9 15.7 63.5 3.3X +Native ORC Vectorized 2724 2732 7 5.8 173.2 1.2X +Native ORC Vectorized (Pushdown) 864 870 5 18.2 54.9 3.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5897 5908 9 2.7 374.9 1.0X -Parquet Vectorized (Pushdown) 4693 4705 13 3.4 298.3 1.3X -Native ORC Vectorized 5145 5195 75 3.1 327.1 1.1X -Native ORC Vectorized (Pushdown) 4134 4139 5 3.8 262.8 1.4X +Parquet Vectorized 6098 6125 40 2.6 387.7 1.0X +Parquet Vectorized (Pushdown) 4842 4859 15 3.2 307.9 1.3X +Native ORC Vectorized 5243 5246 3 3.0 333.4 1.2X +Native ORC Vectorized (Pushdown) 4205 4220 14 3.7 267.4 1.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8540 8549 9 1.8 542.9 1.0X -Parquet Vectorized (Pushdown) 8330 8339 11 1.9 529.6 1.0X -Native ORC Vectorized 7638 7650 11 2.1 485.6 1.1X -Native ORC Vectorized (Pushdown) 7440 7448 11 2.1 473.0 1.1X +Parquet Vectorized 8867 8893 24 1.8 563.7 1.0X +Parquet Vectorized (Pushdown) 8630 8677 38 1.8 548.7 1.0X +Native ORC Vectorized 7897 7900 2 2.0 502.1 1.1X +Native ORC Vectorized (Pushdown) 7700 7716 11 2.0 489.6 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2748 2759 18 5.7 174.7 1.0X -Parquet Vectorized (Pushdown) 70 73 3 224.7 4.5 39.3X -Native ORC Vectorized 1986 1999 18 7.9 126.2 1.4X -Native ORC Vectorized (Pushdown) 39 42 5 407.9 2.5 71.3X +Parquet Vectorized 2783 2801 17 5.7 177.0 1.0X +Parquet Vectorized (Pushdown) 72 75 4 218.9 4.6 38.7X +Native ORC Vectorized 2023 2032 6 7.8 128.6 1.4X +Native ORC Vectorized (Pushdown) 40 43 4 393.7 2.5 69.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3397 3403 7 4.6 216.0 1.0X -Parquet Vectorized (Pushdown) 999 1006 6 15.7 63.5 3.4X -Native ORC Vectorized 2612 2620 12 6.0 166.1 1.3X -Native ORC Vectorized (Pushdown) 876 879 2 18.0 55.7 3.9X +Parquet Vectorized 3474 3479 3 4.5 220.9 1.0X +Parquet Vectorized (Pushdown) 1036 1044 8 15.2 65.9 3.4X +Native ORC Vectorized 2757 2766 7 5.7 175.3 1.3X +Native ORC Vectorized (Pushdown) 910 914 4 17.3 57.9 3.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6116 6123 9 2.6 388.8 1.0X -Parquet Vectorized (Pushdown) 4802 4813 11 3.3 305.3 1.3X -Native ORC Vectorized 5152 5160 7 3.1 327.6 1.2X -Native ORC Vectorized (Pushdown) 4126 4138 9 3.8 262.3 1.5X +Parquet Vectorized 6243 6270 16 2.5 396.9 1.0X +Parquet Vectorized (Pushdown) 4928 4956 36 3.2 313.3 1.3X +Native ORC Vectorized 5326 5332 5 3.0 338.6 1.2X +Native ORC Vectorized (Pushdown) 4262 4272 8 3.7 271.0 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8692 8746 68 1.8 552.6 1.0X -Parquet Vectorized (Pushdown) 8481 8495 15 1.9 539.2 1.0X -Native ORC Vectorized 7644 7653 10 2.1 486.0 1.1X -Native ORC Vectorized (Pushdown) 7449 7462 9 2.1 473.6 1.2X +Parquet Vectorized 8993 9036 38 1.7 571.7 1.0X +Parquet Vectorized (Pushdown) 8777 8803 19 1.8 558.0 1.0X +Native ORC Vectorized 7774 7790 17 2.0 494.3 1.2X +Native ORC Vectorized (Pushdown) 7573 7587 12 2.1 481.5 1.2X ================================================================================================ Pushdown benchmark with many filters ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 58 72 19 0.0 57741888.0 1.0X -Parquet Vectorized (Pushdown) 58 61 3 0.0 58429929.0 1.0X -Native ORC Vectorized 51 53 2 0.0 51359839.0 1.1X -Native ORC Vectorized (Pushdown) 53 55 3 0.0 53142981.0 1.1X +Parquet Vectorized 49 75 18 0.0 49268544.0 1.0X +Parquet Vectorized (Pushdown) 49 52 3 0.0 48949281.0 1.0X +Native ORC Vectorized 43 46 3 0.0 43343584.0 1.1X +Native ORC Vectorized (Pushdown) 44 48 4 0.0 44392858.0 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 400 420 16 0.0 400224642.0 1.0X -Parquet Vectorized (Pushdown) 399 417 11 0.0 399319343.0 1.0X -Native ORC Vectorized 387 393 7 0.0 387215337.0 1.0X -Native ORC Vectorized (Pushdown) 390 396 7 0.0 389851290.0 1.0X +Parquet Vectorized 193 196 3 0.0 192567723.0 1.0X +Parquet Vectorized (Pushdown) 191 206 18 0.0 191266175.0 1.0X +Native ORC Vectorized 178 182 4 0.0 178471724.0 1.1X +Native ORC Vectorized (Pushdown) 184 190 7 0.0 183580008.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 2216 2266 67 0.0 2215862652.0 1.0X -Parquet Vectorized (Pushdown) 2237 2281 47 0.0 2237304947.0 1.0X -Native ORC Vectorized 2202 2257 58 0.0 2202335420.0 1.0X -Native ORC Vectorized (Pushdown) 2219 2262 65 0.0 2219444511.0 1.0X +Parquet Vectorized 581 587 6 0.0 581443562.0 1.0X +Parquet Vectorized (Pushdown) 591 611 14 0.0 591021175.0 1.0X +Native ORC Vectorized 563 580 11 0.0 563194077.0 1.0X +Native ORC Vectorized (Pushdown) 583 597 12 0.0 582533796.0 1.0X diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt index f762a7147d31b..e6f878de0a974 100644 --- a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt +++ b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt @@ -2,733 +2,733 @@ Pushdown for many distinct value case ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6564 6709 108 2.4 417.3 1.0X -Parquet Vectorized (Pushdown) 315 335 18 50.0 20.0 20.9X -Native ORC Vectorized 5085 5205 71 3.1 323.3 1.3X -Native ORC Vectorized (Pushdown) 296 309 11 53.2 18.8 22.2X +Parquet Vectorized 6867 6919 54 2.3 436.6 1.0X +Parquet Vectorized (Pushdown) 313 342 24 50.3 19.9 22.0X +Native ORC Vectorized 5135 5177 42 3.1 326.5 1.3X +Native ORC Vectorized (Pushdown) 314 327 9 50.1 19.9 21.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6573 6616 51 2.4 417.9 1.0X -Parquet Vectorized (Pushdown) 291 309 16 54.1 18.5 22.6X -Native ORC Vectorized 5027 5047 17 3.1 319.6 1.3X -Native ORC Vectorized (Pushdown) 292 316 19 53.9 18.5 22.5X +Parquet Vectorized 6952 6967 17 2.3 442.0 1.0X +Parquet Vectorized (Pushdown) 313 324 14 50.2 19.9 22.2X +Native ORC Vectorized 5212 5234 18 3.0 331.4 1.3X +Native ORC Vectorized (Pushdown) 318 331 8 49.5 20.2 21.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6453 6497 31 2.4 410.3 1.0X -Parquet Vectorized (Pushdown) 289 295 6 54.4 18.4 22.3X -Native ORC Vectorized 4973 5006 25 3.2 316.2 1.3X -Native ORC Vectorized (Pushdown) 276 288 14 57.0 17.5 23.4X +Parquet Vectorized 6921 6940 11 2.3 440.1 1.0X +Parquet Vectorized (Pushdown) 299 310 11 52.7 19.0 23.2X +Native ORC Vectorized 5203 5210 6 3.0 330.8 1.3X +Native ORC Vectorized (Pushdown) 312 319 7 50.4 19.8 22.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6498 6553 74 2.4 413.1 1.0X -Parquet Vectorized (Pushdown) 284 294 7 55.3 18.1 22.9X -Native ORC Vectorized 5070 5087 11 3.1 322.3 1.3X -Native ORC Vectorized (Pushdown) 272 287 14 57.9 17.3 23.9X +Parquet Vectorized 6899 6925 15 2.3 438.6 1.0X +Parquet Vectorized (Pushdown) 286 303 13 55.0 18.2 24.1X +Native ORC Vectorized 5194 5210 15 3.0 330.2 1.3X +Native ORC Vectorized (Pushdown) 296 303 6 53.2 18.8 23.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6558 6601 43 2.4 416.9 1.0X -Parquet Vectorized (Pushdown) 275 288 9 57.1 17.5 23.8X -Native ORC Vectorized 5016 5046 26 3.1 318.9 1.3X -Native ORC Vectorized (Pushdown) 273 289 18 57.5 17.4 24.0X +Parquet Vectorized 6934 6957 27 2.3 440.8 1.0X +Parquet Vectorized (Pushdown) 288 296 8 54.6 18.3 24.1X +Native ORC Vectorized 5212 5229 15 3.0 331.4 1.3X +Native ORC Vectorized (Pushdown) 304 308 3 51.8 19.3 22.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 13444 13642 185 1.2 854.8 1.0X -Parquet Vectorized (Pushdown) 13455 13505 51 1.2 855.4 1.0X -Native ORC Vectorized 12196 12247 41 1.3 775.4 1.1X -Native ORC Vectorized (Pushdown) 12230 12264 21 1.3 777.5 1.1X +Parquet Vectorized 13657 13798 103 1.2 868.3 1.0X +Parquet Vectorized (Pushdown) 13709 13730 14 1.1 871.6 1.0X +Native ORC Vectorized 12028 12061 37 1.3 764.7 1.1X +Native ORC Vectorized (Pushdown) 12105 12152 29 1.3 769.6 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6126 6159 26 2.6 389.5 1.0X -Parquet Vectorized (Pushdown) 260 273 10 60.6 16.5 23.6X -Native ORC Vectorized 4546 4572 18 3.5 289.0 1.3X -Native ORC Vectorized (Pushdown) 260 275 11 60.5 16.5 23.5X +Parquet Vectorized 6544 6575 26 2.4 416.1 1.0X +Parquet Vectorized (Pushdown) 274 283 9 57.4 17.4 23.9X +Native ORC Vectorized 4734 4753 13 3.3 301.0 1.4X +Native ORC Vectorized (Pushdown) 283 293 8 55.5 18.0 23.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6123 6138 11 2.6 389.3 1.0X -Parquet Vectorized (Pushdown) 273 286 15 57.7 17.3 22.4X -Native ORC Vectorized 4557 4590 42 3.5 289.7 1.3X -Native ORC Vectorized (Pushdown) 265 277 8 59.3 16.9 23.1X +Parquet Vectorized 6522 6549 39 2.4 414.7 1.0X +Parquet Vectorized (Pushdown) 285 296 10 55.3 18.1 22.9X +Native ORC Vectorized 4717 4734 11 3.3 299.9 1.4X +Native ORC Vectorized (Pushdown) 290 296 5 54.3 18.4 22.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6105 6212 72 2.6 388.1 1.0X -Parquet Vectorized (Pushdown) 272 277 7 57.8 17.3 22.4X -Native ORC Vectorized 4581 4651 77 3.4 291.2 1.3X -Native ORC Vectorized (Pushdown) 264 275 9 59.7 16.8 23.2X +Parquet Vectorized 6556 6567 10 2.4 416.8 1.0X +Parquet Vectorized (Pushdown) 279 288 6 56.3 17.8 23.5X +Native ORC Vectorized 4778 4790 8 3.3 303.8 1.4X +Native ORC Vectorized (Pushdown) 285 291 4 55.2 18.1 23.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6162 6217 59 2.6 391.8 1.0X -Parquet Vectorized (Pushdown) 263 275 9 59.8 16.7 23.4X -Native ORC Vectorized 4611 4630 23 3.4 293.1 1.3X -Native ORC Vectorized (Pushdown) 259 267 5 60.8 16.4 23.8X +Parquet Vectorized 6561 6594 41 2.4 417.1 1.0X +Parquet Vectorized (Pushdown) 279 284 4 56.4 17.7 23.5X +Native ORC Vectorized 4785 4792 8 3.3 304.2 1.4X +Native ORC Vectorized (Pushdown) 284 292 6 55.3 18.1 23.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6109 6137 21 2.6 388.4 1.0X -Parquet Vectorized (Pushdown) 260 270 5 60.4 16.5 23.5X -Native ORC Vectorized 4596 4621 34 3.4 292.2 1.3X -Native ORC Vectorized (Pushdown) 263 272 7 59.7 16.8 23.2X +Parquet Vectorized 6568 6587 21 2.4 417.6 1.0X +Parquet Vectorized (Pushdown) 277 282 3 56.9 17.6 23.7X +Native ORC Vectorized 4775 4798 19 3.3 303.6 1.4X +Native ORC Vectorized (Pushdown) 284 290 5 55.3 18.1 23.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6104 6142 41 2.6 388.1 1.0X -Parquet Vectorized (Pushdown) 266 278 13 59.0 16.9 22.9X -Native ORC Vectorized 4601 4668 40 3.4 292.5 1.3X -Native ORC Vectorized (Pushdown) 264 271 7 59.5 16.8 23.1X +Parquet Vectorized 6557 6576 20 2.4 416.9 1.0X +Parquet Vectorized (Pushdown) 275 283 7 57.3 17.5 23.9X +Native ORC Vectorized 4783 4807 23 3.3 304.1 1.4X +Native ORC Vectorized (Pushdown) 284 289 4 55.4 18.0 23.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 6775 6878 122 2.3 430.8 1.0X -Parquet Vectorized (Pushdown) 1502 1519 13 10.5 95.5 4.5X -Native ORC Vectorized 5241 5259 17 3.0 333.2 1.3X -Native ORC Vectorized (Pushdown) 1346 1359 11 11.7 85.6 5.0X +Parquet Vectorized 7224 7258 23 2.2 459.3 1.0X +Parquet Vectorized (Pushdown) 1586 1589 4 9.9 100.8 4.6X +Native ORC Vectorized 5423 5455 30 2.9 344.8 1.3X +Native ORC Vectorized (Pushdown) 1408 1430 25 11.2 89.5 5.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 9068 9115 48 1.7 576.5 1.0X -Parquet Vectorized (Pushdown) 6144 6157 17 2.6 390.6 1.5X -Native ORC Vectorized 7649 7712 67 2.1 486.3 1.2X -Native ORC Vectorized (Pushdown) 5542 5561 15 2.8 352.4 1.6X +Parquet Vectorized 9684 9692 5 1.6 615.7 1.0X +Parquet Vectorized (Pushdown) 6559 6581 17 2.4 417.0 1.5X +Native ORC Vectorized 7866 7894 24 2.0 500.1 1.2X +Native ORC Vectorized (Pushdown) 5654 5668 11 2.8 359.5 1.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 11477 11634 171 1.4 729.7 1.0X -Parquet Vectorized (Pushdown) 10963 11008 79 1.4 697.0 1.0X -Native ORC Vectorized 9938 9974 34 1.6 631.9 1.2X -Native ORC Vectorized (Pushdown) 9611 9667 77 1.6 611.1 1.2X +Parquet Vectorized 12234 12243 10 1.3 777.8 1.0X +Parquet Vectorized (Pushdown) 11654 11671 17 1.3 740.9 1.0X +Native ORC Vectorized 10449 10479 23 1.5 664.3 1.2X +Native ORC Vectorized (Pushdown) 10073 10120 40 1.6 640.4 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 11960 12016 47 1.3 760.4 1.0X -Parquet Vectorized (Pushdown) 12071 12135 58 1.3 767.5 1.0X -Native ORC Vectorized 10598 10650 53 1.5 673.8 1.1X -Native ORC Vectorized (Pushdown) 10651 10736 70 1.5 677.2 1.1X +Parquet Vectorized 12733 12756 20 1.2 809.5 1.0X +Parquet Vectorized (Pushdown) 12700 12719 20 1.2 807.4 1.0X +Native ORC Vectorized 10963 10996 31 1.4 697.0 1.2X +Native ORC Vectorized (Pushdown) 11063 11088 23 1.4 703.4 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 11960 11979 19 1.3 760.4 1.0X -Parquet Vectorized (Pushdown) 12058 12147 68 1.3 766.7 1.0X -Native ORC Vectorized 10563 10620 37 1.5 671.6 1.1X -Native ORC Vectorized (Pushdown) 10708 10947 187 1.5 680.8 1.1X +Parquet Vectorized 12663 12687 27 1.2 805.1 1.0X +Parquet Vectorized (Pushdown) 12760 12792 26 1.2 811.2 1.0X +Native ORC Vectorized 10947 10976 42 1.4 696.0 1.2X +Native ORC Vectorized (Pushdown) 11021 11058 24 1.4 700.7 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 12110 12189 60 1.3 769.9 1.0X -Parquet Vectorized (Pushdown) 12337 12422 81 1.3 784.4 1.0X -Native ORC Vectorized 10589 10660 105 1.5 673.2 1.1X -Native ORC Vectorized (Pushdown) 10648 10762 72 1.5 677.0 1.1X +Parquet Vectorized 12632 12656 24 1.2 803.1 1.0X +Parquet Vectorized (Pushdown) 12696 12733 30 1.2 807.2 1.0X +Native ORC Vectorized 10943 10969 22 1.4 695.7 1.2X +Native ORC Vectorized (Pushdown) 11050 11103 41 1.4 702.6 1.1X ================================================================================================ Pushdown for few distinct value case (use dictionary encoding) ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5859 5871 12 2.7 372.5 1.0X -Parquet Vectorized (Pushdown) 237 246 10 66.4 15.1 24.8X -Native ORC Vectorized 6491 6523 29 2.4 412.7 0.9X -Native ORC Vectorized (Pushdown) 907 910 2 17.3 57.6 6.5X +Parquet Vectorized 5890 5925 33 2.7 374.5 1.0X +Parquet Vectorized (Pushdown) 239 251 11 65.8 15.2 24.7X +Native ORC Vectorized 6519 6541 16 2.4 414.5 0.9X +Native ORC Vectorized (Pushdown) 959 961 1 16.4 61.0 6.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 5937 5962 23 2.6 377.5 1.0X -Parquet Vectorized (Pushdown) 239 245 9 65.8 15.2 24.9X -Native ORC Vectorized 6769 6788 34 2.3 430.4 0.9X -Native ORC Vectorized (Pushdown) 914 925 13 17.2 58.1 6.5X +Parquet Vectorized 6021 6046 24 2.6 382.8 1.0X +Parquet Vectorized (Pushdown) 241 257 11 65.2 15.3 25.0X +Native ORC Vectorized 6712 6738 20 2.3 426.7 0.9X +Native ORC Vectorized (Pushdown) 957 970 10 16.4 60.9 6.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5868 5878 6 2.7 373.1 1.0X -Parquet Vectorized (Pushdown) 284 289 3 55.3 18.1 20.6X -Native ORC Vectorized 6676 6696 23 2.4 424.5 0.9X -Native ORC Vectorized (Pushdown) 956 963 11 16.5 60.8 6.1X +Parquet Vectorized 5962 5982 17 2.6 379.1 1.0X +Parquet Vectorized (Pushdown) 288 294 8 54.6 18.3 20.7X +Native ORC Vectorized 6667 6694 27 2.4 423.9 0.9X +Native ORC Vectorized (Pushdown) 986 997 6 15.9 62.7 6.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 5940 5949 6 2.6 377.7 1.0X -Parquet Vectorized (Pushdown) 290 295 5 54.2 18.5 20.5X -Native ORC Vectorized 6733 6746 10 2.3 428.1 0.9X -Native ORC Vectorized (Pushdown) 953 966 12 16.5 60.6 6.2X +Parquet Vectorized 5961 5968 5 2.6 379.0 1.0X +Parquet Vectorized (Pushdown) 286 301 11 54.9 18.2 20.8X +Native ORC Vectorized 6618 6670 68 2.4 420.7 0.9X +Native ORC Vectorized (Pushdown) 988 996 10 15.9 62.8 6.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5946 5966 17 2.6 378.0 1.0X -Parquet Vectorized (Pushdown) 292 296 5 53.9 18.5 20.4X -Native ORC Vectorized 6741 6751 6 2.3 428.6 0.9X -Native ORC Vectorized (Pushdown) 958 964 5 16.4 60.9 6.2X +Parquet Vectorized 6025 6052 15 2.6 383.1 1.0X +Parquet Vectorized (Pushdown) 288 296 8 54.6 18.3 20.9X +Native ORC Vectorized 6727 6756 39 2.3 427.7 0.9X +Native ORC Vectorized (Pushdown) 988 997 10 15.9 62.8 6.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 14342 14388 32 1.1 911.8 1.0X -Parquet Vectorized (Pushdown) 14351 14404 51 1.1 912.4 1.0X -Native ORC Vectorized 14291 14316 30 1.1 908.6 1.0X -Native ORC Vectorized (Pushdown) 14452 14468 10 1.1 918.8 1.0X +Parquet Vectorized 14170 14195 23 1.1 900.9 1.0X +Parquet Vectorized (Pushdown) 14143 14168 22 1.1 899.2 1.0X +Native ORC Vectorized 14438 14488 39 1.1 918.0 1.0X +Native ORC Vectorized (Pushdown) 14638 14703 103 1.1 930.6 1.0X ================================================================================================ Pushdown benchmark for StringStartsWith ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 7092 7104 14 2.2 450.9 1.0X -Parquet Vectorized (Pushdown) 930 935 3 16.9 59.1 7.6X -Native ORC Vectorized 5306 5327 28 3.0 337.3 1.3X -Native ORC Vectorized (Pushdown) 5385 5398 10 2.9 342.4 1.3X +Parquet Vectorized 7156 7212 82 2.2 455.0 1.0X +Parquet Vectorized (Pushdown) 948 952 5 16.6 60.2 7.6X +Native ORC Vectorized 5320 5345 31 3.0 338.2 1.3X +Native ORC Vectorized (Pushdown) 5413 5424 7 2.9 344.1 1.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6867 6913 26 2.3 436.6 1.0X -Parquet Vectorized (Pushdown) 278 283 4 56.5 17.7 24.7X -Native ORC Vectorized 5146 5154 5 3.1 327.2 1.3X -Native ORC Vectorized (Pushdown) 5225 5236 7 3.0 332.2 1.3X +Parquet Vectorized 7011 7034 30 2.2 445.7 1.0X +Parquet Vectorized (Pushdown) 280 286 8 56.1 17.8 25.0X +Native ORC Vectorized 5194 5204 9 3.0 330.2 1.3X +Native ORC Vectorized (Pushdown) 5264 5284 15 3.0 334.7 1.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6831 6849 11 2.3 434.3 1.0X -Parquet Vectorized (Pushdown) 265 272 6 59.2 16.9 25.7X -Native ORC Vectorized 5114 5140 18 3.1 325.2 1.3X -Native ORC Vectorized (Pushdown) 5193 5227 27 3.0 330.1 1.3X +Parquet Vectorized 6995 7025 21 2.2 444.7 1.0X +Parquet Vectorized (Pushdown) 277 287 12 56.7 17.6 25.2X +Native ORC Vectorized 5182 5205 23 3.0 329.5 1.3X +Native ORC Vectorized (Pushdown) 5260 5271 11 3.0 334.4 1.3X ================================================================================================ Pushdown benchmark for StringEndsWith ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringEndsWith filter: (value like '%10'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5890 5903 16 2.7 374.5 1.0X -Parquet Vectorized (Pushdown) 366 369 3 43.0 23.3 16.1X -Native ORC Vectorized 6686 6712 29 2.4 425.1 0.9X -Native ORC Vectorized (Pushdown) 6877 6895 12 2.3 437.2 0.9X +Parquet Vectorized 5961 5985 24 2.6 379.0 1.0X +Parquet Vectorized (Pushdown) 366 375 10 42.9 23.3 16.3X +Native ORC Vectorized 6698 6718 19 2.3 425.9 0.9X +Native ORC Vectorized (Pushdown) 6899 6921 18 2.3 438.6 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringEndsWith filter: (value like '%1000'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5860 5875 13 2.7 372.6 1.0X -Parquet Vectorized (Pushdown) 269 275 7 58.5 17.1 21.8X -Native ORC Vectorized 6606 6637 23 2.4 420.0 0.9X -Native ORC Vectorized (Pushdown) 6803 6830 28 2.3 432.5 0.9X +Parquet Vectorized 5940 5950 7 2.6 377.7 1.0X +Parquet Vectorized (Pushdown) 269 278 9 58.5 17.1 22.1X +Native ORC Vectorized 6613 6643 24 2.4 420.5 0.9X +Native ORC Vectorized (Pushdown) 6856 6870 14 2.3 435.9 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringEndsWith filter: (value like '%786432'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5806 5835 24 2.7 369.1 1.0X -Parquet Vectorized (Pushdown) 263 271 4 59.9 16.7 22.1X -Native ORC Vectorized 6617 6624 5 2.4 420.7 0.9X -Native ORC Vectorized (Pushdown) 6771 6784 13 2.3 430.5 0.9X +Parquet Vectorized 5939 5955 17 2.6 377.6 1.0X +Parquet Vectorized (Pushdown) 270 292 36 58.2 17.2 22.0X +Native ORC Vectorized 6634 6655 20 2.4 421.8 0.9X +Native ORC Vectorized (Pushdown) 6824 6853 25 2.3 433.9 0.9X ================================================================================================ Pushdown benchmark for StringContains ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringContains filter: (value like '%10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5983 6035 43 2.6 380.4 1.0X -Parquet Vectorized (Pushdown) 790 795 4 19.9 50.2 7.6X -Native ORC Vectorized 6785 6803 11 2.3 431.4 0.9X -Native ORC Vectorized (Pushdown) 6943 6977 24 2.3 441.4 0.9X +Parquet Vectorized 6160 6173 14 2.6 391.7 1.0X +Parquet Vectorized (Pushdown) 801 808 5 19.6 50.9 7.7X +Native ORC Vectorized 6872 6881 11 2.3 436.9 0.9X +Native ORC Vectorized (Pushdown) 7071 7082 7 2.2 449.6 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringContains filter: (value like '%1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5790 5839 29 2.7 368.1 1.0X -Parquet Vectorized (Pushdown) 267 271 4 58.9 17.0 21.7X -Native ORC Vectorized 6623 6635 14 2.4 421.1 0.9X -Native ORC Vectorized (Pushdown) 6782 6797 10 2.3 431.2 0.9X +Parquet Vectorized 5949 5968 17 2.6 378.2 1.0X +Parquet Vectorized (Pushdown) 272 278 4 57.9 17.3 21.9X +Native ORC Vectorized 6649 6666 23 2.4 422.7 0.9X +Native ORC Vectorized (Pushdown) 6860 6888 23 2.3 436.1 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor StringContains filter: (value like '%786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 5811 5831 16 2.7 369.4 1.0X -Parquet Vectorized (Pushdown) 263 273 10 59.8 16.7 22.1X -Native ORC Vectorized 6563 6609 27 2.4 417.3 0.9X -Native ORC Vectorized (Pushdown) 6734 6772 26 2.3 428.1 0.9X +Parquet Vectorized 5936 5959 19 2.6 377.4 1.0X +Parquet Vectorized (Pushdown) 271 276 3 58.1 17.2 21.9X +Native ORC Vectorized 6637 6657 15 2.4 422.0 0.9X +Native ORC Vectorized (Pushdown) 6852 6878 24 2.3 435.6 0.9X ================================================================================================ Pushdown benchmark for decimal ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2805 2825 20 5.6 178.3 1.0X -Parquet Vectorized (Pushdown) 70 73 5 226.2 4.4 40.3X -Native ORC Vectorized 3503 3543 26 4.5 222.7 0.8X -Native ORC Vectorized (Pushdown) 55 59 3 286.3 3.5 51.1X +Parquet Vectorized 2866 2879 13 5.5 182.2 1.0X +Parquet Vectorized (Pushdown) 71 74 5 222.8 4.5 40.6X +Native ORC Vectorized 3278 3290 12 4.8 208.4 0.9X +Native ORC Vectorized (Pushdown) 59 63 5 268.1 3.7 48.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 4151 4162 9 3.8 263.9 1.0X -Parquet Vectorized (Pushdown) 1966 1981 15 8.0 125.0 2.1X -Native ORC Vectorized 5029 5082 51 3.1 319.7 0.8X -Native ORC Vectorized (Pushdown) 2193 2203 8 7.2 139.4 1.9X +Parquet Vectorized 4201 4213 14 3.7 267.1 1.0X +Parquet Vectorized (Pushdown) 1950 1959 6 8.1 124.0 2.2X +Native ORC Vectorized 4784 4797 15 3.3 304.2 0.9X +Native ORC Vectorized (Pushdown) 2117 2120 3 7.4 134.6 2.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8776 8810 40 1.8 558.0 1.0X -Parquet Vectorized (Pushdown) 8460 8484 13 1.9 537.9 1.0X -Native ORC Vectorized 9930 9952 17 1.6 631.4 0.9X -Native ORC Vectorized (Pushdown) 9440 9476 30 1.7 600.2 0.9X +Parquet Vectorized 8854 8870 18 1.8 562.9 1.0X +Parquet Vectorized (Pushdown) 8480 8486 8 1.9 539.1 1.0X +Native ORC Vectorized 9614 9653 56 1.6 611.2 0.9X +Native ORC Vectorized (Pushdown) 9180 9242 85 1.7 583.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 10066 10176 84 1.6 640.0 1.0X -Parquet Vectorized (Pushdown) 10147 10173 29 1.6 645.1 1.0X -Native ORC Vectorized 10790 10854 60 1.5 686.0 0.9X -Native ORC Vectorized (Pushdown) 10900 11013 189 1.4 693.0 0.9X +Parquet Vectorized 9942 9968 19 1.6 632.1 1.0X +Parquet Vectorized (Pushdown) 9975 9993 14 1.6 634.2 1.0X +Native ORC Vectorized 10610 10638 19 1.5 674.6 0.9X +Native ORC Vectorized (Pushdown) 10626 10648 16 1.5 675.6 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2978 2995 19 5.3 189.3 1.0X -Parquet Vectorized (Pushdown) 69 72 3 229.4 4.4 43.4X -Native ORC Vectorized 3520 3535 13 4.5 223.8 0.8X -Native ORC Vectorized (Pushdown) 53 56 3 296.7 3.4 56.2X +Parquet Vectorized 3028 3063 27 5.2 192.5 1.0X +Parquet Vectorized (Pushdown) 69 71 2 227.4 4.4 43.8X +Native ORC Vectorized 3306 3322 21 4.8 210.2 0.9X +Native ORC Vectorized (Pushdown) 56 59 4 281.4 3.6 54.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3733 3745 9 4.2 237.4 1.0X -Parquet Vectorized (Pushdown) 1078 1089 6 14.6 68.5 3.5X -Native ORC Vectorized 4305 4316 11 3.7 273.7 0.9X -Native ORC Vectorized (Pushdown) 1110 1113 3 14.2 70.6 3.4X +Parquet Vectorized 3790 3798 7 4.1 241.0 1.0X +Parquet Vectorized (Pushdown) 1082 1086 3 14.5 68.8 3.5X +Native ORC Vectorized 4052 4071 35 3.9 257.6 0.9X +Native ORC Vectorized (Pushdown) 1078 1081 3 14.6 68.5 3.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6729 6741 14 2.3 427.8 1.0X -Parquet Vectorized (Pushdown) 5185 5240 41 3.0 329.7 1.3X -Native ORC Vectorized 7200 7224 21 2.2 457.8 0.9X -Native ORC Vectorized (Pushdown) 5405 5438 22 2.9 343.7 1.2X +Parquet Vectorized 6691 6712 15 2.4 425.4 1.0X +Parquet Vectorized (Pushdown) 5196 5211 17 3.0 330.3 1.3X +Native ORC Vectorized 6925 6934 8 2.3 440.3 1.0X +Native ORC Vectorized (Pushdown) 5264 5279 19 3.0 334.7 1.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9576 9593 18 1.6 608.8 1.0X -Parquet Vectorized (Pushdown) 9301 9312 16 1.7 591.3 1.0X -Native ORC Vectorized 10115 10143 26 1.6 643.1 0.9X -Native ORC Vectorized (Pushdown) 9809 9814 3 1.6 623.6 1.0X +Parquet Vectorized 9504 9527 19 1.7 604.3 1.0X +Parquet Vectorized (Pushdown) 9218 9233 17 1.7 586.1 1.0X +Native ORC Vectorized 9809 9836 18 1.6 623.6 1.0X +Native ORC Vectorized (Pushdown) 9507 9531 17 1.7 604.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 4258 4267 8 3.7 270.7 1.0X -Parquet Vectorized (Pushdown) 75 77 3 210.3 4.8 56.9X -Native ORC Vectorized 3587 3638 85 4.4 228.1 1.2X -Native ORC Vectorized (Pushdown) 52 55 3 302.8 3.3 82.0X +Parquet Vectorized 4264 4282 25 3.7 271.1 1.0X +Parquet Vectorized (Pushdown) 75 79 3 208.7 4.8 56.6X +Native ORC Vectorized 3347 3364 16 4.7 212.8 1.3X +Native ORC Vectorized (Pushdown) 55 59 5 283.6 3.5 76.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5156 5167 18 3.1 327.8 1.0X -Parquet Vectorized (Pushdown) 1386 1395 9 11.3 88.1 3.7X -Native ORC Vectorized 4486 4506 27 3.5 285.2 1.1X -Native ORC Vectorized (Pushdown) 1242 1251 8 12.7 79.0 4.2X +Parquet Vectorized 5214 5231 10 3.0 331.5 1.0X +Parquet Vectorized (Pushdown) 1409 1413 2 11.2 89.6 3.7X +Native ORC Vectorized 4207 4222 25 3.7 267.5 1.2X +Native ORC Vectorized (Pushdown) 1209 1211 2 13.0 76.9 4.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8812 8822 7 1.8 560.2 1.0X -Parquet Vectorized (Pushdown) 6728 6732 4 2.3 427.8 1.3X -Native ORC Vectorized 7787 7836 60 2.0 495.1 1.1X -Native ORC Vectorized (Pushdown) 6007 6023 24 2.6 381.9 1.5X +Parquet Vectorized 8897 8913 12 1.8 565.6 1.0X +Parquet Vectorized (Pushdown) 6816 6830 19 2.3 433.3 1.3X +Native ORC Vectorized 7648 7665 14 2.1 486.3 1.2X +Native ORC Vectorized (Pushdown) 5932 5954 25 2.7 377.1 1.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 12367 12381 9 1.3 786.3 1.0X -Parquet Vectorized (Pushdown) 11977 12018 24 1.3 761.5 1.0X -Native ORC Vectorized 11109 11169 79 1.4 706.3 1.1X -Native ORC Vectorized (Pushdown) 10772 10786 15 1.5 684.9 1.1X +Parquet Vectorized 12548 12558 13 1.3 797.8 1.0X +Parquet Vectorized (Pushdown) 12139 12156 22 1.3 771.8 1.0X +Native ORC Vectorized 11055 11089 25 1.4 702.8 1.1X +Native ORC Vectorized (Pushdown) 10746 10789 41 1.5 683.2 1.2X ================================================================================================ Pushdown benchmark for InSet -> InFilters ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6442 6482 25 2.4 409.6 1.0X -Parquet Vectorized (Pushdown) 284 293 15 55.4 18.0 22.7X -Native ORC Vectorized 4965 4990 17 3.2 315.7 1.3X -Native ORC Vectorized (Pushdown) 281 288 8 56.1 17.8 23.0X +Parquet Vectorized 6561 6633 92 2.4 417.1 1.0X +Parquet Vectorized (Pushdown) 281 287 6 55.9 17.9 23.3X +Native ORC Vectorized 4666 4681 19 3.4 296.6 1.4X +Native ORC Vectorized (Pushdown) 289 298 8 54.4 18.4 22.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6451 6461 9 2.4 410.1 1.0X -Parquet Vectorized (Pushdown) 283 286 3 55.6 18.0 22.8X -Native ORC Vectorized 4937 4981 34 3.2 313.9 1.3X -Native ORC Vectorized (Pushdown) 279 288 11 56.4 17.7 23.1X +Parquet Vectorized 6554 6599 62 2.4 416.7 1.0X +Parquet Vectorized (Pushdown) 284 295 10 55.3 18.1 23.1X +Native ORC Vectorized 4674 4695 23 3.4 297.1 1.4X +Native ORC Vectorized (Pushdown) 293 303 13 53.7 18.6 22.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6449 6462 8 2.4 410.0 1.0X -Parquet Vectorized (Pushdown) 283 293 10 55.6 18.0 22.8X -Native ORC Vectorized 4955 4964 12 3.2 315.0 1.3X -Native ORC Vectorized (Pushdown) 280 284 3 56.2 17.8 23.0X +Parquet Vectorized 6540 6557 16 2.4 415.8 1.0X +Parquet Vectorized (Pushdown) 284 298 12 55.4 18.1 23.0X +Native ORC Vectorized 4667 4680 8 3.4 296.7 1.4X +Native ORC Vectorized (Pushdown) 290 297 7 54.3 18.4 22.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6448 6474 23 2.4 409.9 1.0X -Parquet Vectorized (Pushdown) 302 321 43 52.1 19.2 21.4X -Native ORC Vectorized 4977 4994 16 3.2 316.4 1.3X -Native ORC Vectorized (Pushdown) 297 301 3 53.0 18.9 21.7X +Parquet Vectorized 6564 6587 18 2.4 417.3 1.0X +Parquet Vectorized (Pushdown) 299 306 4 52.6 19.0 21.9X +Native ORC Vectorized 4686 4707 22 3.4 297.9 1.4X +Native ORC Vectorized (Pushdown) 305 310 3 51.5 19.4 21.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6478 6503 36 2.4 411.8 1.0X -Parquet Vectorized (Pushdown) 301 307 4 52.2 19.1 21.5X -Native ORC Vectorized 4972 5002 20 3.2 316.1 1.3X -Native ORC Vectorized (Pushdown) 297 305 11 52.9 18.9 21.8X +Parquet Vectorized 6568 6599 31 2.4 417.6 1.0X +Parquet Vectorized (Pushdown) 307 309 1 51.2 19.5 21.4X +Native ORC Vectorized 4684 4700 19 3.4 297.8 1.4X +Native ORC Vectorized (Pushdown) 302 310 11 52.1 19.2 21.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6506 6522 11 2.4 413.7 1.0X -Parquet Vectorized (Pushdown) 305 309 3 51.6 19.4 21.3X -Native ORC Vectorized 5057 5062 4 3.1 321.5 1.3X -Native ORC Vectorized (Pushdown) 304 309 3 51.7 19.4 21.4X +Parquet Vectorized 6567 6584 12 2.4 417.5 1.0X +Parquet Vectorized (Pushdown) 306 308 3 51.4 19.5 21.5X +Native ORC Vectorized 4684 4694 9 3.4 297.8 1.4X +Native ORC Vectorized (Pushdown) 308 313 3 51.1 19.6 21.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6675 6693 24 2.4 424.4 1.0X -Parquet Vectorized (Pushdown) 896 899 3 17.6 57.0 7.4X -Native ORC Vectorized 5264 5272 9 3.0 334.7 1.3X -Native ORC Vectorized (Pushdown) 407 410 4 38.7 25.9 16.4X +Parquet Vectorized 6743 6760 23 2.3 428.7 1.0X +Parquet Vectorized (Pushdown) 902 909 7 17.4 57.3 7.5X +Native ORC Vectorized 4877 4900 14 3.2 310.1 1.4X +Native ORC Vectorized (Pushdown) 414 415 1 38.0 26.3 16.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6662 6668 8 2.4 423.6 1.0X -Parquet Vectorized (Pushdown) 3357 3364 6 4.7 213.4 2.0X -Native ORC Vectorized 5164 5191 23 3.0 328.3 1.3X -Native ORC Vectorized (Pushdown) 426 429 3 36.9 27.1 15.6X +Parquet Vectorized 6731 6752 22 2.3 427.9 1.0X +Parquet Vectorized (Pushdown) 3328 3339 11 4.7 211.6 2.0X +Native ORC Vectorized 4904 4908 5 3.2 311.8 1.4X +Native ORC Vectorized (Pushdown) 431 433 2 36.5 27.4 15.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6648 6658 12 2.4 422.7 1.0X -Parquet Vectorized (Pushdown) 5843 5860 15 2.7 371.5 1.1X -Native ORC Vectorized 5182 5189 6 3.0 329.5 1.3X -Native ORC Vectorized (Pushdown) 432 436 9 36.4 27.5 15.4X +Parquet Vectorized 6720 6732 8 2.3 427.2 1.0X +Parquet Vectorized (Pushdown) 6064 6085 14 2.6 385.6 1.1X +Native ORC Vectorized 4885 4893 11 3.2 310.6 1.4X +Native ORC Vectorized (Pushdown) 439 451 20 35.9 27.9 15.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6611 6629 16 2.4 420.3 1.0X -Parquet Vectorized (Pushdown) 892 898 5 17.6 56.7 7.4X -Native ORC Vectorized 5126 5154 32 3.1 325.9 1.3X -Native ORC Vectorized (Pushdown) 500 505 3 31.4 31.8 13.2X +Parquet Vectorized 6698 6710 9 2.3 425.8 1.0X +Parquet Vectorized (Pushdown) 927 932 5 17.0 59.0 7.2X +Native ORC Vectorized 4843 4859 31 3.2 307.9 1.4X +Native ORC Vectorized (Pushdown) 509 515 6 30.9 32.4 13.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6556 6583 17 2.4 416.8 1.0X -Parquet Vectorized (Pushdown) 3433 3448 17 4.6 218.2 1.9X -Native ORC Vectorized 5099 5119 12 3.1 324.2 1.3X -Native ORC Vectorized (Pushdown) 570 572 3 27.6 36.2 11.5X +Parquet Vectorized 6698 6717 19 2.3 425.8 1.0X +Parquet Vectorized (Pushdown) 3443 3458 19 4.6 218.9 1.9X +Native ORC Vectorized 4838 4869 27 3.3 307.6 1.4X +Native ORC Vectorized (Pushdown) 571 574 4 27.6 36.3 11.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6631 6642 10 2.4 421.6 1.0X -Parquet Vectorized (Pushdown) 5877 5888 8 2.7 373.6 1.1X -Native ORC Vectorized 5141 5148 8 3.1 326.9 1.3X -Native ORC Vectorized (Pushdown) 585 587 2 26.9 37.2 11.3X +Parquet Vectorized 6694 6714 16 2.3 425.6 1.0X +Parquet Vectorized (Pushdown) 5855 5876 26 2.7 372.2 1.1X +Native ORC Vectorized 4833 4848 14 3.3 307.3 1.4X +Native ORC Vectorized (Pushdown) 559 561 2 28.1 35.5 12.0X ================================================================================================ Pushdown benchmark for tinyint ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3074 3122 77 5.1 195.4 1.0X -Parquet Vectorized (Pushdown) 107 111 6 146.8 6.8 28.7X -Native ORC Vectorized 2473 2482 6 6.4 157.2 1.2X -Native ORC Vectorized (Pushdown) 114 117 5 138.0 7.2 27.0X +Parquet Vectorized 3146 3222 55 5.0 200.0 1.0X +Parquet Vectorized (Pushdown) 108 112 4 145.3 6.9 29.1X +Native ORC Vectorized 2163 2182 29 7.3 137.5 1.5X +Native ORC Vectorized (Pushdown) 115 119 6 137.4 7.3 27.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3739 3758 16 4.2 237.7 1.0X -Parquet Vectorized (Pushdown) 1004 1011 5 15.7 63.8 3.7X -Native ORC Vectorized 3078 3092 17 5.1 195.7 1.2X -Native ORC Vectorized (Pushdown) 918 920 1 17.1 58.4 4.1X +Parquet Vectorized 3769 3787 16 4.2 239.6 1.0X +Parquet Vectorized (Pushdown) 990 998 6 15.9 63.0 3.8X +Native ORC Vectorized 2723 2728 3 5.8 173.1 1.4X +Native ORC Vectorized (Pushdown) 854 857 2 18.4 54.3 4.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6391 6394 2 2.5 406.3 1.0X -Parquet Vectorized (Pushdown) 4890 4907 10 3.2 310.9 1.3X -Native ORC Vectorized 5584 5613 20 2.8 355.0 1.1X -Native ORC Vectorized (Pushdown) 4397 4412 10 3.6 279.6 1.5X +Parquet Vectorized 6447 6463 14 2.4 409.9 1.0X +Parquet Vectorized (Pushdown) 4925 4941 13 3.2 313.1 1.3X +Native ORC Vectorized 5158 5166 9 3.0 327.9 1.2X +Native ORC Vectorized (Pushdown) 4108 4130 29 3.8 261.2 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9020 9050 32 1.7 573.4 1.0X -Parquet Vectorized (Pushdown) 8757 8792 23 1.8 556.7 1.0X -Native ORC Vectorized 8277 8316 42 1.9 526.2 1.1X -Native ORC Vectorized (Pushdown) 8050 8069 14 2.0 511.8 1.1X +Parquet Vectorized 8983 9016 29 1.8 571.1 1.0X +Parquet Vectorized (Pushdown) 8710 8734 14 1.8 553.8 1.0X +Native ORC Vectorized 7637 7670 33 2.1 485.5 1.2X +Native ORC Vectorized (Pushdown) 7453 7479 22 2.1 473.9 1.2X ================================================================================================ Pushdown benchmark for Timestamp ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 timestamp stored as INT96 row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3218 3228 10 4.9 204.6 1.0X -Parquet Vectorized (Pushdown) 3203 3213 10 4.9 203.7 1.0X -Native ORC Vectorized 2387 2391 5 6.6 151.7 1.3X -Native ORC Vectorized (Pushdown) 39 42 4 407.8 2.5 83.4X +Parquet Vectorized 3234 3250 11 4.9 205.6 1.0X +Parquet Vectorized (Pushdown) 3243 3254 10 4.9 206.2 1.0X +Native ORC Vectorized 2077 2092 22 7.6 132.1 1.6X +Native ORC Vectorized (Pushdown) 40 42 4 394.9 2.5 81.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% timestamp stored as INT96 rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3900 3912 11 4.0 247.9 1.0X -Parquet Vectorized (Pushdown) 3896 3903 6 4.0 247.7 1.0X -Native ORC Vectorized 2987 2996 11 5.3 189.9 1.3X -Native ORC Vectorized (Pushdown) 889 892 3 17.7 56.5 4.4X +Parquet Vectorized 3945 3975 41 4.0 250.8 1.0X +Parquet Vectorized (Pushdown) 3936 3950 15 4.0 250.3 1.0X +Native ORC Vectorized 2699 2712 16 5.8 171.6 1.5X +Native ORC Vectorized (Pushdown) 864 871 6 18.2 54.9 4.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% timestamp stored as INT96 rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6698 6711 15 2.3 425.9 1.0X -Parquet Vectorized (Pushdown) 6698 6708 9 2.3 425.8 1.0X -Native ORC Vectorized 5550 5563 22 2.8 352.9 1.2X -Native ORC Vectorized (Pushdown) 4359 4374 18 3.6 277.2 1.5X +Parquet Vectorized 6749 6770 14 2.3 429.1 1.0X +Parquet Vectorized (Pushdown) 6746 6762 24 2.3 428.9 1.0X +Native ORC Vectorized 5192 5218 16 3.0 330.1 1.3X +Native ORC Vectorized (Pushdown) 4140 4152 15 3.8 263.2 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% timestamp stored as INT96 rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9385 9400 18 1.7 596.7 1.0X -Parquet Vectorized (Pushdown) 9378 9390 11 1.7 596.3 1.0X -Native ORC Vectorized 8168 8194 28 1.9 519.3 1.1X -Native ORC Vectorized (Pushdown) 7949 7959 9 2.0 505.4 1.2X +Parquet Vectorized 9447 9478 29 1.7 600.6 1.0X +Parquet Vectorized (Pushdown) 9462 9486 25 1.7 601.6 1.0X +Native ORC Vectorized 7861 7994 77 2.0 499.8 1.2X +Native ORC Vectorized (Pushdown) 7811 7838 17 2.0 496.6 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 timestamp stored as TIMESTAMP_MICROS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 2972 2983 14 5.3 189.0 1.0X -Parquet Vectorized (Pushdown) 68 71 3 231.7 4.3 43.8X -Native ORC Vectorized 2359 2366 4 6.7 150.0 1.3X -Native ORC Vectorized (Pushdown) 38 40 3 416.7 2.4 78.7X +Parquet Vectorized 3007 3018 9 5.2 191.2 1.0X +Parquet Vectorized (Pushdown) 69 71 3 229.6 4.4 43.9X +Native ORC Vectorized 2066 2069 2 7.6 131.4 1.5X +Native ORC Vectorized (Pushdown) 39 41 3 399.5 2.5 76.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3649 3652 4 4.3 232.0 1.0X -Parquet Vectorized (Pushdown) 1043 1047 3 15.1 66.3 3.5X -Native ORC Vectorized 2989 2989 1 5.3 190.0 1.2X -Native ORC Vectorized (Pushdown) 888 892 2 17.7 56.5 4.1X +Parquet Vectorized 3717 3738 27 4.2 236.3 1.0X +Parquet Vectorized (Pushdown) 1052 1055 4 14.9 66.9 3.5X +Native ORC Vectorized 2695 2702 7 5.8 171.3 1.4X +Native ORC Vectorized (Pushdown) 863 867 5 18.2 54.9 4.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6413 6426 8 2.5 407.7 1.0X -Parquet Vectorized (Pushdown) 5009 5049 48 3.1 318.5 1.3X -Native ORC Vectorized 5548 5553 11 2.8 352.7 1.2X -Native ORC Vectorized (Pushdown) 4359 4368 9 3.6 277.1 1.5X +Parquet Vectorized 6536 6551 9 2.4 415.5 1.0X +Parquet Vectorized (Pushdown) 5041 5059 13 3.1 320.5 1.3X +Native ORC Vectorized 5201 5223 21 3.0 330.7 1.3X +Native ORC Vectorized (Pushdown) 4134 4139 5 3.8 262.9 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9114 9138 26 1.7 579.5 1.0X -Parquet Vectorized (Pushdown) 8869 8880 9 1.8 563.9 1.0X -Native ORC Vectorized 8175 8193 20 1.9 519.8 1.1X -Native ORC Vectorized (Pushdown) 7947 7956 11 2.0 505.3 1.1X +Parquet Vectorized 9201 9222 27 1.7 585.0 1.0X +Parquet Vectorized (Pushdown) 8940 8961 23 1.8 568.4 1.0X +Native ORC Vectorized 7987 8023 27 2.0 507.8 1.2X +Native ORC Vectorized (Pushdown) 7792 7808 21 2.0 495.4 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3001 3005 6 5.2 190.8 1.0X -Parquet Vectorized (Pushdown) 68 70 3 232.2 4.3 44.3X -Native ORC Vectorized 2359 2362 3 6.7 150.0 1.3X -Native ORC Vectorized (Pushdown) 38 40 4 415.7 2.4 79.3X +Parquet Vectorized 3037 3044 8 5.2 193.1 1.0X +Parquet Vectorized (Pushdown) 68 71 3 230.6 4.3 44.5X +Native ORC Vectorized 2068 2082 23 7.6 131.5 1.5X +Native ORC Vectorized (Pushdown) 39 42 3 400.5 2.5 77.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3686 3693 5 4.3 234.4 1.0X -Parquet Vectorized (Pushdown) 1044 1048 4 15.1 66.4 3.5X -Native ORC Vectorized 2984 2989 4 5.3 189.7 1.2X -Native ORC Vectorized (Pushdown) 889 891 2 17.7 56.5 4.1X +Parquet Vectorized 3730 3735 5 4.2 237.1 1.0X +Parquet Vectorized (Pushdown) 1047 1052 4 15.0 66.5 3.6X +Native ORC Vectorized 2700 2704 4 5.8 171.7 1.4X +Native ORC Vectorized (Pushdown) 861 877 22 18.3 54.8 4.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6454 6464 8 2.4 410.3 1.0X -Parquet Vectorized (Pushdown) 5018 5033 26 3.1 319.0 1.3X -Native ORC Vectorized 5545 5556 9 2.8 352.5 1.2X -Native ORC Vectorized (Pushdown) 4357 4377 14 3.6 277.0 1.5X +Parquet Vectorized 6517 6528 18 2.4 414.4 1.0X +Parquet Vectorized (Pushdown) 5046 5050 5 3.1 320.8 1.3X +Native ORC Vectorized 5189 5203 15 3.0 329.9 1.3X +Native ORC Vectorized (Pushdown) 4131 4148 22 3.8 262.6 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 9143 9162 28 1.7 581.3 1.0X -Parquet Vectorized (Pushdown) 8888 8895 6 1.8 565.1 1.0X -Native ORC Vectorized 8163 8178 25 1.9 519.0 1.1X -Native ORC Vectorized (Pushdown) 7942 7966 35 2.0 504.9 1.2X +Parquet Vectorized 9216 9225 7 1.7 585.9 1.0X +Parquet Vectorized (Pushdown) 8966 8976 15 1.8 570.0 1.0X +Native ORC Vectorized 7990 8006 18 2.0 508.0 1.2X +Native ORC Vectorized (Pushdown) 7788 7804 19 2.0 495.1 1.2X ================================================================================================ Pushdown benchmark with many filters ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 55 57 2 0.0 55430933.0 1.0X -Parquet Vectorized (Pushdown) 56 59 5 0.0 56257088.0 1.0X -Native ORC Vectorized 50 52 2 0.0 50120677.0 1.1X -Native ORC Vectorized (Pushdown) 52 55 3 0.0 52126525.0 1.1X +Parquet Vectorized 48 50 2 0.0 47822192.0 1.0X +Parquet Vectorized (Pushdown) 49 52 4 0.0 48715892.0 1.0X +Native ORC Vectorized 43 44 2 0.0 42630483.0 1.1X +Native ORC Vectorized (Pushdown) 44 47 4 0.0 44086388.0 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 405 417 11 0.0 404938298.0 1.0X -Parquet Vectorized (Pushdown) 407 412 6 0.0 406571487.0 1.0X -Native ORC Vectorized 394 397 4 0.0 394366762.0 1.0X -Native ORC Vectorized (Pushdown) 397 406 7 0.0 396723685.0 1.0X +Parquet Vectorized 189 197 6 0.0 189302685.0 1.0X +Parquet Vectorized (Pushdown) 192 196 5 0.0 191858297.0 1.0X +Native ORC Vectorized 182 189 7 0.0 182429398.0 1.0X +Native ORC Vectorized (Pushdown) 186 190 3 0.0 185920182.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Parquet Vectorized 2255 2270 9 0.0 2255338602.0 1.0X -Parquet Vectorized (Pushdown) 2258 2279 13 0.0 2258126416.0 1.0X -Native ORC Vectorized 2244 2260 10 0.0 2243733317.0 1.0X -Native ORC Vectorized (Pushdown) 2255 2290 29 0.0 2254729481.0 1.0X +Parquet Vectorized 600 607 9 0.0 599862493.0 1.0X +Parquet Vectorized (Pushdown) 606 622 12 0.0 605756895.0 1.0X +Native ORC Vectorized 591 600 12 0.0 591069360.0 1.0X +Native ORC Vectorized (Pushdown) 595 609 9 0.0 594620092.0 1.0X diff --git a/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt b/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt index bd83ba8858f29..ae1f8694afbf5 100644 --- a/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt @@ -2,11 +2,11 @@ GenerateExec benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor GenerateExec Benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -GenerateExec Benchmark wholestage off 71281 71290 12 1.4 712.8 1.0X -GenerateExec Benchmark wholestage on 21377 22190 461 4.7 213.8 3.3X +GenerateExec Benchmark wholestage off 73608 73642 47 1.4 736.1 1.0X +GenerateExec Benchmark wholestage on 20481 20591 165 4.9 204.8 3.6X diff --git a/sql/core/benchmarks/GenerateExecBenchmark-results.txt b/sql/core/benchmarks/GenerateExecBenchmark-results.txt index 7aaa8fad9e560..6790608ad6b2b 100644 --- a/sql/core/benchmarks/GenerateExecBenchmark-results.txt +++ b/sql/core/benchmarks/GenerateExecBenchmark-results.txt @@ -2,11 +2,11 @@ GenerateExec benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor GenerateExec Benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -GenerateExec Benchmark wholestage off 73307 73512 290 1.4 733.1 1.0X -GenerateExec Benchmark wholestage on 24438 24523 84 4.1 244.4 3.0X +GenerateExec Benchmark wholestage off 72745 72752 10 1.4 727.4 1.0X +GenerateExec Benchmark wholestage on 23957 24433 339 4.2 239.6 3.0X diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt index 8e47f7e27a85b..e326f00783419 100644 --- a/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt @@ -2,10 +2,10 @@ LongToUnsafeRowMap metrics ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LongToUnsafeRowMap metrics: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -LongToUnsafeRowMap 255 259 3 2.0 510.5 1.0X +LongToUnsafeRowMap 266 269 3 1.9 532.1 1.0X diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt index f201c27de387e..5364545cd8af7 100644 --- a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt +++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt @@ -2,10 +2,10 @@ LongToUnsafeRowMap metrics ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor LongToUnsafeRowMap metrics: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -LongToUnsafeRowMap 261 268 6 1.9 521.5 1.0X +LongToUnsafeRowMap 260 264 3 1.9 519.7 1.0X diff --git a/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt b/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt index e571db07479a0..4ee151d851b96 100644 --- a/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt @@ -2,739 +2,739 @@ In Expression Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 46 61 9 217.9 4.6 1.0X -InSet expression 68 73 6 146.3 6.8 0.7X +In expression 39 52 10 254.0 3.9 1.0X +InSet expression 61 68 5 162.9 6.1 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 36 42 6 275.1 3.6 1.0X -InSet expression 62 66 4 160.3 6.2 0.6X +In expression 37 42 6 267.8 3.7 1.0X +InSet expression 57 62 4 173.9 5.7 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 65 70 6 153.5 6.5 1.0X -InSet expression 77 81 4 130.5 7.7 0.9X +In expression 66 72 5 151.0 6.6 1.0X +InSet expression 82 85 4 122.2 8.2 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 106 111 7 94.6 10.6 1.0X -InSet expression 84 87 3 119.7 8.4 1.3X +In expression 106 111 5 94.1 10.6 1.0X +InSet expression 96 101 6 103.7 9.6 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 184 189 4 54.3 18.4 1.0X -InSet expression 98 102 4 102.1 9.8 1.9X +In expression 185 189 5 54.1 18.5 1.0X +InSet expression 124 128 4 80.4 12.4 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 393 395 2 25.5 39.3 1.0X -InSet expression 187 192 6 53.5 18.7 2.1X +In expression 397 403 9 25.2 39.7 1.0X +InSet expression 187 190 3 53.4 18.7 2.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 29 33 5 345.2 2.9 1.0X -InSet expression 75 79 3 132.5 7.5 0.4X +In expression 32 35 4 315.0 3.2 1.0X +InSet expression 85 88 4 117.4 8.5 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 36 41 5 274.6 3.6 1.0X -InSet expression 92 95 2 109.1 9.2 0.4X +In expression 41 44 5 244.9 4.1 1.0X +InSet expression 98 101 2 101.8 9.8 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 62 66 4 161.1 6.2 1.0X -InSet expression 91 93 2 110.1 9.1 0.7X +In expression 63 65 3 159.0 6.3 1.0X +InSet expression 98 100 2 102.4 9.8 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 119 122 4 84.3 11.9 1.0X -InSet expression 128 129 1 78.4 12.8 0.9X +In expression 120 123 4 83.6 12.0 1.0X +InSet expression 133 137 4 74.9 13.3 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 186 189 5 53.7 18.6 1.0X -InSet expression 114 116 3 87.5 11.4 1.6X +In expression 197 201 4 50.7 19.7 1.0X +InSet expression 120 124 4 83.1 12.0 1.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 368 372 5 27.2 36.8 1.0X -InSet expression 122 124 1 81.9 12.2 3.0X +In expression 370 372 2 27.1 37.0 1.0X +InSet expression 132 135 3 76.0 13.2 2.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 551 559 5 18.1 55.1 1.0X -InSet expression 135 138 2 74.0 13.5 4.1X +In expression 556 557 2 18.0 55.6 1.0X +InSet expression 145 148 3 68.9 14.5 3.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 734 737 3 13.6 73.4 1.0X -InSet expression 148 151 2 67.5 14.8 4.9X +In expression 731 736 4 13.7 73.1 1.0X +InSet expression 158 160 2 63.4 15.8 4.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 941 947 5 10.6 94.1 1.0X -InSet expression 162 165 2 61.7 16.2 5.8X +In expression 944 945 2 10.6 94.4 1.0X +InSet expression 172 174 2 58.2 17.2 5.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 27 31 6 374.5 2.7 1.0X -InSet expression 73 75 3 137.7 7.3 0.4X +In expression 27 30 4 372.4 2.7 1.0X +InSet expression 81 84 2 122.8 8.1 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 36 41 7 276.7 3.6 1.0X -InSet expression 91 93 1 109.7 9.1 0.4X +In expression 37 39 3 270.3 3.7 1.0X +InSet expression 98 100 2 102.4 9.8 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 60 64 4 166.6 6.0 1.0X -InSet expression 120 122 1 83.5 12.0 0.5X +In expression 62 63 3 162.1 6.2 1.0X +InSet expression 123 125 1 81.1 12.3 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 111 116 5 89.8 11.1 1.0X -InSet expression 134 137 4 74.8 13.4 0.8X +In expression 112 116 10 89.1 11.2 1.0X +InSet expression 140 142 2 71.3 14.0 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 195 197 3 51.3 19.5 1.0X -InSet expression 116 119 3 85.8 11.6 1.7X +In expression 194 198 4 51.4 19.4 1.0X +InSet expression 123 126 4 81.3 12.3 1.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 348 351 4 28.7 34.8 1.0X -InSet expression 122 125 2 81.7 12.2 2.8X +In expression 344 347 3 29.1 34.4 1.0X +InSet expression 128 130 2 77.8 12.8 2.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 513 516 5 19.5 51.3 1.0X -InSet expression 133 135 2 75.1 13.3 3.9X +In expression 514 517 3 19.5 51.4 1.0X +InSet expression 143 146 3 70.2 14.3 3.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 658 663 4 15.2 65.8 1.0X -InSet expression 146 149 3 68.3 14.6 4.5X +In expression 664 670 6 15.1 66.4 1.0X +InSet expression 156 159 2 64.0 15.6 4.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 847 853 7 11.8 84.7 1.0X -InSet expression 159 162 2 62.7 15.9 5.3X +In expression 848 851 4 11.8 84.8 1.0X +InSet expression 169 172 2 59.2 16.9 5.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 27 30 4 368.5 2.7 1.0X -InSet expression 80 83 3 124.6 8.0 0.3X +In expression 28 30 4 359.2 2.8 1.0X +InSet expression 82 84 2 121.6 8.2 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 35 38 5 285.9 3.5 1.0X -InSet expression 97 99 1 103.0 9.7 0.4X +In expression 39 41 3 259.6 3.9 1.0X +InSet expression 99 101 2 101.1 9.9 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 63 65 3 157.7 6.3 1.0X -InSet expression 97 100 4 102.8 9.7 0.7X +In expression 61 63 3 164.4 6.1 1.0X +InSet expression 99 102 2 101.0 9.9 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 116 119 4 86.1 11.6 1.0X -InSet expression 135 137 1 74.3 13.5 0.9X +In expression 111 112 3 89.9 11.1 1.0X +InSet expression 136 138 2 73.4 13.6 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 184 197 6 54.5 18.4 1.0X -InSet expression 117 119 2 85.4 11.7 1.6X +In expression 196 199 4 50.9 19.6 1.0X +InSet expression 118 121 2 84.4 11.8 1.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 342 351 8 29.2 34.2 1.0X -InSet expression 124 126 1 80.6 12.4 2.8X +In expression 360 364 3 27.8 36.0 1.0X +InSet expression 127 131 6 78.8 12.7 2.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 511 518 9 19.6 51.1 1.0X -InSet expression 136 140 3 73.3 13.6 3.7X +In expression 514 517 3 19.4 51.4 1.0X +InSet expression 139 141 2 72.0 13.9 3.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 668 675 9 15.0 66.8 1.0X -InSet expression 149 152 4 67.1 14.9 4.5X +In expression 673 687 16 14.8 67.3 1.0X +InSet expression 151 153 2 66.4 15.1 4.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 826 831 3 12.1 82.6 1.0X -InSet expression 161 163 1 62.2 16.1 5.1X +In expression 833 836 3 12.0 83.3 1.0X +InSet expression 163 166 3 61.5 16.3 5.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 20 24 4 497.1 2.0 1.0X -InSet expression 77 80 1 129.1 7.7 0.3X +In expression 21 25 4 466.0 2.1 1.0X +InSet expression 79 81 1 126.7 7.9 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 28 31 4 356.5 2.8 1.0X -InSet expression 94 96 1 105.9 9.4 0.3X +In expression 30 32 5 335.1 3.0 1.0X +InSet expression 96 98 2 104.5 9.6 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 56 60 4 177.3 5.6 1.0X -InSet expression 108 110 1 92.5 10.8 0.5X +In expression 58 60 3 171.6 5.8 1.0X +InSet expression 109 111 2 91.5 10.9 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 100 102 3 99.7 10.0 1.0X -InSet expression 133 135 1 75.1 13.3 0.8X +In expression 101 103 3 98.6 10.1 1.0X +InSet expression 134 136 2 74.6 13.4 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 179 182 4 55.7 17.9 1.0X -InSet expression 120 123 3 83.2 12.0 1.5X +In expression 180 182 3 55.5 18.0 1.0X +InSet expression 121 124 4 82.4 12.1 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 337 347 7 29.6 33.7 1.0X -InSet expression 127 131 9 78.9 12.7 2.7X +In expression 339 344 3 29.5 33.9 1.0X +InSet expression 127 130 2 78.4 12.7 2.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 506 517 21 19.8 50.6 1.0X -InSet expression 135 139 4 73.8 13.5 3.7X +In expression 507 507 0 19.7 50.7 1.0X +InSet expression 138 140 2 72.3 13.8 3.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 666 672 4 15.0 66.6 1.0X -InSet expression 148 152 3 67.4 14.8 4.5X +In expression 664 675 10 15.1 66.4 1.0X +InSet expression 151 153 1 66.2 15.1 4.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 831 860 62 12.0 83.1 1.0X -InSet expression 159 162 1 62.7 15.9 5.2X +In expression 833 867 63 12.0 83.3 1.0X +InSet expression 162 165 2 61.8 16.2 5.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 19 20 2 526.2 1.9 1.0X -InSet expression 86 87 1 116.9 8.6 0.2X +In expression 20 22 3 501.8 2.0 1.0X +InSet expression 89 90 1 113.0 8.9 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 28 31 3 352.8 2.8 1.0X -InSet expression 101 103 2 98.9 10.1 0.3X +In expression 30 33 4 334.8 3.0 1.0X +InSet expression 105 107 2 95.6 10.5 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 53 56 3 190.2 5.3 1.0X -InSet expression 101 103 2 98.9 10.1 0.5X +In expression 54 58 5 184.2 5.4 1.0X +InSet expression 104 106 2 96.5 10.4 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 92 94 3 108.2 9.2 1.0X -InSet expression 135 138 2 73.8 13.5 0.7X +In expression 93 95 3 107.1 9.3 1.0X +InSet expression 137 139 3 73.2 13.7 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 171 173 2 58.4 17.1 1.0X -InSet expression 121 123 2 82.4 12.1 1.4X +In expression 172 173 3 58.1 17.2 1.0X +InSet expression 120 122 2 83.2 12.0 1.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 328 331 3 30.5 32.8 1.0X -InSet expression 129 131 2 77.7 12.9 2.5X +In expression 332 337 6 30.2 33.2 1.0X +InSet expression 129 131 2 77.8 12.9 2.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 36 40 4 274.9 3.6 1.0X -InSet expression 77 81 8 130.3 7.7 0.5X +In expression 36 40 4 281.6 3.6 1.0X +InSet expression 78 80 2 127.9 7.8 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 61 63 3 164.9 6.1 1.0X -InSet expression 98 99 1 102.5 9.8 0.6X +In expression 61 65 3 164.8 6.1 1.0X +InSet expression 100 102 2 99.6 10.0 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 147 150 3 67.8 14.7 1.0X -InSet expression 99 101 1 101.1 9.9 1.5X +In expression 149 151 3 67.0 14.9 1.0X +InSet expression 100 102 2 100.1 10.0 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 227 232 6 44.0 22.7 1.0X -InSet expression 144 146 2 69.5 14.4 1.6X +In expression 231 234 3 43.3 23.1 1.0X +InSet expression 146 147 2 68.7 14.6 1.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 384 387 4 26.0 38.4 1.0X -InSet expression 116 118 1 86.0 11.6 3.3X +In expression 387 389 2 25.9 38.7 1.0X +InSet expression 117 119 2 85.5 11.7 3.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1895 1931 68 5.3 189.5 1.0X -InSet expression 120 122 2 83.6 12.0 15.9X +In expression 1905 1963 66 5.2 190.5 1.0X +InSet expression 122 123 1 81.9 12.2 15.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 36 39 3 281.1 3.6 1.0X -InSet expression 77 80 2 129.7 7.7 0.5X +In expression 39 40 3 258.5 3.9 1.0X +InSet expression 79 81 2 126.5 7.9 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 61 64 4 163.7 6.1 1.0X -InSet expression 98 100 1 101.8 9.8 0.6X +In expression 62 63 3 161.7 6.2 1.0X +InSet expression 100 102 2 99.8 10.0 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 151 153 3 66.3 15.1 1.0X -InSet expression 100 103 3 99.9 10.0 1.5X +In expression 150 152 3 66.7 15.0 1.0X +InSet expression 99 101 4 101.1 9.9 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 227 232 10 44.0 22.7 1.0X -InSet expression 143 145 1 70.1 14.3 1.6X +In expression 231 233 2 43.3 23.1 1.0X +InSet expression 147 149 4 68.1 14.7 1.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 386 389 3 25.9 38.6 1.0X -InSet expression 116 117 1 86.2 11.6 3.3X +In expression 390 391 2 25.6 39.0 1.0X +InSet expression 119 121 2 84.3 11.9 3.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 2266 2298 49 4.4 226.6 1.0X -InSet expression 119 121 1 83.9 11.9 19.0X +In expression 2236 2268 70 4.5 223.6 1.0X +InSet expression 122 124 1 81.8 12.2 18.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 20 22 3 49.4 20.2 1.0X -InSet expression 59 61 2 17.0 58.8 0.3X +In expression 21 22 2 47.2 21.2 1.0X +InSet expression 58 60 2 17.3 58.0 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 26 28 3 37.9 26.4 1.0X -InSet expression 61 63 2 16.3 61.3 0.4X +In expression 27 29 2 37.0 27.0 1.0X +InSet expression 60 62 2 16.6 60.1 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 40 43 3 24.7 40.4 1.0X -InSet expression 62 65 4 16.2 61.7 0.7X +In expression 42 43 2 23.9 41.8 1.0X +InSet expression 61 63 1 16.3 61.2 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 79 81 3 12.6 79.4 1.0X -InSet expression 67 69 2 14.8 67.4 1.2X +In expression 80 82 2 12.5 80.3 1.0X +InSet expression 65 67 1 15.3 65.4 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 240 245 7 4.2 240.0 1.0X -InSet expression 65 68 4 15.4 65.1 3.7X +In expression 241 243 3 4.1 241.2 1.0X +InSet expression 64 66 3 15.6 64.1 3.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 572 576 5 1.7 571.9 1.0X -InSet expression 66 68 1 15.1 66.4 8.6X +In expression 581 582 1 1.7 580.6 1.0X +InSet expression 66 68 3 15.1 66.2 8.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 5 6 2 199.2 5.0 1.0X -InSet expression 5 6 2 211.3 4.7 1.1X +In expression 5 6 2 201.8 5.0 1.0X +InSet expression 5 6 2 211.8 4.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 5 6 2 205.8 4.9 1.0X -InSet expression 5 5 2 210.7 4.7 1.0X +In expression 5 6 2 207.5 4.8 1.0X +InSet expression 5 6 2 207.7 4.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 5 6 2 194.7 5.1 1.0X -InSet expression 5 6 2 191.2 5.2 1.0X +In expression 5 7 3 193.2 5.2 1.0X +InSet expression 5 6 2 190.2 5.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 6 7 2 167.7 6.0 1.0X -InSet expression 6 7 2 167.6 6.0 1.0X +In expression 6 7 2 167.4 6.0 1.0X +InSet expression 6 7 2 168.1 5.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 7 8 2 134.1 7.5 1.0X -InSet expression 7 8 2 135.4 7.4 1.0X +In expression 8 9 3 132.3 7.6 1.0X +InSet expression 8 9 3 133.1 7.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 11 11 2 94.8 10.6 1.0X -InSet expression 11 11 1 95.0 10.5 1.0X +In expression 11 13 3 91.4 10.9 1.0X +InSet expression 11 13 3 93.3 10.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 29 30 1 34.3 29.1 1.0X -InSet expression 43 45 2 23.1 43.3 0.7X +In expression 29 34 5 34.2 29.3 1.0X +InSet expression 43 46 3 23.2 43.1 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 34 36 2 29.0 34.5 1.0X -InSet expression 46 47 1 21.9 45.6 0.8X +In expression 34 38 4 29.4 34.1 1.0X +InSet expression 46 50 3 21.9 45.7 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 44 45 2 23.0 43.5 1.0X -InSet expression 50 51 1 19.9 50.3 0.9X +In expression 43 47 3 23.2 43.1 1.0X +InSet expression 60 62 3 16.5 60.5 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 58 60 2 17.1 58.4 1.0X -InSet expression 54 55 1 18.5 54.2 1.1X +In expression 59 59 1 17.0 58.7 1.0X +InSet expression 54 54 1 18.6 53.7 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 90 92 2 11.1 90.4 1.0X -InSet expression 51 53 1 19.6 51.1 1.8X +In expression 89 90 2 11.2 89.4 1.0X +InSet expression 51 51 1 19.7 50.8 1.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 408 413 5 2.4 408.2 1.0X -InSet expression 51 53 2 19.6 50.9 8.0X +In expression 408 410 4 2.5 407.6 1.0X +InSet expression 52 52 1 19.4 51.6 7.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 17 18 2 593.9 1.7 1.0X -InSet expression 81 83 2 123.5 8.1 0.2X +In expression 17 19 2 579.0 1.7 1.0X +InSet expression 84 85 2 118.9 8.4 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 23 25 4 442.1 2.3 1.0X -InSet expression 95 96 1 105.7 9.5 0.2X +In expression 23 24 2 435.8 2.3 1.0X +InSet expression 97 98 1 103.3 9.7 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 48 50 2 209.9 4.8 1.0X -InSet expression 128 130 1 78.1 12.8 0.4X +In expression 49 49 2 206.1 4.9 1.0X +InSet expression 129 130 1 77.8 12.9 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 81 84 3 123.3 8.1 1.0X -InSet expression 161 163 1 62.0 16.1 0.5X +In expression 82 83 2 122.1 8.2 1.0X +InSet expression 160 162 1 62.5 16.0 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 152 154 2 65.6 15.2 1.0X -InSet expression 137 138 1 73.0 13.7 1.1X +In expression 153 154 2 65.3 15.3 1.0X +InSet expression 138 140 1 72.4 13.8 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 295 306 11 33.9 29.5 1.0X -InSet expression 133 134 1 75.4 13.3 2.2X +In expression 296 308 11 33.8 29.6 1.0X +InSet expression 134 136 1 74.7 13.4 2.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 304 306 3 32.9 30.4 1.0X -InSet expression 300 303 3 33.3 30.0 1.0X +In expression 288 289 1 34.7 28.8 1.0X +InSet expression 284 288 3 35.2 28.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 307 313 4 32.6 30.7 1.0X -InSet expression 300 302 2 33.3 30.0 1.0X +In expression 290 294 2 34.5 29.0 1.0X +InSet expression 284 287 2 35.2 28.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 344 345 1 29.1 34.4 1.0X -InSet expression 300 301 1 33.4 30.0 1.1X +In expression 328 329 1 30.5 32.8 1.0X +InSet expression 287 289 2 34.8 28.7 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 387 393 7 25.9 38.7 1.0X -InSet expression 300 302 1 33.3 30.0 1.3X +In expression 381 389 17 26.3 38.1 1.0X +InSet expression 285 289 4 35.1 28.5 1.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 488 489 1 20.5 48.8 1.0X -InSet expression 300 305 3 33.3 30.0 1.6X +In expression 474 477 3 21.1 47.4 1.0X +InSet expression 287 289 1 34.9 28.7 1.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 643 646 3 15.6 64.3 1.0X -InSet expression 303 305 2 33.0 30.3 2.1X +In expression 618 620 1 16.2 61.8 1.0X +InSet expression 287 290 3 34.8 28.7 2.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 803 805 2 12.5 80.3 1.0X -InSet expression 305 306 1 32.8 30.5 2.6X +In expression 789 793 3 12.7 78.9 1.0X +InSet expression 291 295 3 34.3 29.1 2.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 960 972 8 10.4 96.0 1.0X -InSet expression 306 308 2 32.7 30.6 3.1X +In expression 952 973 27 10.5 95.2 1.0X +InSet expression 292 294 2 34.2 29.2 3.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1122 1145 26 8.9 112.2 1.0X -InSet expression 371 374 2 27.0 37.1 3.0X +In expression 1110 1118 7 9.0 111.0 1.0X +InSet expression 369 371 2 27.1 36.9 3.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 26 27 2 38.8 25.8 1.0X -InSet expression 58 60 2 17.2 58.2 0.4X +In expression 27 28 2 37.7 26.5 1.0X +InSet expression 59 60 2 17.0 58.8 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 41 43 1 24.3 41.1 1.0X -InSet expression 87 90 4 11.5 87.1 0.5X +In expression 42 43 2 24.1 41.6 1.0X +InSet expression 87 89 2 11.5 86.9 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 162 165 3 6.2 162.3 1.0X -InSet expression 102 104 2 9.8 101.6 1.6X +In expression 172 174 3 5.8 171.8 1.0X +InSet expression 102 103 1 9.8 101.6 1.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 393 400 12 2.5 393.1 1.0X -InSet expression 130 134 3 7.7 130.4 3.0X +In expression 399 401 2 2.5 398.8 1.0X +InSet expression 131 133 1 7.7 130.7 3.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 818 822 4 1.2 818.1 1.0X -InSet expression 146 149 2 6.8 146.5 5.6X +In expression 780 785 5 1.3 779.7 1.0X +InSet expression 146 149 3 6.8 146.3 5.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1772 1922 315 0.6 1772.2 1.0X -InSet expression 164 167 2 6.1 164.2 10.8X +In expression 1715 1871 325 0.6 1714.6 1.0X +InSet expression 164 166 2 6.1 163.6 10.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 17 18 1 58.3 17.1 1.0X -InSet expression 83 87 9 12.1 82.9 0.2X +In expression 17 20 3 57.7 17.3 1.0X +InSet expression 87 90 6 11.5 86.9 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 24 25 1 42.0 23.8 1.0X -InSet expression 126 128 1 7.9 126.2 0.2X +In expression 25 27 2 40.8 24.5 1.0X +InSet expression 134 136 2 7.5 133.9 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 71 73 1 14.1 71.1 1.0X -InSet expression 149 150 1 6.7 148.8 0.5X +In expression 72 73 1 13.8 72.2 1.0X +InSet expression 157 163 14 6.4 156.9 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 191 192 1 5.2 191.2 1.0X -InSet expression 190 194 4 5.3 189.9 1.0X +In expression 198 199 2 5.0 198.0 1.0X +InSet expression 202 204 2 5.0 202.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 564 582 35 1.8 563.9 1.0X -InSet expression 214 217 2 4.7 214.2 2.6X +In expression 555 573 35 1.8 554.6 1.0X +InSet expression 229 233 3 4.4 229.2 2.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1282 1479 291 0.8 1282.4 1.0X -InSet expression 243 252 7 4.1 243.3 5.3X +In expression 1395 1508 239 0.7 1395.2 1.0X +InSet expression 262 264 2 3.8 261.5 5.3X diff --git a/sql/core/benchmarks/InExpressionBenchmark-results.txt b/sql/core/benchmarks/InExpressionBenchmark-results.txt index 5178c51124c4f..539cb9a5060c9 100644 --- a/sql/core/benchmarks/InExpressionBenchmark-results.txt +++ b/sql/core/benchmarks/InExpressionBenchmark-results.txt @@ -2,739 +2,739 @@ In Expression Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 43 61 11 230.8 4.3 1.0X -InSet expression 88 94 7 113.6 8.8 0.5X +In expression 48 60 8 210.3 4.8 1.0X +InSet expression 85 92 6 117.9 8.5 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 36 42 5 279.2 3.6 1.0X -InSet expression 82 86 4 121.3 8.2 0.4X +In expression 38 43 6 263.4 3.8 1.0X +InSet expression 79 84 4 125.8 7.9 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 65 69 3 154.3 6.5 1.0X -InSet expression 83 87 4 121.0 8.3 0.8X +In expression 67 70 4 149.6 6.7 1.0X +InSet expression 84 88 3 119.3 8.4 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 105 107 2 95.4 10.5 1.0X -InSet expression 88 91 3 113.8 8.8 1.2X +In expression 107 110 4 93.9 10.7 1.0X +InSet expression 89 92 3 112.7 8.9 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 193 196 5 51.8 19.3 1.0X -InSet expression 98 100 3 102.2 9.8 2.0X +In expression 194 198 4 51.5 19.4 1.0X +InSet expression 99 102 3 101.5 9.9 2.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 388 390 2 25.8 38.8 1.0X -InSet expression 178 180 2 56.1 17.8 2.2X +In expression 397 401 3 25.2 39.7 1.0X +InSet expression 191 194 5 52.5 19.1 2.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 30 34 4 330.9 3.0 1.0X -InSet expression 91 95 3 109.5 9.1 0.3X +In expression 32 35 4 316.4 3.2 1.0X +InSet expression 96 100 2 104.6 9.6 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 38 42 6 259.9 3.8 1.0X -InSet expression 106 109 4 94.7 10.6 0.4X +In expression 37 40 4 268.0 3.7 1.0X +InSet expression 109 113 4 91.4 10.9 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 61 65 6 164.1 6.1 1.0X -InSet expression 106 108 2 94.6 10.6 0.6X +In expression 63 65 3 159.7 6.3 1.0X +InSet expression 109 116 18 91.6 10.9 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 116 118 2 86.3 11.6 1.0X -InSet expression 136 138 2 73.4 13.6 0.9X +In expression 118 120 2 84.6 11.8 1.0X +InSet expression 139 142 3 72.0 13.9 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 197 199 3 50.8 19.7 1.0X -InSet expression 119 121 1 84.0 11.9 1.7X +In expression 186 188 2 53.8 18.6 1.0X +InSet expression 122 126 3 81.7 12.2 1.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 364 367 3 27.5 36.4 1.0X -InSet expression 126 128 1 79.3 12.6 2.9X +In expression 366 368 3 27.3 36.6 1.0X +InSet expression 133 135 1 75.3 13.3 2.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 548 551 3 18.2 54.8 1.0X -InSet expression 139 143 3 71.8 13.9 3.9X +In expression 551 555 7 18.1 55.1 1.0X +InSet expression 145 147 2 69.0 14.5 3.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 666 672 4 15.0 66.6 1.0X -InSet expression 153 155 3 65.3 15.3 4.4X +In expression 664 785 87 15.1 66.4 1.0X +InSet expression 157 161 2 63.6 15.7 4.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 934 938 5 10.7 93.4 1.0X -InSet expression 165 168 2 60.4 16.5 5.6X +In expression 884 892 5 11.3 88.4 1.0X +InSet expression 170 173 3 58.7 17.0 5.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 28 30 2 352.5 2.8 1.0X -InSet expression 87 89 4 114.7 8.7 0.3X +In expression 30 32 3 334.7 3.0 1.0X +InSet expression 92 94 3 108.7 9.2 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 34 37 4 291.1 3.4 1.0X -InSet expression 106 108 2 94.2 10.6 0.3X +In expression 36 39 4 277.4 3.6 1.0X +InSet expression 109 110 1 91.7 10.9 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 69 71 1 144.2 6.9 1.0X -InSet expression 128 133 8 78.0 12.8 0.5X +In expression 70 72 2 142.4 7.0 1.0X +InSet expression 131 133 1 76.3 13.1 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 105 107 2 95.0 10.5 1.0X -InSet expression 143 145 2 70.1 14.3 0.7X +In expression 106 107 2 94.6 10.6 1.0X +InSet expression 146 148 1 68.5 14.6 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 195 197 2 51.2 19.5 1.0X -InSet expression 123 127 5 81.4 12.3 1.6X +In expression 196 197 2 51.1 19.6 1.0X +InSet expression 126 130 5 79.6 12.6 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 353 356 4 28.3 35.3 1.0X -InSet expression 127 129 1 78.7 12.7 2.8X +In expression 354 357 4 28.3 35.4 1.0X +InSet expression 130 132 1 76.9 13.0 2.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 500 502 3 20.0 50.0 1.0X -InSet expression 137 142 7 73.1 13.7 3.7X +In expression 501 504 4 20.0 50.1 1.0X +InSet expression 143 148 5 70.0 14.3 3.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 666 670 4 15.0 66.6 1.0X -InSet expression 151 154 2 66.1 15.1 4.4X +In expression 665 667 4 15.0 66.5 1.0X +InSet expression 155 158 1 64.4 15.5 4.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 854 858 4 11.7 85.4 1.0X -InSet expression 162 164 1 61.9 16.2 5.3X +In expression 858 861 3 11.7 85.8 1.0X +InSet expression 167 171 4 59.9 16.7 5.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 27 29 3 372.2 2.7 1.0X -InSet expression 89 92 2 111.7 8.9 0.3X +In expression 28 30 2 356.5 2.8 1.0X +InSet expression 91 93 2 109.8 9.1 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 34 39 2 291.3 3.4 1.0X -InSet expression 107 109 1 93.5 10.7 0.3X +In expression 35 37 2 283.2 3.5 1.0X +InSet expression 107 110 2 93.1 10.7 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 60 65 9 166.2 6.0 1.0X -InSet expression 109 112 2 91.9 10.9 0.6X +In expression 62 66 11 160.7 6.2 1.0X +InSet expression 110 113 3 90.6 11.0 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 103 104 1 97.2 10.3 1.0X -InSet expression 140 144 8 71.5 14.0 0.7X +In expression 105 106 1 95.4 10.5 1.0X +InSet expression 142 144 1 70.3 14.2 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 183 184 3 54.8 18.3 1.0X -InSet expression 119 121 1 83.9 11.9 1.5X +In expression 195 195 1 51.4 19.5 1.0X +InSet expression 122 124 2 81.7 12.2 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 353 355 2 28.3 35.3 1.0X -InSet expression 122 127 3 81.6 12.2 2.9X +In expression 360 364 3 27.8 36.0 1.0X +InSet expression 130 132 2 76.7 13.0 2.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 508 510 3 19.7 50.8 1.0X -InSet expression 135 140 8 74.0 13.5 3.8X +In expression 509 514 8 19.6 50.9 1.0X +InSet expression 142 143 1 70.5 14.2 3.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 668 692 33 15.0 66.8 1.0X -InSet expression 147 149 2 68.2 14.7 4.6X +In expression 668 683 23 15.0 66.8 1.0X +InSet expression 153 155 1 65.4 15.3 4.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 835 841 8 12.0 83.5 1.0X -InSet expression 160 162 2 62.6 16.0 5.2X +In expression 830 854 45 12.1 83.0 1.0X +InSet expression 165 167 1 60.8 16.5 5.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 22 24 3 456.0 2.2 1.0X -InSet expression 86 89 4 116.4 8.6 0.3X +In expression 23 24 2 444.2 2.3 1.0X +InSet expression 88 91 3 113.3 8.8 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 29 31 3 350.9 2.9 1.0X -InSet expression 103 105 3 97.1 10.3 0.3X +In expression 29 31 2 345.8 2.9 1.0X +InSet expression 104 106 1 95.9 10.4 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 56 57 2 179.0 5.6 1.0X -InSet expression 118 120 1 84.6 11.8 0.5X +In expression 57 58 2 176.4 5.7 1.0X +InSet expression 119 121 1 83.8 11.9 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 100 101 1 100.0 10.0 1.0X -InSet expression 138 140 2 72.6 13.8 0.7X +In expression 101 102 1 99.2 10.1 1.0X +InSet expression 139 142 3 71.7 13.9 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 179 181 2 55.8 17.9 1.0X -InSet expression 122 124 1 82.2 12.2 1.5X +In expression 180 181 2 55.6 18.0 1.0X +InSet expression 125 128 4 80.1 12.5 1.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 344 347 3 29.1 34.4 1.0X -InSet expression 126 128 2 79.7 12.6 2.7X +In expression 346 350 4 28.9 34.6 1.0X +InSet expression 130 131 2 77.1 13.0 2.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 505 506 2 19.8 50.5 1.0X -InSet expression 136 139 2 73.5 13.6 3.7X +In expression 506 508 3 19.8 50.6 1.0X +InSet expression 141 144 2 71.0 14.1 3.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 661 665 3 15.1 66.1 1.0X -InSet expression 147 149 1 68.1 14.7 4.5X +In expression 658 665 4 15.2 65.8 1.0X +InSet expression 153 155 2 65.5 15.3 4.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 818 856 78 12.2 81.8 1.0X -InSet expression 159 161 3 63.0 15.9 5.2X +In expression 821 858 76 12.2 82.1 1.0X +InSet expression 164 166 1 61.1 16.4 5.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 22 24 4 459.0 2.2 1.0X -InSet expression 82 86 5 121.2 8.2 0.3X +In expression 22 24 3 456.1 2.2 1.0X +InSet expression 88 90 2 113.9 8.8 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 28 29 2 355.6 2.8 1.0X -InSet expression 99 101 1 101.1 9.9 0.3X +In expression 30 31 2 337.8 3.0 1.0X +InSet expression 103 106 2 96.8 10.3 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 53 55 2 190.1 5.3 1.0X -InSet expression 102 105 5 98.5 10.2 0.5X +In expression 55 56 2 180.8 5.5 1.0X +InSet expression 106 108 2 94.5 10.6 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 91 94 3 109.4 9.1 1.0X -InSet expression 132 134 1 75.7 13.2 0.7X +In expression 95 98 2 105.3 9.5 1.0X +InSet expression 136 139 4 73.5 13.6 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 175 183 10 57.2 17.5 1.0X -InSet expression 112 114 2 89.5 11.2 1.6X +In expression 172 177 5 58.0 17.2 1.0X +InSet expression 116 119 4 86.3 11.6 1.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 327 332 4 30.6 32.7 1.0X -InSet expression 119 128 18 84.3 11.9 2.8X +In expression 330 347 10 30.3 33.0 1.0X +InSet expression 125 127 2 80.2 12.5 2.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 35 37 3 286.9 3.5 1.0X -InSet expression 113 115 1 88.8 11.3 0.3X +In expression 37 38 3 273.6 3.7 1.0X +InSet expression 114 116 2 87.4 11.4 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 64 65 2 157.2 6.4 1.0X -InSet expression 143 148 10 70.2 14.3 0.4X +In expression 65 66 1 154.2 6.5 1.0X +InSet expression 143 145 3 70.1 14.3 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 125 127 2 79.8 12.5 1.0X -InSet expression 143 147 5 70.1 14.3 0.9X +In expression 128 130 3 78.1 12.8 1.0X +InSet expression 144 146 3 69.4 14.4 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 208 210 1 48.0 20.8 1.0X -InSet expression 188 190 2 53.3 18.8 1.1X +In expression 210 211 2 47.7 21.0 1.0X +InSet expression 191 192 1 52.3 19.1 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 365 369 6 27.4 36.5 1.0X -InSet expression 148 154 15 67.6 14.8 2.5X +In expression 367 369 4 27.3 36.7 1.0X +InSet expression 149 151 2 67.1 14.9 2.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1682 1771 104 5.9 168.2 1.0X -InSet expression 148 150 1 67.5 14.8 11.3X +In expression 1682 1789 90 5.9 168.2 1.0X +InSet expression 151 152 1 66.2 15.1 11.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 35 36 2 286.9 3.5 1.0X -InSet expression 95 97 2 105.0 9.5 0.4X +In expression 41 42 3 246.1 4.1 1.0X +InSet expression 116 117 2 86.6 11.6 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 63 64 2 159.6 6.3 1.0X -InSet expression 116 118 1 85.9 11.6 0.5X +In expression 63 65 6 158.1 6.3 1.0X +InSet expression 144 147 3 69.3 14.4 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 127 129 1 78.6 12.7 1.0X -InSet expression 118 120 1 84.9 11.8 1.1X +In expression 128 129 2 78.2 12.8 1.0X +InSet expression 144 146 4 69.6 14.4 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 207 210 3 48.3 20.7 1.0X -InSet expression 155 158 1 64.3 15.5 1.3X +In expression 210 210 2 47.7 21.0 1.0X +InSet expression 196 198 3 51.1 19.6 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 364 370 12 27.4 36.4 1.0X -InSet expression 126 129 3 79.3 12.6 2.9X +In expression 367 367 1 27.3 36.7 1.0X +InSet expression 152 154 1 65.6 15.2 2.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1920 1928 7 5.2 192.0 1.0X -InSet expression 134 139 5 74.8 13.4 14.4X +In expression 1925 2082 125 5.2 192.5 1.0X +InSet expression 155 158 2 64.3 15.5 12.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 20 21 3 51.3 19.5 1.0X -InSet expression 56 57 1 17.9 56.0 0.3X +In expression 21 23 2 47.8 20.9 1.0X +InSet expression 58 61 7 17.2 58.1 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 25 27 2 40.2 24.9 1.0X -InSet expression 58 60 1 17.3 57.9 0.4X +In expression 26 28 2 38.7 25.9 1.0X +InSet expression 60 63 2 16.5 60.5 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 40 41 2 25.3 39.5 1.0X -InSet expression 58 61 4 17.3 58.0 0.7X +In expression 42 43 2 24.1 41.5 1.0X +InSet expression 61 63 2 16.4 60.8 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 76 77 1 13.2 75.8 1.0X -InSet expression 62 64 1 16.1 62.2 1.2X +In expression 80 80 1 12.6 79.6 1.0X +InSet expression 65 67 1 15.3 65.3 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 234 236 4 4.3 234.3 1.0X -InSet expression 60 62 2 16.7 60.0 3.9X +In expression 237 238 2 4.2 236.9 1.0X +InSet expression 63 65 2 15.9 62.8 3.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 548 605 125 1.8 547.7 1.0X -InSet expression 63 65 4 15.9 62.7 8.7X +In expression 555 611 124 1.8 554.9 1.0X +InSet expression 65 68 4 15.4 64.8 8.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 5 6 1 202.4 4.9 1.0X -InSet expression 5 6 1 207.4 4.8 1.0X +In expression 5 6 2 194.4 5.1 1.0X +InSet expression 5 6 2 205.4 4.9 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 5 6 1 201.8 5.0 1.0X -InSet expression 5 5 1 203.6 4.9 1.0X +In expression 5 6 2 199.6 5.0 1.0X +InSet expression 5 6 1 205.5 4.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 5 6 1 187.1 5.3 1.0X -InSet expression 5 6 1 185.3 5.4 1.0X +In expression 5 7 2 182.1 5.5 1.0X +InSet expression 5 6 1 187.6 5.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 6 7 1 163.5 6.1 1.0X -InSet expression 6 7 1 161.6 6.2 1.0X +In expression 6 7 2 160.2 6.2 1.0X +InSet expression 6 7 1 160.7 6.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 8 8 1 127.7 7.8 1.0X -InSet expression 8 9 2 127.7 7.8 1.0X +In expression 8 9 1 125.2 8.0 1.0X +InSet expression 8 9 1 122.6 8.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 11 12 1 90.6 11.0 1.0X -InSet expression 11 12 1 90.6 11.0 1.0X +In expression 12 13 1 84.6 11.8 1.0X +InSet expression 12 13 1 84.8 11.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 29 32 4 34.6 28.9 1.0X -InSet expression 45 46 2 22.3 44.8 0.6X +In expression 30 32 2 33.2 30.2 1.0X +InSet expression 49 51 2 20.2 49.4 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 34 36 2 29.3 34.1 1.0X -InSet expression 48 49 1 21.0 47.6 0.7X +In expression 36 38 2 27.8 36.0 1.0X +InSet expression 51 53 2 19.5 51.4 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 43 44 2 23.5 42.5 1.0X -InSet expression 51 52 1 19.6 50.9 0.8X +In expression 44 45 1 22.7 44.0 1.0X +InSet expression 56 57 1 17.9 55.7 0.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 58 59 2 17.4 57.6 1.0X -InSet expression 55 57 1 18.2 54.8 1.1X +In expression 59 60 2 16.9 59.3 1.0X +InSet expression 60 61 1 16.6 60.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 88 90 1 11.3 88.4 1.0X -InSet expression 51 52 1 19.6 51.1 1.7X +In expression 138 139 1 7.3 137.5 1.0X +InSet expression 56 57 1 17.9 56.0 2.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 382 388 12 2.6 381.7 1.0X -InSet expression 52 53 1 19.4 51.5 7.4X +In expression 392 393 2 2.6 392.0 1.0X +InSet expression 56 58 6 17.8 56.0 7.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 16 17 1 622.8 1.6 1.0X -InSet expression 89 92 3 112.0 8.9 0.2X +In expression 17 18 2 602.6 1.7 1.0X +InSet expression 91 93 2 109.3 9.1 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 23 25 4 427.6 2.3 1.0X -InSet expression 99 101 1 101.0 9.9 0.2X +In expression 24 26 2 412.8 2.4 1.0X +InSet expression 101 103 2 98.5 10.1 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 45 46 2 220.5 4.5 1.0X -InSet expression 125 127 1 80.0 12.5 0.4X +In expression 47 48 1 212.0 4.7 1.0X +InSet expression 127 129 3 78.5 12.7 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 80 82 1 124.5 8.0 1.0X -InSet expression 151 153 2 66.2 15.1 0.5X +In expression 82 83 1 121.4 8.2 1.0X +InSet expression 155 157 1 64.5 15.5 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 152 153 2 65.8 15.2 1.0X -InSet expression 130 132 1 76.7 13.0 1.2X +In expression 154 155 2 65.1 15.4 1.0X +InSet expression 133 137 6 74.9 13.3 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 295 304 9 33.9 29.5 1.0X -InSet expression 127 129 1 78.8 12.7 2.3X +In expression 296 307 12 33.8 29.6 1.0X +InSet expression 128 130 1 77.8 12.8 2.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 441 442 1 22.7 44.1 1.0X -InSet expression 434 440 9 23.1 43.4 1.0X +In expression 441 443 2 22.7 44.1 1.0X +InSet expression 437 440 4 22.9 43.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 445 450 7 22.5 44.5 1.0X -InSet expression 437 438 3 22.9 43.7 1.0X +In expression 447 452 4 22.4 44.7 1.0X +InSet expression 441 443 2 22.7 44.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 485 486 1 20.6 48.5 1.0X -InSet expression 436 438 2 23.0 43.6 1.1X +In expression 470 471 1 21.3 47.0 1.0X +InSet expression 438 440 3 22.9 43.8 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 547 549 1 18.3 54.7 1.0X -InSet expression 441 445 4 22.7 44.1 1.2X +In expression 542 543 2 18.5 54.2 1.0X +InSet expression 440 443 3 22.7 44.0 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 608 610 1 16.4 60.8 1.0X -InSet expression 440 441 1 22.7 44.0 1.4X +In expression 619 620 1 16.1 61.9 1.0X +InSet expression 442 445 3 22.6 44.2 1.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 776 782 7 12.9 77.6 1.0X -InSet expression 436 440 2 22.9 43.6 1.8X +In expression 785 790 4 12.7 78.5 1.0X +InSet expression 441 448 7 22.7 44.1 1.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 300 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 939 945 6 10.6 93.9 1.0X -InSet expression 441 443 2 22.7 44.1 2.1X +In expression 933 937 6 10.7 93.3 1.0X +InSet expression 441 444 2 22.7 44.1 2.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 400 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1098 1105 10 9.1 109.8 1.0X -InSet expression 447 450 2 22.4 44.7 2.5X +In expression 1096 1106 7 9.1 109.6 1.0X +InSet expression 443 444 1 22.6 44.3 2.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 500 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1297 1305 7 7.7 129.7 1.0X -InSet expression 537 539 1 18.6 53.7 2.4X +In expression 1270 1273 2 7.9 127.0 1.0X +InSet expression 551 554 3 18.1 55.1 2.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 24 25 2 41.2 24.3 1.0X -InSet expression 57 58 1 17.6 56.8 0.4X +In expression 26 28 4 38.9 25.7 1.0X +InSet expression 58 59 2 17.4 57.6 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 41 42 1 24.4 41.0 1.0X -InSet expression 83 86 4 12.0 83.4 0.5X +In expression 42 43 2 24.1 41.5 1.0X +InSet expression 85 87 1 11.7 85.5 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 157 158 2 6.4 156.6 1.0X -InSet expression 98 99 1 10.2 97.6 1.6X +In expression 156 158 2 6.4 156.2 1.0X +InSet expression 100 102 2 10.0 99.8 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 371 374 2 2.7 371.2 1.0X -InSet expression 125 127 1 8.0 125.2 3.0X +In expression 388 389 1 2.6 387.5 1.0X +InSet expression 128 130 3 7.8 128.2 3.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 757 759 3 1.3 756.8 1.0X -InSet expression 142 144 1 7.0 142.5 5.3X +In expression 761 767 6 1.3 761.2 1.0X +InSet expression 143 149 13 7.0 143.1 5.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1671 1822 219 0.6 1671.2 1.0X -InSet expression 159 173 37 6.3 159.2 10.5X +In expression 1682 1860 303 0.6 1682.0 1.0X +InSet expression 160 163 2 6.3 160.0 10.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 5 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 20 21 1 49.8 20.1 1.0X -InSet expression 76 78 5 13.2 75.6 0.3X +In expression 22 23 2 46.5 21.5 1.0X +InSet expression 81 85 3 12.3 81.5 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 10 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 29 30 2 34.3 29.1 1.0X -InSet expression 116 117 2 8.7 115.6 0.3X +In expression 31 33 2 31.9 31.4 1.0X +InSet expression 122 125 3 8.2 122.4 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 25 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 68 70 1 14.6 68.3 1.0X -InSet expression 131 137 4 7.6 131.2 0.5X +In expression 71 73 1 14.0 71.2 1.0X +InSet expression 144 146 2 6.9 144.0 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 50 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 187 189 3 5.4 186.5 1.0X -InSet expression 176 179 3 5.7 175.5 1.1X +In expression 209 210 1 4.8 209.0 1.0X +InSet expression 186 189 3 5.4 186.1 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 100 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 491 498 8 2.0 491.0 1.0X -InSet expression 199 201 1 5.0 199.0 2.5X +In expression 490 502 7 2.0 489.9 1.0X +InSet expression 209 213 4 4.8 209.2 2.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor 200 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1139 1377 266 0.9 1139.0 1.0X -InSet expression 227 232 8 4.4 227.2 5.0X +In expression 1212 1422 205 0.8 1211.6 1.0X +InSet expression 239 242 5 4.2 238.6 5.1X diff --git a/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt b/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt index 4d79ea0b65033..94ffd3ca73811 100644 --- a/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt @@ -2,11 +2,11 @@ Int In-memory with 1000000 rows ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int In-Memory scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -columnar deserialization + columnar-to-row 147 200 59 6.8 147.4 1.0X -row-based deserialization 129 158 42 7.8 129.0 1.1X +columnar deserialization + columnar-to-row 171 215 38 5.8 171.5 1.0X +row-based deserialization 136 139 2 7.3 136.2 1.3X diff --git a/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt b/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt index 6787b645563b3..3feaaca07c885 100644 --- a/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt +++ b/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt @@ -2,11 +2,11 @@ Int In-memory with 1000000 rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int In-Memory scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -columnar deserialization + columnar-to-row 188 199 12 5.3 187.6 1.0X -row-based deserialization 142 216 115 7.0 141.9 1.3X +columnar deserialization + columnar-to-row 184 210 23 5.4 184.0 1.0X +row-based deserialization 142 144 2 7.0 142.2 1.3X diff --git a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt index 85601d9e9757f..f78e7551cefe1 100644 --- a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt @@ -1,8 +1,8 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dynamic insert table benchmark, totalRows = 200000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------- -one partition column, 100 partitions 9762 9793 43 0.0 48810.6 1.0X -two partition columns, 500 partitions 25446 25796 495 0.0 127230.3 0.4X -three partition columns, 2000 partitions 68971 69095 176 0.0 344853.7 0.1X +one partition column, 100 partitions 8137 8169 44 0.0 40687.0 1.0X +two partition columns, 500 partitions 20814 20937 174 0.0 104067.7 0.4X +three partition columns, 2000 partitions 56067 56122 78 0.0 280335.3 0.1X diff --git a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt index a8b6b9b48805d..8ca9c389f7348 100644 --- a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt +++ b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt @@ -1,8 +1,8 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor dynamic insert table benchmark, totalRows = 200000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------- -one partition column, 100 partitions 9336 9513 250 0.0 46681.2 1.0X -two partition columns, 500 partitions 25266 25745 677 0.0 126332.0 0.4X -three partition columns, 2000 partitions 69778 70117 479 0.0 348891.4 0.1X +one partition column, 100 partitions 7555 7583 41 0.0 37772.8 1.0X +two partition columns, 500 partitions 20496 20667 242 0.0 102480.4 0.4X +three partition columns, 2000 partitions 56071 56093 30 0.0 280357.3 0.1X diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt index 260eec63f5118..8e46de244bcc0 100644 --- a/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt @@ -1,40 +1,40 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare string w/ interval 427 498 62 2.3 426.8 1.0X -prepare string w/o interval 385 389 4 2.6 384.8 1.1X -1 units w/ interval 340 343 4 2.9 340.0 1.3X -1 units w/o interval 380 387 6 2.6 380.3 1.1X -2 units w/ interval 549 557 7 1.8 549.2 0.8X -2 units w/o interval 553 555 4 1.8 553.1 0.8X -3 units w/ interval 1125 1126 2 0.9 1124.7 0.4X -3 units w/o interval 1144 1149 7 0.9 1143.7 0.4X -4 units w/ interval 1338 1341 3 0.7 1337.7 0.3X -4 units w/o interval 1351 1367 18 0.7 1351.1 0.3X -5 units w/ interval 1506 1510 5 0.7 1506.4 0.3X -5 units w/o interval 1522 1523 1 0.7 1521.6 0.3X -6 units w/ interval 1644 1651 11 0.6 1643.6 0.3X -6 units w/o interval 1654 1661 10 0.6 1653.8 0.3X -7 units w/ interval 2058 2066 9 0.5 2058.2 0.2X -7 units w/o interval 2069 2072 5 0.5 2068.7 0.2X -8 units w/ interval 2291 2295 6 0.4 2290.9 0.2X -8 units w/o interval 2348 2358 12 0.4 2347.9 0.2X -9 units w/ interval 2453 2457 5 0.4 2452.8 0.2X -9 units w/o interval 2460 2472 16 0.4 2460.0 0.2X -10 units w/ interval 2709 2716 6 0.4 2709.3 0.2X -10 units w/o interval 2706 2707 1 0.4 2705.6 0.2X -11 units w/ interval 3049 3055 7 0.3 3048.7 0.1X -11 units w/o interval 3043 3050 7 0.3 3042.5 0.1X +prepare string w/ interval 397 422 27 2.5 396.9 1.0X +prepare string w/o interval 365 395 43 2.7 365.0 1.1X +1 units w/ interval 337 347 15 3.0 337.4 1.2X +1 units w/o interval 358 365 10 2.8 357.6 1.1X +2 units w/ interval 526 528 2 1.9 526.3 0.8X +2 units w/o interval 535 539 5 1.9 535.3 0.7X +3 units w/ interval 1126 1133 7 0.9 1125.5 0.4X +3 units w/o interval 1115 1118 3 0.9 1115.0 0.4X +4 units w/ interval 1310 1315 4 0.8 1310.3 0.3X +4 units w/o interval 1327 1333 5 0.8 1327.1 0.3X +5 units w/ interval 1453 1457 7 0.7 1452.9 0.3X +5 units w/o interval 1467 1472 4 0.7 1467.5 0.3X +6 units w/ interval 1615 1618 3 0.6 1614.7 0.2X +6 units w/o interval 1617 1617 1 0.6 1616.6 0.2X +7 units w/ interval 2046 2053 6 0.5 2046.0 0.2X +7 units w/o interval 2067 2072 4 0.5 2067.4 0.2X +8 units w/ interval 2277 2288 11 0.4 2277.3 0.2X +8 units w/o interval 2291 2297 5 0.4 2290.8 0.2X +9 units w/ interval 2612 2618 8 0.4 2611.7 0.2X +9 units w/o interval 2647 2651 4 0.4 2646.5 0.1X +10 units w/ interval 2842 2849 8 0.4 2841.8 0.1X +10 units w/o interval 2838 2841 3 0.4 2838.4 0.1X +11 units w/ interval 3071 3077 9 0.3 3070.8 0.1X +11 units w/o interval 3083 3088 6 0.3 3082.8 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor make_interval(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -prepare make_interval() 351 355 4 2.9 350.8 1.0X -make_interval(0, 1, 2, 3, 4, 5, 50.123456) 42 44 2 23.9 41.9 8.4X -make_interval(*, *, 2, 3, 4, 5, 50.123456) 52 54 3 19.2 52.0 6.7X -make_interval(0, 1, *, *, 4, 5, 50.123456) 61 64 3 16.4 60.9 5.8X -make_interval(0, 1, 2, 3, *, *, *) 344 348 5 2.9 344.5 1.0X -make_interval(*, *, *, *, *, *, *) 359 363 6 2.8 359.2 1.0X +prepare make_interval() 356 357 2 2.8 355.5 1.0X +make_interval(0, 1, 2, 3, 4, 5, 50.123456) 44 53 8 22.8 43.8 8.1X +make_interval(*, *, 2, 3, 4, 5, 50.123456) 53 57 4 18.8 53.3 6.7X +make_interval(0, 1, *, *, 4, 5, 50.123456) 56 56 0 17.9 55.8 6.4X +make_interval(0, 1, 2, 3, *, *, *) 326 327 1 3.1 326.1 1.1X +make_interval(*, *, *, *, *, *, *) 342 345 3 2.9 341.6 1.0X diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt index f09ebdc4d121f..5bd12d7b15ef5 100644 --- a/sql/core/benchmarks/IntervalBenchmark-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -1,40 +1,40 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare string w/ interval 409 467 97 2.4 409.1 1.0X -prepare string w/o interval 371 378 11 2.7 371.2 1.1X -1 units w/ interval 358 373 14 2.8 358.5 1.1X -1 units w/o interval 382 395 22 2.6 382.1 1.1X -2 units w/ interval 532 536 4 1.9 531.9 0.8X -2 units w/o interval 538 544 6 1.9 537.9 0.8X -3 units w/ interval 1202 1205 3 0.8 1202.1 0.3X -3 units w/o interval 1222 1227 5 0.8 1222.0 0.3X -4 units w/ interval 1403 1408 5 0.7 1403.3 0.3X -4 units w/o interval 1432 1435 4 0.7 1431.6 0.3X -5 units w/ interval 1552 1564 16 0.6 1551.5 0.3X -5 units w/o interval 1559 1562 3 0.6 1558.8 0.3X -6 units w/ interval 1700 1705 5 0.6 1700.2 0.2X -6 units w/o interval 1721 1728 8 0.6 1720.8 0.2X -7 units w/ interval 2241 2244 4 0.4 2241.0 0.2X -7 units w/o interval 2254 2265 10 0.4 2254.3 0.2X -8 units w/ interval 2505 2519 15 0.4 2505.0 0.2X -8 units w/o interval 2505 2508 3 0.4 2505.5 0.2X -9 units w/ interval 2621 2629 7 0.4 2621.2 0.2X -9 units w/o interval 2623 2628 4 0.4 2623.1 0.2X -10 units w/ interval 2844 2849 6 0.4 2843.6 0.1X -10 units w/o interval 2829 2842 20 0.4 2829.2 0.1X -11 units w/ interval 3143 3146 3 0.3 3142.7 0.1X -11 units w/o interval 3147 3156 10 0.3 3146.9 0.1X +prepare string w/ interval 407 418 12 2.5 406.7 1.0X +prepare string w/o interval 375 383 8 2.7 374.6 1.1X +1 units w/ interval 386 387 1 2.6 385.8 1.1X +1 units w/o interval 343 352 11 2.9 343.0 1.2X +2 units w/ interval 511 513 2 2.0 511.1 0.8X +2 units w/o interval 526 529 3 1.9 526.5 0.8X +3 units w/ interval 1196 1199 4 0.8 1196.3 0.3X +3 units w/o interval 1171 1174 3 0.9 1171.0 0.3X +4 units w/ interval 1389 1392 3 0.7 1389.3 0.3X +4 units w/o interval 1401 1403 2 0.7 1400.5 0.3X +5 units w/ interval 1545 1549 4 0.6 1545.2 0.3X +5 units w/o interval 1545 1552 8 0.6 1544.9 0.3X +6 units w/ interval 1689 1692 3 0.6 1689.0 0.2X +6 units w/o interval 1703 1706 5 0.6 1702.5 0.2X +7 units w/ interval 2287 2287 1 0.4 2286.6 0.2X +7 units w/o interval 2267 2272 4 0.4 2267.2 0.2X +8 units w/ interval 2475 2479 5 0.4 2474.8 0.2X +8 units w/o interval 2471 2476 4 0.4 2471.1 0.2X +9 units w/ interval 2625 2629 3 0.4 2625.4 0.2X +9 units w/o interval 2616 2624 12 0.4 2616.0 0.2X +10 units w/ interval 2850 2852 2 0.4 2850.5 0.1X +10 units w/o interval 2842 2845 4 0.4 2842.3 0.1X +11 units w/ interval 3177 3180 4 0.3 3177.3 0.1X +11 units w/o interval 3164 3174 8 0.3 3164.1 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor make_interval(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -prepare make_interval() 368 374 5 2.7 368.5 1.0X -make_interval(0, 1, 2, 3, 4, 5, 50.123456) 47 50 3 21.1 47.3 7.8X -make_interval(*, *, 2, 3, 4, 5, 50.123456) 59 62 5 17.0 58.8 6.3X -make_interval(0, 1, *, *, 4, 5, 50.123456) 62 64 3 16.0 62.3 5.9X -make_interval(0, 1, 2, 3, *, *, *) 342 345 2 2.9 342.0 1.1X -make_interval(*, *, *, *, *, *, *) 351 357 7 2.8 350.9 1.1X +prepare make_interval() 337 340 3 3.0 337.0 1.0X +make_interval(0, 1, 2, 3, 4, 5, 50.123456) 42 43 1 23.6 42.4 7.9X +make_interval(*, *, 2, 3, 4, 5, 50.123456) 53 55 3 19.0 52.7 6.4X +make_interval(0, 1, *, *, 4, 5, 50.123456) 56 60 5 17.9 55.7 6.0X +make_interval(0, 1, 2, 3, *, *, *) 341 345 5 2.9 341.2 1.0X +make_interval(*, *, *, *, *, *, *) 343 344 1 2.9 342.8 1.0X diff --git a/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt b/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt index 473cfdde4d76d..b908a2502d766 100644 --- a/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt @@ -2,81 +2,81 @@ Join Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long wholestage off 2088 2099 15 10.0 99.6 1.0X -Join w long wholestage on 918 947 28 22.8 43.8 2.3X +Join w long wholestage off 2048 2052 5 10.2 97.7 1.0X +Join w long wholestage on 884 926 37 23.7 42.1 2.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long duplicated wholestage off 1991 1993 3 10.5 94.9 1.0X -Join w long duplicated wholestage on 911 923 16 23.0 43.4 2.2X +Join w long duplicated wholestage off 2023 2028 7 10.4 96.5 1.0X +Join w long duplicated wholestage on 887 904 18 23.6 42.3 2.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 ints wholestage off 106730 106790 85 0.2 5089.3 1.0X -Join w 2 ints wholestage on 105489 105534 40 0.2 5030.1 1.0X +Join w 2 ints wholestage off 107738 107744 9 0.2 5137.3 1.0X +Join w 2 ints wholestage on 105798 105824 18 0.2 5044.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs wholestage off 3315 3323 12 6.3 158.1 1.0X -Join w 2 longs wholestage on 1972 1997 25 10.6 94.0 1.7X +Join w 2 longs wholestage off 3236 3370 189 6.5 154.3 1.0X +Join w 2 longs wholestage on 1977 2014 37 10.6 94.3 1.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs duplicated wholestage off 8534 8563 42 2.5 406.9 1.0X -Join w 2 longs duplicated wholestage on 5521 5729 121 3.8 263.3 1.5X +Join w 2 longs duplicated wholestage off 8713 8742 42 2.4 415.5 1.0X +Join w 2 longs duplicated wholestage on 5435 5556 105 3.9 259.2 1.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -outer join w long wholestage off 1590 1593 5 13.2 75.8 1.0X -outer join w long wholestage on 948 978 46 22.1 45.2 1.7X +outer join w long wholestage off 1586 1675 126 13.2 75.6 1.0X +outer join w long wholestage on 904 935 30 23.2 43.1 1.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -semi join w long wholestage off 1053 1055 3 19.9 50.2 1.0X -semi join w long wholestage on 568 585 15 37.0 27.1 1.9X +semi join w long wholestage off 1052 1053 1 19.9 50.2 1.0X +semi join w long wholestage on 522 528 4 40.2 24.9 2.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort merge join wholestage off 519 527 11 4.0 247.7 1.0X -sort merge join wholestage on 467 493 27 4.5 222.5 1.1X +sort merge join wholestage off 525 536 15 4.0 250.4 1.0X +sort merge join wholestage on 460 465 5 4.6 219.4 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -sort merge join with duplicates wholestage off 1031 1042 15 2.0 491.7 1.0X -sort merge join with duplicates wholestage on 960 968 8 2.2 457.8 1.1X +sort merge join with duplicates wholestage off 1008 1020 17 2.1 480.5 1.0X +sort merge join with duplicates wholestage on 920 934 14 2.3 438.5 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -shuffle hash join wholestage off 530 537 10 7.9 126.4 1.0X -shuffle hash join wholestage on 415 434 12 10.1 99.1 1.3X +shuffle hash join wholestage off 494 508 20 8.5 117.7 1.0X +shuffle hash join wholestage on 412 426 17 10.2 98.1 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor broadcast nested loop join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -broadcast nested loop join wholestage off 25590 25605 22 0.8 1220.2 1.0X -broadcast nested loop join wholestage on 18711 18767 79 1.1 892.2 1.4X +broadcast nested loop join wholestage off 25779 25822 61 0.8 1229.2 1.0X +broadcast nested loop join wholestage on 18110 18272 148 1.2 863.6 1.4X diff --git a/sql/core/benchmarks/JoinBenchmark-results.txt b/sql/core/benchmarks/JoinBenchmark-results.txt index 9c460f39d1ae7..abf8364e533d7 100644 --- a/sql/core/benchmarks/JoinBenchmark-results.txt +++ b/sql/core/benchmarks/JoinBenchmark-results.txt @@ -2,81 +2,81 @@ Join Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long wholestage off 2221 2232 15 9.4 105.9 1.0X -Join w long wholestage on 1032 1080 56 20.3 49.2 2.2X +Join w long wholestage off 2021 2057 50 10.4 96.4 1.0X +Join w long wholestage on 922 960 38 22.7 44.0 2.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long duplicated wholestage off 2180 2181 1 9.6 104.0 1.0X -Join w long duplicated wholestage on 917 927 10 22.9 43.7 2.4X +Join w long duplicated wholestage off 2060 2199 197 10.2 98.2 1.0X +Join w long duplicated wholestage on 927 935 11 22.6 44.2 2.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 ints wholestage off 112458 112473 21 0.2 5362.4 1.0X -Join w 2 ints wholestage on 110885 110937 68 0.2 5287.4 1.0X +Join w 2 ints wholestage off 112712 112721 12 0.2 5374.5 1.0X +Join w 2 ints wholestage on 111144 111183 26 0.2 5299.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs wholestage off 3502 3507 7 6.0 167.0 1.0X -Join w 2 longs wholestage on 2071 2085 10 10.1 98.8 1.7X +Join w 2 longs wholestage off 3114 3126 17 6.7 148.5 1.0X +Join w 2 longs wholestage on 1971 1991 17 10.6 94.0 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs duplicated wholestage off 9384 9385 2 2.2 447.4 1.0X -Join w 2 longs duplicated wholestage on 5493 5515 16 3.8 261.9 1.7X +Join w 2 longs duplicated wholestage off 8230 8239 13 2.5 392.4 1.0X +Join w 2 longs duplicated wholestage on 5478 5494 16 3.8 261.2 1.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -outer join w long wholestage off 1871 1884 19 11.2 89.2 1.0X -outer join w long wholestage on 1031 1054 30 20.4 49.1 1.8X +outer join w long wholestage off 1607 1627 28 13.1 76.6 1.0X +outer join w long wholestage on 906 914 6 23.1 43.2 1.8X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -semi join w long wholestage off 1189 1195 8 17.6 56.7 1.0X -semi join w long wholestage on 549 569 35 38.2 26.2 2.2X +semi join w long wholestage off 1047 1050 3 20.0 49.9 1.0X +semi join w long wholestage on 522 530 5 40.2 24.9 2.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort merge join wholestage off 526 535 13 4.0 250.9 1.0X -sort merge join wholestage on 461 470 6 4.5 220.0 1.1X +sort merge join wholestage off 512 516 5 4.1 244.2 1.0X +sort merge join wholestage on 459 477 13 4.6 218.7 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -sort merge join with duplicates wholestage off 1026 1054 39 2.0 489.2 1.0X -sort merge join with duplicates wholestage on 922 948 28 2.3 439.4 1.1X +sort merge join with duplicates wholestage off 970 984 20 2.2 462.4 1.0X +sort merge join with duplicates wholestage on 868 879 10 2.4 413.9 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -shuffle hash join wholestage off 521 533 16 8.0 124.3 1.0X -shuffle hash join wholestage on 383 393 10 11.0 91.3 1.4X +shuffle hash join wholestage off 512 520 13 8.2 122.0 1.0X +shuffle hash join wholestage on 353 369 20 11.9 84.1 1.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor broadcast nested loop join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -broadcast nested loop join wholestage off 29667 29788 171 0.7 1414.6 1.0X -broadcast nested loop join wholestage on 18946 19016 66 1.1 903.4 1.6X +broadcast nested loop join wholestage off 25058 25234 249 0.8 1194.9 1.0X +broadcast nested loop join wholestage on 18197 18557 692 1.2 867.7 1.4X diff --git a/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt index d87eb6530a855..381f30f110867 100644 --- a/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt @@ -3,128 +3,128 @@ Benchmark for performance of JSON parsing ================================================================================================ Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 2240 2300 98 2.2 448.0 1.0X -UTF-8 is set 3325 3333 8 1.5 665.0 0.7X +No encoding 2632 2713 96 1.9 526.3 1.0X +UTF-8 is set 4814 4824 12 1.0 962.8 0.5X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 1890 1917 23 2.6 378.1 1.0X -UTF-8 is set 3155 3158 3 1.6 630.9 0.6X +No encoding 2193 2256 82 2.3 438.6 1.0X +UTF-8 is set 4539 4544 5 1.1 907.8 0.5X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 5079 5266 188 0.2 5078.9 1.0X -UTF-8 is set 4272 4280 6 0.2 4272.5 1.2X +No encoding 4593 4651 87 0.2 4592.9 1.0X +UTF-8 is set 4837 4856 30 0.2 4837.0 0.9X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 9614 9866 271 0.0 192271.0 1.0X -UTF-8 is set 10517 10608 80 0.0 210331.2 0.9X +No encoding 9423 9596 234 0.0 188463.6 1.0X +UTF-8 is set 10747 10797 49 0.0 214934.1 0.9X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 1661 1666 6 0.6 1660.6 1.0X -Select 1 column 1078 1081 2 0.9 1078.3 1.5X +Select 10 columns 1769 1779 11 0.6 1769.0 1.0X +Select 1 column 1217 1220 3 0.8 1217.4 1.5X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 604 612 10 1.7 604.3 1.0X -Short column with UTF-8 828 839 15 1.2 828.3 0.7X -Wide column without encoding 7212 7255 38 0.1 7212.1 0.1X -Wide column with UTF-8 7446 7462 15 0.1 7445.8 0.1X +Short column without encoding 658 665 6 1.5 658.4 1.0X +Short column with UTF-8 1144 1162 16 0.9 1143.9 0.6X +Wide column without encoding 5152 5164 19 0.2 5151.8 0.1X +Wide column with UTF-8 7246 7274 28 0.1 7246.1 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 60 63 2 16.6 60.1 1.0X -from_json 1168 1175 7 0.9 1168.4 0.1X -json_tuple 1158 1170 16 0.9 1158.4 0.1X -get_json_object wholestage off 1075 1081 6 0.9 1074.8 0.1X -get_json_object wholestage on 1018 1029 13 1.0 1018.1 0.1X +Text read 59 62 2 16.9 59.0 1.0X +from_json 1119 1125 6 0.9 1119.4 0.1X +json_tuple 1039 1044 6 1.0 1039.4 0.1X +get_json_object wholestage off 1054 1060 5 0.9 1053.7 0.1X +get_json_object wholestage on 991 998 6 1.0 991.2 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 232 238 10 21.6 46.4 1.0X -schema inferring 1919 1928 9 2.6 383.7 0.1X -parsing 2717 2724 7 1.8 543.4 0.1X +Text read 235 242 12 21.3 46.9 1.0X +schema inferring 1966 1972 9 2.5 393.1 0.1X +parsing 2961 2978 24 1.7 592.2 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 562 569 7 8.9 112.5 1.0X -Schema inferring 2424 2432 9 2.1 484.8 0.2X -Parsing without charset 2808 2810 3 1.8 561.7 0.2X -Parsing with UTF-8 3993 4001 12 1.3 798.5 0.1X +Text read 563 569 6 8.9 112.5 1.0X +Schema inferring 2535 2538 3 2.0 507.0 0.2X +Parsing without charset 3072 3102 36 1.6 614.4 0.2X +Parsing with UTF-8 5607 5629 33 0.9 1121.5 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 101 108 7 9.9 101.4 1.0X -to_json(timestamp) 705 707 2 1.4 704.6 0.1X -write timestamps to files 598 611 20 1.7 598.3 0.2X -Create a dataset of dates 112 118 10 8.9 111.9 0.9X -to_json(date) 546 548 2 1.8 546.3 0.2X -write dates to files 393 399 9 2.5 393.1 0.3X +Create a dataset of timestamps 103 105 2 9.7 102.7 1.0X +to_json(timestamp) 555 557 3 1.8 554.8 0.2X +write timestamps to files 591 597 7 1.7 591.0 0.2X +Create a dataset of dates 121 125 4 8.3 120.8 0.8X +to_json(date) 420 422 3 2.4 419.6 0.2X +write dates to files 393 394 1 2.5 392.6 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -read timestamp text from files 149 153 4 6.7 149.4 1.0X -read timestamps from files 1044 1049 5 1.0 1043.6 0.1X -infer timestamps from files 1973 1983 12 0.5 1972.7 0.1X -read date text from files 140 143 5 7.1 140.0 1.1X -read date from files 690 698 9 1.4 690.3 0.2X -timestamp strings 141 149 7 7.1 140.8 1.1X -parse timestamps from Dataset[String] 1265 1266 2 0.8 1264.5 0.1X -infer timestamps from Dataset[String] 2160 2169 12 0.5 2160.5 0.1X -date strings 248 250 2 4.0 248.3 0.6X -parse dates from Dataset[String] 1010 1015 6 1.0 1009.6 0.1X -from_json(timestamp) 1781 1810 27 0.6 1781.1 0.1X -from_json(date) 1510 1514 4 0.7 1510.0 0.1X -infer error timestamps from Dataset[String] with default format 1412 1420 8 0.7 1412.2 0.1X -infer error timestamps from Dataset[String] with user-provided format 1372 1378 6 0.7 1371.6 0.1X -infer error timestamps from Dataset[String] with legacy format 1427 1439 18 0.7 1426.6 0.1X +read timestamp text from files 143 149 9 7.0 143.4 1.0X +read timestamps from files 1102 1110 13 0.9 1101.8 0.1X +infer timestamps from files 2042 2051 14 0.5 2041.7 0.1X +read date text from files 140 143 4 7.2 139.6 1.0X +read date from files 739 764 33 1.4 739.1 0.2X +timestamp strings 135 136 1 7.4 134.6 1.1X +parse timestamps from Dataset[String] 1321 1328 7 0.8 1320.8 0.1X +infer timestamps from Dataset[String] 2235 2239 5 0.4 2235.3 0.1X +date strings 194 196 3 5.2 193.6 0.7X +parse dates from Dataset[String] 1054 1058 4 0.9 1054.1 0.1X +from_json(timestamp) 1750 1753 4 0.6 1750.0 0.1X +from_json(date) 1476 1480 6 0.7 1476.1 0.1X +infer error timestamps from Dataset[String] with default format 1499 1502 4 0.7 1499.4 0.1X +infer error timestamps from Dataset[String] with user-provided format 1491 1496 7 0.7 1491.1 0.1X +infer error timestamps from Dataset[String] with legacy format 1528 1538 9 0.7 1527.8 0.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 6100 6124 33 0.0 61003.7 1.0X -pushdown disabled 5957 5981 31 0.0 59569.9 1.0X -w/ filters 729 737 8 0.1 7291.0 8.4X +w/o filters 6122 6143 24 0.0 61217.0 1.0X +pushdown disabled 5947 5957 10 0.0 59474.6 1.0X +w/ filters 700 703 4 0.1 7004.2 8.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partial JSON results: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -parse invalid JSON 2476 2480 5 0.0 247550.8 1.0X +parse invalid JSON 2496 2508 16 0.0 249615.2 1.0X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index bf7662a428dfb..106ad732614c0 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -3,128 +3,128 @@ Benchmark for performance of JSON parsing ================================================================================================ Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 2335 2405 102 2.1 467.1 1.0X -UTF-8 is set 3188 3205 17 1.6 637.5 0.7X +No encoding 2318 2390 99 2.2 463.5 1.0X +UTF-8 is set 4814 4832 16 1.0 962.8 0.5X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 2017 2130 130 2.5 403.4 1.0X -UTF-8 is set 3090 3104 22 1.6 618.0 0.7X +No encoding 2303 2369 93 2.2 460.6 1.0X +UTF-8 is set 4841 4855 12 1.0 968.2 0.5X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 3264 3390 134 0.3 3264.0 1.0X -UTF-8 is set 4385 4419 40 0.2 4384.9 0.7X +No encoding 4324 4400 95 0.2 4324.1 1.0X +UTF-8 is set 4825 4836 10 0.2 4824.6 0.9X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 8549 8761 245 0.0 170970.8 1.0X -UTF-8 is set 9833 9868 31 0.0 196661.2 0.9X +No encoding 9577 9734 206 0.0 191531.0 1.0X +UTF-8 is set 10240 10256 14 0.0 204805.9 0.9X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 1485 1498 22 0.7 1484.5 1.0X -Select 1 column 1056 1063 6 0.9 1055.6 1.4X +Select 10 columns 1602 1605 3 0.6 1601.6 1.0X +Select 1 column 1160 1169 12 0.9 1160.0 1.4X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 622 630 9 1.6 622.2 1.0X -Short column with UTF-8 792 802 13 1.3 792.1 0.8X -Wide column without encoding 7214 7321 111 0.1 7214.3 0.1X -Wide column with UTF-8 6455 6493 54 0.2 6454.8 0.1X +Short column without encoding 661 665 4 1.5 660.7 1.0X +Short column with UTF-8 1188 1188 1 0.8 1187.9 0.6X +Wide column without encoding 5314 5336 21 0.2 5313.6 0.1X +Wide column with UTF-8 7265 7267 2 0.1 7265.1 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 59 61 3 16.9 59.1 1.0X -from_json 1083 1088 5 0.9 1082.5 0.1X -json_tuple 1125 1133 7 0.9 1125.5 0.1X -get_json_object wholestage off 1049 1062 12 1.0 1048.6 0.1X -get_json_object wholestage on 968 975 7 1.0 968.1 0.1X +Text read 58 64 6 17.4 57.5 1.0X +from_json 1092 1106 21 0.9 1091.9 0.1X +json_tuple 1075 1077 3 0.9 1074.7 0.1X +get_json_object wholestage off 1071 1081 10 0.9 1071.1 0.1X +get_json_object wholestage on 1020 1033 20 1.0 1019.7 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 234 238 5 21.4 46.8 1.0X -schema inferring 1774 1776 2 2.8 354.8 0.1X -parsing 2648 2686 33 1.9 529.6 0.1X +Text read 233 270 61 21.5 46.6 1.0X +schema inferring 1786 1790 8 2.8 357.2 0.1X +parsing 2837 2844 9 1.8 567.4 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 615 634 23 8.1 123.0 1.0X -Schema inferring 2319 2330 10 2.2 463.8 0.3X -Parsing without charset 2834 2844 9 1.8 566.8 0.2X -Parsing with UTF-8 3741 3758 17 1.3 748.1 0.2X +Text read 622 625 4 8.0 124.4 1.0X +Schema inferring 2378 2386 7 2.1 475.7 0.3X +Parsing without charset 3016 3019 3 1.7 603.2 0.2X +Parsing with UTF-8 5651 5660 11 0.9 1130.2 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 117 125 10 8.6 116.8 1.0X -to_json(timestamp) 803 809 5 1.2 803.0 0.1X -write timestamps to files 698 701 4 1.4 697.6 0.2X -Create a dataset of dates 123 128 6 8.1 123.2 0.9X -to_json(date) 594 602 7 1.7 594.2 0.2X -write dates to files 471 479 7 2.1 471.4 0.2X +Create a dataset of timestamps 100 101 2 10.0 99.6 1.0X +to_json(timestamp) 648 653 5 1.5 648.1 0.2X +write timestamps to files 677 683 6 1.5 677.2 0.1X +Create a dataset of dates 128 130 3 7.8 128.1 0.8X +to_json(date) 455 466 10 2.2 455.1 0.2X +write dates to files 444 448 5 2.3 443.8 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -read timestamp text from files 157 161 3 6.4 156.8 1.0X -read timestamps from files 1010 1019 10 1.0 1009.8 0.2X -infer timestamps from files 1924 1930 10 0.5 1923.9 0.1X -read date text from files 147 150 4 6.8 146.6 1.1X -read date from files 705 710 6 1.4 705.5 0.2X -timestamp strings 151 159 7 6.6 150.9 1.0X -parse timestamps from Dataset[String] 1191 1193 1 0.8 1191.3 0.1X -infer timestamps from Dataset[String] 2049 2055 7 0.5 2049.2 0.1X -date strings 228 235 6 4.4 228.3 0.7X -parse dates from Dataset[String] 955 967 14 1.0 954.8 0.2X -from_json(timestamp) 1669 1681 12 0.6 1669.4 0.1X -from_json(date) 1444 1447 5 0.7 1443.9 0.1X -infer error timestamps from Dataset[String] with default format 1398 1401 5 0.7 1397.9 0.1X -infer error timestamps from Dataset[String] with user-provided format 1420 1423 2 0.7 1420.2 0.1X -infer error timestamps from Dataset[String] with legacy format 1419 1437 21 0.7 1418.7 0.1X +read timestamp text from files 155 157 3 6.5 154.7 1.0X +read timestamps from files 1091 1093 2 0.9 1091.1 0.1X +infer timestamps from files 2016 2017 1 0.5 2016.2 0.1X +read date text from files 141 142 1 7.1 141.2 1.1X +read date from files 744 751 6 1.3 744.2 0.2X +timestamp strings 132 135 3 7.6 131.5 1.2X +parse timestamps from Dataset[String] 1247 1249 2 0.8 1246.8 0.1X +infer timestamps from Dataset[String] 2129 2134 7 0.5 2128.7 0.1X +date strings 203 204 1 4.9 202.6 0.8X +parse dates from Dataset[String] 1005 1006 1 1.0 1004.7 0.2X +from_json(timestamp) 1659 1672 11 0.6 1659.4 0.1X +from_json(date) 1413 1416 4 0.7 1413.2 0.1X +infer error timestamps from Dataset[String] with default format 1400 1407 6 0.7 1400.4 0.1X +infer error timestamps from Dataset[String] with user-provided format 1411 1420 13 0.7 1410.7 0.1X +infer error timestamps from Dataset[String] with legacy format 1441 1461 20 0.7 1441.2 0.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 6592 6601 7 0.0 65920.4 1.0X -pushdown disabled 5825 5829 4 0.0 58246.5 1.1X -w/ filters 664 802 200 0.2 6643.7 9.9X +w/o filters 5832 5837 9 0.0 58320.8 1.0X +pushdown disabled 5810 5821 10 0.0 58100.1 1.0X +w/ filters 679 767 147 0.1 6792.3 8.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partial JSON results: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -parse invalid JSON 2714 2828 195 0.0 271356.0 1.0X +parse invalid JSON 2325 2417 159 0.0 232496.5 1.0X diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt index 3f95fc73de078..ba1261bd77389 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt @@ -1,22 +1,22 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 2328 2360 31 43.0 23.3 1.0X -make_date(2019, 9, 16) 1883 1936 46 53.1 18.8 1.2X -make_date(*, *, *) 4034 4050 20 24.8 40.3 0.6X +prepare make_date() 2319 2381 55 43.1 23.2 1.0X +make_date(2019, 9, 16) 2021 2048 28 49.5 20.2 1.1X +make_date(*, *, *) 3857 3872 14 25.9 38.6 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -prepare make_timestamp() 358 367 11 2.8 358.0 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 43 46 4 23.3 42.9 8.3X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 37 40 3 26.8 37.3 9.6X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 37 48 10 27.3 36.6 9.8X -make_timestamp(*, *, *, 3, 4, 50.123456) 170 171 2 5.9 169.9 2.1X -make_timestamp(*, *, *, *, *, 0) 113 116 3 8.9 112.5 3.2X -make_timestamp(*, *, *, *, *, 60.0) 158 162 4 6.3 158.2 2.3X -make_timestamp(2019, 1, 2, *, *, *) 478 479 1 2.1 477.9 0.7X -make_timestamp(*, *, *, *, *, *) 491 495 6 2.0 491.5 0.7X +prepare make_timestamp() 346 351 4 2.9 346.3 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 39 43 4 25.8 38.8 8.9X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 42 48 8 23.8 42.0 8.2X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 33 37 6 29.9 33.4 10.4X +make_timestamp(*, *, *, 3, 4, 50.123456) 160 162 1 6.2 160.2 2.2X +make_timestamp(*, *, *, *, *, 0) 103 109 6 9.7 102.6 3.4X +make_timestamp(*, *, *, *, *, 60.0) 144 148 5 6.9 144.2 2.4X +make_timestamp(2019, 1, 2, *, *, *) 422 424 3 2.4 422.0 0.8X +make_timestamp(*, *, *, *, *, *) 459 460 1 2.2 459.3 0.8X diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt index 34855593dd93f..acfb4fa3ddbb1 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt @@ -1,22 +1,22 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 2195 2443 263 45.5 22.0 1.0X -make_date(2019, 9, 16) 1806 1860 81 55.4 18.1 1.2X -make_date(*, *, *) 4107 4186 74 24.4 41.1 0.5X +prepare make_date() 2164 2170 6 46.2 21.6 1.0X +make_date(2019, 9, 16) 1823 1836 11 54.9 18.2 1.2X +make_date(*, *, *) 4074 4085 16 24.5 40.7 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -prepare make_timestamp() 354 364 9 2.8 354.3 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 44 48 3 22.5 44.5 8.0X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 48 53 4 20.8 48.1 7.4X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 34 37 4 29.3 34.1 10.4X -make_timestamp(*, *, *, 3, 4, 50.123456) 146 158 16 6.9 146.0 2.4X -make_timestamp(*, *, *, *, *, 0) 107 114 9 9.4 106.5 3.3X -make_timestamp(*, *, *, *, *, 60.0) 145 150 8 6.9 144.7 2.4X -make_timestamp(2019, 1, 2, *, *, *) 453 454 2 2.2 452.7 0.8X -make_timestamp(*, *, *, *, *, *) 475 480 6 2.1 475.2 0.7X +prepare make_timestamp() 336 350 13 3.0 336.0 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 44 50 8 22.6 44.2 7.6X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 37 43 8 26.8 37.4 9.0X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 37 38 0 26.8 37.3 9.0X +make_timestamp(*, *, *, 3, 4, 50.123456) 155 159 5 6.4 155.2 2.2X +make_timestamp(*, *, *, *, *, 0) 109 111 4 9.2 109.1 3.1X +make_timestamp(*, *, *, *, *, 60.0) 141 144 5 7.1 140.9 2.4X +make_timestamp(2019, 1, 2, *, *, *) 451 452 1 2.2 451.1 0.7X +make_timestamp(*, *, *, *, *, *) 480 483 2 2.1 479.8 0.7X diff --git a/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt b/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt index 8f1696638d097..1b9b8408c42ed 100644 --- a/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt @@ -2,45 +2,45 @@ Metadata Struct Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Vectorized Parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -no metadata columns 614 644 14 8.1 122.7 1.0X -_metadata.file_path 737 752 10 6.8 147.5 0.8X -_metadata.file_name 737 751 17 6.8 147.3 0.8X -_metadata.file_size 638 678 14 7.8 127.5 1.0X -_metadata.file_block_start 632 654 11 7.9 126.5 1.0X -_metadata.file_block_length 665 676 7 7.5 133.0 0.9X -_metadata.file_modification_time 636 655 13 7.9 127.1 1.0X -_metadata.row_index 714 728 8 7.0 142.9 0.9X -_metadata 966 993 15 5.2 193.2 0.6X +no metadata columns 615 646 20 8.1 122.9 1.0X +_metadata.file_path 731 748 9 6.8 146.2 0.8X +_metadata.file_name 720 749 12 6.9 144.0 0.9X +_metadata.file_size 668 682 10 7.5 133.7 0.9X +_metadata.file_block_start 670 679 11 7.5 134.0 0.9X +_metadata.file_block_length 668 679 8 7.5 133.6 0.9X +_metadata.file_modification_time 647 674 8 7.7 129.5 0.9X +_metadata.row_index 702 721 11 7.1 140.4 0.9X +_metadata 965 991 17 5.2 192.9 0.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet-mr: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -no metadata columns 2804 2862 34 1.8 560.9 1.0X -_metadata.file_path 3567 3624 33 1.4 713.3 0.8X -_metadata.file_name 3614 3648 27 1.4 722.8 0.8X -_metadata.file_size 3459 3485 21 1.4 691.8 0.8X -_metadata.file_block_start 3460 3498 25 1.4 692.0 0.8X -_metadata.file_block_length 3396 3432 32 1.5 679.3 0.8X -_metadata.file_modification_time 3385 3416 19 1.5 677.1 0.8X -_metadata.row_index 3734 3762 18 1.3 746.8 0.8X -_metadata 4804 4837 20 1.0 960.8 0.6X +no metadata columns 2555 2601 25 2.0 511.1 1.0X +_metadata.file_path 3338 3382 19 1.5 667.6 0.8X +_metadata.file_name 3325 3365 20 1.5 665.0 0.8X +_metadata.file_size 3141 3164 16 1.6 628.2 0.8X +_metadata.file_block_start 3123 3172 22 1.6 624.7 0.8X +_metadata.file_block_length 3138 3158 11 1.6 627.6 0.8X +_metadata.file_modification_time 3120 3152 31 1.6 624.1 0.8X +_metadata.row_index 3664 3700 23 1.4 732.9 0.7X +_metadata 4819 4856 26 1.0 963.8 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -no metadata columns 6626 6678 89 0.8 1325.2 1.0X -_metadata.file_path 7476 7498 14 0.7 1495.3 0.9X -_metadata.file_name 7468 7485 15 0.7 1493.7 0.9X -_metadata.file_size 7302 7326 18 0.7 1460.3 0.9X -_metadata.file_block_start 7303 7327 14 0.7 1460.5 0.9X -_metadata.file_block_length 7312 7337 14 0.7 1462.4 0.9X -_metadata.file_modification_time 7322 7340 11 0.7 1464.3 0.9X -_metadata 8135 8155 14 0.6 1627.0 0.8X +no metadata columns 7218 7289 102 0.7 1443.5 1.0X +_metadata.file_path 8149 8178 19 0.6 1629.8 0.9X +_metadata.file_name 8137 8164 22 0.6 1627.5 0.9X +_metadata.file_size 7942 7964 14 0.6 1588.5 0.9X +_metadata.file_block_start 7916 7959 20 0.6 1583.3 0.9X +_metadata.file_block_length 7931 7958 17 0.6 1586.2 0.9X +_metadata.file_modification_time 7934 7956 16 0.6 1586.7 0.9X +_metadata 8829 8857 27 0.6 1765.8 0.8X diff --git a/sql/core/benchmarks/MetadataStructBenchmark-results.txt b/sql/core/benchmarks/MetadataStructBenchmark-results.txt index 82429601dab29..bd14214994cd1 100644 --- a/sql/core/benchmarks/MetadataStructBenchmark-results.txt +++ b/sql/core/benchmarks/MetadataStructBenchmark-results.txt @@ -2,45 +2,45 @@ Metadata Struct Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Vectorized Parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -no metadata columns 650 670 20 7.7 129.9 1.0X -_metadata.file_path 743 762 12 6.7 148.7 0.9X -_metadata.file_name 742 752 7 6.7 148.4 0.9X -_metadata.file_size 677 685 6 7.4 135.4 1.0X -_metadata.file_block_start 675 686 12 7.4 134.9 1.0X -_metadata.file_block_length 677 683 5 7.4 135.5 1.0X -_metadata.file_modification_time 673 682 7 7.4 134.7 1.0X -_metadata.row_index 718 728 8 7.0 143.6 0.9X -_metadata 1023 1033 6 4.9 204.6 0.6X +no metadata columns 639 660 20 7.8 127.8 1.0X +_metadata.file_path 745 760 13 6.7 148.9 0.9X +_metadata.file_name 738 747 8 6.8 147.6 0.9X +_metadata.file_size 672 678 4 7.4 134.3 1.0X +_metadata.file_block_start 671 678 4 7.4 134.3 1.0X +_metadata.file_block_length 670 677 5 7.5 134.0 1.0X +_metadata.file_modification_time 669 678 9 7.5 133.8 1.0X +_metadata.row_index 726 731 4 6.9 145.1 0.9X +_metadata 1022 1031 6 4.9 204.5 0.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Parquet-mr: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -no metadata columns 2582 2621 41 1.9 516.4 1.0X -_metadata.file_path 3488 3505 12 1.4 697.5 0.7X -_metadata.file_name 3481 3502 10 1.4 696.1 0.7X -_metadata.file_size 3193 3223 18 1.6 638.6 0.8X -_metadata.file_block_start 3198 3217 15 1.6 639.7 0.8X -_metadata.file_block_length 3191 3216 19 1.6 638.2 0.8X -_metadata.file_modification_time 3188 3204 13 1.6 637.7 0.8X -_metadata.row_index 3714 3736 18 1.3 742.8 0.7X -_metadata 4935 4958 24 1.0 986.9 0.5X +no metadata columns 2658 2694 18 1.9 531.7 1.0X +_metadata.file_path 3480 3504 16 1.4 696.1 0.8X +_metadata.file_name 3465 3486 17 1.4 693.0 0.8X +_metadata.file_size 3244 3268 18 1.5 648.8 0.8X +_metadata.file_block_start 3268 3291 15 1.5 653.6 0.8X +_metadata.file_block_length 3269 3296 33 1.5 653.8 0.8X +_metadata.file_modification_time 3275 3301 19 1.5 655.1 0.8X +_metadata.row_index 3727 3742 13 1.3 745.3 0.7X +_metadata 4986 5019 24 1.0 997.2 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor JSON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -no metadata columns 7127 7171 21 0.7 1425.3 1.0X -_metadata.file_path 8122 8147 13 0.6 1624.4 0.9X -_metadata.file_name 8143 8166 24 0.6 1628.6 0.9X -_metadata.file_size 7914 7943 14 0.6 1582.8 0.9X -_metadata.file_block_start 7947 7978 16 0.6 1589.4 0.9X -_metadata.file_block_length 7964 7991 20 0.6 1592.8 0.9X -_metadata.file_modification_time 7950 7977 20 0.6 1590.1 0.9X -_metadata 8869 8888 15 0.6 1773.7 0.8X +no metadata columns 7065 7085 12 0.7 1413.0 1.0X +_metadata.file_path 8095 8116 11 0.6 1619.0 0.9X +_metadata.file_name 8133 8148 10 0.6 1626.6 0.9X +_metadata.file_size 7787 7810 10 0.6 1557.3 0.9X +_metadata.file_block_start 7787 7803 9 0.6 1557.4 0.9X +_metadata.file_block_length 7774 7802 13 0.6 1554.8 0.9X +_metadata.file_modification_time 7788 7803 11 0.6 1557.6 0.9X +_metadata 8705 8724 12 0.6 1741.0 0.8X diff --git a/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt b/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt index 220e9da2e088c..edc395387882b 100644 --- a/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt @@ -1,12 +1,12 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor metrics aggregation (50 metrics, 100000 tasks per stage): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -1 stage(s) 565 623 52 0.0 564588687.0 1.0X -2 stage(s) 1318 1347 41 0.0 1318133868.0 0.4X -3 stage(s) 2044 2136 130 0.0 2043877303.0 0.3X +1 stage(s) 602 678 78 0.0 601969935.0 1.0X +2 stage(s) 1150 1277 180 0.0 1149781938.0 0.5X +3 stage(s) 1992 2072 113 0.0 1992188122.0 0.3X Stage Count Stage Proc. Time Aggreg. Time - 1 339 58 - 2 333 213 - 3 376 256 + 1 339 53 + 2 344 156 + 3 387 299 diff --git a/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt b/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt index 5ca7125aa3bc0..d3e2611541552 100644 --- a/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt +++ b/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt @@ -1,12 +1,12 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor metrics aggregation (50 metrics, 100000 tasks per stage): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ---------------------------------------------------------------------------------------------------------------------------------------- -1 stage(s) 684 710 28 0.0 683720517.0 1.0X -2 stage(s) 1368 1407 55 0.0 1367925138.0 0.5X -3 stage(s) 1606 1850 346 0.0 1605768734.0 0.4X +1 stage(s) 667 689 20 0.0 666605773.0 1.0X +2 stage(s) 1366 1403 52 0.0 1366457850.0 0.5X +3 stage(s) 2087 2127 57 0.0 2086587364.0 0.3X Stage Count Stage Proc. Time Aggreg. Time - 1 306 92 - 2 437 150 - 3 368 219 + 1 388 98 + 2 346 232 + 3 384 244 diff --git a/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt b/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt index 7228d0a184011..0bccc882d5d0d 100644 --- a/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt @@ -2,126 +2,126 @@ filter & aggregate without group ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/filter/sum wholestage off 36694 36725 44 57.2 17.5 1.0X -range/filter/sum wholestage on 3483 3597 69 602.2 1.7 10.5X +range/filter/sum wholestage off 35356 36325 1371 59.3 16.9 1.0X +range/filter/sum wholestage on 2827 3470 362 741.9 1.3 12.5X ================================================================================================ range/limit/sum ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/limit/sum wholestage off 70 95 35 7458.5 0.1 1.0X -range/limit/sum wholestage on 66 82 13 7909.4 0.1 1.1X +range/limit/sum wholestage off 59 60 1 8883.8 0.1 1.0X +range/limit/sum wholestage on 66 70 7 7984.2 0.1 0.9X ================================================================================================ sample ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sample with replacement wholestage off 8132 8156 35 16.1 62.0 1.0X -sample with replacement wholestage on 5075 5185 154 25.8 38.7 1.6X +sample with replacement wholestage off 7811 7961 212 16.8 59.6 1.0X +sample with replacement wholestage on 5125 5152 40 25.6 39.1 1.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -sample without replacement wholestage off 1885 1893 12 69.5 14.4 1.0X -sample without replacement wholestage on 651 668 20 201.2 5.0 2.9X +sample without replacement wholestage off 1837 1839 3 71.4 14.0 1.0X +sample without replacement wholestage on 660 672 10 198.5 5.0 2.8X ================================================================================================ collect ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect 1 million 159 232 53 6.6 151.6 1.0X -collect 2 millions 295 441 85 3.6 281.2 0.5X -collect 4 millions 818 832 12 1.3 780.2 0.2X +collect 1 million 159 228 64 6.6 151.8 1.0X +collect 2 millions 331 404 65 3.2 316.1 0.5X +collect 4 millions 743 912 148 1.4 708.4 0.2X ================================================================================================ collect limit ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect limit 1 million 147 224 53 7.1 140.6 1.0X -collect limit 2 millions 301 404 86 3.5 287.3 0.5X +collect limit 1 million 161 240 54 6.5 153.6 1.0X +collect limit 2 millions 302 421 79 3.5 287.8 0.5X ================================================================================================ generate explode ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode array wholestage off 12316 12347 45 1.4 734.1 1.0X -generate explode array wholestage on 2800 2856 69 6.0 166.9 4.4X +generate explode array wholestage off 12439 12453 20 1.3 741.4 1.0X +generate explode array wholestage on 2842 3047 135 5.9 169.4 4.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode map wholestage off 23670 23767 138 0.7 1410.8 1.0X -generate explode map wholestage on 9745 9872 100 1.7 580.8 2.4X +generate explode map wholestage off 23498 23624 178 0.7 1400.6 1.0X +generate explode map wholestage on 9976 10151 128 1.7 594.6 2.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate posexplode array wholestage off 12583 12709 178 1.3 750.0 1.0X -generate posexplode array wholestage on 2992 3053 67 5.6 178.3 4.2X +generate posexplode array wholestage off 12883 13108 318 1.3 767.9 1.0X +generate posexplode array wholestage on 2971 3070 66 5.6 177.1 4.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate inline array wholestage off 6914 6972 81 2.4 412.1 1.0X -generate inline array wholestage on 2418 2524 90 6.9 144.1 2.9X +generate inline array wholestage off 7289 7311 31 2.3 434.5 1.0X +generate inline array wholestage on 2378 2456 100 7.1 141.7 3.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate big struct array wholestage off 200 216 22 0.3 3334.6 1.0X -generate big struct array wholestage on 164 180 13 0.4 2733.2 1.2X +generate big struct array wholestage off 191 208 25 0.3 3181.0 1.0X +generate big struct array wholestage on 155 179 24 0.4 2575.5 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -generate big nested struct array wholestage off 18684 18738 77 0.0 311401.3 1.0X -generate big nested struct array wholestage on 152 161 10 0.4 2533.3 122.9X +generate big nested struct array wholestage off 17003 17476 669 0.0 283383.8 1.0X +generate big nested struct array wholestage on 146 149 3 0.4 2436.9 116.3X ================================================================================================ generate regular generator ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate stack wholestage off 14179 14202 33 1.2 845.1 1.0X -generate stack wholestage on 3091 3114 26 5.4 184.2 4.6X +generate stack wholestage off 14560 14589 42 1.2 867.8 1.0X +generate stack wholestage on 3365 3428 53 5.0 200.6 4.3X diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt index 8a3e9921dbe4b..8df4b3a8c4d0a 100644 --- a/sql/core/benchmarks/MiscBenchmark-results.txt +++ b/sql/core/benchmarks/MiscBenchmark-results.txt @@ -2,126 +2,126 @@ filter & aggregate without group ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/filter/sum wholestage off 33428 35668 3169 62.7 15.9 1.0X -range/filter/sum wholestage on 2842 3756 511 737.8 1.4 11.8X +range/filter/sum wholestage off 36850 37568 1016 56.9 17.6 1.0X +range/filter/sum wholestage on 3456 3738 158 606.8 1.6 10.7X ================================================================================================ range/limit/sum ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/limit/sum wholestage off 98 107 13 5332.3 0.2 1.0X -range/limit/sum wholestage on 67 77 11 7806.1 0.1 1.5X +range/limit/sum wholestage off 111 116 7 4736.2 0.2 1.0X +range/limit/sum wholestage on 91 94 3 5739.7 0.2 1.2X ================================================================================================ sample ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sample with replacement wholestage off 8058 8215 221 16.3 61.5 1.0X -sample with replacement wholestage on 4994 5005 8 26.2 38.1 1.6X +sample with replacement wholestage off 7793 7904 158 16.8 59.5 1.0X +sample with replacement wholestage on 4935 5027 54 26.6 37.7 1.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -sample without replacement wholestage off 1974 1983 13 66.4 15.1 1.0X -sample without replacement wholestage on 701 713 12 186.9 5.4 2.8X +sample without replacement wholestage off 1857 1874 24 70.6 14.2 1.0X +sample without replacement wholestage on 707 721 14 185.3 5.4 2.6X ================================================================================================ collect ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect 1 million 170 227 81 6.2 161.8 1.0X -collect 2 millions 361 470 71 2.9 344.1 0.5X -collect 4 millions 727 753 33 1.4 693.1 0.2X +collect 1 million 161 231 79 6.5 153.9 1.0X +collect 2 millions 307 439 78 3.4 292.6 0.5X +collect 4 millions 692 771 82 1.5 660.2 0.2X ================================================================================================ collect limit ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect limit 1 million 153 229 120 6.9 145.6 1.0X -collect limit 2 millions 283 420 139 3.7 269.6 0.5X +collect limit 1 million 156 223 80 6.7 148.9 1.0X +collect limit 2 millions 290 412 108 3.6 276.8 0.5X ================================================================================================ generate explode ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode array wholestage off 13769 13835 94 1.2 820.7 1.0X -generate explode array wholestage on 2901 2973 67 5.8 172.9 4.7X +generate explode array wholestage off 12014 12148 189 1.4 716.1 1.0X +generate explode array wholestage on 2952 3034 65 5.7 175.9 4.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode map wholestage off 25526 25775 352 0.7 1521.4 1.0X -generate explode map wholestage on 9201 9259 68 1.8 548.4 2.8X +generate explode map wholestage off 24447 24465 25 0.7 1457.1 1.0X +generate explode map wholestage on 9883 9992 65 1.7 589.1 2.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate posexplode array wholestage off 14288 14334 65 1.2 851.6 1.0X -generate posexplode array wholestage on 2959 3006 49 5.7 176.3 4.8X +generate posexplode array wholestage off 12403 12597 274 1.4 739.3 1.0X +generate posexplode array wholestage on 2935 3002 62 5.7 174.9 4.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate inline array wholestage off 7176 7181 8 2.3 427.7 1.0X -generate inline array wholestage on 2383 2471 61 7.0 142.0 3.0X +generate inline array wholestage off 6683 6740 81 2.5 398.4 1.0X +generate inline array wholestage on 2455 2517 45 6.8 146.4 2.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate big struct array wholestage off 209 296 123 0.3 3483.4 1.0X -generate big struct array wholestage on 178 188 11 0.3 2965.6 1.2X +generate big struct array wholestage off 236 250 20 0.3 3938.8 1.0X +generate big struct array wholestage on 185 217 27 0.3 3089.8 1.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------- -generate big nested struct array wholestage off 18690 20677 2809 0.0 311503.1 1.0X -generate big nested struct array wholestage on 172 186 15 0.3 2860.7 108.9X +generate big nested struct array wholestage off 18566 21321 3897 0.0 309426.4 1.0X +generate big nested struct array wholestage on 188 206 33 0.3 3132.8 98.8X ================================================================================================ generate regular generator ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate stack wholestage off 15372 15414 60 1.1 916.2 1.0X -generate stack wholestage on 3053 3069 15 5.5 182.0 5.0X +generate stack wholestage off 13373 13401 39 1.3 797.1 1.0X +generate stack wholestage on 3053 3068 13 5.5 182.0 4.4X diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt index d54a37baa5770..922ec22d5e0af 100644 --- a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v1 ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 64 85 12 15.7 63.6 1.0X -Nested column 58 65 7 17.3 57.9 1.1X -Nested column in array 165 170 5 6.1 164.6 0.4X +Top-level column 57 69 8 17.6 56.7 1.0X +Nested column 55 64 6 18.1 55.3 1.0X +Nested column in array 165 174 6 6.1 165.3 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 236 264 18 4.2 235.7 1.0X -Nested column 241 259 11 4.2 240.7 1.0X -Nested column in array 518 537 11 1.9 518.3 0.5X +Top-level column 237 250 15 4.2 236.6 1.0X +Nested column 241 255 12 4.1 241.1 1.0X +Nested column in array 527 544 16 1.9 527.0 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 219 230 7 4.6 218.5 1.0X -Nested column 224 235 10 4.5 223.6 1.0X -Nested column in array 483 487 5 2.1 482.6 0.5X +Top-level column 207 219 8 4.8 206.6 1.0X +Nested column 218 229 7 4.6 218.1 0.9X +Nested column in array 476 480 4 2.1 476.0 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 218 226 5 4.6 218.5 1.0X -Nested column 240 251 7 4.2 240.2 0.9X -Nested column in array 511 515 4 2.0 510.7 0.4X +Top-level column 207 219 8 4.8 206.6 1.0X +Nested column 228 244 15 4.4 227.7 0.9X +Nested column in array 504 508 4 2.0 503.9 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 65 78 13 15.5 64.6 1.0X -Nested column 74 89 15 13.5 74.0 0.9X -Nested column in array 200 219 16 5.0 199.9 0.3X +Top-level column 65 91 21 15.3 65.2 1.0X +Nested column 75 86 6 13.4 74.6 0.9X +Nested column in array 207 234 16 4.8 206.6 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 302 318 22 3.3 301.5 1.0X -Nested column 353 368 17 2.8 352.9 0.9X -Nested column in array 720 755 35 1.4 720.5 0.4X +Top-level column 292 305 14 3.4 291.9 1.0X +Nested column 344 357 18 2.9 344.2 0.8X +Nested column in array 713 760 41 1.4 712.9 0.4X diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt index 77a9e92525691..262ce4adc26e0 100644 --- a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v1 ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 67 83 11 14.8 67.4 1.0X -Nested column 58 65 7 17.3 57.6 1.2X -Nested column in array 158 163 5 6.3 158.2 0.4X +Top-level column 64 82 12 15.7 63.7 1.0X +Nested column 58 67 6 17.2 58.1 1.1X +Nested column in array 162 169 4 6.2 161.6 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 238 263 18 4.2 237.5 1.0X -Nested column 237 255 12 4.2 237.2 1.0X -Nested column in array 523 534 6 1.9 523.3 0.5X +Top-level column 231 253 18 4.3 230.7 1.0X +Nested column 233 252 13 4.3 233.0 1.0X +Nested column in array 501 520 19 2.0 500.6 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 212 221 8 4.7 212.3 1.0X -Nested column 219 230 9 4.6 219.4 1.0X -Nested column in array 470 477 5 2.1 470.4 0.5X +Top-level column 207 213 4 4.8 207.1 1.0X +Nested column 210 219 6 4.8 209.8 1.0X +Nested column in array 448 452 2 2.2 448.2 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 213 218 6 4.7 213.2 1.0X -Nested column 237 245 7 4.2 236.6 0.9X -Nested column in array 504 510 6 2.0 503.8 0.4X +Top-level column 202 211 8 4.9 202.3 1.0X +Nested column 227 233 5 4.4 227.4 0.9X +Nested column in array 480 487 7 2.1 480.2 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 66 78 8 15.1 66.4 1.0X -Nested column 73 84 12 13.7 73.0 0.9X -Nested column in array 202 234 31 4.9 202.3 0.3X +Top-level column 61 77 10 16.5 60.7 1.0X +Nested column 74 90 15 13.4 74.5 0.8X +Nested column in array 218 239 12 4.6 218.5 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 304 317 12 3.3 303.9 1.0X -Nested column 350 358 7 2.9 350.3 0.9X -Nested column in array 722 730 7 1.4 721.8 0.4X +Top-level column 289 300 9 3.5 289.2 1.0X +Nested column 340 348 4 2.9 340.4 0.8X +Nested column in array 685 699 11 1.5 685.2 0.4X diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt index 5585eabfe717b..4676ae5a6bb3c 100644 --- a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v2 ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 66 84 14 15.1 66.0 1.0X -Nested column 57 68 7 17.6 56.7 1.2X -Nested column in array 167 173 5 6.0 166.9 0.4X +Top-level column 59 73 9 17.1 58.5 1.0X +Nested column 58 67 7 17.4 57.5 1.0X +Nested column in array 171 177 4 5.9 170.9 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 243 260 12 4.1 242.6 1.0X -Nested column 245 260 12 4.1 244.9 1.0X -Nested column in array 515 532 12 1.9 514.8 0.5X +Top-level column 233 256 18 4.3 232.9 1.0X +Nested column 229 241 8 4.4 229.5 1.0X +Nested column in array 520 551 30 1.9 520.3 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 218 230 7 4.6 218.1 1.0X -Nested column 223 234 14 4.5 223.4 1.0X -Nested column in array 472 484 7 2.1 472.1 0.5X +Top-level column 203 213 6 4.9 203.0 1.0X +Nested column 208 219 6 4.8 207.8 1.0X +Nested column in array 467 476 4 2.1 467.3 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 213 231 14 4.7 213.3 1.0X -Nested column 240 252 8 4.2 240.1 0.9X -Nested column in array 510 514 3 2.0 509.7 0.4X +Top-level column 201 206 5 5.0 201.2 1.0X +Nested column 238 244 4 4.2 238.0 0.8X +Nested column in array 504 522 30 2.0 503.6 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 65 77 10 15.3 65.3 1.0X -Nested column 74 89 15 13.6 73.7 0.9X -Nested column in array 206 242 28 4.8 206.2 0.3X +Top-level column 53 76 10 18.9 53.0 1.0X +Nested column 73 87 9 13.6 73.5 0.7X +Nested column in array 208 247 24 4.8 208.4 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 301 316 21 3.3 300.7 1.0X -Nested column 361 378 21 2.8 360.9 0.8X -Nested column in array 723 765 42 1.4 722.8 0.4X +Top-level column 289 297 12 3.5 289.3 1.0X +Nested column 334 344 14 3.0 333.8 0.9X +Nested column in array 702 767 41 1.4 701.8 0.4X diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt index e2eba2b51fb49..c7825eb9f4a8f 100644 --- a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v2 ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 66 83 11 15.2 66.0 1.0X -Nested column 60 68 7 16.6 60.4 1.1X -Nested column in array 165 170 3 6.1 164.6 0.4X +Top-level column 62 78 12 16.2 61.8 1.0X +Nested column 58 65 5 17.3 57.8 1.1X +Nested column in array 164 173 6 6.1 164.1 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 252 265 16 4.0 252.0 1.0X -Nested column 243 258 14 4.1 242.8 1.0X -Nested column in array 503 532 14 2.0 503.4 0.5X +Top-level column 232 252 15 4.3 231.9 1.0X +Nested column 230 243 8 4.3 229.9 1.0X +Nested column in array 498 525 22 2.0 497.6 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 216 221 5 4.6 216.1 1.0X -Nested column 221 228 5 4.5 220.8 1.0X -Nested column in array 471 476 3 2.1 470.9 0.5X +Top-level column 210 215 6 4.8 209.9 1.0X +Nested column 210 220 9 4.8 210.3 1.0X +Nested column in array 449 455 5 2.2 449.2 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 213 219 5 4.7 213.2 1.0X -Nested column 238 247 7 4.2 237.9 0.9X -Nested column in array 504 510 5 2.0 503.9 0.4X +Top-level column 204 215 23 4.9 203.7 1.0X +Nested column 227 232 4 4.4 226.9 0.9X +Nested column in array 483 491 5 2.1 483.4 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 60 76 7 16.5 60.4 1.0X -Nested column 71 89 15 14.2 70.5 0.9X -Nested column in array 202 242 47 4.9 202.5 0.3X +Top-level column 69 88 23 14.5 68.8 1.0X +Nested column 73 92 16 13.8 72.7 0.9X +Nested column in array 223 242 15 4.5 223.2 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 309 317 12 3.2 309.4 1.0X -Nested column 355 362 5 2.8 354.8 0.9X -Nested column in array 710 732 19 1.4 710.5 0.4X +Top-level column 290 300 7 3.4 290.0 1.0X +Nested column 342 348 5 2.9 341.9 0.8X +Nested column in array 707 714 7 1.4 706.7 0.4X diff --git a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt index 743331fb4dae2..11fbaf8abd6cd 100644 --- a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt @@ -1,21 +1,21 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Can skip all row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without nested predicate Pushdown 6375 6430 47 16.4 60.8 1.0X -With nested predicate Pushdown 50 65 14 2093.7 0.5 127.3X +Without nested predicate Pushdown 6543 6575 34 16.0 62.4 1.0X +With nested predicate Pushdown 53 69 9 1979.9 0.5 123.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Can skip some row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without nested predicate Pushdown 6877 6916 20 15.2 65.6 1.0X -With nested predicate Pushdown 45 60 10 2345.3 0.4 153.8X +Without nested predicate Pushdown 7146 7174 30 14.7 68.1 1.0X +With nested predicate Pushdown 48 61 11 2176.9 0.5 148.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Can skip no row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without nested predicate Pushdown 13281 13345 48 7.9 126.7 1.0X -With nested predicate Pushdown 13310 13352 34 7.9 126.9 1.0X +Without nested predicate Pushdown 13410 13505 61 7.8 127.9 1.0X +With nested predicate Pushdown 13459 13550 71 7.8 128.4 1.0X diff --git a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt index f6a914114a017..aecea8e69fe1d 100644 --- a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt +++ b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt @@ -1,21 +1,21 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Can skip all row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without nested predicate Pushdown 7157 7297 99 14.7 68.3 1.0X -With nested predicate Pushdown 82 99 13 1279.1 0.8 87.3X +Without nested predicate Pushdown 7218 7274 50 14.5 68.8 1.0X +With nested predicate Pushdown 57 78 12 1844.1 0.5 126.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Can skip some row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without nested predicate Pushdown 7499 7833 119 14.0 71.5 1.0X -With nested predicate Pushdown 61 74 6 1714.3 0.6 122.6X +Without nested predicate Pushdown 7511 7878 130 14.0 71.6 1.0X +With nested predicate Pushdown 68 79 10 1535.8 0.7 110.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Can skip no row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without nested predicate Pushdown 14150 14216 75 7.4 134.9 1.0X -With nested predicate Pushdown 14150 14221 50 7.4 134.9 1.0X +Without nested predicate Pushdown 14244 14289 41 7.4 135.8 1.0X +With nested predicate Pushdown 14288 14318 24 7.3 136.3 1.0X diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt index 9f48b560d615a..e2dde58903157 100644 --- a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For Parquet ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 66 82 12 15.2 66.0 1.0X -Nested column 67 77 6 15.0 66.6 1.0X -Nested column in array 230 239 8 4.4 229.7 0.3X +Top-level column 65 77 12 15.5 64.7 1.0X +Nested column 66 74 12 15.1 66.3 1.0X +Nested column in array 221 255 19 4.5 220.9 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 248 266 13 4.0 247.7 1.0X -Nested column 242 259 12 4.1 242.3 1.0X -Nested column in array 564 594 22 1.8 563.9 0.4X +Top-level column 242 257 11 4.1 241.8 1.0X +Nested column 240 258 17 4.2 239.7 1.0X +Nested column in array 560 588 25 1.8 560.5 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 218 229 7 4.6 218.4 1.0X -Nested column 222 234 10 4.5 221.8 1.0X -Nested column in array 521 537 8 1.9 521.2 0.4X +Top-level column 218 227 14 4.6 218.1 1.0X +Nested column 220 230 14 4.5 219.9 1.0X +Nested column in array 509 525 14 2.0 509.5 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 216 224 7 4.6 216.4 1.0X -Nested column 237 251 10 4.2 236.9 0.9X -Nested column in array 559 568 7 1.8 558.7 0.4X +Top-level column 216 228 13 4.6 215.9 1.0X +Nested column 242 254 12 4.1 241.8 0.9X +Nested column in array 542 559 16 1.8 542.4 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 74 91 10 13.4 74.4 1.0X -Nested column 81 98 12 12.3 81.2 0.9X -Nested column in array 264 287 18 3.8 264.1 0.3X +Top-level column 73 85 13 13.8 72.6 1.0X +Nested column 75 96 17 13.3 75.2 1.0X +Nested column in array 274 310 31 3.6 274.3 0.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 303 370 49 3.3 302.9 1.0X -Nested column 432 447 11 2.3 432.0 0.7X -Nested column in array 779 833 37 1.3 779.2 0.4X +Top-level column 298 314 22 3.4 298.2 1.0X +Nested column 339 353 17 2.9 339.3 0.9X +Nested column in array 757 781 30 1.3 757.3 0.4X diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt index 094a254580f30..87e3441303e76 100644 --- a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For Parquet ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 68 84 11 14.6 68.4 1.0X -Nested column 69 76 5 14.5 69.2 1.0X -Nested column in array 224 229 5 4.5 224.0 0.3X +Top-level column 75 98 14 13.3 75.0 1.0X +Nested column 67 77 8 15.0 66.8 1.1X +Nested column in array 226 233 6 4.4 225.7 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 241 267 25 4.1 241.1 1.0X -Nested column 244 258 10 4.1 243.8 1.0X -Nested column in array 562 583 18 1.8 562.2 0.4X +Top-level column 250 274 20 4.0 250.3 1.0X +Nested column 244 261 14 4.1 244.3 1.0X +Nested column in array 552 569 8 1.8 551.9 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 220 226 3 4.5 220.2 1.0X -Nested column 223 230 8 4.5 223.0 1.0X -Nested column in array 525 530 4 1.9 525.2 0.4X +Top-level column 217 223 5 4.6 217.3 1.0X +Nested column 223 231 6 4.5 222.8 1.0X +Nested column in array 513 521 10 1.9 513.1 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 216 221 5 4.6 216.0 1.0X -Nested column 242 252 6 4.1 242.0 0.9X -Nested column in array 561 565 4 1.8 561.1 0.4X +Top-level column 215 220 3 4.7 215.0 1.0X +Nested column 240 244 3 4.2 239.7 0.9X +Nested column in array 551 559 10 1.8 551.1 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 74 84 8 13.5 73.8 1.0X -Nested column 79 94 12 12.6 79.4 0.9X -Nested column in array 263 283 13 3.8 263.3 0.3X +Top-level column 78 91 12 12.8 78.3 1.0X +Nested column 84 96 9 11.9 84.0 0.9X +Nested column in array 285 301 11 3.5 284.6 0.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 309 323 11 3.2 309.0 1.0X -Nested column 358 367 7 2.8 358.3 0.9X -Nested column in array 771 793 23 1.3 770.7 0.4X +Top-level column 303 315 10 3.3 302.6 1.0X +Nested column 356 360 3 2.8 356.5 0.8X +Nested column in array 759 778 14 1.3 758.8 0.4X diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt index 3c57cee485c54..fec73347a3711 100644 --- a/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt @@ -2,11 +2,11 @@ Write primitive arrays in dataset ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write an array in Dataset: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 211 241 25 39.8 25.2 1.0X -Double 287 303 12 29.3 34.2 0.7X +Int 165 203 26 50.9 19.7 1.0X +Double 226 252 21 37.1 27.0 0.7X diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt index 1f8ea79f262be..50acc6af2e6f9 100644 --- a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt +++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt @@ -2,11 +2,11 @@ Write primitive arrays in dataset ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write an array in Dataset: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 256 291 24 32.8 30.5 1.0X -Double 305 327 15 27.5 36.4 0.8X +Int 199 230 18 42.2 23.7 1.0X +Double 282 290 8 29.7 33.6 0.7X diff --git a/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt b/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt index 33fa0ff972d15..fe289c99daef8 100644 --- a/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt @@ -2,14 +2,14 @@ range ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -full scan 10051 10241 242 52.2 19.2 1.0X -limit after range 36 48 10 14566.1 0.1 279.3X -filter after range 1003 1012 7 522.9 1.9 10.0X -count after range 344 371 42 1522.3 0.7 29.2X -count after limit after range 40 50 10 13166.5 0.1 252.4X +full scan 10142 10196 93 51.7 19.3 1.0X +limit after range 44 47 3 12012.9 0.1 232.4X +filter after range 1005 1019 15 521.5 1.9 10.1X +count after range 345 354 9 1519.4 0.7 29.4X +count after limit after range 48 52 4 10846.5 0.1 209.8X diff --git a/sql/core/benchmarks/RangeBenchmark-results.txt b/sql/core/benchmarks/RangeBenchmark-results.txt index faca550c9e2d5..fdae5cc68e8ce 100644 --- a/sql/core/benchmarks/RangeBenchmark-results.txt +++ b/sql/core/benchmarks/RangeBenchmark-results.txt @@ -2,14 +2,14 @@ range ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -full scan 9920 10204 190 52.9 18.9 1.0X -limit after range 44 51 7 11786.7 0.1 223.0X -filter after range 1011 1031 16 518.5 1.9 9.8X -count after range 370 372 2 1417.0 0.7 26.8X -count after limit after range 47 49 1 11082.6 0.1 209.7X +full scan 9433 9964 355 55.6 18.0 1.0X +limit after range 53 56 2 9946.9 0.1 179.0X +filter after range 1004 1039 48 522.3 1.9 9.4X +count after range 348 353 6 1508.6 0.7 27.1X +count after limit after range 65 72 10 8068.8 0.1 145.2X diff --git a/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt b/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt index 31c29c93fb9e5..37a2d749eb195 100644 --- a/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt @@ -2,12 +2,12 @@ Set Operations Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-Ubuntu-1ubuntu120.04 on Linux 5.4.0-1131-aws-fips -Intel(R) Xeon(R) Platinum 8375C CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor Parsing + Analysis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UNION ALL 342 358 22 0.0 22784.2 1.0X -EXCEPT ALL 310 351 44 0.0 20675.4 1.1X -INTERSECT ALL 305 309 5 0.0 20301.6 1.1X +UNION ALL 319 415 124 0.0 21283.9 1.0X +EXCEPT ALL 259 266 7 0.1 17287.8 1.2X +INTERSECT ALL 257 263 4 0.1 17101.2 1.2X diff --git a/sql/core/benchmarks/SetOperationsBenchmark-results.txt b/sql/core/benchmarks/SetOperationsBenchmark-results.txt index 56bd366579e82..cb944d1d61fd0 100644 --- a/sql/core/benchmarks/SetOperationsBenchmark-results.txt +++ b/sql/core/benchmarks/SetOperationsBenchmark-results.txt @@ -2,12 +2,12 @@ Set Operations Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-Ubuntu-1ubuntu220.04 on Linux 5.4.0-1131-aws-fips -Intel(R) Xeon(R) Platinum 8375C CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor Parsing + Analysis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UNION ALL 360 423 70 0.0 24019.4 1.0X -EXCEPT ALL 322 328 5 0.0 21463.2 1.1X -INTERSECT ALL 327 360 33 0.0 21777.2 1.1X +UNION ALL 325 445 92 0.0 21641.7 1.0X +EXCEPT ALL 272 277 5 0.1 18110.7 1.2X +INTERSECT ALL 269 276 6 0.1 17938.3 1.2X diff --git a/sql/core/benchmarks/SortBenchmark-jdk21-results.txt b/sql/core/benchmarks/SortBenchmark-jdk21-results.txt index 4125c72bce4ab..683aa1f400f5c 100644 --- a/sql/core/benchmarks/SortBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/SortBenchmark-jdk21-results.txt @@ -2,15 +2,15 @@ radix sort ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor radix sort 25000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -reference TimSort key prefix array 8456 8460 5 3.0 338.3 1.0X -reference Arrays.sort 2041 2067 37 12.2 81.6 4.1X -radix sort one byte 68 76 8 368.5 2.7 124.6X -radix sort two bytes 125 133 7 200.4 5.0 67.8X -radix sort eight bytes 479 494 17 52.2 19.2 17.7X -radix sort key prefix array 564 584 33 44.3 22.6 15.0X +reference TimSort key prefix array 8456 9400 1334 3.0 338.3 1.0X +reference Arrays.sort 2181 2207 37 11.5 87.2 3.9X +radix sort one byte 67 72 4 372.6 2.7 126.0X +radix sort two bytes 123 129 6 202.7 4.9 68.6X +radix sort eight bytes 473 478 6 52.9 18.9 17.9X +radix sort key prefix array 575 585 10 43.4 23.0 14.7X diff --git a/sql/core/benchmarks/SortBenchmark-results.txt b/sql/core/benchmarks/SortBenchmark-results.txt index 533049edd2237..49205fbe84798 100644 --- a/sql/core/benchmarks/SortBenchmark-results.txt +++ b/sql/core/benchmarks/SortBenchmark-results.txt @@ -2,15 +2,15 @@ radix sort ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor radix sort 25000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -reference TimSort key prefix array 8140 8157 23 3.1 325.6 1.0X -reference Arrays.sort 2063 2087 35 12.1 82.5 3.9X -radix sort one byte 64 73 6 393.0 2.5 128.0X -radix sort two bytes 116 129 8 216.1 4.6 70.4X -radix sort eight bytes 454 475 16 55.1 18.2 17.9X -radix sort key prefix array 885 896 11 28.3 35.4 9.2X +reference TimSort key prefix array 8207 8241 48 3.0 328.3 1.0X +reference Arrays.sort 2069 2095 37 12.1 82.8 4.0X +radix sort one byte 62 71 5 400.4 2.5 131.4X +radix sort two bytes 127 137 6 196.3 5.1 64.4X +radix sort eight bytes 488 500 8 51.2 19.5 16.8X +radix sort key prefix array 929 930 2 26.9 37.1 8.8X diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt index 4ab5f6d0061cc..9ec0af2d17a71 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.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 10 14 1 1.0 1006.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 43 45 2 0.2 4345.4 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 17 1 0.6 1547.6 0.7X +In-memory 10 12 1 1.0 1034.4 1.0X +RocksDB (trackTotalNumberOfRows: true) 45 47 2 0.2 4504.4 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1617.3 0.6X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 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 12 1 1.0 1011.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 46 1 0.2 4441.2 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1521.7 0.7X +In-memory 9 11 1 1.1 903.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4592.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1614.6 0.6X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 10 1 1.1 940.8 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 46 1 0.2 4425.1 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1515.2 0.6X +In-memory 9 11 1 1.1 941.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4572.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1589.0 0.6X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 11 2 1.1 932.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 46 1 0.2 4400.3 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 17 1 0.7 1506.0 0.6X +In-memory 8 9 1 1.3 797.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4468.9 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1575.2 0.5X ================================================================================================ merge rows ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -RocksDB (trackTotalNumberOfRows: true) 532 547 8 0.0 53154.1 1.0X -RocksDB (trackTotalNumberOfRows: false) 174 180 3 0.1 17410.5 3.1X +RocksDB (trackTotalNumberOfRows: true) 566 584 6 0.0 56623.9 1.0X +RocksDB (trackTotalNumberOfRows: false) 175 185 3 0.1 17469.9 3.2X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------ -RocksDB (trackTotalNumberOfRows: true) 472 484 5 0.0 47228.8 1.0X -RocksDB (trackTotalNumberOfRows: false) 174 179 3 0.1 17433.5 2.7X +RocksDB (trackTotalNumberOfRows: true) 488 502 5 0.0 48798.8 1.0X +RocksDB (trackTotalNumberOfRows: false) 177 184 3 0.1 17675.0 2.8X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------ -RocksDB (trackTotalNumberOfRows: true) 422 434 5 0.0 42226.0 1.0X -RocksDB (trackTotalNumberOfRows: false) 172 179 3 0.1 17235.9 2.4X +RocksDB (trackTotalNumberOfRows: true) 424 437 6 0.0 42429.3 1.0X +RocksDB (trackTotalNumberOfRows: false) 176 182 3 0.1 17608.0 2.4X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------- -RocksDB (trackTotalNumberOfRows: true) 406 419 7 0.0 40646.7 1.0X -RocksDB (trackTotalNumberOfRows: false) 173 179 3 0.1 17265.8 2.4X +RocksDB (trackTotalNumberOfRows: true) 406 420 6 0.0 40630.6 1.0X +RocksDB (trackTotalNumberOfRows: false) 171 178 3 0.1 17137.0 2.4X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 0 1 0 27.0 37.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 46 1 0.2 4447.0 0.0X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1453.0 0.0X +In-memory 0 1 0 26.6 37.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 43 45 1 0.2 4303.7 0.0X +RocksDB (trackTotalNumberOfRows: false) 15 16 0 0.6 1543.9 0.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 1 1.3 796.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4384.0 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 15 0 0.7 1463.5 0.5X +In-memory 7 7 1 1.5 650.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4469.5 0.1X +RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1496.4 0.4X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 9 1 1.2 853.3 1.0X -RocksDB (trackTotalNumberOfRows: true) 43 44 1 0.2 4278.0 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 15 1 0.7 1460.7 0.6X +In-memory 7 7 0 1.5 687.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4357.2 0.2X +RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1479.4 0.5X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 10 2 1.2 854.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 42 44 1 0.2 4183.1 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1457.0 0.6X +In-memory 7 7 0 1.4 697.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 43 45 1 0.2 4332.4 0.2X +RocksDB (trackTotalNumberOfRows: false) 14 15 0 0.7 1448.6 0.5X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 0 1.2 837.4 1.0X -RocksDB (trackTotalNumberOfRows: true) 41 42 1 0.2 4146.8 0.2X -RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1623.1 0.5X +In-memory 7 7 0 1.5 683.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 43 44 1 0.2 4257.5 0.2X +RocksDB (trackTotalNumberOfRows: false) 17 17 1 0.6 1669.2 0.4X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 8 9 1 1.3 798.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 22 23 1 0.5 2201.4 0.4X -RocksDB (trackTotalNumberOfRows: false) 10 10 1 1.0 956.5 0.8X +In-memory 6 7 0 1.6 643.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 23 24 1 0.4 2349.6 0.3X +RocksDB (trackTotalNumberOfRows: false) 10 10 0 1.0 999.9 0.6X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 1 1.4 724.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 7 7 0 1.4 698.4 1.0X -RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.2 450.9 1.6X +In-memory 6 6 0 1.7 590.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 8 8 0 1.3 758.5 0.8X +RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 491.5 1.2X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 0 0 0 24.0 41.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 3 3 1 3.2 317.3 0.1X -RocksDB (trackTotalNumberOfRows: false) 3 3 0 3.2 317.2 0.1X +In-memory 0 0 0 23.7 42.1 1.0X +RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.8 354.3 0.1X +RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.8 354.8 0.1X diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt index 856985b5d071f..a8e4c83be80e1 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt @@ -2,143 +2,143 @@ put rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 10 10 1 1.0 953.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 43 44 2 0.2 4269.8 0.2X -RocksDB (trackTotalNumberOfRows: false) 16 16 1 0.6 1550.5 0.6X +In-memory 8 10 1 1.2 843.4 1.0X +RocksDB (trackTotalNumberOfRows: true) 44 46 1 0.2 4423.7 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1616.4 0.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 10 0 1.1 930.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4387.9 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1521.4 0.6X +In-memory 8 9 1 1.2 830.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 45 47 1 0.2 4506.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1576.7 0.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 10 0 1.1 918.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4441.6 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1521.7 0.6X +In-memory 8 8 0 1.2 808.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4489.0 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1588.0 0.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 10 0 1.1 916.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4413.7 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 0 0.7 1522.0 0.6X +In-memory 8 8 0 1.3 796.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 44 46 1 0.2 4437.2 0.2X +RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1573.0 0.5X ================================================================================================ merge rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -RocksDB (trackTotalNumberOfRows: true) 542 553 6 0.0 54222.4 1.0X -RocksDB (trackTotalNumberOfRows: false) 174 179 3 0.1 17391.9 3.1X +RocksDB (trackTotalNumberOfRows: true) 549 562 6 0.0 54902.6 1.0X +RocksDB (trackTotalNumberOfRows: false) 179 184 2 0.1 17887.1 3.1X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------ -RocksDB (trackTotalNumberOfRows: true) 479 490 5 0.0 47921.1 1.0X -RocksDB (trackTotalNumberOfRows: false) 174 179 3 0.1 17446.2 2.7X +RocksDB (trackTotalNumberOfRows: true) 486 496 6 0.0 48554.8 1.0X +RocksDB (trackTotalNumberOfRows: false) 180 185 3 0.1 17973.1 2.7X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------ -RocksDB (trackTotalNumberOfRows: true) 423 433 5 0.0 42311.4 1.0X -RocksDB (trackTotalNumberOfRows: false) 173 178 3 0.1 17309.1 2.4X +RocksDB (trackTotalNumberOfRows: true) 429 440 4 0.0 42859.9 1.0X +RocksDB (trackTotalNumberOfRows: false) 178 184 2 0.1 17776.5 2.4X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor merging 10000 rows with 10 values per key (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------- -RocksDB (trackTotalNumberOfRows: true) 408 419 5 0.0 40762.3 1.0X -RocksDB (trackTotalNumberOfRows: false) 174 183 3 0.1 17377.7 2.3X +RocksDB (trackTotalNumberOfRows: true) 409 423 4 0.0 40946.3 1.0X +RocksDB (trackTotalNumberOfRows: false) 178 183 2 0.1 17820.2 2.3X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 0 0 0 26.1 38.3 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 46 1 0.2 4444.2 0.0X -RocksDB (trackTotalNumberOfRows: false) 15 15 0 0.7 1489.6 0.0X +In-memory 1 1 0 19.6 51.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4403.8 0.0X +RocksDB (trackTotalNumberOfRows: false) 15 16 0 0.7 1527.7 0.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 8 0 1.3 788.8 1.0X -RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4425.4 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1499.2 0.5X +In-memory 7 7 0 1.5 671.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4484.7 0.1X +RocksDB (trackTotalNumberOfRows: false) 15 16 0 0.7 1516.5 0.4X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 0 1.2 841.3 1.0X -RocksDB (trackTotalNumberOfRows: true) 43 44 1 0.2 4336.9 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.7 1493.6 0.6X +In-memory 7 8 0 1.4 727.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4433.9 0.2X +RocksDB (trackTotalNumberOfRows: false) 15 16 0 0.7 1512.5 0.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 0 1.2 848.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 42 43 1 0.2 4216.8 0.2X -RocksDB (trackTotalNumberOfRows: false) 15 15 0 0.7 1467.4 0.6X +In-memory 7 8 1 1.4 740.3 1.0X +RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4390.9 0.2X +RocksDB (trackTotalNumberOfRows: false) 15 16 0 0.7 1500.6 0.5X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 0 1.2 836.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 42 43 2 0.2 4182.0 0.2X -RocksDB (trackTotalNumberOfRows: false) 16 17 0 0.6 1645.0 0.5X +In-memory 7 7 0 1.5 688.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 43 44 1 0.2 4337.8 0.2X +RocksDB (trackTotalNumberOfRows: false) 17 17 0 0.6 1678.8 0.4X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 8 8 0 1.3 785.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 23 23 1 0.4 2258.3 0.3X -RocksDB (trackTotalNumberOfRows: false) 10 10 0 1.0 999.7 0.8X +In-memory 6 7 0 1.5 645.3 1.0X +RocksDB (trackTotalNumberOfRows: true) 24 24 1 0.4 2370.3 0.3X +RocksDB (trackTotalNumberOfRows: false) 11 11 0 0.9 1082.4 0.6X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 0 1.4 726.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 7 8 0 1.4 736.8 1.0X -RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.1 487.0 1.5X +In-memory 6 6 0 1.7 587.3 1.0X +RocksDB (trackTotalNumberOfRows: true) 8 8 0 1.3 788.1 0.7X +RocksDB (trackTotalNumberOfRows: false) 6 6 0 1.8 554.1 1.1X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 0 0 0 22.8 43.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.8 354.8 0.1X -RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.8 353.1 0.1X +In-memory 0 0 0 23.9 41.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.6 387.4 0.1X +RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.6 389.4 0.1X diff --git a/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt b/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt index 04720fb50b41c..ac6a3ea26717f 100644 --- a/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt @@ -2,10 +2,10 @@ SQL string functions ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor regexp_replace: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -regexp_replace('*-*', '(\\d+)', 'num') 503 527 32 2.0 503.0 1.0X +regexp_replace('*-*', '(\\d+)', 'num') 505 521 11 2.0 505.0 1.0X diff --git a/sql/core/benchmarks/StringFunctionsBenchmark-results.txt b/sql/core/benchmarks/StringFunctionsBenchmark-results.txt index c1b9bdb4ea3da..7ecccc7a826f4 100644 --- a/sql/core/benchmarks/StringFunctionsBenchmark-results.txt +++ b/sql/core/benchmarks/StringFunctionsBenchmark-results.txt @@ -2,10 +2,10 @@ SQL string functions ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor regexp_replace: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -regexp_replace('*-*', '(\\d+)', 'num') 512 529 24 2.0 512.0 1.0X +regexp_replace('*-*', '(\\d+)', 'num') 509 533 15 2.0 509.1 1.0X diff --git a/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt b/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt index 8a79199cd92ed..8e75b2b3d4697 100644 --- a/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt @@ -1,810 +1,810 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q1 756 854 165 0.6 1639.1 1.0X +q1 672 845 214 0.7 1456.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q2 841 929 113 2.7 376.8 1.0X +q2 890 937 57 2.5 398.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q3 239 288 37 12.4 80.6 1.0X +q3 244 297 49 12.2 82.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q4 5033 5148 163 1.0 965.7 1.0X +q4 4653 5006 500 1.1 892.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q5 1125 1352 321 5.0 199.9 1.0X +q5 1406 1469 89 4.0 249.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q6 1095 1121 37 2.8 350.9 1.0X +q6 1122 1134 17 2.8 359.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q7 601 631 48 8.1 122.9 1.0X +q7 567 603 39 8.6 116.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q8 453 501 54 6.8 146.1 1.0X +q8 455 510 51 6.8 146.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q9 895 929 36 0.0 25559860.1 1.0X +q9 874 935 90 0.0 24957685.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q10 1917 1986 98 1.1 925.6 1.0X +q10 1916 2021 148 1.1 925.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q11 1897 2025 181 2.0 502.9 1.0X +q11 1781 2094 443 2.1 472.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q12 164 205 47 4.9 203.0 1.0X +q12 165 209 61 4.9 204.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q13 778 820 49 6.3 157.8 1.0X +q13 767 808 44 6.4 155.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14a 4952 5091 196 1.0 965.3 1.0X +q14a 4824 5200 532 1.1 940.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14b 3747 3786 56 1.4 730.4 1.0X +q14b 4151 4159 11 1.2 809.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q15 443 505 48 3.8 266.7 1.0X +q15 452 485 38 3.7 271.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q16 618 652 44 2.5 395.3 1.0X +q16 589 628 35 2.7 377.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q17 1531 1632 144 3.1 325.8 1.0X +q17 1392 1400 12 3.4 296.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q18 1107 1243 193 3.3 307.3 1.0X +q18 1055 1107 73 3.4 293.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q19 299 335 42 10.4 95.9 1.0X +q19 302 326 28 10.3 96.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q20 180 205 33 8.5 117.3 1.0X +q20 176 213 52 8.7 115.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q21 595 639 32 19.9 50.3 1.0X +q21 576 621 51 20.5 48.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22 3548 3579 43 3.3 299.8 1.0X +q22 3218 3259 58 3.7 271.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q23a 6699 6712 20 0.8 1280.9 1.0X +q23a 5971 6291 453 0.9 1141.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q23b 6338 6683 488 0.8 1211.9 1.0X +q23b 6194 6443 353 0.8 1184.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24a 118 248 53 28.3 35.4 1.0X +q24a 214 254 46 15.6 64.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24b 214 264 41 15.6 64.1 1.0X +q24b 155 229 49 21.6 46.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q25 1381 1456 106 3.4 293.9 1.0X +q25 1379 1525 207 3.4 293.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q26 344 378 37 10.0 99.6 1.0X +q26 318 362 48 10.8 92.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q27 531 580 41 9.2 108.5 1.0X +q27 538 555 14 9.1 110.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q28 1202 1337 190 2.4 417.6 1.0X +q28 1187 1291 147 2.4 412.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q29 1540 1703 230 3.1 327.7 1.0X +q29 1566 1636 99 3.0 333.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q30 402 447 43 0.7 1364.7 1.0X +q30 381 444 53 0.8 1292.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q31 839 851 14 4.4 225.5 1.0X +q31 770 871 89 4.8 207.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q32 198 246 53 7.7 129.0 1.0X +q32 192 213 28 8.0 125.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q33 405 447 56 12.8 78.2 1.0X +q33 393 423 33 13.2 75.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q34 353 384 32 8.7 115.3 1.0X +q34 350 393 35 8.7 114.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35 1296 1367 101 1.6 625.7 1.0X +q35 1281 1303 32 1.6 618.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q36 544 583 43 5.5 183.2 1.0X +q36 516 544 35 5.8 173.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q37 765 807 45 17.4 57.6 1.0X +q37 780 797 21 17.0 58.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q38 696 740 39 7.5 133.5 1.0X +q38 639 690 49 8.2 122.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q39a 1244 1409 233 9.5 105.1 1.0X +q39a 1442 1443 1 8.2 121.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q39b 1269 1285 23 9.3 107.2 1.0X +q39b 1298 1305 9 9.1 109.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q40 398 446 51 4.2 237.8 1.0X +q40 319 386 46 5.2 190.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q41 140 170 37 0.1 7757.0 1.0X +q41 143 156 20 0.1 7952.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q42 155 169 24 19.2 52.2 1.0X +q42 147 165 34 20.2 49.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q43 308 335 30 9.6 104.4 1.0X +q43 304 337 39 9.7 103.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q44 338 403 45 8.6 116.6 1.0X +q44 325 371 39 8.9 112.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q45 196 226 31 4.9 204.6 1.0X +q45 197 227 27 4.9 204.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q46 462 505 67 6.7 148.4 1.0X +q46 452 485 35 6.9 145.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q47 1578 1797 310 1.9 531.1 1.0X +q47 1500 1649 211 2.0 504.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q48 924 945 25 5.3 187.7 1.0X +q48 825 848 38 6.0 167.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q49 670 761 131 8.4 119.3 1.0X +q49 544 603 56 10.3 96.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q50 784 841 51 4.1 241.8 1.0X +q50 564 636 69 5.7 174.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q51 2769 2809 56 1.3 754.1 1.0X +q51 2540 2716 250 1.4 691.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q52 153 186 33 19.5 51.3 1.0X +q52 140 157 21 21.2 47.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q53 277 340 52 10.7 93.1 1.0X +q53 265 290 42 11.2 89.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q54 1250 1262 17 4.2 236.7 1.0X +q54 1253 1262 13 4.2 237.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q55 154 168 19 19.3 51.7 1.0X +q55 143 155 17 20.8 48.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q56 408 473 67 12.7 78.8 1.0X +q56 415 460 29 12.5 80.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q57 778 837 65 2.0 508.0 1.0X +q57 758 807 45 2.0 494.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q58 412 543 195 12.4 80.4 1.0X +q58 438 561 203 11.7 85.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q59 669 747 118 4.4 226.6 1.0X +q59 635 700 58 4.7 215.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q60 427 473 27 12.1 82.4 1.0X +q60 416 452 46 12.5 80.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q61 556 586 30 5.6 178.2 1.0X +q61 537 574 54 5.8 172.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q62 183 204 25 4.3 230.9 1.0X +q62 177 201 32 4.5 223.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q63 281 301 18 10.6 94.7 1.0X +q63 276 306 31 10.8 93.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q64 2377 2586 296 2.9 343.5 1.0X +q64 2141 2374 330 3.2 309.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q65 707 782 71 4.2 237.8 1.0X +q65 607 670 64 4.9 204.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q66 510 565 54 4.5 219.8 1.0X +q66 530 565 37 4.4 228.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q67 5734 5829 134 0.5 1930.2 1.0X +q67 5561 5583 31 0.5 1871.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q68 539 559 20 5.8 173.2 1.0X +q68 454 485 22 6.8 146.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q69 1756 1826 99 1.2 848.0 1.0X +q69 1669 1751 115 1.2 806.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q70 587 620 35 5.0 198.7 1.0X +q70 547 581 36 5.4 185.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q71 352 394 43 14.8 67.5 1.0X +q71 344 368 38 15.2 65.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q72 136543 138257 2425 0.1 8896.5 1.0X +q72 119142 119748 857 0.1 7762.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q73 349 380 32 8.8 114.1 1.0X +q73 328 355 27 9.3 107.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q74 1353 1607 359 2.8 358.8 1.0X +q74 1250 1720 665 3.0 331.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q75 1482 1683 285 3.8 263.0 1.0X +q75 1347 1539 272 4.2 239.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q76 298 335 28 17.2 58.1 1.0X +q76 291 319 33 17.6 56.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q77 610 791 184 9.2 108.7 1.0X +q77 544 719 162 10.3 96.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q78 2709 2767 82 2.1 482.4 1.0X +q78 2167 2456 408 2.6 386.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q79 422 449 29 7.3 137.8 1.0X +q79 414 442 26 7.4 135.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q80 1318 1555 335 4.3 233.5 1.0X +q80 1401 1542 198 4.0 248.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q81 371 421 42 1.0 1012.0 1.0X +q81 336 416 66 1.1 916.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q82 1019 1064 64 14.4 69.2 1.0X +q82 1003 1036 46 14.7 68.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q83 230 286 27 2.6 387.4 1.0X +q83 235 267 31 2.5 394.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q84 685 705 25 3.5 289.6 1.0X +q84 659 707 82 3.6 278.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q85 1656 1798 200 1.7 584.3 1.0X +q85 1993 2062 97 1.4 703.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q86 184 204 30 4.4 227.5 1.0X +q86 189 209 23 4.3 233.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q87 660 727 65 7.9 126.7 1.0X +q87 693 729 31 7.5 133.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q88 1241 1402 229 2.4 417.3 1.0X +q88 1156 1373 307 2.6 388.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q89 309 357 53 9.6 103.9 1.0X +q89 293 334 36 10.1 98.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q90 132 159 19 6.1 162.8 1.0X +q90 113 136 20 7.2 139.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q91 327 365 38 7.0 142.4 1.0X +q91 330 355 20 7.0 143.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q92 135 160 19 6.0 166.3 1.0X +q92 133 173 70 6.1 164.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q93 529 549 38 6.0 166.9 1.0X +q93 423 452 41 7.5 133.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q94 307 364 42 2.7 364.6 1.0X +q94 303 329 24 2.8 359.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q95 5173 5213 57 0.2 6143.9 1.0X +q95 5193 5248 78 0.2 6167.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q96 163 179 17 18.3 54.7 1.0X +q96 160 174 17 18.6 53.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q97 1257 1323 93 3.5 286.2 1.0X +q97 1153 1170 24 3.8 262.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q98 268 304 31 11.1 90.4 1.0X +q98 267 305 45 11.1 89.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q99 264 295 38 5.7 174.4 1.0X +q99 262 285 22 5.8 172.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q5a-v2.7 1294 1378 120 4.3 229.9 1.0X +q5a-v2.7 1134 1159 35 5.0 201.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q6-v2.7 952 977 22 3.3 305.1 1.0X +q6-v2.7 916 932 26 3.4 293.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q10a-v2.7 1806 1867 88 1.1 871.9 1.0X +q10a-v2.7 1757 1846 126 1.2 848.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q11-v2.7 1867 2114 350 2.0 494.9 1.0X +q11-v2.7 1725 2043 449 2.2 457.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q12-v2.7 128 143 18 6.4 157.4 1.0X +q12-v2.7 125 139 19 6.5 154.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14-v2.7 3604 3857 358 1.4 702.5 1.0X +q14-v2.7 3771 3829 82 1.4 735.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14a-v2.7 6933 7124 270 0.7 1351.6 1.0X +q14a-v2.7 6402 6605 287 0.8 1248.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q18a-v2.7 1805 2075 382 2.0 501.2 1.0X +q18a-v2.7 1831 2129 421 2.0 508.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q20-v2.7 157 175 20 9.7 102.6 1.0X +q20-v2.7 153 174 18 10.0 100.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22-v2.7 13585 13655 98 0.9 1147.8 1.0X +q22-v2.7 12999 13185 264 0.9 1098.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22a-v2.7 2043 2046 4 5.8 172.6 1.0X +q22a-v2.7 1909 1970 87 6.2 161.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24-v2.7 207 238 34 16.1 61.9 1.0X +q24-v2.7 190 235 32 17.5 57.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q27a-v2.7 1452 1553 143 3.4 296.8 1.0X +q27a-v2.7 1710 1727 23 2.9 349.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q34-v2.7 363 390 39 8.4 118.7 1.0X +q34-v2.7 336 369 35 9.1 109.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35-v2.7 1310 1320 14 1.6 632.5 1.0X +q35-v2.7 1195 1215 28 1.7 577.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35a-v2.7 1283 1290 10 1.6 619.7 1.0X +q35a-v2.7 1174 1214 56 1.8 567.0 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q36a-v2.7 492 538 44 6.0 165.5 1.0X +q36a-v2.7 481 510 50 6.2 161.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q47-v2.7 1573 1759 262 1.9 529.6 1.0X +q47-v2.7 1567 1672 148 1.9 527.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q49-v2.7 560 618 39 10.0 99.8 1.0X +q49-v2.7 534 636 127 10.5 95.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q51a-v2.7 15223 15578 502 0.2 4146.0 1.0X +q51a-v2.7 14944 15519 814 0.2 4069.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q57-v2.7 756 801 50 2.0 494.0 1.0X +q57-v2.7 686 789 120 2.2 448.1 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q64-v2.7 2553 2715 230 2.7 368.8 1.0X +q64-v2.7 1878 2187 436 3.7 271.4 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q67a-v2.7 7363 7679 446 0.4 2478.4 1.0X +q67a-v2.7 6928 7238 439 0.4 2331.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q70a-v2.7 630 674 35 4.7 213.5 1.0X +q70a-v2.7 618 674 52 4.8 209.3 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q72-v2.7 137936 138063 180 0.1 8987.2 1.0X +q72-v2.7 125912 126950 1468 0.1 8203.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q74-v2.7 1287 1678 553 2.9 341.3 1.0X +q74-v2.7 1078 1567 691 3.5 285.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q75-v2.7 1379 1615 333 4.1 244.9 1.0X +q75-v2.7 1297 1566 380 4.3 230.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q77a-v2.7 898 1007 155 6.3 159.8 1.0X +q77a-v2.7 885 1077 187 6.3 157.6 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q78-v2.7 2389 2652 372 2.4 425.5 1.0X +q78-v2.7 2429 2558 183 2.3 432.5 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q80a-v2.7 1616 2053 618 3.5 286.2 1.0X +q80a-v2.7 1613 1917 430 3.5 285.7 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q86a-v2.7 237 278 36 3.4 292.6 1.0X +q86a-v2.7 240 265 36 3.4 295.8 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q98-v2.7 259 296 65 11.5 87.0 1.0X +q98-v2.7 260 287 22 11.4 87.7 1.0X diff --git a/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt index 4831dffceecd1..7458fd93a4f38 100644 --- a/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt +++ b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt @@ -1,810 +1,810 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q1 600 728 155 0.8 1300.2 1.0X +q1 661 864 259 0.7 1432.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q2 823 845 23 2.7 368.5 1.0X +q2 817 864 59 2.7 366.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q3 225 259 25 13.2 75.6 1.0X +q3 234 277 31 12.7 78.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q4 4365 4759 557 1.2 837.4 1.0X +q4 4856 5073 308 1.1 931.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q5 1027 1178 214 5.5 182.5 1.0X +q5 1118 1254 191 5.0 198.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q6 1062 1102 56 2.9 340.4 1.0X +q6 1228 1362 191 2.5 393.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q7 583 611 30 8.4 119.1 1.0X +q7 756 776 17 6.5 154.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q8 462 483 23 6.7 149.0 1.0X +q8 567 619 61 5.5 182.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q9 878 890 11 0.0 25071759.3 1.0X +q9 907 945 55 0.0 25911119.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q10 1901 2002 143 1.1 917.8 1.0X +q10 2016 2062 65 1.0 973.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q11 1901 2269 521 2.0 504.1 1.0X +q11 1845 2147 427 2.0 489.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q12 228 263 28 3.5 281.8 1.0X +q12 202 240 29 4.0 250.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q13 856 885 27 5.8 173.6 1.0X +q13 852 889 33 5.8 172.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14a 4584 5058 670 1.1 893.5 1.0X +q14a 4637 4894 363 1.1 904.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14b 3771 3852 115 1.4 735.1 1.0X +q14b 3528 3702 246 1.5 687.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q15 383 418 39 4.3 230.4 1.0X +q15 402 413 12 4.1 241.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q16 645 781 127 2.4 412.7 1.0X +q16 750 791 37 2.1 480.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q17 1451 1466 21 3.2 308.8 1.0X +q17 1691 1730 55 2.8 359.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q18 1305 1446 200 2.8 362.3 1.0X +q18 1414 1569 219 2.5 392.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q19 359 415 50 8.7 114.9 1.0X +q19 376 403 43 8.3 120.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q20 185 203 26 8.3 121.1 1.0X +q20 190 204 14 8.1 123.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q21 699 727 27 16.9 59.0 1.0X +q21 672 712 54 17.6 56.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22 3295 3404 154 3.6 278.4 1.0X +q22 3351 3407 80 3.5 283.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q23a 5808 5881 103 0.9 1110.6 1.0X +q23a 5597 5928 467 0.9 1070.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q23b 5902 5980 111 0.9 1128.5 1.0X +q23b 5925 6131 291 0.9 1133.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24a 146 308 88 22.8 43.8 1.0X +q24a 231 263 30 14.4 69.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24b 210 253 43 15.9 62.9 1.0X +q24b 217 257 49 15.4 65.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q25 1249 1294 63 3.8 265.8 1.0X +q25 1468 1480 17 3.2 312.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q26 391 436 39 8.8 113.3 1.0X +q26 426 443 14 8.1 123.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q27 515 575 51 9.5 105.2 1.0X +q27 603 618 16 8.1 123.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q28 1188 1378 268 2.4 412.6 1.0X +q28 1402 1654 356 2.1 486.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q29 1239 1246 9 3.8 263.8 1.0X +q29 1496 1569 104 3.1 318.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q30 473 511 36 0.6 1606.0 1.0X +q30 453 511 72 0.7 1537.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q31 1043 1236 273 3.6 280.2 1.0X +q31 992 1280 407 3.8 266.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q32 254 310 63 6.0 165.6 1.0X +q32 247 291 37 6.2 161.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q33 484 537 38 10.7 93.5 1.0X +q33 407 441 28 12.7 78.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q34 362 393 55 8.4 118.3 1.0X +q34 409 425 20 7.5 133.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35 1405 1427 31 1.5 678.3 1.0X +q35 1442 1460 26 1.4 696.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q36 559 580 18 5.3 188.1 1.0X +q36 570 582 20 5.2 191.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q37 931 945 23 14.3 70.1 1.0X +q37 896 901 6 14.8 67.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q38 759 830 85 6.9 145.7 1.0X +q38 888 1151 371 5.9 170.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q39a 1611 1833 314 7.3 136.1 1.0X +q39a 1533 1756 315 7.7 129.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q39b 1526 1610 118 7.8 129.0 1.0X +q39b 1514 1748 331 7.8 127.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q40 331 358 19 5.1 197.7 1.0X +q40 355 377 20 4.7 212.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q41 164 180 13 0.1 9114.0 1.0X +q41 166 184 10 0.1 9202.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q42 169 196 20 17.6 57.0 1.0X +q42 151 163 13 19.6 51.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q43 330 344 15 9.0 111.7 1.0X +q43 306 328 16 9.6 103.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q44 401 431 38 7.2 138.4 1.0X +q44 338 347 9 8.6 116.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q45 207 249 24 4.6 215.6 1.0X +q45 187 214 28 5.1 194.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q46 523 534 14 5.9 168.2 1.0X +q46 471 491 21 6.6 151.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q47 1714 1776 88 1.7 576.8 1.0X +q47 1780 1899 168 1.7 599.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q48 887 897 13 5.6 180.1 1.0X +q48 937 945 7 5.3 190.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q49 681 732 49 8.2 121.3 1.0X +q49 728 753 41 7.7 129.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q50 672 694 37 4.8 207.3 1.0X +q50 754 786 28 4.3 232.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q51 2761 2791 41 1.3 752.1 1.0X +q51 2470 2795 459 1.5 672.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q52 159 174 17 18.6 53.7 1.0X +q52 151 158 6 19.7 50.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q53 279 320 31 10.7 93.9 1.0X +q53 299 311 23 9.9 100.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q54 1304 1308 5 4.0 247.0 1.0X +q54 1292 1341 70 4.1 244.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q55 161 186 13 18.5 54.1 1.0X +q55 169 186 14 17.6 56.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q56 525 550 22 9.9 101.4 1.0X +q56 491 507 11 10.5 94.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q57 761 854 105 2.0 496.8 1.0X +q57 1032 1203 241 1.5 674.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q58 483 529 39 10.6 94.1 1.0X +q58 509 580 99 10.1 99.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q59 680 692 14 4.3 230.3 1.0X +q59 753 754 2 3.9 255.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q60 500 565 49 10.4 96.5 1.0X +q60 581 692 167 8.9 112.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q61 579 630 56 5.4 185.4 1.0X +q61 578 605 52 5.4 185.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q62 190 215 18 4.2 239.6 1.0X +q62 176 186 4 4.5 221.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q63 268 282 14 11.1 90.3 1.0X +q63 320 351 28 9.3 107.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q64 2422 2684 371 2.9 350.0 1.0X +q64 2602 2865 372 2.7 376.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q65 767 815 45 3.9 258.3 1.0X +q65 603 668 53 4.9 202.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q66 681 707 26 3.4 293.7 1.0X +q66 538 600 82 4.3 232.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q67 5411 5483 101 0.5 1821.5 1.0X +q67 5377 5515 195 0.6 1809.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q68 511 535 24 6.1 164.3 1.0X +q68 495 550 38 6.3 159.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q69 1532 1706 247 1.4 739.6 1.0X +q69 1742 1842 140 1.2 841.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q70 548 560 9 5.4 185.5 1.0X +q70 651 667 14 4.5 220.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q71 408 420 15 12.8 78.3 1.0X +q71 435 455 16 12.0 83.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q72 93843 95348 2129 0.2 6114.3 1.0X +q72 123701 123925 317 0.1 8059.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q73 389 404 11 7.9 127.1 1.0X +q73 336 373 31 9.1 109.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q74 1330 1616 405 2.8 352.6 1.0X +q74 1419 1584 233 2.7 376.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q75 1481 1759 394 3.8 262.9 1.0X +q75 1665 1951 404 3.4 295.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q76 325 364 30 15.8 63.4 1.0X +q76 358 369 11 14.3 69.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q77 573 780 185 9.8 102.0 1.0X +q77 711 859 128 7.9 126.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q78 2164 2460 420 2.6 385.3 1.0X +q78 2705 2843 195 2.1 481.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q79 450 464 11 6.8 147.0 1.0X +q79 484 492 8 6.3 158.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q80 1596 1722 178 3.5 282.8 1.0X +q80 1533 1855 455 3.7 271.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q81 408 458 53 0.9 1113.9 1.0X +q81 364 447 82 1.0 991.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q82 1177 1192 22 12.5 80.0 1.0X +q82 1176 1196 27 12.5 79.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q83 284 324 30 2.1 477.7 1.0X +q83 323 348 20 1.8 542.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q84 776 778 2 3.0 328.0 1.0X +q84 742 767 28 3.2 313.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q85 1721 2185 656 1.6 607.0 1.0X +q85 1817 2120 428 1.6 641.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q86 207 227 14 3.9 255.4 1.0X +q86 210 226 9 3.8 259.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q87 731 791 74 7.1 140.3 1.0X +q87 919 1068 211 5.7 176.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q88 1414 1665 355 2.1 475.6 1.0X +q88 1471 1642 243 2.0 494.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q89 346 391 43 8.6 116.6 1.0X +q89 313 362 62 9.5 105.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q90 146 171 21 5.5 180.3 1.0X +q90 145 165 20 5.6 178.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q91 365 393 26 6.3 159.1 1.0X +q91 388 429 29 5.9 169.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q92 153 183 22 5.3 189.5 1.0X +q92 152 173 17 5.3 188.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q93 430 445 9 7.4 135.8 1.0X +q93 501 515 10 6.3 158.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q94 356 377 23 2.4 422.7 1.0X +q94 367 387 12 2.3 436.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q95 5268 5437 240 0.2 6256.5 1.0X +q95 5374 5470 137 0.2 6382.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q96 188 207 20 15.8 63.3 1.0X +q96 186 208 19 16.0 62.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q97 1214 1299 120 3.6 276.4 1.0X +q97 1318 1354 51 3.3 300.1 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q98 314 351 41 9.5 105.8 1.0X +q98 304 334 33 9.8 102.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q99 312 321 11 4.8 206.2 1.0X +q99 310 330 19 4.9 205.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q5a-v2.7 1495 1520 36 3.8 265.7 1.0X +q5a-v2.7 1498 1674 250 3.8 266.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q6-v2.7 1002 1015 18 3.1 321.0 1.0X +q6-v2.7 980 1001 18 3.2 314.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q10a-v2.7 1792 1914 172 1.2 865.6 1.0X +q10a-v2.7 1863 2003 199 1.1 899.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q11-v2.7 1809 2158 493 2.1 479.7 1.0X +q11-v2.7 1811 2162 497 2.1 480.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q12-v2.7 146 168 18 5.5 180.8 1.0X +q12-v2.7 140 162 24 5.8 173.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14-v2.7 4035 4204 239 1.3 786.6 1.0X +q14-v2.7 3748 3965 307 1.4 730.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14a-v2.7 7068 7371 429 0.7 1377.8 1.0X +q14a-v2.7 7129 7256 179 0.7 1389.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q18a-v2.7 2182 2292 156 1.7 605.7 1.0X +q18a-v2.7 2101 2292 270 1.7 583.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q20-v2.7 179 193 11 8.5 117.1 1.0X +q20-v2.7 162 182 16 9.5 105.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22-v2.7 13689 13818 183 0.9 1156.5 1.0X +q22-v2.7 14202 14286 118 0.8 1199.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22a-v2.7 1981 2114 189 6.0 167.3 1.0X +q22a-v2.7 2164 2288 176 5.5 182.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24-v2.7 228 270 21 14.6 68.3 1.0X +q24-v2.7 254 278 26 13.1 76.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q27a-v2.7 1361 1495 189 3.6 278.3 1.0X +q27a-v2.7 1449 1664 304 3.4 296.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q34-v2.7 403 409 8 7.6 131.6 1.0X +q34-v2.7 404 415 11 7.6 131.9 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35-v2.7 1326 1387 87 1.6 640.1 1.0X +q35-v2.7 1433 1462 41 1.4 691.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35a-v2.7 1296 1311 20 1.6 626.0 1.0X +q35a-v2.7 1346 1394 68 1.5 650.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q36a-v2.7 533 556 16 5.6 179.5 1.0X +q36a-v2.7 575 607 44 5.2 193.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q47-v2.7 1618 1744 178 1.8 544.6 1.0X +q47-v2.7 1841 2000 226 1.6 619.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q49-v2.7 617 669 35 9.1 109.8 1.0X +q49-v2.7 620 682 63 9.1 110.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q51a-v2.7 13989 14478 692 0.3 3809.9 1.0X +q51a-v2.7 14407 14835 605 0.3 3923.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q57-v2.7 799 867 82 1.9 521.6 1.0X +q57-v2.7 966 1157 271 1.6 630.8 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q64-v2.7 2391 2749 506 2.9 345.5 1.0X +q64-v2.7 2494 2897 570 2.8 360.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q67a-v2.7 7040 7300 367 0.4 2369.8 1.0X +q67a-v2.7 7449 7556 152 0.4 2507.3 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q70a-v2.7 702 723 33 4.2 237.8 1.0X +q70a-v2.7 713 751 47 4.1 241.6 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q72-v2.7 92914 94378 2071 0.2 6053.8 1.0X +q72-v2.7 121071 121592 736 0.1 7888.4 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q74-v2.7 1301 1522 314 2.9 344.9 1.0X +q74-v2.7 1213 1347 190 3.1 321.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q75-v2.7 1729 1814 121 3.3 306.9 1.0X +q75-v2.7 1379 1739 509 4.1 244.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q77a-v2.7 809 874 92 6.9 144.1 1.0X +q77a-v2.7 1231 1282 72 4.6 219.2 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q78-v2.7 2053 2458 573 2.7 365.5 1.0X +q78-v2.7 2218 2579 511 2.5 395.0 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q80a-v2.7 1612 1662 72 3.5 285.5 1.0X +q80a-v2.7 1873 1912 56 3.0 331.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q86a-v2.7 244 278 29 3.3 300.6 1.0X +q86a-v2.7 232 262 24 3.5 286.7 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q98-v2.7 289 308 19 10.3 97.2 1.0X +q98-v2.7 285 300 12 10.4 95.8 1.0X diff --git a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt index c746cde05060a..4888365b839e0 100644 --- a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TakeOrderedAndProject with SMJ: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -TakeOrderedAndProject with SMJ for doExecute 214 243 27 0.0 21428.5 1.0X -TakeOrderedAndProject with SMJ for executeCollect 97 102 4 0.1 9748.1 2.2X +TakeOrderedAndProject with SMJ for doExecute 160 200 39 0.1 15968.9 1.0X +TakeOrderedAndProject with SMJ for executeCollect 96 101 5 0.1 9562.9 1.7X diff --git a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt index 1fa4496d6aea0..0cbc1823e7d29 100644 --- a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt +++ b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor TakeOrderedAndProject with SMJ: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -TakeOrderedAndProject with SMJ for doExecute 262 286 31 0.0 26165.4 1.0X -TakeOrderedAndProject with SMJ for executeCollect 107 113 7 0.1 10681.8 2.4X +TakeOrderedAndProject with SMJ for doExecute 194 275 70 0.1 19414.1 1.0X +TakeOrderedAndProject with SMJ for executeCollect 118 119 2 0.1 11785.9 1.6X diff --git a/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt b/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt index 269fdd7c815a2..7c2ab96ac4ec2 100644 --- a/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt @@ -2,21 +2,21 @@ Top-K Computation ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark Top-K: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------- -ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 9148 9493 278 2.3 436.2 1.0X -ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1698 1731 48 12.4 81.0 5.4X -ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12103 12157 56 1.7 577.1 0.8X -ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5049 5211 98 4.2 240.8 1.8X -RANK (PARTITION: , WindowGroupLimit: false) 9596 9842 149 2.2 457.6 1.0X -RANK (PARTITION: , WindowGroupLimit: true) 1896 2059 112 11.1 90.4 4.8X -RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12338 12642 150 1.7 588.3 0.7X -RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4985 5179 95 4.2 237.7 1.8X -DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9389 9628 171 2.2 447.7 1.0X -DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1849 1900 71 11.3 88.2 4.9X -DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12027 12393 186 1.7 573.5 0.8X -DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5018 5083 47 4.2 239.3 1.8X +ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 8651 8928 175 2.4 412.5 1.0X +ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1629 1647 13 12.9 77.7 5.3X +ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11038 11214 143 1.9 526.3 0.8X +ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5061 5109 33 4.1 241.3 1.7X +RANK (PARTITION: , WindowGroupLimit: false) 9458 9640 159 2.2 451.0 0.9X +RANK (PARTITION: , WindowGroupLimit: true) 1728 1749 16 12.1 82.4 5.0X +RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11732 11986 175 1.8 559.4 0.7X +RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4823 4980 69 4.3 230.0 1.8X +DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9262 9491 104 2.3 441.6 0.9X +DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1961 1978 19 10.7 93.5 4.4X +DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11727 11982 139 1.8 559.2 0.7X +DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4876 4978 120 4.3 232.5 1.8X diff --git a/sql/core/benchmarks/TopKBenchmark-results.txt b/sql/core/benchmarks/TopKBenchmark-results.txt index 76efbf1397b08..4b335ce3e2d98 100644 --- a/sql/core/benchmarks/TopKBenchmark-results.txt +++ b/sql/core/benchmarks/TopKBenchmark-results.txt @@ -2,21 +2,21 @@ Top-K Computation ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Benchmark Top-K: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------- -ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 9462 9625 131 2.2 451.2 1.0X -ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1653 1694 28 12.7 78.8 5.7X -ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11977 12058 82 1.8 571.1 0.8X -ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5021 5081 35 4.2 239.4 1.9X -RANK (PARTITION: , WindowGroupLimit: false) 10017 10132 79 2.1 477.6 0.9X -RANK (PARTITION: , WindowGroupLimit: true) 1948 1984 22 10.8 92.9 4.9X -RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12477 12533 73 1.7 594.9 0.8X -RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5033 5090 36 4.2 240.0 1.9X -DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9757 9841 63 2.1 465.3 1.0X -DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1968 1996 30 10.7 93.8 4.8X -DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12419 12483 47 1.7 592.2 0.8X -DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5060 5128 53 4.1 241.3 1.9X +ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 9179 9279 81 2.3 437.7 1.0X +ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1609 1637 16 13.0 76.7 5.7X +ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11629 11673 34 1.8 554.5 0.8X +ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5008 5038 23 4.2 238.8 1.8X +RANK (PARTITION: , WindowGroupLimit: false) 9720 9859 339 2.2 463.5 0.9X +RANK (PARTITION: , WindowGroupLimit: true) 1955 1990 15 10.7 93.2 4.7X +RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12101 12208 135 1.7 577.0 0.8X +RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5019 5127 167 4.2 239.3 1.8X +DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9673 9748 62 2.2 461.3 0.9X +DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1972 2007 31 10.6 94.0 4.7X +DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12099 12148 42 1.7 576.9 0.8X +DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4982 5028 28 4.2 237.5 1.8X diff --git a/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt b/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt index 81efa0b9b3a72..7c2f56761d1cd 100644 --- a/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt @@ -2,58 +2,58 @@ UDF with mixed input types ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -long/nullable int/string to string wholestage off 129 165 50 0.8 1291.3 1.0X -long/nullable int/string to string wholestage on 64 74 6 1.6 638.6 2.0X +long/nullable int/string to string wholestage off 137 144 9 0.7 1370.6 1.0X +long/nullable int/string to string wholestage on 72 82 11 1.4 719.9 1.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -long/nullable int/string to option wholestage off 47 66 28 2.2 465.1 1.0X -long/nullable int/string to option wholestage on 34 39 6 2.9 343.2 1.4X +long/nullable int/string to option wholestage off 43 49 9 2.3 427.1 1.0X +long/nullable int/string to option wholestage on 37 42 6 2.7 374.6 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int/string to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to primitive wholestage off 30 31 1 3.3 299.6 1.0X -long/nullable int/string to primitive wholestage on 28 29 2 3.6 280.4 1.1X +long/nullable int/string to primitive wholestage off 34 39 7 2.9 340.1 1.0X +long/nullable int/string to primitive wholestage on 32 35 4 3.2 315.5 1.1X ================================================================================================ UDF with primitive types ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -long/nullable int to string wholestage off 30 31 2 3.4 297.4 1.0X -long/nullable int to string wholestage on 28 35 5 3.5 283.9 1.0X +long/nullable int to string wholestage off 30 30 0 3.3 301.5 1.0X +long/nullable int to string wholestage on 31 33 1 3.3 306.2 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -long/nullable int to option wholestage off 22 26 6 4.6 219.3 1.0X -long/nullable int to option wholestage on 21 23 1 4.7 214.0 1.0X +long/nullable int to option wholestage off 22 25 4 4.5 224.4 1.0X +long/nullable int to option wholestage on 23 28 6 4.4 228.9 1.0X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -long/nullable int to primitive wholestage off 18 19 1 5.6 179.0 1.0X -long/nullable int to primitive wholestage on 18 19 1 5.6 179.7 1.0X +long/nullable int to primitive wholestage off 24 25 2 4.2 235.5 1.0X +long/nullable int to primitive wholestage on 19 20 1 5.3 189.2 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Baseline 13 19 8 7.9 125.8 1.0X -With identity UDF 16 18 2 6.2 160.4 0.8X +Baseline 13 15 2 7.9 126.9 1.0X +With identity UDF 18 20 3 5.7 176.4 0.7X diff --git a/sql/core/benchmarks/UDFBenchmark-results.txt b/sql/core/benchmarks/UDFBenchmark-results.txt index 818b51532da74..4e6c9dfdc74e6 100644 --- a/sql/core/benchmarks/UDFBenchmark-results.txt +++ b/sql/core/benchmarks/UDFBenchmark-results.txt @@ -2,58 +2,58 @@ UDF with mixed input types ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -long/nullable int/string to string wholestage off 165 166 1 0.6 1648.2 1.0X -long/nullable int/string to string wholestage on 87 114 18 1.1 869.9 1.9X +long/nullable int/string to string wholestage off 130 141 15 0.8 1299.8 1.0X +long/nullable int/string to string wholestage on 92 97 6 1.1 922.8 1.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -long/nullable int/string to option wholestage off 66 68 3 1.5 659.8 1.0X -long/nullable int/string to option wholestage on 53 63 6 1.9 525.8 1.3X +long/nullable int/string to option wholestage off 52 57 7 1.9 523.5 1.0X +long/nullable int/string to option wholestage on 42 49 5 2.4 420.5 1.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int/string to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to primitive wholestage off 34 40 8 3.0 338.5 1.0X -long/nullable int/string to primitive wholestage on 28 30 1 3.6 280.4 1.2X +long/nullable int/string to primitive wholestage off 30 34 5 3.3 301.7 1.0X +long/nullable int/string to primitive wholestage on 31 33 2 3.2 312.3 1.0X ================================================================================================ UDF with primitive types ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -long/nullable int to string wholestage off 28 30 2 3.5 284.0 1.0X -long/nullable int to string wholestage on 29 33 5 3.4 293.0 1.0X +long/nullable int to string wholestage off 31 32 1 3.2 312.6 1.0X +long/nullable int to string wholestage on 31 32 1 3.2 309.5 1.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------- -long/nullable int to option wholestage off 22 23 1 4.6 218.1 1.0X -long/nullable int to option wholestage on 22 23 1 4.5 224.7 1.0X +long/nullable int to option wholestage off 23 23 0 4.4 226.9 1.0X +long/nullable int to option wholestage on 24 25 2 4.2 240.8 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------- -long/nullable int to primitive wholestage off 16 16 0 6.4 157.3 1.0X -long/nullable int to primitive wholestage on 18 21 4 5.7 175.0 0.9X +long/nullable int to primitive wholestage off 18 20 3 5.5 180.6 1.0X +long/nullable int to primitive wholestage on 19 21 3 5.2 193.0 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Baseline 14 16 1 7.1 141.4 1.0X -With identity UDF 14 16 3 6.9 144.3 1.0X +Baseline 13 16 2 7.5 133.4 1.0X +With identity UDF 17 18 1 6.1 165.2 0.8X diff --git a/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt b/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt index d11fd0406e1b4..8148196e6b688 100644 --- a/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt @@ -2,32 +2,32 @@ Benchmark UnsafeArrayData ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 74 74 1 2281.5 0.4 1.0X -Double 158 158 0 1064.2 0.9 0.5X +Int 76 77 1 2202.3 0.5 1.0X +Double 159 159 0 1055.9 0.9 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 12 13 1 1709.6 0.6 1.0X -Double 28 33 4 737.4 1.4 0.4X +Int 12 13 1 1797.9 0.6 1.0X +Double 28 33 4 748.6 1.3 0.4X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Get primitive array from UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 19 21 1 3322.4 0.3 1.0X -Double 39 42 2 1600.5 0.6 0.5X +Int 20 21 1 3164.3 0.3 1.0X +Double 40 42 1 1561.8 0.6 0.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Create UnsafeArrayData from primitive array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Int 20 24 2 3069.8 0.3 1.0X -Double 44 49 3 1444.2 0.7 0.5X +Int 20 22 1 3198.7 0.3 1.0X +Double 42 45 2 1508.3 0.7 0.5X diff --git a/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt b/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt index 79032e13c0de3..1c9c5f4f5b4c3 100644 --- a/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt +++ b/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt @@ -2,32 +2,32 @@ Benchmark UnsafeArrayData ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Read UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 76 76 0 2215.1 0.5 1.0X -Double 158 158 0 1062.9 0.9 0.5X +Int 76 77 3 2206.0 0.5 1.0X +Double 158 159 1 1060.4 0.9 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Write UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 12 14 1 1690.2 0.6 1.0X -Double 31 33 1 687.1 1.5 0.4X +Int 13 15 1 1671.6 0.6 1.0X +Double 28 34 3 738.8 1.4 0.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Get primitive array from UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Int 21 23 2 3025.8 0.3 1.0X -Double 45 48 1 1410.8 0.7 0.5X +Int 19 22 2 3250.9 0.3 1.0X +Double 40 43 2 1572.4 0.6 0.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Create UnsafeArrayData from primitive array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------- -Int 22 24 1 2902.4 0.3 1.0X -Double 46 49 1 1374.3 0.7 0.5X +Int 20 23 2 3201.2 0.3 1.0X +Double 43 46 1 1460.5 0.7 0.5X diff --git a/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt b/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt index c5756342d99d5..eac137fad9594 100644 --- a/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt @@ -2,25 +2,25 @@ Add 2 columns and drop 2 columns at 3 different depths of nesting ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Add 2 columns and drop 2 columns at 3 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -To non-nullable StructTypes using performant method 3 4 1 0.0 Infinity 1.0X -To nullable StructTypes using performant method 1 2 0 0.0 Infinity 1.9X -To non-nullable StructTypes using non-performant method 18 21 2 0.0 Infinity 0.1X -To nullable StructTypes using non-performant method 792 836 39 0.0 Infinity 0.0X +To non-nullable StructTypes using performant method 1 2 1 0.0 Infinity 1.0X +To nullable StructTypes using performant method 1 1 0 0.0 Infinity 1.4X +To non-nullable StructTypes using non-performant method 15 16 1 0.0 Infinity 0.1X +To nullable StructTypes using non-performant method 542 561 13 0.0 Infinity 0.0X ================================================================================================ Add 50 columns and drop 50 columns at 100 different depths of nesting ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Add 50 columns and drop 50 columns at 100 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -To non-nullable StructTypes using performant method 1088 1112 34 0.0 Infinity 1.0X -To nullable StructTypes using performant method 1150 1151 1 0.0 Infinity 0.9X +To non-nullable StructTypes using performant method 168 172 4 0.0 Infinity 1.0X +To nullable StructTypes using performant method 196 200 5 0.0 Infinity 0.9X diff --git a/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt b/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt index 7bc440e192516..6e137bec68e30 100644 --- a/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt +++ b/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt @@ -2,25 +2,25 @@ Add 2 columns and drop 2 columns at 3 different depths of nesting ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Add 2 columns and drop 2 columns at 3 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -To non-nullable StructTypes using performant method 2 3 1 0.0 Infinity 1.0X +To non-nullable StructTypes using performant method 2 2 1 0.0 Infinity 1.0X To nullable StructTypes using performant method 1 1 0 0.0 Infinity 1.4X -To non-nullable StructTypes using non-performant method 18 19 2 0.0 Infinity 0.1X -To nullable StructTypes using non-performant method 846 885 45 0.0 Infinity 0.0X +To non-nullable StructTypes using non-performant method 16 17 1 0.0 Infinity 0.1X +To nullable StructTypes using non-performant method 565 597 25 0.0 Infinity 0.0X ================================================================================================ Add 50 columns and drop 50 columns at 100 different depths of nesting ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Add 50 columns and drop 50 columns at 100 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -To non-nullable StructTypes using performant method 1087 1109 31 0.0 Infinity 1.0X -To nullable StructTypes using performant method 1123 1190 95 0.0 Infinity 1.0X +To non-nullable StructTypes using performant method 178 188 8 0.0 Infinity 1.0X +To nullable StructTypes using performant method 207 212 5 0.0 Infinity 0.9X diff --git a/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt b/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt index 49cf58086a51c..143c6c57232e1 100644 --- a/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt @@ -1,44 +1,44 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = true codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------ -native_long_add 9542 9554 11 52.4 19.1 1.0X -java_long_add_default 22433 22756 485 22.3 44.9 0.4X -java_long_add_magic 11747 11782 44 42.6 23.5 0.8X -java_long_add_static_magic 11539 11594 48 43.3 23.1 0.8X -scala_long_add_default 23789 25196 2336 21.0 47.6 0.4X -scala_long_add_magic 11714 11758 38 42.7 23.4 0.8X +native_long_add 9807 10151 549 51.0 19.6 1.0X +java_long_add_default 22932 22997 56 21.8 45.9 0.4X +java_long_add_magic 11408 11651 246 43.8 22.8 0.9X +java_long_add_static_magic 11451 11487 52 43.7 22.9 0.9X +scala_long_add_default 23554 23574 22 21.2 47.1 0.4X +scala_long_add_magic 11686 11710 33 42.8 23.4 0.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = false codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -native_long_add 10296 10347 45 48.6 20.6 1.0X -java_long_add_default 22464 23279 1403 22.3 44.9 0.5X -java_long_add_magic 11775 11807 33 42.5 23.5 0.9X -java_long_add_static_magic 10049 10065 16 49.8 20.1 1.0X -scala_long_add_default 22436 24439 3455 22.3 44.9 0.5X -scala_long_add_magic 11815 11895 108 42.3 23.6 0.9X +native_long_add 10397 10439 41 48.1 20.8 1.0X +java_long_add_default 22679 22712 33 22.0 45.4 0.5X +java_long_add_magic 11595 11695 100 43.1 23.2 0.9X +java_long_add_static_magic 10111 10146 38 49.5 20.2 1.0X +scala_long_add_default 22592 22624 27 22.1 45.2 0.5X +scala_long_add_magic 11593 11648 52 43.1 23.2 0.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = true codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -native_long_add 22445 22924 448 22.3 44.9 1.0X -java_long_add_default 26468 26478 10 18.9 52.9 0.8X -java_long_add_magic 32917 32937 26 15.2 65.8 0.7X -java_long_add_static_magic 31424 31496 108 15.9 62.8 0.7X -scala_long_add_default 26265 26358 100 19.0 52.5 0.9X -scala_long_add_magic 33764 34033 423 14.8 67.5 0.7X +native_long_add 22769 22882 160 22.0 45.5 1.0X +java_long_add_default 27959 28164 261 17.9 55.9 0.8X +java_long_add_magic 32664 32703 36 15.3 65.3 0.7X +java_long_add_static_magic 31044 31293 407 16.1 62.1 0.7X +scala_long_add_default 26593 26614 18 18.8 53.2 0.9X +scala_long_add_magic 32909 33049 170 15.2 65.8 0.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = false codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -native_long_add 21582 22336 1304 23.2 43.2 1.0X -java_long_add_default 25194 25472 475 19.8 50.4 0.9X -java_long_add_magic 32678 32727 43 15.3 65.4 0.7X -java_long_add_static_magic 30357 30481 214 16.5 60.7 0.7X -scala_long_add_default 25166 25413 392 19.9 50.3 0.9X -scala_long_add_magic 32759 32773 12 15.3 65.5 0.7X +native_long_add 22966 22981 26 21.8 45.9 1.0X +java_long_add_default 26581 26697 133 18.8 53.2 0.9X +java_long_add_magic 32925 33042 131 15.2 65.9 0.7X +java_long_add_static_magic 31046 32306 2072 16.1 62.1 0.7X +scala_long_add_default 26648 26670 19 18.8 53.3 0.9X +scala_long_add_magic 32969 33052 129 15.2 65.9 0.7X diff --git a/sql/core/benchmarks/V2FunctionBenchmark-results.txt b/sql/core/benchmarks/V2FunctionBenchmark-results.txt index dca57e380c1a2..8dcacf05fa0eb 100644 --- a/sql/core/benchmarks/V2FunctionBenchmark-results.txt +++ b/sql/core/benchmarks/V2FunctionBenchmark-results.txt @@ -1,44 +1,44 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = true codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------ -native_long_add 9323 9391 60 53.6 18.6 1.0X -java_long_add_default 22346 22797 425 22.4 44.7 0.4X -java_long_add_magic 10786 10800 13 46.4 21.6 0.9X -java_long_add_static_magic 10625 10748 169 47.1 21.2 0.9X -scala_long_add_default 22788 22840 47 21.9 45.6 0.4X -scala_long_add_magic 10709 10767 51 46.7 21.4 0.9X +native_long_add 9192 9271 105 54.4 18.4 1.0X +java_long_add_default 22377 22680 265 22.3 44.8 0.4X +java_long_add_magic 10753 10776 28 46.5 21.5 0.9X +java_long_add_static_magic 10564 11517 825 47.3 21.1 0.9X +scala_long_add_default 23011 23250 273 21.7 46.0 0.4X +scala_long_add_magic 10654 10734 97 46.9 21.3 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = false codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -native_long_add 9743 9901 137 51.3 19.5 1.0X -java_long_add_default 22268 22278 10 22.5 44.5 0.4X -java_long_add_magic 10735 10785 44 46.6 21.5 0.9X -java_long_add_static_magic 9964 10028 94 50.2 19.9 1.0X -scala_long_add_default 21995 22058 63 22.7 44.0 0.4X -scala_long_add_magic 10726 10757 42 46.6 21.5 0.9X +native_long_add 10044 10057 12 49.8 20.1 1.0X +java_long_add_default 22261 22343 81 22.5 44.5 0.5X +java_long_add_magic 10632 10644 17 47.0 21.3 0.9X +java_long_add_static_magic 9940 9974 59 50.3 19.9 1.0X +scala_long_add_default 22279 22349 68 22.4 44.6 0.5X +scala_long_add_magic 10616 10639 21 47.1 21.2 0.9X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = true codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -native_long_add 22837 22861 31 21.9 45.7 1.0X -java_long_add_default 28062 28099 41 17.8 56.1 0.8X -java_long_add_magic 32026 33081 1131 15.6 64.1 0.7X -java_long_add_static_magic 32031 32038 8 15.6 64.1 0.7X -scala_long_add_default 26219 26263 63 19.1 52.4 0.9X -scala_long_add_magic 32113 32182 65 15.6 64.2 0.7X +native_long_add 22642 23179 680 22.1 45.3 1.0X +java_long_add_default 27400 27497 102 18.2 54.8 0.8X +java_long_add_magic 31896 31958 66 15.7 63.8 0.7X +java_long_add_static_magic 30630 31059 389 16.3 61.3 0.7X +scala_long_add_default 26240 26339 156 19.1 52.5 0.9X +scala_long_add_magic 32268 32325 90 15.5 64.5 0.7X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor scalar function (long + long) -> long, result_nullable = false codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative -------------------------------------------------------------------------------------------------------------------------------------------------------------- -native_long_add 22056 22271 294 22.7 44.1 1.0X -java_long_add_default 25840 25884 40 19.3 51.7 0.9X -java_long_add_magic 31928 31992 55 15.7 63.9 0.7X -java_long_add_static_magic 31464 31507 46 15.9 62.9 0.7X -scala_long_add_default 25851 25932 107 19.3 51.7 0.9X -scala_long_add_magic 32315 32881 629 15.5 64.6 0.7X +native_long_add 21853 22201 538 22.9 43.7 1.0X +java_long_add_default 25860 25886 34 19.3 51.7 0.8X +java_long_add_magic 32191 32350 218 15.5 64.4 0.7X +java_long_add_static_magic 30755 30812 52 16.3 61.5 0.7X +scala_long_add_default 25872 25923 69 19.3 51.7 0.8X +scala_long_add_magic 31910 31922 14 15.7 63.8 0.7X diff --git a/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt index c4b6ef29d7074..91e51c6833aa5 100644 --- a/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt @@ -2,157 +2,157 @@ parsing large select expressions ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor parsing large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 select expressions 1 2 1 0.0 1196151.0 1.0X -100 select expressions 2 3 1 0.0 2095800.0 0.6X -2500 select expressions 36 39 4 0.0 35701821.0 0.0X +1 select expressions 1 1 0 0.0 669297.0 1.0X +100 select expressions 3 3 1 0.0 2920356.0 0.2X +2500 select expressions 63 65 1 0.0 63383411.0 0.0X ================================================================================================ optimize large select expressions ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor optimize large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -100 columns 4 5 1 0.0 4384067.0 1.0X -1000 columns 28 29 1 0.0 27845199.0 0.2X -10000 columns 287 294 7 0.0 286788665.0 0.0X +100 columns 6 7 1 0.0 6257029.0 1.0X +1000 columns 48 49 1 0.0 47583298.0 0.1X +10000 columns 488 504 11 0.0 487843016.0 0.0X ================================================================================================ many column field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor many column field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 cols x 100000 rows (read in-mem) 16 22 5 6.4 157.3 1.0X -1 cols x 100000 rows (exec in-mem) 18 23 5 5.6 179.9 0.9X -1 cols x 100000 rows (read parquet) 30 37 7 3.3 302.7 0.5X -1 cols x 100000 rows (write parquet) 98 106 8 1.0 978.0 0.2X -100 cols x 1000 rows (read in-mem) 12 17 4 8.1 123.9 1.3X -100 cols x 1000 rows (exec in-mem) 15 19 5 6.5 153.2 1.0X -100 cols x 1000 rows (read parquet) 24 30 7 4.1 244.2 0.6X -100 cols x 1000 rows (write parquet) 93 103 9 1.1 932.3 0.2X -2500 cols x 40 rows (read in-mem) 55 58 4 1.8 545.9 0.3X -2500 cols x 40 rows (exec in-mem) 100 107 6 1.0 995.7 0.2X -2500 cols x 40 rows (read parquet) 306 308 3 0.3 3060.5 0.1X -2500 cols x 40 rows (write parquet) 135 144 10 0.7 1349.9 0.1X +1 cols x 100000 rows (read in-mem) 15 22 5 6.8 147.1 1.0X +1 cols x 100000 rows (exec in-mem) 16 23 5 6.3 158.4 0.9X +1 cols x 100000 rows (read parquet) 28 35 7 3.6 281.5 0.5X +1 cols x 100000 rows (write parquet) 90 102 8 1.1 897.3 0.2X +100 cols x 1000 rows (read in-mem) 12 16 4 8.3 120.4 1.2X +100 cols x 1000 rows (exec in-mem) 15 18 4 6.8 146.1 1.0X +100 cols x 1000 rows (read parquet) 24 28 7 4.2 237.6 0.6X +100 cols x 1000 rows (write parquet) 85 92 6 1.2 847.2 0.2X +2500 cols x 40 rows (read in-mem) 43 45 5 2.3 426.0 0.3X +2500 cols x 40 rows (exec in-mem) 71 74 4 1.4 708.9 0.2X +2500 cols x 40 rows (read parquet) 295 301 4 0.3 2945.7 0.0X +2500 cols x 40 rows (write parquet) 115 119 5 0.9 1153.7 0.1X ================================================================================================ wide shallowly nested struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor wide shallowly nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 20 26 6 5.0 201.6 1.0X -1 wide x 100000 rows (exec in-mem) 22 28 7 4.5 223.4 0.9X -1 wide x 100000 rows (read parquet) 25 31 8 4.0 249.3 0.8X -1 wide x 100000 rows (write parquet) 99 109 9 1.0 992.7 0.2X -100 wide x 1000 rows (read in-mem) 15 19 5 6.6 151.9 1.3X -100 wide x 1000 rows (exec in-mem) 23 28 6 4.4 229.1 0.9X -100 wide x 1000 rows (read parquet) 22 29 7 4.5 223.1 0.9X -100 wide x 1000 rows (write parquet) 95 103 7 1.1 947.8 0.2X -2500 wide x 40 rows (read in-mem) 23 27 5 4.3 231.0 0.9X -2500 wide x 40 rows (exec in-mem) 192 201 7 0.5 1920.9 0.1X -2500 wide x 40 rows (read parquet) 68 73 6 1.5 681.4 0.3X -2500 wide x 40 rows (write parquet) 102 107 7 1.0 1019.1 0.2X +1 wide x 100000 rows (read in-mem) 20 25 6 4.9 203.7 1.0X +1 wide x 100000 rows (exec in-mem) 21 26 7 4.7 211.7 1.0X +1 wide x 100000 rows (read parquet) 22 30 8 4.5 221.6 0.9X +1 wide x 100000 rows (write parquet) 94 103 9 1.1 939.9 0.2X +100 wide x 1000 rows (read in-mem) 15 18 5 6.8 147.8 1.4X +100 wide x 1000 rows (exec in-mem) 16 19 6 6.3 159.1 1.3X +100 wide x 1000 rows (read parquet) 22 27 7 4.6 216.4 0.9X +100 wide x 1000 rows (write parquet) 88 97 10 1.1 880.7 0.2X +2500 wide x 40 rows (read in-mem) 20 26 7 4.9 204.7 1.0X +2500 wide x 40 rows (exec in-mem) 22 27 7 4.6 216.8 0.9X +2500 wide x 40 rows (read parquet) 66 68 5 1.5 658.4 0.3X +2500 wide x 40 rows (write parquet) 94 103 10 1.1 941.6 0.2X ================================================================================================ deeply nested struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 deep x 100000 rows (read in-mem) 16 20 6 6.3 158.7 1.0X -1 deep x 100000 rows (exec in-mem) 18 20 4 5.6 177.3 0.9X -1 deep x 100000 rows (read parquet) 19 23 6 5.4 185.3 0.9X -1 deep x 100000 rows (write parquet) 95 104 8 1.1 951.5 0.2X -100 deep x 1000 rows (read in-mem) 44 47 3 2.3 444.0 0.4X -100 deep x 1000 rows (exec in-mem) 452 466 13 0.2 4520.5 0.0X -100 deep x 1000 rows (read parquet) 433 442 13 0.2 4329.1 0.0X -100 deep x 1000 rows (write parquet) 122 129 6 0.8 1224.3 0.1X -250 deep x 400 rows (read in-mem) 191 195 3 0.5 1909.7 0.1X -250 deep x 400 rows (exec in-mem) 2893 2909 23 0.0 28927.4 0.0X -250 deep x 400 rows (read parquet) 2595 2598 4 0.0 25951.8 0.0X -250 deep x 400 rows (write parquet) 268 273 4 0.4 2675.1 0.1X +1 deep x 100000 rows (read in-mem) 15 19 6 6.5 154.5 1.0X +1 deep x 100000 rows (exec in-mem) 18 21 6 5.7 175.5 0.9X +1 deep x 100000 rows (read parquet) 18 23 7 5.6 178.4 0.9X +1 deep x 100000 rows (write parquet) 89 96 8 1.1 887.6 0.2X +100 deep x 1000 rows (read in-mem) 43 46 5 2.3 432.1 0.4X +100 deep x 1000 rows (exec in-mem) 445 453 8 0.2 4448.2 0.0X +100 deep x 1000 rows (read parquet) 445 453 6 0.2 4449.9 0.0X +100 deep x 1000 rows (write parquet) 117 129 9 0.9 1168.8 0.1X +250 deep x 400 rows (read in-mem) 192 195 3 0.5 1915.7 0.1X +250 deep x 400 rows (exec in-mem) 2694 2700 9 0.0 26937.2 0.0X +250 deep x 400 rows (read parquet) 2683 2688 6 0.0 26833.1 0.0X +250 deep x 400 rows (write parquet) 267 273 4 0.4 2667.1 0.1X ================================================================================================ bushy struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor bushy struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -1 x 1 deep x 100000 rows (read in-mem) 13 16 4 7.4 134.4 1.0X -1 x 1 deep x 100000 rows (exec in-mem) 15 19 5 6.6 150.9 0.9X -1 x 1 deep x 100000 rows (read parquet) 18 22 5 5.6 177.6 0.8X -1 x 1 deep x 100000 rows (write parquet) 92 98 7 1.1 921.7 0.1X -128 x 8 deep x 1000 rows (read in-mem) 13 16 4 7.4 134.3 1.0X -128 x 8 deep x 1000 rows (exec in-mem) 25 28 5 4.0 248.9 0.5X -128 x 8 deep x 1000 rows (read parquet) 21 26 6 4.7 213.3 0.6X -128 x 8 deep x 1000 rows (write parquet) 91 98 8 1.1 911.6 0.1X -1024 x 11 deep x 100 rows (read in-mem) 19 22 4 5.3 187.6 0.7X -1024 x 11 deep x 100 rows (exec in-mem) 129 133 6 0.8 1286.3 0.1X -1024 x 11 deep x 100 rows (read parquet) 36 40 4 2.8 363.4 0.4X -1024 x 11 deep x 100 rows (write parquet) 96 102 10 1.0 962.5 0.1X +1 x 1 deep x 100000 rows (read in-mem) 13 15 4 7.7 130.2 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 15 18 4 6.6 150.8 0.9X +1 x 1 deep x 100000 rows (read parquet) 18 21 5 5.6 177.4 0.7X +1 x 1 deep x 100000 rows (write parquet) 88 94 6 1.1 875.7 0.1X +128 x 8 deep x 1000 rows (read in-mem) 13 16 5 7.6 131.7 1.0X +128 x 8 deep x 1000 rows (exec in-mem) 15 18 5 6.7 148.7 0.9X +128 x 8 deep x 1000 rows (read parquet) 22 26 6 4.6 215.6 0.6X +128 x 8 deep x 1000 rows (write parquet) 86 93 6 1.2 864.9 0.2X +1024 x 11 deep x 100 rows (read in-mem) 18 21 6 5.7 176.2 0.7X +1024 x 11 deep x 100 rows (exec in-mem) 19 23 6 5.4 185.8 0.7X +1024 x 11 deep x 100 rows (read parquet) 35 37 4 2.9 349.8 0.4X +1024 x 11 deep x 100 rows (write parquet) 91 94 5 1.1 912.5 0.1X ================================================================================================ wide array field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor wide array field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 15 18 4 6.6 151.1 1.0X -1 wide x 100000 rows (exec in-mem) 17 19 4 5.8 171.5 0.9X -1 wide x 100000 rows (read parquet) 17 21 5 5.8 172.7 0.9X -1 wide x 100000 rows (write parquet) 93 103 8 1.1 931.5 0.2X -100 wide x 1000 rows (read in-mem) 11 13 4 8.9 112.1 1.3X -100 wide x 1000 rows (exec in-mem) 13 15 4 7.8 128.9 1.2X -100 wide x 1000 rows (read parquet) 17 21 5 5.9 170.7 0.9X -100 wide x 1000 rows (write parquet) 90 98 9 1.1 900.2 0.2X -2500 wide x 40 rows (read in-mem) 11 13 3 9.0 111.5 1.4X -2500 wide x 40 rows (exec in-mem) 13 16 4 7.7 129.6 1.2X -2500 wide x 40 rows (read parquet) 17 19 4 5.9 168.5 0.9X -2500 wide x 40 rows (write parquet) 91 98 7 1.1 906.0 0.2X +1 wide x 100000 rows (read in-mem) 15 17 5 6.7 148.8 1.0X +1 wide x 100000 rows (exec in-mem) 17 19 4 5.9 170.7 0.9X +1 wide x 100000 rows (read parquet) 17 21 6 5.8 172.8 0.9X +1 wide x 100000 rows (write parquet) 88 93 9 1.1 878.8 0.2X +100 wide x 1000 rows (read in-mem) 11 14 5 9.1 110.1 1.4X +100 wide x 1000 rows (exec in-mem) 13 16 5 7.9 127.1 1.2X +100 wide x 1000 rows (read parquet) 17 21 6 5.9 168.2 0.9X +100 wide x 1000 rows (write parquet) 83 90 8 1.2 832.1 0.2X +2500 wide x 40 rows (read in-mem) 11 15 6 9.1 109.8 1.4X +2500 wide x 40 rows (exec in-mem) 12 17 6 8.0 125.0 1.2X +2500 wide x 40 rows (read parquet) 17 23 7 6.0 167.5 0.9X +2500 wide x 40 rows (write parquet) 84 94 8 1.2 841.2 0.2X ================================================================================================ wide map field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor wide map field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 12 14 3 8.3 121.0 1.0X -1 wide x 100000 rows (exec in-mem) 16 18 3 6.3 159.0 0.8X -1 wide x 100000 rows (read parquet) 21 24 5 4.7 213.2 0.6X -1 wide x 100000 rows (write parquet) 91 96 6 1.1 905.2 0.1X -100 wide x 1000 rows (read in-mem) 8 9 3 13.3 75.2 1.6X -100 wide x 1000 rows (exec in-mem) 10 12 3 10.4 96.1 1.3X -100 wide x 1000 rows (read parquet) 19 21 4 5.3 187.5 0.6X -100 wide x 1000 rows (write parquet) 86 90 5 1.2 858.4 0.1X -2500 wide x 40 rows (read in-mem) 9 11 2 10.8 92.8 1.3X -2500 wide x 40 rows (exec in-mem) 11 13 3 9.0 111.5 1.1X -2500 wide x 40 rows (read parquet) 19 22 4 5.2 191.8 0.6X -2500 wide x 40 rows (write parquet) 90 94 5 1.1 899.2 0.1X +1 wide x 100000 rows (read in-mem) 12 15 4 8.3 121.2 1.0X +1 wide x 100000 rows (exec in-mem) 16 21 5 6.3 159.3 0.8X +1 wide x 100000 rows (read parquet) 21 24 5 4.8 208.1 0.6X +1 wide x 100000 rows (write parquet) 84 89 5 1.2 842.0 0.1X +100 wide x 1000 rows (read in-mem) 8 9 2 13.1 76.5 1.6X +100 wide x 1000 rows (exec in-mem) 10 11 3 10.5 95.3 1.3X +100 wide x 1000 rows (read parquet) 18 20 6 5.6 178.2 0.7X +100 wide x 1000 rows (write parquet) 80 85 6 1.2 801.0 0.2X +2500 wide x 40 rows (read in-mem) 9 10 3 10.7 93.4 1.3X +2500 wide x 40 rows (exec in-mem) 11 12 2 8.9 111.7 1.1X +2500 wide x 40 rows (read parquet) 18 21 6 5.5 183.2 0.7X +2500 wide x 40 rows (write parquet) 82 89 8 1.2 822.1 0.1X diff --git a/sql/core/benchmarks/WideSchemaBenchmark-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-results.txt index e61b27a7c727f..4931872223cfe 100644 --- a/sql/core/benchmarks/WideSchemaBenchmark-results.txt +++ b/sql/core/benchmarks/WideSchemaBenchmark-results.txt @@ -2,157 +2,157 @@ parsing large select expressions ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor parsing large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 select expressions 1 2 1 0.0 1177503.0 1.0X -100 select expressions 2 3 1 0.0 2179549.0 0.5X -2500 select expressions 40 43 4 0.0 39575214.0 0.0X +1 select expressions 1 1 0 0.0 671442.0 1.0X +100 select expressions 3 3 0 0.0 3181250.0 0.2X +2500 select expressions 69 72 2 0.0 69457245.0 0.0X ================================================================================================ optimize large select expressions ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor optimize large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -100 columns 5 5 1 0.0 4710103.0 1.0X -1000 columns 31 32 1 0.0 30879997.0 0.2X -10000 columns 309 327 10 0.0 309351929.0 0.0X +100 columns 7 8 1 0.0 6678458.0 1.0X +1000 columns 53 55 2 0.0 53079798.0 0.1X +10000 columns 530 538 7 0.0 529951923.0 0.0X ================================================================================================ many column field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor many column field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 cols x 100000 rows (read in-mem) 16 22 3 6.1 163.3 1.0X -1 cols x 100000 rows (exec in-mem) 16 22 3 6.1 162.8 1.0X -1 cols x 100000 rows (read parquet) 29 36 6 3.4 290.4 0.6X -1 cols x 100000 rows (write parquet) 96 105 8 1.0 961.4 0.2X -100 cols x 1000 rows (read in-mem) 13 16 3 7.8 128.8 1.3X -100 cols x 1000 rows (exec in-mem) 16 19 3 6.3 159.4 1.0X -100 cols x 1000 rows (read parquet) 24 28 4 4.2 240.6 0.7X -100 cols x 1000 rows (write parquet) 93 98 5 1.1 931.7 0.2X -2500 cols x 40 rows (read in-mem) 57 61 4 1.8 566.2 0.3X -2500 cols x 40 rows (exec in-mem) 105 108 5 0.9 1054.4 0.2X -2500 cols x 40 rows (read parquet) 285 287 2 0.4 2852.5 0.1X -2500 cols x 40 rows (write parquet) 136 143 7 0.7 1358.0 0.1X +1 cols x 100000 rows (read in-mem) 18 23 4 5.6 177.9 1.0X +1 cols x 100000 rows (exec in-mem) 17 23 4 5.8 171.6 1.0X +1 cols x 100000 rows (read parquet) 30 36 6 3.3 302.9 0.6X +1 cols x 100000 rows (write parquet) 91 101 11 1.1 909.2 0.2X +100 cols x 1000 rows (read in-mem) 12 16 3 8.0 124.3 1.4X +100 cols x 1000 rows (exec in-mem) 15 19 3 6.5 154.0 1.2X +100 cols x 1000 rows (read parquet) 25 29 4 4.0 248.6 0.7X +100 cols x 1000 rows (write parquet) 87 96 8 1.1 871.9 0.2X +2500 cols x 40 rows (read in-mem) 46 50 4 2.2 461.8 0.4X +2500 cols x 40 rows (exec in-mem) 77 81 4 1.3 766.0 0.2X +2500 cols x 40 rows (read parquet) 285 290 3 0.4 2849.8 0.1X +2500 cols x 40 rows (write parquet) 127 134 6 0.8 1265.8 0.1X ================================================================================================ wide shallowly nested struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor wide shallowly nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 21 25 3 4.8 208.5 1.0X -1 wide x 100000 rows (exec in-mem) 23 26 4 4.4 228.5 0.9X -1 wide x 100000 rows (read parquet) 23 28 4 4.3 231.2 0.9X -1 wide x 100000 rows (write parquet) 100 109 6 1.0 1002.6 0.2X -100 wide x 1000 rows (read in-mem) 15 18 4 6.7 148.9 1.4X -100 wide x 1000 rows (exec in-mem) 21 25 4 4.7 214.8 1.0X -100 wide x 1000 rows (read parquet) 22 26 4 4.6 218.0 1.0X -100 wide x 1000 rows (write parquet) 98 102 5 1.0 975.5 0.2X -2500 wide x 40 rows (read in-mem) 23 27 3 4.4 227.3 0.9X -2500 wide x 40 rows (exec in-mem) 195 199 4 0.5 1951.3 0.1X -2500 wide x 40 rows (read parquet) 71 75 5 1.4 707.3 0.3X -2500 wide x 40 rows (write parquet) 107 110 4 0.9 1065.6 0.2X +1 wide x 100000 rows (read in-mem) 24 30 6 4.3 235.0 1.0X +1 wide x 100000 rows (exec in-mem) 25 29 4 4.0 252.3 0.9X +1 wide x 100000 rows (read parquet) 25 29 4 3.9 254.4 0.9X +1 wide x 100000 rows (write parquet) 101 110 7 1.0 1010.2 0.2X +100 wide x 1000 rows (read in-mem) 20 23 4 5.1 195.0 1.2X +100 wide x 1000 rows (exec in-mem) 19 22 3 5.2 192.1 1.2X +100 wide x 1000 rows (read parquet) 25 28 4 4.0 249.4 0.9X +100 wide x 1000 rows (write parquet) 96 102 7 1.0 957.9 0.2X +2500 wide x 40 rows (read in-mem) 24 27 4 4.2 240.1 1.0X +2500 wide x 40 rows (exec in-mem) 25 29 5 3.9 253.4 0.9X +2500 wide x 40 rows (read parquet) 73 77 4 1.4 727.0 0.3X +2500 wide x 40 rows (write parquet) 106 111 4 0.9 1055.1 0.2X ================================================================================================ deeply nested struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 deep x 100000 rows (read in-mem) 17 20 3 5.8 171.8 1.0X -1 deep x 100000 rows (exec in-mem) 18 21 3 5.5 183.1 0.9X -1 deep x 100000 rows (read parquet) 19 22 4 5.4 186.1 0.9X -1 deep x 100000 rows (write parquet) 96 102 6 1.0 962.2 0.2X -100 deep x 1000 rows (read in-mem) 31 33 3 3.2 314.7 0.5X -100 deep x 1000 rows (exec in-mem) 462 469 4 0.2 4622.7 0.0X -100 deep x 1000 rows (read parquet) 458 465 8 0.2 4576.2 0.0X -100 deep x 1000 rows (write parquet) 110 116 4 0.9 1100.9 0.2X -250 deep x 400 rows (read in-mem) 123 127 4 0.8 1230.2 0.1X -250 deep x 400 rows (exec in-mem) 2940 2943 4 0.0 29395.9 0.0X -250 deep x 400 rows (read parquet) 2723 2741 25 0.0 27229.1 0.0X -250 deep x 400 rows (write parquet) 206 219 11 0.5 2055.2 0.1X +1 deep x 100000 rows (read in-mem) 17 20 4 5.8 172.3 1.0X +1 deep x 100000 rows (exec in-mem) 20 23 4 5.1 195.7 0.9X +1 deep x 100000 rows (read parquet) 21 24 4 4.7 211.9 0.8X +1 deep x 100000 rows (write parquet) 93 100 7 1.1 931.1 0.2X +100 deep x 1000 rows (read in-mem) 39 41 3 2.6 389.5 0.4X +100 deep x 1000 rows (exec in-mem) 430 434 6 0.2 4300.2 0.0X +100 deep x 1000 rows (read parquet) 439 450 6 0.2 4388.3 0.0X +100 deep x 1000 rows (write parquet) 114 118 4 0.9 1141.3 0.2X +250 deep x 400 rows (read in-mem) 155 160 5 0.6 1552.9 0.1X +250 deep x 400 rows (exec in-mem) 2583 2589 9 0.0 25828.0 0.0X +250 deep x 400 rows (read parquet) 2598 2615 24 0.0 25976.7 0.0X +250 deep x 400 rows (write parquet) 233 249 13 0.4 2334.4 0.1X ================================================================================================ bushy struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor bushy struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -1 x 1 deep x 100000 rows (read in-mem) 13 15 3 7.6 132.0 1.0X -1 x 1 deep x 100000 rows (exec in-mem) 15 18 4 6.8 147.3 0.9X -1 x 1 deep x 100000 rows (read parquet) 17 19 3 5.8 172.4 0.8X -1 x 1 deep x 100000 rows (write parquet) 97 100 4 1.0 965.0 0.1X -128 x 8 deep x 1000 rows (read in-mem) 15 17 3 6.9 145.3 0.9X -128 x 8 deep x 1000 rows (exec in-mem) 26 28 3 3.9 257.4 0.5X -128 x 8 deep x 1000 rows (read parquet) 22 24 3 4.5 221.1 0.6X -128 x 8 deep x 1000 rows (write parquet) 92 95 5 1.1 916.0 0.1X -1024 x 11 deep x 100 rows (read in-mem) 19 22 3 5.3 188.5 0.7X -1024 x 11 deep x 100 rows (exec in-mem) 126 128 2 0.8 1257.4 0.1X -1024 x 11 deep x 100 rows (read parquet) 37 39 3 2.7 368.9 0.4X -1024 x 11 deep x 100 rows (write parquet) 97 102 5 1.0 971.1 0.1X +1 x 1 deep x 100000 rows (read in-mem) 15 18 3 6.5 153.9 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 17 19 3 5.9 168.3 0.9X +1 x 1 deep x 100000 rows (read parquet) 20 23 4 5.0 200.6 0.8X +1 x 1 deep x 100000 rows (write parquet) 92 96 4 1.1 919.2 0.2X +128 x 8 deep x 1000 rows (read in-mem) 16 19 4 6.1 164.8 0.9X +128 x 8 deep x 1000 rows (exec in-mem) 16 19 4 6.2 161.9 1.0X +128 x 8 deep x 1000 rows (read parquet) 22 26 4 4.5 223.7 0.7X +128 x 8 deep x 1000 rows (write parquet) 90 95 7 1.1 900.9 0.2X +1024 x 11 deep x 100 rows (read in-mem) 19 21 3 5.4 186.8 0.8X +1024 x 11 deep x 100 rows (exec in-mem) 21 23 3 4.8 206.9 0.7X +1024 x 11 deep x 100 rows (read parquet) 37 40 4 2.7 373.4 0.4X +1024 x 11 deep x 100 rows (write parquet) 96 105 11 1.0 965.0 0.2X ================================================================================================ wide array field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor wide array field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 15 17 3 6.8 147.6 1.0X -1 wide x 100000 rows (exec in-mem) 17 19 3 6.0 167.8 0.9X -1 wide x 100000 rows (read parquet) 17 20 3 5.9 170.6 0.9X -1 wide x 100000 rows (write parquet) 93 96 3 1.1 926.3 0.2X -100 wide x 1000 rows (read in-mem) 11 12 3 9.1 109.4 1.3X -100 wide x 1000 rows (exec in-mem) 12 14 3 8.0 125.0 1.2X -100 wide x 1000 rows (read parquet) 17 19 3 6.0 165.8 0.9X -100 wide x 1000 rows (write parquet) 89 94 4 1.1 885.3 0.2X -2500 wide x 40 rows (read in-mem) 11 12 3 9.4 106.5 1.4X -2500 wide x 40 rows (exec in-mem) 12 14 3 8.2 121.9 1.2X -2500 wide x 40 rows (read parquet) 16 18 3 6.2 162.4 0.9X -2500 wide x 40 rows (write parquet) 89 94 5 1.1 885.5 0.2X +1 wide x 100000 rows (read in-mem) 17 20 4 5.7 174.7 1.0X +1 wide x 100000 rows (exec in-mem) 19 23 4 5.1 194.6 0.9X +1 wide x 100000 rows (read parquet) 20 26 10 5.1 196.4 0.9X +1 wide x 100000 rows (write parquet) 92 98 5 1.1 920.3 0.2X +100 wide x 1000 rows (read in-mem) 12 15 4 8.1 123.2 1.4X +100 wide x 1000 rows (exec in-mem) 15 19 4 6.9 145.0 1.2X +100 wide x 1000 rows (read parquet) 19 23 4 5.3 190.3 0.9X +100 wide x 1000 rows (write parquet) 89 95 4 1.1 894.3 0.2X +2500 wide x 40 rows (read in-mem) 13 16 4 7.8 128.9 1.4X +2500 wide x 40 rows (exec in-mem) 15 17 3 6.7 149.5 1.2X +2500 wide x 40 rows (read parquet) 19 21 4 5.4 185.9 0.9X +2500 wide x 40 rows (write parquet) 88 93 7 1.1 877.3 0.2X ================================================================================================ wide map field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor wide map field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 12 13 2 8.3 120.7 1.0X -1 wide x 100000 rows (exec in-mem) 15 16 2 6.8 147.6 0.8X -1 wide x 100000 rows (read parquet) 20 26 4 5.0 201.5 0.6X -1 wide x 100000 rows (write parquet) 89 93 4 1.1 894.7 0.1X -100 wide x 1000 rows (read in-mem) 7 8 1 13.5 74.1 1.6X -100 wide x 1000 rows (exec in-mem) 9 10 2 10.9 91.6 1.3X -100 wide x 1000 rows (read parquet) 18 20 3 5.6 177.1 0.7X -100 wide x 1000 rows (write parquet) 84 87 4 1.2 843.3 0.1X -2500 wide x 40 rows (read in-mem) 9 10 1 11.0 91.3 1.3X -2500 wide x 40 rows (exec in-mem) 11 12 2 9.2 108.4 1.1X -2500 wide x 40 rows (read parquet) 18 20 3 5.6 180.1 0.7X -2500 wide x 40 rows (write parquet) 88 92 4 1.1 881.3 0.1X +1 wide x 100000 rows (read in-mem) 14 15 2 7.3 136.4 1.0X +1 wide x 100000 rows (exec in-mem) 17 18 2 6.1 165.2 0.8X +1 wide x 100000 rows (read parquet) 22 25 5 4.6 217.3 0.6X +1 wide x 100000 rows (write parquet) 87 91 6 1.2 866.7 0.2X +100 wide x 1000 rows (read in-mem) 8 10 3 12.5 80.3 1.7X +100 wide x 1000 rows (exec in-mem) 10 12 2 9.7 103.2 1.3X +100 wide x 1000 rows (read parquet) 21 24 4 4.9 205.3 0.7X +100 wide x 1000 rows (write parquet) 82 87 6 1.2 821.1 0.2X +2500 wide x 40 rows (read in-mem) 10 12 3 9.7 103.1 1.3X +2500 wide x 40 rows (exec in-mem) 12 14 3 8.2 121.4 1.1X +2500 wide x 40 rows (read parquet) 20 22 4 5.0 199.0 0.7X +2500 wide x 40 rows (write parquet) 84 89 7 1.2 842.7 0.2X diff --git a/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt b/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt index 04f1737afb586..62aea5f496f92 100644 --- a/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt @@ -2,16 +2,16 @@ projection on wide table ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor projection on wide table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -split threshold 10 2606 2701 94 0.4 2485.4 1.0X -split threshold 100 2174 2193 24 0.5 2073.8 1.2X -split threshold 1024 1652 1662 9 0.6 1575.6 1.6X -split threshold 2048 1618 1625 6 0.6 1543.3 1.6X -split threshold 4096 1713 1734 12 0.6 1633.6 1.5X -split threshold 8192 2321 2336 25 0.5 2213.4 1.1X -split threshold 65536 20726 20950 265 0.1 19765.7 0.1X +split threshold 10 2606 2655 71 0.4 2485.4 1.0X +split threshold 100 2142 2160 14 0.5 2043.1 1.2X +split threshold 1024 1632 1711 124 0.6 1556.0 1.6X +split threshold 2048 1608 1623 14 0.7 1533.1 1.6X +split threshold 4096 1725 1741 21 0.6 1644.7 1.5X +split threshold 8192 2456 2464 7 0.4 2342.5 1.1X +split threshold 65536 21150 21518 353 0.0 20170.3 0.1X diff --git a/sql/core/benchmarks/WideTableBenchmark-results.txt b/sql/core/benchmarks/WideTableBenchmark-results.txt index 1dda0fdd03fb9..e3f5c9bebeee6 100644 --- a/sql/core/benchmarks/WideTableBenchmark-results.txt +++ b/sql/core/benchmarks/WideTableBenchmark-results.txt @@ -2,16 +2,16 @@ projection on wide table ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor projection on wide table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -split threshold 10 2543 2625 73 0.4 2425.1 1.0X -split threshold 100 2035 2074 30 0.5 1940.5 1.2X -split threshold 1024 1641 1658 12 0.6 1565.4 1.5X -split threshold 2048 1609 1625 12 0.7 1534.9 1.6X -split threshold 4096 1668 1681 15 0.6 1590.4 1.5X -split threshold 8192 2119 2153 50 0.5 2021.2 1.2X -split threshold 65536 21512 21816 366 0.0 20515.1 0.1X +split threshold 10 2549 2628 72 0.4 2431.4 1.0X +split threshold 100 2035 2068 32 0.5 1940.7 1.3X +split threshold 1024 1674 1703 27 0.6 1596.6 1.5X +split threshold 2048 1612 1618 6 0.7 1537.6 1.6X +split threshold 4096 1663 1686 17 0.6 1585.9 1.5X +split threshold 8192 2151 2162 13 0.5 2051.2 1.2X +split threshold 65536 21995 22268 233 0.0 20976.1 0.1X diff --git a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive2.3-results.txt b/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive2.3-results.txt index 6e8c140c72dcc..4c44860c4618a 100644 --- a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive2.3-results.txt +++ b/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive2.3-results.txt @@ -1,11 +1,11 @@ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor insert hive table benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -INSERT INTO DYNAMIC 3695 3849 218 0.0 360836.1 1.0X -INSERT INTO HYBRID 536 551 17 0.0 52374.2 6.9X -INSERT INTO STATIC 151 177 15 0.1 14737.4 24.5X -INSERT OVERWRITE DYNAMIC 3057 3228 241 0.0 298536.0 1.2X -INSERT OVERWRITE HYBRID 455 467 15 0.0 44443.5 8.1X -INSERT OVERWRITE STATIC 173 180 4 0.1 16911.3 21.3X +INSERT INTO DYNAMIC 3480 3775 417 0.0 339817.0 1.0X +INSERT INTO HYBRID 562 581 13 0.0 54901.2 6.2X +INSERT INTO STATIC 157 174 14 0.1 15316.1 22.2X +INSERT OVERWRITE DYNAMIC 2961 3195 331 0.0 289121.3 1.2X +INSERT OVERWRITE HYBRID 426 431 6 0.0 41557.2 8.2X +INSERT OVERWRITE STATIC 161 168 5 0.1 15682.4 21.7X diff --git a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-jdk21-hive2.3-results.txt b/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-jdk21-hive2.3-results.txt index 7a901f75ddb35..38e3b10eb5d00 100644 --- a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-jdk21-hive2.3-results.txt +++ b/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-jdk21-hive2.3-results.txt @@ -1,11 +1,11 @@ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor insert hive table benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -INSERT INTO DYNAMIC 3762 3968 292 0.0 367406.9 1.0X -INSERT INTO HYBRID 516 591 80 0.0 50355.2 7.3X -INSERT INTO STATIC 168 192 24 0.1 16403.7 22.4X -INSERT OVERWRITE DYNAMIC 3524 3643 169 0.0 344143.1 1.1X -INSERT OVERWRITE HYBRID 493 510 13 0.0 48137.8 7.6X -INSERT OVERWRITE STATIC 178 190 14 0.1 17346.8 21.2X +INSERT INTO DYNAMIC 3406 3754 493 0.0 332568.8 1.0X +INSERT INTO HYBRID 496 523 22 0.0 48481.9 6.9X +INSERT INTO STATIC 146 168 20 0.1 14228.9 23.4X +INSERT OVERWRITE DYNAMIC 3031 3148 166 0.0 295998.0 1.1X +INSERT OVERWRITE HYBRID 419 444 26 0.0 40901.7 8.1X +INSERT OVERWRITE STATIC 158 169 12 0.1 15420.2 21.6X diff --git a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk21-results.txt b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk21-results.txt index f185c50f929bf..9f7cd0bfd8762 100644 --- a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk21-results.txt +++ b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk21-results.txt @@ -2,44 +2,44 @@ Hive UDAF vs Spark AF ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor hive udaf vs spark af: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hive udaf w/o group by 3232 3292 46 0.0 49313.1 1.0X -spark af w/o group by 20 26 4 3.3 303.1 162.7X -hive udaf w/ group by 2002 2055 40 0.0 30540.8 1.6X -spark af w/ group by w/o fallback 22 25 3 3.0 334.3 147.5X -spark af w/ group by w/ fallback 25 27 3 2.7 376.5 131.0X +hive udaf w/o group by 3071 3289 131 0.0 46855.4 1.0X +spark af w/o group by 21 27 5 3.2 315.4 148.6X +hive udaf w/ group by 2138 2161 26 0.0 32618.6 1.4X +spark af w/ group by w/o fallback 22 26 5 3.0 338.3 138.5X +spark af w/ group by w/ fallback 26 30 7 2.5 395.2 118.6X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - typed_count ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort agg w/ group by 23962 24276 445 4.4 228.5 1.0X -object agg w/ group by w/o fallback 7346 7389 41 14.3 70.1 3.3X -object agg w/ group by w/ fallback 15904 16415 443 6.6 151.7 1.5X -sort agg w/o group by 4041 4060 17 26.0 38.5 5.9X -object agg w/o group by w/o fallback 3872 3914 42 27.1 36.9 6.2X +sort agg w/ group by 23012 23051 55 4.6 219.5 1.0X +object agg w/ group by w/o fallback 6670 7292 278 15.7 63.6 3.5X +object agg w/ group by w/ fallback 15467 15512 48 6.8 147.5 1.5X +sort agg w/o group by 4075 4142 34 25.7 38.9 5.6X +object agg w/o group by w/o fallback 3715 3810 67 28.2 35.4 6.2X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - percentile_approx ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort agg w/ group by 417 449 16 5.0 198.9 1.0X -object agg w/ group by w/o fallback 328 339 5 6.4 156.5 1.3X -object agg w/ group by w/ fallback 467 501 15 4.5 222.4 0.9X -sort agg w/o group by 274 283 6 7.6 130.8 1.5X -object agg w/o group by w/o fallback 271 277 3 7.7 129.3 1.5X +sort agg w/ group by 412 436 14 5.1 196.4 1.0X +object agg w/ group by w/o fallback 324 333 6 6.5 154.7 1.3X +object agg w/ group by w/ fallback 414 421 7 5.1 197.3 1.0X +sort agg w/o group by 238 242 3 8.8 113.5 1.7X +object agg w/o group by w/o fallback 227 234 6 9.2 108.4 1.8X diff --git a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt index fb426c84414ba..1e143f39fbf91 100644 --- a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt +++ b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt @@ -2,44 +2,44 @@ Hive UDAF vs Spark AF ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor hive udaf vs spark af: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hive udaf w/o group by 3271 3305 29 0.0 49904.4 1.0X -spark af w/o group by 21 26 4 3.2 316.4 157.7X -hive udaf w/ group by 2070 2109 30 0.0 31591.0 1.6X -spark af w/ group by w/o fallback 22 26 3 3.0 335.8 148.6X -spark af w/ group by w/ fallback 25 27 3 2.6 379.4 131.5X +hive udaf w/o group by 3797 3861 45 0.0 57941.7 1.0X +spark af w/o group by 21 27 5 3.2 314.9 184.0X +hive udaf w/ group by 2569 2575 11 0.0 39194.8 1.5X +spark af w/ group by w/o fallback 22 26 3 3.0 333.7 173.6X +spark af w/ group by w/ fallback 25 28 3 2.6 388.1 149.3X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - typed_count ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort agg w/ group by 24310 24337 39 4.3 231.8 1.0X -object agg w/ group by w/o fallback 6916 7223 137 15.2 66.0 3.5X -object agg w/ group by w/ fallback 14558 14693 128 7.2 138.8 1.7X -sort agg w/o group by 4079 4125 48 25.7 38.9 6.0X -object agg w/o group by w/o fallback 3577 3608 22 29.3 34.1 6.8X +sort agg w/ group by 24523 24678 220 4.3 233.9 1.0X +object agg w/ group by w/o fallback 6979 7355 177 15.0 66.6 3.5X +object agg w/ group by w/ fallback 14572 14619 33 7.2 139.0 1.7X +sort agg w/o group by 4265 4283 19 24.6 40.7 5.7X +object agg w/o group by w/o fallback 3614 3660 29 29.0 34.5 6.8X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - percentile_approx ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort agg w/ group by 403 412 6 5.2 192.3 1.0X -object agg w/ group by w/o fallback 341 347 5 6.1 162.7 1.2X -object agg w/ group by w/ fallback 469 473 4 4.5 223.6 0.9X -sort agg w/o group by 304 310 4 6.9 144.9 1.3X -object agg w/o group by w/o fallback 297 305 3 7.1 141.4 1.4X +sort agg w/ group by 413 422 7 5.1 196.8 1.0X +object agg w/ group by w/o fallback 335 343 4 6.3 159.6 1.2X +object agg w/ group by w/ fallback 446 453 5 4.7 212.7 0.9X +sort agg w/o group by 274 280 4 7.7 130.6 1.5X +object agg w/o group by w/o fallback 266 273 4 7.9 126.9 1.6X diff --git a/sql/hive/benchmarks/OrcReadBenchmark-jdk21-results.txt b/sql/hive/benchmarks/OrcReadBenchmark-jdk21-results.txt index b941571563401..25ba0a0602b47 100644 --- a/sql/hive/benchmarks/OrcReadBenchmark-jdk21-results.txt +++ b/sql/hive/benchmarks/OrcReadBenchmark-jdk21-results.txt @@ -2,221 +2,221 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 675 696 17 23.3 42.9 1.0X -Native ORC MR 745 759 24 21.1 47.3 0.9X -Native ORC Vectorized 91 118 9 172.4 5.8 7.4X +Hive built-in ORC 711 756 43 22.1 45.2 1.0X +Native ORC MR 762 842 92 20.7 48.4 0.9X +Native ORC Vectorized 94 115 17 167.8 6.0 7.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 680 728 47 23.1 43.3 1.0X -Native ORC MR 726 755 25 21.7 46.1 0.9X -Native ORC Vectorized 83 99 11 190.0 5.3 8.2X +Hive built-in ORC 693 722 32 22.7 44.0 1.0X +Native ORC MR 738 767 35 21.3 46.9 0.9X +Native ORC Vectorized 81 100 15 193.2 5.2 8.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 696 716 28 22.6 44.3 1.0X -Native ORC MR 741 766 32 21.2 47.1 0.9X -Native ORC Vectorized 86 98 12 181.9 5.5 8.0X +Hive built-in ORC 776 792 27 20.3 49.3 1.0X +Native ORC MR 895 907 18 17.6 56.9 0.9X +Native ORC Vectorized 102 120 14 154.7 6.5 7.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 720 729 14 21.9 45.8 1.0X -Native ORC MR 766 783 16 20.5 48.7 0.9X -Native ORC Vectorized 92 108 11 171.7 5.8 7.9X +Hive built-in ORC 831 857 34 18.9 52.9 1.0X +Native ORC MR 938 996 55 16.8 59.6 0.9X +Native ORC Vectorized 100 116 22 157.1 6.4 8.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 754 792 65 20.9 47.9 1.0X -Native ORC MR 861 879 27 18.3 54.7 0.9X -Native ORC Vectorized 147 164 13 107.3 9.3 5.1X +Hive built-in ORC 768 806 36 20.5 48.8 1.0X +Native ORC MR 950 972 25 16.6 60.4 0.8X +Native ORC Vectorized 139 160 34 113.4 8.8 5.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 826 833 6 19.0 52.5 1.0X -Native ORC MR 947 975 43 16.6 60.2 0.9X -Native ORC Vectorized 218 234 24 72.0 13.9 3.8X +Hive built-in ORC 914 959 38 17.2 58.1 1.0X +Native ORC MR 994 1007 18 15.8 63.2 0.9X +Native ORC Vectorized 223 240 31 70.6 14.2 4.1X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1632 1653 30 6.4 155.6 1.0X -Native ORC MR 1523 1528 8 6.9 145.2 1.1X -Native ORC Vectorized 610 643 24 17.2 58.2 2.7X +Hive built-in ORC 1770 1819 69 5.9 168.8 1.0X +Native ORC MR 1606 1611 6 6.5 153.2 1.1X +Native ORC Vectorized 606 646 44 17.3 57.8 2.9X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column - Hive built-in ORC 937 953 14 16.8 59.6 1.0X -Data column - Native ORC MR 988 1040 73 15.9 62.8 0.9X -Data column - Native ORC Vectorized 89 107 13 177.2 5.6 10.6X -Partition column - Hive built-in ORC 640 690 55 24.6 40.7 1.5X -Partition column - Native ORC MR 695 708 16 22.6 44.2 1.3X -Partition column - Native ORC Vectorized 38 49 9 416.8 2.4 24.8X -Both columns - Hive built-in ORC 978 1015 42 16.1 62.2 1.0X -Both columns - Native ORC MR 1055 1076 29 14.9 67.1 0.9X -Both columns - Native ORC Vectorized 102 125 24 153.8 6.5 9.2X +Data column - Hive built-in ORC 989 1049 85 15.9 62.8 1.0X +Data column - Native ORC MR 1076 1078 2 14.6 68.4 0.9X +Data column - Native ORC Vectorized 103 143 29 152.9 6.5 9.6X +Partition column - Hive built-in ORC 648 687 35 24.3 41.2 1.5X +Partition column - Native ORC MR 680 716 32 23.1 43.2 1.5X +Partition column - Native ORC Vectorized 36 55 17 431.5 2.3 27.1X +Both columns - Hive built-in ORC 993 1019 38 15.8 63.1 1.0X +Both columns - Native ORC MR 1137 1173 51 13.8 72.3 0.9X +Both columns - Native ORC Vectorized 138 188 25 114.1 8.8 7.2X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 928 944 14 11.3 88.5 1.0X -Native ORC MR 711 733 25 14.8 67.8 1.3X -Native ORC Vectorized 127 139 19 82.9 12.1 7.3X +Hive built-in ORC 947 974 24 11.1 90.3 1.0X +Native ORC MR 934 950 18 11.2 89.0 1.0X +Native ORC Vectorized 127 132 6 82.8 12.1 7.5X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1539 1597 83 6.8 146.7 1.0X -Native ORC MR 1223 1232 12 8.6 116.7 1.3X -Native ORC Vectorized 286 320 27 36.6 27.3 5.4X +Hive built-in ORC 1476 1489 17 7.1 140.8 1.0X +Native ORC MR 1310 1328 25 8.0 125.0 1.1X +Native ORC Vectorized 308 350 29 34.1 29.3 4.8X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1381 1397 22 7.6 131.7 1.0X -Native ORC MR 1112 1124 17 9.4 106.0 1.2X -Native ORC Vectorized 363 394 30 28.9 34.6 3.8X +Hive built-in ORC 1258 1259 2 8.3 119.9 1.0X +Native ORC MR 1168 1173 7 9.0 111.4 1.1X +Native ORC Vectorized 362 408 50 29.0 34.5 3.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 733 751 24 14.3 69.9 1.0X -Native ORC MR 742 771 48 14.1 70.8 1.0X -Native ORC Vectorized 148 171 26 70.8 14.1 5.0X +Hive built-in ORC 749 774 43 14.0 71.4 1.0X +Native ORC MR 797 830 51 13.2 76.0 0.9X +Native ORC Vectorized 148 168 22 71.1 14.1 5.1X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 562 588 25 1.9 536.0 1.0X -Native ORC MR 87 109 15 12.0 83.3 6.4X -Native ORC Vectorized 30 37 6 34.9 28.7 18.7X +Hive built-in ORC 558 611 65 1.9 532.2 1.0X +Native ORC MR 90 110 25 11.7 85.5 6.2X +Native ORC Vectorized 32 40 10 33.1 30.2 17.6X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1022 1040 26 1.0 974.3 1.0X -Native ORC MR 100 114 11 10.5 95.2 10.2X -Native ORC Vectorized 37 44 7 28.6 35.0 27.8X +Hive built-in ORC 1029 1033 5 1.0 981.4 1.0X +Native ORC MR 98 117 20 10.7 93.2 10.5X +Native ORC Vectorized 39 50 9 26.7 37.5 26.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1522 1617 134 0.7 1451.1 1.0X -Native ORC MR 104 114 9 10.1 99.4 14.6X -Native ORC Vectorized 49 65 12 21.4 46.7 31.1X +Hive built-in ORC 1512 1536 35 0.7 1441.8 1.0X +Native ORC MR 106 128 17 9.9 101.1 14.3X +Native ORC Vectorized 46 67 17 22.9 43.7 33.0X ================================================================================================ Struct scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 10 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 285 321 35 3.7 272.0 1.0X -Native ORC MR 208 274 55 5.1 198.0 1.4X -Native ORC Vectorized 97 119 25 10.8 92.8 2.9X +Hive built-in ORC 324 369 49 3.2 309.0 1.0X +Native ORC MR 213 245 34 4.9 203.5 1.5X +Native ORC Vectorized 99 118 19 10.6 94.1 3.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 100 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Hive built-in ORC 1963 2005 59 0.5 1871.9 1.0X -Native ORC MR 1612 1677 92 0.7 1537.5 1.2X -Native ORC Vectorized 859 944 92 1.2 819.4 2.3X +Hive built-in ORC 2169 2204 48 0.5 2069.0 1.0X +Native ORC MR 1765 1841 107 0.6 1683.6 1.2X +Native ORC Vectorized 858 940 100 1.2 818.1 2.5X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 300 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Hive built-in ORC 5793 5868 107 0.2 5524.2 1.0X -Native ORC MR 5247 5321 105 0.2 5003.5 1.1X -Native ORC Vectorized 5404 5425 30 0.2 5153.5 1.1X +Hive built-in ORC 6111 6228 166 0.2 5828.0 1.0X +Native ORC MR 5474 5540 93 0.2 5220.3 1.1X +Native ORC Vectorized 5605 5658 74 0.2 5345.5 1.1X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 600 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Hive built-in ORC 12664 12690 37 0.1 12077.5 1.0X -Native ORC MR 12398 12513 162 0.1 11823.9 1.0X -Native ORC Vectorized 12552 12553 1 0.1 11970.4 1.0X +Hive built-in ORC 13063 13093 42 0.1 12458.1 1.0X +Native ORC MR 12754 12782 39 0.1 12163.1 1.0X +Native ORC Vectorized 13004 13082 111 0.1 12401.2 1.0X ================================================================================================ Nested Struct scan ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Nested Struct Scan with 10 Elements, 10 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1981 2003 30 0.5 1889.3 1.0X -Native ORC MR 2095 2133 54 0.5 1997.9 0.9X -Native ORC Vectorized 564 605 45 1.9 537.6 3.5X +Hive built-in ORC 2130 2182 73 0.5 2031.7 1.0X +Native ORC MR 2179 2290 156 0.5 2078.2 1.0X +Native ORC Vectorized 568 575 7 1.8 541.9 3.7X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Nested Struct Scan with 30 Elements, 10 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 5412 5426 21 0.2 5161.0 1.0X -Native ORC MR 4556 4639 117 0.2 4345.2 1.2X -Native ORC Vectorized 1478 1506 39 0.7 1409.7 3.7X +Hive built-in ORC 5890 5894 5 0.2 5617.5 1.0X +Native ORC MR 5089 5121 45 0.2 4853.2 1.2X +Native ORC Vectorized 1512 1550 53 0.7 1442.1 3.9X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Nested Struct Scan with 10 Elements, 30 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 5018 5079 87 0.2 4785.1 1.0X -Native ORC MR 5380 5388 11 0.2 5130.5 0.9X -Native ORC Vectorized 1975 2012 52 0.5 1883.8 2.5X +Hive built-in ORC 5276 5277 2 0.2 5031.7 1.0X +Native ORC MR 5272 5293 29 0.2 5027.8 1.0X +Native ORC Vectorized 1906 1913 9 0.6 1818.0 2.8X diff --git a/sql/hive/benchmarks/OrcReadBenchmark-results.txt b/sql/hive/benchmarks/OrcReadBenchmark-results.txt index 64d738858b1a2..7eca721b2d23d 100644 --- a/sql/hive/benchmarks/OrcReadBenchmark-results.txt +++ b/sql/hive/benchmarks/OrcReadBenchmark-results.txt @@ -2,221 +2,221 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 724 754 46 21.7 46.0 1.0X -Native ORC MR 838 865 38 18.8 53.3 0.9X -Native ORC Vectorized 83 104 10 188.5 5.3 8.7X +Hive built-in ORC 738 797 51 21.3 46.9 1.0X +Native ORC MR 814 860 41 19.3 51.8 0.9X +Native ORC Vectorized 112 127 13 140.1 7.1 6.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 709 746 43 22.2 45.1 1.0X -Native ORC MR 791 822 28 19.9 50.3 0.9X -Native ORC Vectorized 85 101 9 184.6 5.4 8.3X +Hive built-in ORC 640 730 82 24.6 40.7 1.0X +Native ORC MR 713 744 35 22.0 45.4 0.9X +Native ORC Vectorized 91 110 15 173.3 5.8 7.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 766 777 17 20.5 48.7 1.0X -Native ORC MR 772 801 25 20.4 49.1 1.0X -Native ORC Vectorized 89 98 6 177.0 5.7 8.6X +Hive built-in ORC 673 679 10 23.4 42.8 1.0X +Native ORC MR 787 816 29 20.0 50.0 0.9X +Native ORC Vectorized 91 103 9 172.3 5.8 7.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 755 762 6 20.8 48.0 1.0X -Native ORC MR 811 818 10 19.4 51.6 0.9X -Native ORC Vectorized 87 101 11 181.7 5.5 8.7X +Hive built-in ORC 648 662 11 24.3 41.2 1.0X +Native ORC MR 749 768 20 21.0 47.6 0.9X +Native ORC Vectorized 88 103 11 178.9 5.6 7.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 775 794 18 20.3 49.3 1.0X -Native ORC MR 847 857 9 18.6 53.9 0.9X -Native ORC Vectorized 141 157 17 111.6 9.0 5.5X +Hive built-in ORC 715 728 23 22.0 45.4 1.0X +Native ORC MR 785 801 14 20.0 49.9 0.9X +Native ORC Vectorized 140 149 8 112.4 8.9 5.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 867 875 7 18.1 55.1 1.0X -Native ORC MR 914 940 22 17.2 58.1 0.9X -Native ORC Vectorized 219 232 15 71.8 13.9 4.0X +Hive built-in ORC 785 794 9 20.0 49.9 1.0X +Native ORC MR 871 899 43 18.1 55.4 0.9X +Native ORC Vectorized 221 239 24 71.2 14.0 3.6X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1588 1596 12 6.6 151.4 1.0X -Native ORC MR 1563 1567 6 6.7 149.1 1.0X -Native ORC Vectorized 628 676 63 16.7 59.8 2.5X +Hive built-in ORC 1494 1514 28 7.0 142.5 1.0X +Native ORC MR 1427 1433 8 7.3 136.1 1.0X +Native ORC Vectorized 602 610 12 17.4 57.4 2.5X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column - Hive built-in ORC 1117 1162 63 14.1 71.0 1.0X -Data column - Native ORC MR 1293 1306 17 12.2 82.2 0.9X -Data column - Native ORC Vectorized 91 103 10 173.7 5.8 12.3X -Partition column - Hive built-in ORC 717 722 7 21.9 45.6 1.6X -Partition column - Native ORC MR 633 673 42 24.8 40.3 1.8X -Partition column - Native ORC Vectorized 37 50 7 419.5 2.4 29.8X -Both columns - Hive built-in ORC 948 1010 69 16.6 60.3 1.2X -Both columns - Native ORC MR 1102 1109 10 14.3 70.1 1.0X -Both columns - Native ORC Vectorized 105 121 13 149.7 6.7 10.6X +Data column - Hive built-in ORC 803 826 24 19.6 51.0 1.0X +Data column - Native ORC MR 910 941 39 17.3 57.9 0.9X +Data column - Native ORC Vectorized 91 105 11 172.8 5.8 8.8X +Partition column - Hive built-in ORC 589 612 19 26.7 37.5 1.4X +Partition column - Native ORC MR 616 640 20 25.5 39.2 1.3X +Partition column - Native ORC Vectorized 37 49 8 422.4 2.4 21.6X +Both columns - Hive built-in ORC 935 945 12 16.8 59.4 0.9X +Both columns - Native ORC MR 994 1000 9 15.8 63.2 0.8X +Both columns - Native ORC Vectorized 103 123 15 153.0 6.5 7.8X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 904 909 5 11.6 86.2 1.0X -Native ORC MR 804 812 7 13.0 76.7 1.1X -Native ORC Vectorized 128 148 19 82.0 12.2 7.1X +Hive built-in ORC 830 846 27 12.6 79.1 1.0X +Native ORC MR 747 751 3 14.0 71.3 1.1X +Native ORC Vectorized 131 147 15 80.0 12.5 6.3X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1429 1453 33 7.3 136.3 1.0X -Native ORC MR 1288 1291 4 8.1 122.9 1.1X -Native ORC Vectorized 294 300 6 35.7 28.0 4.9X +Hive built-in ORC 1475 1479 5 7.1 140.7 1.0X +Native ORC MR 1230 1236 7 8.5 117.3 1.2X +Native ORC Vectorized 294 311 21 35.6 28.1 5.0X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1282 1290 13 8.2 122.2 1.0X -Native ORC MR 1195 1199 7 8.8 113.9 1.1X -Native ORC Vectorized 346 382 45 30.3 33.0 3.7X +Hive built-in ORC 1243 1244 2 8.4 118.5 1.0X +Native ORC MR 1157 1166 13 9.1 110.3 1.1X +Native ORC Vectorized 349 377 20 30.0 33.3 3.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 771 803 29 13.6 73.5 1.0X -Native ORC MR 776 784 12 13.5 74.0 1.0X -Native ORC Vectorized 149 166 13 70.4 14.2 5.2X +Hive built-in ORC 689 715 35 15.2 65.7 1.0X +Native ORC MR 762 767 5 13.8 72.6 0.9X +Native ORC Vectorized 149 174 20 70.5 14.2 4.6X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 400 431 29 2.6 381.3 1.0X -Native ORC MR 89 102 11 11.8 84.9 4.5X -Native ORC Vectorized 32 38 6 33.3 30.1 12.7X +Hive built-in ORC 447 510 71 2.3 426.4 1.0X +Native ORC MR 86 101 11 12.1 82.4 5.2X +Native ORC Vectorized 32 38 6 33.0 30.3 14.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 742 748 8 1.4 707.9 1.0X -Native ORC MR 95 108 12 11.0 90.6 7.8X -Native ORC Vectorized 38 44 5 27.8 36.0 19.7X +Hive built-in ORC 733 744 19 1.4 698.9 1.0X +Native ORC MR 94 109 10 11.1 89.8 7.8X +Native ORC Vectorized 38 46 7 27.7 36.1 19.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 1056 1100 62 1.0 1007.5 1.0X -Native ORC MR 104 114 8 10.1 99.4 10.1X -Native ORC Vectorized 47 54 5 22.5 44.5 22.7X +Hive built-in ORC 1079 1079 0 1.0 1028.8 1.0X +Native ORC MR 103 118 14 10.2 98.2 10.5X +Native ORC Vectorized 47 55 9 22.5 44.4 23.2X ================================================================================================ Struct scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 10 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 434 451 12 2.4 413.9 1.0X -Native ORC MR 273 294 14 3.8 260.8 1.6X -Native ORC Vectorized 104 139 22 10.0 99.6 4.2X +Hive built-in ORC 366 439 60 2.9 348.8 1.0X +Native ORC MR 268 290 23 3.9 255.2 1.4X +Native ORC Vectorized 104 116 15 10.1 99.5 3.5X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 100 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Hive built-in ORC 2530 2556 38 0.4 2412.6 1.0X -Native ORC MR 1530 1598 97 0.7 1458.7 1.7X -Native ORC Vectorized 802 891 89 1.3 764.7 3.2X +Hive built-in ORC 2452 2519 94 0.4 2338.9 1.0X +Native ORC MR 1620 1739 169 0.6 1544.6 1.5X +Native ORC Vectorized 940 1015 106 1.1 896.2 2.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 300 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Hive built-in ORC 7576 7591 20 0.1 7225.4 1.0X -Native ORC MR 5344 5377 47 0.2 5096.4 1.4X -Native ORC Vectorized 5351 5375 35 0.2 5102.9 1.4X +Hive built-in ORC 7321 7391 100 0.1 6981.8 1.0X +Native ORC MR 5367 5395 40 0.2 5118.2 1.4X +Native ORC Vectorized 5121 5225 148 0.2 4883.4 1.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Single Struct Column Scan with 600 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Hive built-in ORC 16196 16239 60 0.1 15446.2 1.0X -Native ORC MR 12920 12974 76 0.1 12321.6 1.3X -Native ORC Vectorized 12604 12735 185 0.1 12019.9 1.3X +Hive built-in ORC 15706 15710 6 0.1 14978.4 1.0X +Native ORC MR 12801 12832 45 0.1 12208.0 1.2X +Native ORC Vectorized 12607 12815 294 0.1 12023.4 1.2X ================================================================================================ Nested Struct scan ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Nested Struct Scan with 10 Elements, 10 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 2614 2637 32 0.4 2493.1 1.0X -Native ORC MR 2025 2027 2 0.5 1931.2 1.3X -Native ORC Vectorized 629 638 10 1.7 599.7 4.2X +Hive built-in ORC 2792 2816 34 0.4 2662.9 1.0X +Native ORC MR 2210 2291 114 0.5 2107.9 1.3X +Native ORC Vectorized 657 684 24 1.6 626.2 4.3X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Nested Struct Scan with 30 Elements, 10 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 7193 7232 55 0.1 6860.0 1.0X -Native ORC MR 4480 4694 302 0.2 4272.6 1.6X -Native ORC Vectorized 1453 1458 6 0.7 1386.2 4.9X +Hive built-in ORC 7307 7509 286 0.1 6968.8 1.0X +Native ORC MR 4974 5189 305 0.2 4743.4 1.5X +Native ORC Vectorized 1578 1604 37 0.7 1504.9 4.6X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure AMD EPYC 7763 64-Core Processor Nested Struct Scan with 10 Elements, 30 Fields: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------ -Hive built-in ORC 6660 6679 27 0.2 6351.1 1.0X -Native ORC MR 5078 5085 9 0.2 4842.7 1.3X -Native ORC Vectorized 1762 1793 43 0.6 1680.6 3.8X +Hive built-in ORC 6461 6466 7 0.2 6161.5 1.0X +Native ORC MR 5289 5352 89 0.2 5043.9 1.2X +Native ORC Vectorized 2077 2086 13 0.5 1980.8 3.1X From bbb8aca0b51008bf65ba8f9232ba96c166e84f8e Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 8 Jan 2025 09:28:13 -0800 Subject: [PATCH 360/438] [SPARK-50754][BUILD] Upgrade commons-codec to 1.17.2 ### What changes were proposed in this pull request? The pr aims to upgrade `commons-codec` from `1.17.1` to `1.17.2`. ### Why are the changes needed? The full release notes: https://commons.apache.org/proper/commons-codec/changes-report.html#a1.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 #49398 from panbingkun/SPARK-50754. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- 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 aa171f0cbfffd..9be4986101351 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -37,7 +37,7 @@ checker-qual/3.43.0//checker-qual-3.43.0.jar chill-java/0.10.0//chill-java-0.10.0.jar chill_2.13/0.10.0//chill_2.13-0.10.0.jar commons-cli/1.9.0//commons-cli-1.9.0.jar -commons-codec/1.17.1//commons-codec-1.17.1.jar +commons-codec/1.17.2//commons-codec-1.17.2.jar commons-collections/3.2.2//commons-collections-3.2.2.jar commons-collections4/4.4//commons-collections4-4.4.jar commons-compiler/3.1.9//commons-compiler-3.1.9.jar diff --git a/pom.xml b/pom.xml index 3a8cbd1f033c2..3b242b9ea865d 100644 --- a/pom.xml +++ b/pom.xml @@ -185,7 +185,7 @@ 2.3.1 1.1.10.7 3.0.3 - 1.17.1 + 1.17.2 1.27.1 2.18.0 From cb093e68f312b25e58d56ff9825b63cf732a8e13 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 9 Jan 2025 13:02:05 +0900 Subject: [PATCH 361/438] [SPARK-50685][PYTHON][FOLLOW-UP] Improve Py4J performance by leveraging getattr ### What changes were proposed in this pull request? This PR is. a followup of https://github.com/apache/spark/pull/49313 that fixes more places missed. This PR fixes Core, SQL, ML and Structured Streaming. Tests codes, MLLib and DStream are not affected. ### Why are the changes needed? To reduce the overhead of Py4J calls. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually tested as demonstrated in https://github.com/apache/spark/pull/49312 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49412 from HyukjinKwon/SPARK-50685-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/core/context.py | 10 +++++----- python/pyspark/core/rdd.py | 4 +++- python/pyspark/ml/feature.py | 8 ++++---- python/pyspark/ml/stat.py | 6 +++--- python/pyspark/resource/requests.py | 12 ++++++------ python/pyspark/sql/avro/functions.py | 6 +++--- python/pyspark/sql/classic/window.py | 20 ++++++++++---------- python/pyspark/sql/protobuf/functions.py | 24 ++++++++++++------------ python/pyspark/sql/readwriter.py | 4 +++- 9 files changed, 49 insertions(+), 45 deletions(-) diff --git a/python/pyspark/core/context.py b/python/pyspark/core/context.py index 42a368555ae9e..5fcd4ffb09210 100644 --- a/python/pyspark/core/context.py +++ b/python/pyspark/core/context.py @@ -559,7 +559,7 @@ def setSystemProperty(cls, key: str, value: str) -> None: """ SparkContext._ensure_initialized() assert SparkContext._jvm is not None - SparkContext._jvm.java.lang.System.setProperty(key, value) + getattr(SparkContext._jvm, "java.lang.System").setProperty(key, value) @classmethod def getSystemProperty(cls, key: str) -> str: @@ -581,7 +581,7 @@ def getSystemProperty(cls, key: str) -> str: """ SparkContext._ensure_initialized() assert SparkContext._jvm is not None - return SparkContext._jvm.java.lang.System.getProperty(key) + return getattr(SparkContext._jvm, "java.lang.System").getProperty(key) @property def version(self) -> str: @@ -1206,7 +1206,7 @@ def binaryRecords(self, path: str, recordLength: int) -> RDD[bytes]: def _dictToJavaMap(self, d: Optional[Dict[str, str]]) -> JavaMap: assert self._jvm is not None - jm = self._jvm.java.util.HashMap() + jm = getattr(self._jvm, "java.util.HashMap")() if not d: d = {} for k, v in d.items(): @@ -1938,7 +1938,7 @@ def listFiles(self) -> List[str]: :meth:`SparkContext.addFile` """ return list( - self._jvm.scala.jdk.javaapi.CollectionConverters.asJava( # type: ignore[union-attr] + getattr(self._jvm, "scala.jdk.javaapi.CollectionConverters").asJava( self._jsc.sc().listFiles() ) ) @@ -2066,7 +2066,7 @@ def listArchives(self) -> List[str]: :meth:`SparkContext.addArchive` """ return list( - self._jvm.scala.jdk.javaapi.CollectionConverters.asJava( # type: ignore[union-attr] + getattr(self._jvm, "scala.jdk.javaapi.CollectionConverters").asJava( self._jsc.sc().listArchives() ) ) diff --git a/python/pyspark/core/rdd.py b/python/pyspark/core/rdd.py index 8106e07df478f..bbf17dbed7fa2 100644 --- a/python/pyspark/core/rdd.py +++ b/python/pyspark/core/rdd.py @@ -3286,7 +3286,9 @@ def func(split: int, iterator: Iterable[Any]) -> Iterable[bytes]: assert self.ctx._jvm is not None if compressionCodecClass: - compressionCodec = self.ctx._jvm.java.lang.Class.forName(compressionCodecClass) + compressionCodec = getattr(self.ctx._jvm, "java.lang.Class").forName( + compressionCodecClass + ) keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path, compressionCodec) else: keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index e053ea273140c..cf12a5390746f 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1208,7 +1208,7 @@ def from_vocabulary( sc = SparkContext._active_spark_context assert sc is not None and sc._gateway is not None - java_class = sc._gateway.jvm.java.lang.String + java_class = getattr(sc._gateway.jvm, "java.lang.String") jvocab = CountVectorizerModel._new_java_array(vocabulary, java_class) model = CountVectorizerModel._create_from_java_class( "org.apache.spark.ml.feature.CountVectorizerModel", jvocab @@ -4799,7 +4799,7 @@ def from_labels( sc = SparkContext._active_spark_context assert sc is not None and sc._gateway is not None - java_class = sc._gateway.jvm.java.lang.String + java_class = getattr(sc._gateway.jvm, "java.lang.String") jlabels = StringIndexerModel._new_java_array(labels, java_class) model = StringIndexerModel._create_from_java_class( "org.apache.spark.ml.feature.StringIndexerModel", jlabels @@ -4828,7 +4828,7 @@ def from_arrays_of_labels( sc = SparkContext._active_spark_context assert sc is not None and sc._gateway is not None - java_class = sc._gateway.jvm.java.lang.String + java_class = getattr(sc._gateway.jvm, "java.lang.String") jlabels = StringIndexerModel._new_java_array(arrayOfLabels, java_class) model = StringIndexerModel._create_from_java_class( "org.apache.spark.ml.feature.StringIndexerModel", jlabels @@ -5198,7 +5198,7 @@ def loadDefaultStopWords(language: str) -> List[str]: Supported languages: danish, dutch, english, finnish, french, german, hungarian, italian, norwegian, portuguese, russian, spanish, swedish, turkish """ - stopWordsObj = _jvm().org.apache.spark.ml.feature.StopWordsRemover + stopWordsObj = getattr(_jvm(), "org.apache.spark.ml.feature.StopWordsRemover") return list(stopWordsObj.loadDefaultStopWords(language)) diff --git a/python/pyspark/ml/stat.py b/python/pyspark/ml/stat.py index 4dcc961909520..04b0c7278a717 100644 --- a/python/pyspark/ml/stat.py +++ b/python/pyspark/ml/stat.py @@ -107,7 +107,7 @@ def test( sc = SparkContext._active_spark_context assert sc is not None - javaTestObj = _jvm().org.apache.spark.ml.stat.ChiSquareTest + javaTestObj = getattr(_jvm(), "org.apache.spark.ml.stat.ChiSquareTest") args = [_py2java(sc, arg) for arg in (dataset, featuresCol, labelCol, flatten)] return _java2py(sc, javaTestObj.test(*args)) @@ -178,7 +178,7 @@ def corr(dataset: DataFrame, column: str, method: str = "pearson") -> DataFrame: sc = SparkContext._active_spark_context assert sc is not None - javaCorrObj = _jvm().org.apache.spark.ml.stat.Correlation + javaCorrObj = getattr(_jvm(), "org.apache.spark.ml.stat.Correlation") args = [_py2java(sc, arg) for arg in (dataset, column, method)] return _java2py(sc, javaCorrObj.corr(*args)) @@ -248,7 +248,7 @@ def test(dataset: DataFrame, sampleCol: str, distName: str, *params: float) -> D sc = SparkContext._active_spark_context assert sc is not None - javaTestObj = _jvm().org.apache.spark.ml.stat.KolmogorovSmirnovTest + javaTestObj = getattr(_jvm(), "org.apache.spark.ml.stat.KolmogorovSmirnovTest") dataset = _py2java(sc, dataset) params = [float(param) for param in params] # type: ignore[assignment] return _java2py( diff --git a/python/pyspark/resource/requests.py b/python/pyspark/resource/requests.py index fa8bb43ee2c49..805cecd5dbbe0 100644 --- a/python/pyspark/resource/requests.py +++ b/python/pyspark/resource/requests.py @@ -173,9 +173,9 @@ def __init__( jvm = _jvm or SparkContext._jvm if jvm is not None: - self._java_executor_resource_requests = ( - jvm.org.apache.spark.resource.ExecutorResourceRequests() - ) + self._java_executor_resource_requests = getattr( + jvm, "org.apache.spark.resource.ExecutorResourceRequests" + )() if _requests is not None: for k, v in _requests.items(): if k == self._MEMORY: @@ -474,9 +474,9 @@ def __init__( jvm = _jvm or SparkContext._jvm if jvm is not None: - self._java_task_resource_requests: Optional[ - "JavaObject" - ] = jvm.org.apache.spark.resource.TaskResourceRequests() + self._java_task_resource_requests: Optional["JavaObject"] = getattr( + jvm, "org.apache.spark.resource.TaskResourceRequests" + )() if _requests is not None: for k, v in _requests.items(): if k == self._CPUS: diff --git a/python/pyspark/sql/avro/functions.py b/python/pyspark/sql/avro/functions.py index a9e41f20357e8..0b18212faf605 100644 --- a/python/pyspark/sql/avro/functions.py +++ b/python/pyspark/sql/avro/functions.py @@ -102,7 +102,7 @@ def from_avro( sc = get_active_spark_context() try: - jc = cast(JVMView, sc._jvm).org.apache.spark.sql.avro.functions.from_avro( + jc = getattr(cast(JVMView, sc._jvm), "org.apache.spark.sql.avro.functions").from_avro( _to_java_column(data), jsonFormatSchema, options or {} ) except TypeError as e: @@ -168,11 +168,11 @@ def to_avro(data: "ColumnOrName", jsonFormatSchema: str = "") -> Column: sc = get_active_spark_context() try: if jsonFormatSchema == "": - jc = cast(JVMView, sc._jvm).org.apache.spark.sql.avro.functions.to_avro( + jc = getattr(cast(JVMView, sc._jvm), "org.apache.spark.sql.avro.functions").to_avro( _to_java_column(data) ) else: - jc = cast(JVMView, sc._jvm).org.apache.spark.sql.avro.functions.to_avro( + jc = getattr(cast(JVMView, sc._jvm), "org.apache.spark.sql.avro.functions").to_avro( _to_java_column(data), jsonFormatSchema ) except TypeError as e: diff --git a/python/pyspark/sql/classic/window.py b/python/pyspark/sql/classic/window.py index 63e9a337c0c2e..c7bc92739b240 100644 --- a/python/pyspark/sql/classic/window.py +++ b/python/pyspark/sql/classic/window.py @@ -48,9 +48,9 @@ def partitionBy(*cols: Union["ColumnOrName", Sequence["ColumnOrName"]]) -> Paren from py4j.java_gateway import JVMView sc = get_active_spark_context() - jspec = cast(JVMView, sc._jvm).org.apache.spark.sql.expressions.Window.partitionBy( - _to_java_cols(cols) - ) + jspec = getattr( + cast(JVMView, sc._jvm), "org.apache.spark.sql.expressions.Window" + ).partitionBy(_to_java_cols(cols)) return WindowSpec(jspec) @staticmethod @@ -58,7 +58,7 @@ def orderBy(*cols: Union["ColumnOrName", Sequence["ColumnOrName"]]) -> ParentWin from py4j.java_gateway import JVMView sc = get_active_spark_context() - jspec = cast(JVMView, sc._jvm).org.apache.spark.sql.expressions.Window.orderBy( + jspec = getattr(cast(JVMView, sc._jvm), "org.apache.spark.sql.expressions.Window").orderBy( _to_java_cols(cols) ) return WindowSpec(jspec) @@ -72,9 +72,9 @@ def rowsBetween(start: int, end: int) -> ParentWindowSpec: if end >= Window._FOLLOWING_THRESHOLD: end = Window.unboundedFollowing sc = get_active_spark_context() - jspec = cast(JVMView, sc._jvm).org.apache.spark.sql.expressions.Window.rowsBetween( - start, end - ) + jspec = getattr( + cast(JVMView, sc._jvm), "org.apache.spark.sql.expressions.Window" + ).rowsBetween(start, end) return WindowSpec(jspec) @staticmethod @@ -86,9 +86,9 @@ def rangeBetween(start: int, end: int) -> ParentWindowSpec: if end >= Window._FOLLOWING_THRESHOLD: end = Window.unboundedFollowing sc = get_active_spark_context() - jspec = cast(JVMView, sc._jvm).org.apache.spark.sql.expressions.Window.rangeBetween( - start, end - ) + jspec = getattr( + cast(JVMView, sc._jvm), "org.apache.spark.sql.expressions.Window" + ).rangeBetween(start, end) return WindowSpec(jspec) diff --git a/python/pyspark/sql/protobuf/functions.py b/python/pyspark/sql/protobuf/functions.py index 1e75874e75f9a..ece450a77f4f3 100644 --- a/python/pyspark/sql/protobuf/functions.py +++ b/python/pyspark/sql/protobuf/functions.py @@ -149,13 +149,13 @@ def from_protobuf( elif descFilePath is not None: binary_proto = _read_descriptor_set_file(descFilePath) if binary_proto is not None: - jc = cast(JVMView, sc._jvm).org.apache.spark.sql.protobuf.functions.from_protobuf( - _to_java_column(data), messageName, binary_proto, options or {} - ) + jc = getattr( + cast(JVMView, sc._jvm), "org.apache.spark.sql.protobuf.functions" + ).from_protobuf(_to_java_column(data), messageName, binary_proto, options or {}) else: - jc = cast(JVMView, sc._jvm).org.apache.spark.sql.protobuf.functions.from_protobuf( - _to_java_column(data), messageName, options or {} - ) + jc = getattr( + cast(JVMView, sc._jvm), "org.apache.spark.sql.protobuf.functions" + ).from_protobuf(_to_java_column(data), messageName, options or {}) except TypeError as e: if str(e) == "'JavaPackage' object is not callable": _print_missing_jar("Protobuf", "protobuf", "protobuf", sc.version) @@ -271,13 +271,13 @@ def to_protobuf( elif descFilePath is not None: binary_proto = _read_descriptor_set_file(descFilePath) if binary_proto is not None: - jc = cast(JVMView, sc._jvm).org.apache.spark.sql.protobuf.functions.to_protobuf( - _to_java_column(data), messageName, binary_proto, options or {} - ) + jc = getattr( + cast(JVMView, sc._jvm), "org.apache.spark.sql.protobuf.functions" + ).to_protobuf(_to_java_column(data), messageName, binary_proto, options or {}) else: - jc = cast(JVMView, sc._jvm).org.apache.spark.sql.protobuf.functions.to_protobuf( - _to_java_column(data), messageName, options or {} - ) + jc = getattr( + cast(JVMView, sc._jvm), "org.apache.spark.sql.protobuf.functions" + ).to_protobuf(_to_java_column(data), messageName, options or {}) except TypeError as e: if str(e) == "'JavaPackage' object is not callable": diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 2113f0707f910..96c8f8a475b26 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -1174,7 +1174,9 @@ def jdbc( if predicates is not None: gateway = self._spark._sc._gateway assert gateway is not None - jpredicates = utils.to_java_array(gateway, gateway.jvm.java.lang.String, predicates) + jpredicates = utils.to_java_array( + gateway, getattr(gateway.jvm, "java.lang.String"), predicates + ) return self._df(self._jreader.jdbc(url, table, jpredicates, jprop)) return self._df(self._jreader.jdbc(url, table, jprop)) From b84dc909a8856388faddc154c6a1d3aba271474e Mon Sep 17 00:00:00 2001 From: Linhong Liu Date: Thu, 9 Jan 2025 12:08:16 +0800 Subject: [PATCH 362/438] [SPARK-50769][SQL] Fix ClassCastException in HistogramNumeric ### What changes were proposed in this pull request? The `HistogramNumeric` accepts `NumberType` but it doesn't properly handle the `DecimalType` in the execution. Therefore, the `ClassCastException` when trying to change a Decimal to Double. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT ``` build/sbt "sql/testOnly *SQLQueryTestSuite -- -z group-by.sql" ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #49418 from linhongliu-db/SPARK-50769. Authored-by: Linhong Liu Signed-off-by: Wenchen Fan --- .../expressions/aggregate/HistogramNumeric.scala | 10 +++++++++- .../sql-tests/analyzer-results/group-by.sql.out | 9 +++++++++ .../src/test/resources/sql-tests/inputs/group-by.sql | 2 ++ .../test/resources/sql-tests/results/group-by.sql.out | 9 +++++++++ 4 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HistogramNumeric.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HistogramNumeric.scala index eda2c742ab4b5..142f4a4eae4c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HistogramNumeric.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HistogramNumeric.scala @@ -126,7 +126,10 @@ case class HistogramNumeric( // Ignore empty rows, for example: histogram_numeric(null) if (value != null) { // Convert the value to a double value - val doubleValue = value.asInstanceOf[Number].doubleValue + val doubleValue = value match { + case d: Decimal => d.toDouble + case o => o.asInstanceOf[Number].doubleValue() + } buffer.add(doubleValue) } buffer @@ -162,6 +165,11 @@ case class HistogramNumeric( case ShortType => coord.x.toShort case _: DayTimeIntervalType | LongType | TimestampType | TimestampNTZType => coord.x.toLong + case d: DecimalType => + val bigDecimal = BigDecimal + .decimal(coord.x, new java.math.MathContext(d.precision)) + .setScale(d.scale, BigDecimal.RoundingMode.HALF_UP) + Decimal(bigDecimal) case _ => coord.x } array(index) = InternalRow.apply(result, coord.y) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out index 34ff2a2186f0b..304b382c7bbec 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out @@ -1055,6 +1055,15 @@ Aggregate [histogram_numeric(col#xL, 3, 0, 0) AS histogram_numeric(col, 3)#x] +- LocalRelation [col#xL] +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS DECIMAL(4, 2))), (CAST(2 AS DECIMAL(4, 2))), (CAST(3 AS DECIMAL(4, 2))) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + -- !query SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 6dd0adbc87221..0cc1f62b0583a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -221,6 +221,8 @@ SELECT histogram_numeric(col, 3) FROM VALUES (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col); SELECT histogram_numeric(col, 3) FROM VALUES (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS DECIMAL(4, 2))), (CAST(2 AS DECIMAL(4, 2))), (CAST(3 AS DECIMAL(4, 2))) AS tab(col); SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col); SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 633133ad7e4d6..98ad1a0a5bba1 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -987,6 +987,15 @@ struct>> [{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS DECIMAL(4, 2))), (CAST(2 AS DECIMAL(4, 2))), (CAST(3 AS DECIMAL(4, 2))) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.00,"y":1.0},{"x":2.00,"y":1.0},{"x":3.00,"y":1.0}] + + -- !query SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) From 22735bedc497e620d3acc9da1b8fc1e70ad02d32 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 9 Jan 2025 13:10:43 +0800 Subject: [PATCH 363/438] [SPARK-50772][SQL] Retain table aliases after SET, EXTEND, DROP operators ### What changes were proposed in this pull request? Per initial feedback from testing, users would like table aliases (such as those mapping to left and right side inputs to a prior join) to remain available after SET, DROP, and EXTEND operators. Here is an example that should work: ``` values (0), (1) lhs(a) |> inner join values (1), (2) rhs(a) using (a) |> extend lhs.a + rhs.a as z1 |> extend lhs.a - rhs.a as z2 |> drop z1 |> where z2 = 0 |> order by lhs.a, rhs.a, z2 |> set z2 = 4 |> limit 2 |> select lhs.a, rhs.a, z2; 1 1 4 ``` To implement this: * Previously, the `|> where` or `|> order by` operators added a `SubqueryAlias` with an auto-generated table alias to the end of the logical plan under construction, in order to prevent the analyzer from adding attributes to the previous plan later (from `ColumnResolutionHelper.resolveExprsAndAddMissingAttrs`). * This PR replaces that behavior with a new `PipeOperator` instead, to avoid replacing the table alias while maintaining correct behavior. * This PR also updates docs to mention the improved table alias behavior. ### Why are the changes needed? This makes SQL pipe syntax easier to use. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds and updates golden file based testing. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49420 from dtenedor/fix-table-aliases. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan --- docs/sql-pipe-syntax.md | 161 ++++++++++++--- .../analysis/ColumnResolutionHelper.scala | 7 +- .../catalyst/expressions/pipeOperators.scala | 22 ++ .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../sql/catalyst/parser/AstBuilder.scala | 15 +- .../sql/catalyst/rules/RuleIdCollection.scala | 1 + .../sql/catalyst/trees/TreePatterns.scala | 1 + .../analyzer-results/pipe-operators.sql.out | 195 +++++++++++------- .../sql-tests/inputs/pipe-operators.sql | 12 ++ .../sql-tests/results/pipe-operators.sql.out | 27 ++- 10 files changed, 327 insertions(+), 115 deletions(-) diff --git a/docs/sql-pipe-syntax.md b/docs/sql-pipe-syntax.md index 829aa0e607bde..3d757db966239 100644 --- a/docs/sql-pipe-syntax.md +++ b/docs/sql-pipe-syntax.md @@ -198,12 +198,22 @@ TABLE t; Evaluates the provided expressions over each of the rows of the input table. +In general, this operator is not always required with SQL pipe syntax. It is possible to use it at +or near the end of a query to evaluate expressions or specify a list of output columns. + +Since the final query result always comprises the columns returned from the last pipe operator, +when this `SELECT` operator does not appear, the output includes all columns from the full row. +This behavior is similar to `SELECT *` in standard SQL syntax. + It is possible to use `DISTINCT` and `*` as needed.
    This works like the outermost `SELECT` in a table subquery in regular Spark SQL. Window functions are supported in the `SELECT` list as well. To use them, the `OVER` clause must be provided. You may provide the window specification in the `WINDOW` clause. +Aggregate functions are not supported in this operator. To perform aggregation, use the `AGGREGATE` +operator instead. + For example: ```sql @@ -226,7 +236,12 @@ FROM t |> EXTEND [[AS] alias], ... ``` -Appends new columns to the input table by evaluating the specified expressions over each of the input rows. +Appends new columns to the input table by evaluating the specified expressions over each of the +input rows. + +After an `EXTEND` operation, top-level column names are updated but table aliases still refer to the +original row values (such as an inner join between two tables `lhs` and `rhs` with a subsequent +`EXTEND` and then `SELECT lhs.col, rhs.col`). For example: @@ -248,7 +263,17 @@ VALUES (0), (1) tab(col) |> SET = , ... ``` -Updates columns of the input table by replacing them with the result of evaluating the provided expressions. +Updates columns of the input table by replacing them with the result of evaluating the provided +expressions. Each such column reference must appear in the input table exactly once. + +This is similar to `SELECT * EXCEPT (column), AS column` in regular Spark SQL. + +It is possible to perform multiple assignments in a single `SET` clause. Each assignment may refer +to the result of previous assignments. + +After an assignment, top-level column names are updated but table aliases still refer to the +original row values (such as an inner join between two tables `lhs` and `rhs` with a subsequent +`SET` and then `SELECT lhs.col, rhs.col`). For example: @@ -256,6 +281,16 @@ For example: VALUES (0), (1) tab(col) |> SET col = col * 2; ++---+ +|col| ++---+ +| 0| +| 2| ++---+ + +VALUES (0), (1) tab(col) +|> SET col = col * 2; + +---+ |col| +---+ @@ -270,7 +305,14 @@ VALUES (0), (1) tab(col) |> DROP , ... ``` -Drops columns of the input table by name. +Drops columns of the input table by name. Each such column reference must appear in the input table +exactly once. + +This is similar to `SELECT * EXCEPT (column)` in regular Spark SQL. + +After a `DROP` operation, top-level column names are updated but table aliases still refer to the +original row values (such as an inner join between two tables `lhs` and `rhs` with a subsequent +`DROP` and then `SELECT lhs.col, rhs.col`). For example: @@ -293,18 +335,25 @@ VALUES (0, 1) tab(col1, col2) Retains the same rows and column names of the input table but with a new table alias. +This operator is useful for introducing a new alias for the input table, which can then be referred +to in subsequent operators. Any existing alias for the table is replaced by the new alias. + +It is useful to use this operator after adding new columns with `SELECT` or `EXTEND` or after +performing aggregation with `AGGREGATE`. This simplifies the process of referring to the columns +from subsequent `JOIN` operators and allows for more readable queries. + For example: ```sql VALUES (0, 1) tab(col1, col2) -|> AS new_tab; -|> SELECT * FROM new_tab; +|> AS new_tab +|> SELECT col1 + col2 FROM new_tab; -+----+----+ -|col1|col2| -+----+----+ -| 0| 1| -+----+----+ ++-----------+ +|col1 + col2| ++-----------+ +| 1| ++-----------+ ``` #### WHERE @@ -357,22 +406,48 @@ VALUES (0), (0) tab(col) #### AGGREGATE ```sql +-- Full-table aggregation |> AGGREGATE [[AS] alias], ... -``` - -Performs full-table aggregation, returning one result row with a column for each aggregate expression. -```sql +-- Aggregation with grouping |> AGGREGATE [ [[AS] alias], ...] GROUP BY [AS alias], ... ``` -Performs aggregation with grouping, returning one row per group. The column list includes the -grouping columns first and then the aggregate columns afterward. Aliases can be assigned directly -on grouping expressions. +Performs aggregation across grouped rows or across the entire input table. + +If no `GROUP BY` clause is present, this performs full-table aggregation, returning one result row +with a column for each aggregate expression. Othwrise, this performs aggregation with grouping, +returning one row per group. Aliases can be assigned directly on grouping expressions. + +The output column list of this operator includes the grouping columns first (if any), and then the +aggregate columns afterward. + +Each `` expression can include standard aggregate function(s) like `COUNT`, `SUM`, `AVG`, +`MIN`, or any other aggregate function(s) that Spark SQL supports. Additional expressions may appear +below or above the aggregate function(s), such as `MIN(FLOOR(col)) + 1`. Each `` +expression must contain at least one aggregate function (or otherwise the query returns an error). +Each `` expression may include a column alias with `AS `, and may also +include a `DISTINCT` keyword to remove duplicate values before applying the aggregate function (for +example, `COUNT(DISTINCT col)`). + +If present, the `GROUP BY` clause can include any number of grouping expressions, and each +`` expression will evaluate over each unique combination of values of the grouping +expressions. The output table contains the evaluated grouping expressions followed by the evaluated +aggregate functions. The `GROUP BY` expressions may include one-based ordinals. Unlike regular SQL +in which such ordinals refer to the expressions in the accompanying `SELECT` clause, in SQL pipe +syntax, they refer to the columns of the relation produced by the preceding operator instead. For +example, in `TABLE t |> AGGREGATE COUNT(*) GROUP BY 2`, we refer to the second column of the input +table `t`. + +There is no need to repeat entire expressions between `GROUP BY` and `SELECT`, since the `AGGREGATE` +operator automatically includes the evaluated grouping expressions in its output. By the same token, +after an `AGGREGATE` operator, it is often unnecessary to issue a following `SELECT` operator, since +`AGGREGATE` returns both the grouping columns and the aggregate columns in a single step. For example: ```sql +-- Full-table aggregation VALUES (0), (1) tab(col) |> AGGREGATE COUNT(col) AS count; @@ -382,6 +457,7 @@ VALUES (0), (1) tab(col) | 2| +-----+ +-- Aggregation with grouping VALUES (0, 1), (0, 2) tab(col1, col2) |> AGGREGATE COUNT(col2) AS count GROUP BY col1; @@ -398,19 +474,45 @@ VALUES (0, 1), (0, 2) tab(col1, col2) |> [LEFT | RIGHT | FULL | CROSS | SEMI | ANTI | NATURAL | LATERAL] JOIN [ON | USING(col, ...)] ``` -Joins rows from both inputs, returning a filtered cross-product of the pipe input table and the table expression following the JOIN keyword. +Joins rows from both inputs, returning a filtered cross-product of the pipe input table and the +table expression following the JOIN keyword. This behaves a similar manner as the `JOIN` clause in +regular SQL where the pipe operator input table becomes the left side of the join and the table +argument becomes the right side of the join. + +Standard join modifiers like `LEFT`, `RIGHT`, and `FULL` are supported before the `JOIN` keyword. + +The join predicate may need to refer to columns from both inputs to the join. In this case, it may +be necessary to use table aliases to differentiate between columns in the event that both inputs +have columns with the same names. The `AS` operator can be useful here to introduce a new alias for +the pipe input table that becomes the left side of the join. Use standard syntax to assign an alias +to the table argument that becomes the right side of the join, if needed. For example: ```sql -VALUES (0, 1) tab(a, b) -|> JOIN VALUES (0, 2) tab(c, d) ON a = c; +SELECT 0 AS a, 1 AS b +|> AS lhs +|> JOIN VALUES (0, 2) rhs(a, b) ON (lhs.a = rhs.a); +---+---+---+---+ | a| b| c| d| +---+---+---+---+ | 0| 1| 0| 2| +---+---+---+---+ + +VALUES ('apples', 3), ('bananas', 4) t(item, sales) +|> AS produce_sales +|> LEFT JOIN + (SELECT "apples" AS item, 123 AS id) AS produce_data + USING (item) +|> SELECT produce_sales.item, sales, id; + +/*---------+-------+------+ + | item | sales | id | + +---------+-------+------+ + | apples | 3 | 123 | + | bananas | 4 | NULL | + +---------+-------+------*/ ``` #### ORDER BY @@ -419,7 +521,8 @@ VALUES (0, 1) tab(a, b) |> ORDER BY [ASC | DESC], ... ``` -Returns the input rows after sorting as indicated. Standard modifiers are supported including NULLS FIRST/LAST. +Returns the input rows after sorting as indicated. Standard modifiers are supported including NULLS +FIRST/LAST. For example: @@ -438,10 +541,10 @@ VALUES (0), (1) tab(col) #### UNION, INTERSECT, EXCEPT ```sql -|> {UNION | INTERSECT | EXCEPT} {ALL | DISTINCT} (), (), ... +|> {UNION | INTERSECT | EXCEPT} {ALL | DISTINCT} () ``` -Performs the union or other set operation over the combined rows from the input table plus one or more tables provided as input arguments. +Performs the union or other set operation over the combined rows from the input table or subquery. For example: @@ -469,12 +572,22 @@ For example: ```sql VALUES (0), (0), (0), (0) tab(col) -|> TABLESAMPLE BERNOULLI(1 ROWS); +|> TABLESAMPLE (1 ROWS); + ++---+ +|col| ++---+ +| 0| ++---+ + +VALUES (0), (0) tab(col) +|> TABLESAMPLE (100 PERCENT); +---+ |col| +---+ | 0| +| 0| +---+ ``` diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index 36fd4d02f8da1..56b2103c555db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -53,9 +53,10 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { (exprs, plan) } else { plan match { - // For `Distinct` and `SubqueryAlias`, we can't recursively resolve and add attributes - // via its children. - case u: UnaryNode if !u.isInstanceOf[Distinct] && !u.isInstanceOf[SubqueryAlias] => + // For `Distinct` and `SubqueryAlias` and `PipeOperator`, we can't recursively resolve and + // add attributes via its children. + case u: UnaryNode if !u.isInstanceOf[Distinct] && !u.isInstanceOf[SubqueryAlias] + && !u.isInstanceOf[PipeOperator] => val (newExprs, newChild) = { // Resolving expressions against current plan. val maybeResolvedExprs = exprs.map(resolveExpressionByPlanOutput(_, u)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala index fe8f0f264e85f..40d7d24263a78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala @@ -18,6 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{PIPE_OPERATOR, TreePattern} import org.apache.spark.sql.errors.QueryCompilationErrors /** @@ -57,6 +60,25 @@ case class PipeExpression(child: Expression, isAggregate: Boolean, clause: Strin } } +/** + * Represents the location within a logical plan that a SQL pipe operator appeared. + * This acts as a logical boundary that works to prevent the analyzer from modifying the logical + * operators above and below the boundary. + */ +case class PipeOperator(child: LogicalPlan) extends UnaryNode { + final override val nodePatterns: Seq[TreePattern] = Seq(PIPE_OPERATOR) + override def output: Seq[Attribute] = child.output + override def withNewChildInternal(newChild: LogicalPlan): PipeOperator = copy(child = newChild) +} + +/** This rule removes all PipeOperator nodes from a logical plan at the end of analysis. */ +object EliminatePipeOperators extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( + _.containsPattern(PIPE_OPERATOR), ruleId) { + case PipeOperator(child) => child + } +} + object PipeOperators { // These are definitions of query result clauses that can be used with the pipe operator. val aggregateClause = "AGGREGATE" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b141d2be04c32..c0c76dd44ad5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -313,6 +313,7 @@ abstract class Optimizer(catalogManager: CatalogManager) private val rules = Seq( EliminateResolvedHint, EliminateSubqueryAliases, + EliminatePipeOperators, EliminateView, ReplaceExpressions, RewriteNonCorrelatedExists, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1f9c148303647..f4f6d2b310f49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -6028,17 +6028,6 @@ class AstBuilder extends DataTypeAstBuilder if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) { operationNotAllowed("Operator pipe SQL syntax using |>", ctx) } - // This helper function adds a table subquery boundary between the new operator to be added - // (such as a filter or sort) and the input plan if one does not already exist. This helps the - // analyzer behave as if we had added the corresponding SQL clause after a table subquery - // containing the input plan. - def withSubqueryAlias(): LogicalPlan = left match { - case _: SubqueryAlias | _: UnresolvedRelation | _: Join | _: Filter | - _: GlobalLimit | _: LocalLimit | _: Offset | _: Sort => - left - case _ => - SubqueryAlias(SubqueryAlias.generateSubqueryName(), left) - } Option(ctx.selectClause).map { c => withSelectQuerySpecification( ctx = ctx, @@ -6082,7 +6071,7 @@ class AstBuilder extends DataTypeAstBuilder if (ctx.windowClause() != null) { throw QueryParsingErrors.windowClauseInPipeOperatorWhereClauseNotAllowedError(ctx) } - withWhereClause(c, withSubqueryAlias()) + withWhereClause(c, PipeOperator(left)) }.getOrElse(Option(ctx.pivotClause()).map { c => if (ctx.unpivotClause() != null) { throw QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx) @@ -6101,7 +6090,7 @@ class AstBuilder extends DataTypeAstBuilder val all = Option(ctx.setQuantifier()).exists(_.ALL != null) visitSetOperationImpl(left, plan(ctx.right), all, c.getType) }.getOrElse(Option(ctx.queryOrganization).map { c => - withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true) + withQueryResultClauses(c, PipeOperator(left), forPipeOperators = true) }.getOrElse( visitOperatorPipeAggregate(ctx, left) )))))))))))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index 0918306de62e0..3f79e74b18a45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -108,6 +108,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.UpdateOuterReferences" :: "org.apache.spark.sql.catalyst.analysis.UpdateAttributeNullability" :: "org.apache.spark.sql.catalyst.analysis.ResolveUpdateEventTimeWatermarkColumn" :: + "org.apache.spark.sql.catalyst.expressions.EliminatePipeOperators" :: // Catalyst Optimizer rules "org.apache.spark.sql.catalyst.optimizer.BooleanSimplification" :: "org.apache.spark.sql.catalyst.optimizer.CollapseProject" :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 80531da4a0ab7..25ef341b8cef3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -79,6 +79,7 @@ object TreePattern extends Enumeration { val OUTER_REFERENCE: Value = Value val PARAMETER: Value = Value val PARAMETERIZED_QUERY: Value = Value + val PIPE_OPERATOR: Value = Value val PIVOT: Value = Value val PLAN_EXPRESSION: Value = Value val PYTHON_UDF: Value = Value diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index 70de582fb7b25..ac74fea1dbfba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -306,7 +306,7 @@ from t as t_alias -- !query analysis Project [tx#x] +- Filter (ty#x = def) - +- SubqueryAlias __auto_generated_subquery_name + +- PipeOperator +- Project [pipeexpression(x#x, false, SELECT) AS tx#x, pipeexpression(y#x, false, SELECT) AS ty#x] +- SubqueryAlias t_alias +- SubqueryAlias spark_catalog.default.t @@ -769,8 +769,9 @@ Filter exists#x [x#x] : +- Project [a#x, b#x, pipeexpression(outer(x#x), false, EXTEND) AS pipeexpression(outer(spark_catalog.default.t.x))#x] : +- SubqueryAlias spark_catalog.default.other : +- Relation spark_catalog.default.other[a#x,b#x] json -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1002,6 +1003,38 @@ Project [x#x, y#x, z#x] +- Relation spark_catalog.default.t[x#x,y#x] csv +-- !query +values (0), (1) lhs(a) +|> inner join values (1), (2) rhs(a) using (a) +|> extend lhs.a + rhs.a as z1 +|> extend lhs.a - rhs.a as z2 +|> drop z1 +|> where z2 = 0 +|> order by lhs.a, rhs.a, z2 +|> set z2 = 4 +|> limit 2 +|> select lhs.a, rhs.a, z2 +-- !query analysis +Project [a#x, a#x, z2#x] ++- GlobalLimit 2 + +- LocalLimit 2 + +- PipeOperator + +- Project [a#x, pipeexpression(4, false, SET) AS z2#x, a#x] + +- Sort [a#x ASC NULLS FIRST, a#x ASC NULLS FIRST, z2#x ASC NULLS FIRST], true + +- PipeOperator + +- Filter (z2#x = 0) + +- PipeOperator + +- Project [a#x, z2#x, a#x] + +- Project [a#x, z1#x, pipeexpression((a#x - a#x), false, EXTEND) AS z2#x, a#x] + +- Project [a#x, pipeexpression((a#x + a#x), false, EXTEND) AS z1#x, a#x] + +- Project [a#x, a#x, a#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias lhs + : +- LocalRelation [a#x] + +- SubqueryAlias rhs + +- LocalRelation [a#x] + + -- !query table t |> set z = 1 @@ -1241,9 +1274,10 @@ table t |> where u.x = 1 -- !query analysis Filter (x#x = 1) -+- SubqueryAlias u - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias u + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1325,8 +1359,9 @@ table t |> where true -- !query analysis Filter true -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1334,8 +1369,9 @@ table t |> where x + length(y) < 4 -- !query analysis Filter ((x#x + length(y#x)) < 4) -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1344,9 +1380,11 @@ table t |> where x + length(y) < 3 -- !query analysis Filter ((x#x + length(y#x)) < 3) -+- Filter ((x#x + length(y#x)) < 4) - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- Filter ((x#x + length(y#x)) < 4) + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1354,7 +1392,7 @@ Filter ((x#x + length(y#x)) < 3) |> where x = 1 -- !query analysis Filter (x#x = 1) -+- SubqueryAlias __auto_generated_subquery_name ++- PipeOperator +- Aggregate [x#x], [x#x, sum(length(y#x)) AS sum_len#xL] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -1365,8 +1403,9 @@ table t |> where t.x = 1 -- !query analysis Filter (x#x = 1) -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1374,8 +1413,9 @@ table t |> where spark_catalog.default.t.x = 1 -- !query analysis Filter (x#x = 1) -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1383,7 +1423,7 @@ Filter (x#x = 1) |> where col.i1 = 1 -- !query analysis Filter (col#x.i1 = 1) -+- SubqueryAlias __auto_generated_subquery_name ++- PipeOperator +- Project [col#x] +- SubqueryAlias spark_catalog.default.st +- Relation spark_catalog.default.st[x#x,col#x] parquet @@ -1394,8 +1434,9 @@ table st |> where st.col.i1 = 2 -- !query analysis Filter (col#x.i1 = 2) -+- SubqueryAlias spark_catalog.default.st - +- Relation spark_catalog.default.st[x#x,col#x] parquet ++- PipeOperator + +- SubqueryAlias spark_catalog.default.st + +- Relation spark_catalog.default.st[x#x,col#x] parquet -- !query @@ -1409,8 +1450,9 @@ Filter exists#x [x#x] : +- Filter (outer(x#x) = a#x) : +- SubqueryAlias spark_catalog.default.other : +- Relation spark_catalog.default.other[a#x,b#x] json -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1424,8 +1466,9 @@ Filter (scalar-subquery#x [x#x] = 1) : +- Filter (outer(x#x) = a#x) : +- SubqueryAlias spark_catalog.default.other : +- Relation spark_catalog.default.other[a#x,b#x] json -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -1527,7 +1570,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1551,7 +1594,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1575,7 +1618,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1599,7 +1642,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1621,7 +1664,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `sum_len`" + "proposal" : "`sum_len`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -2583,20 +2626,21 @@ table natural_join_test_t1 |> where k = "one" -- !query analysis Filter (k#x = one) -+- Project [k#x, v1#x, v2#x] - +- Join Inner, (k#x = k#x) - :- SubqueryAlias natural_join_test_t1 - : +- View (`natural_join_test_t1`, [k#x, v1#x]) - : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] - : +- Project [k#x, v1#x] - : +- SubqueryAlias natural_join_test_t1 - : +- LocalRelation [k#x, v1#x] - +- SubqueryAlias natural_join_test_t2 - +- View (`natural_join_test_t2`, [k#x, v2#x]) - +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] - +- Project [k#x, v2#x] - +- SubqueryAlias natural_join_test_t2 - +- LocalRelation [k#x, v2#x] ++- PipeOperator + +- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias natural_join_test_t1 + : +- View (`natural_join_test_t1`, [k#x, v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias natural_join_test_t1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias natural_join_test_t2 + +- View (`natural_join_test_t2`, [k#x, v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias natural_join_test_t2 + +- LocalRelation [k#x, v2#x] -- !query @@ -2774,7 +2818,7 @@ values (2, 'xyz') tab(x, y) |> where x = 0 -- !query analysis Filter (x#x = 0) -+- SubqueryAlias __auto_generated_subquery_name ++- PipeOperator +- Distinct +- Union false, false :- SubqueryAlias tab @@ -2932,8 +2976,9 @@ table t |> order by x -- !query analysis Sort [x#x ASC NULLS FIRST], true -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -2941,7 +2986,7 @@ Sort [x#x ASC NULLS FIRST], true |> order by x -- !query analysis Sort [x#x ASC NULLS FIRST], true -+- SubqueryAlias __auto_generated_subquery_name ++- PipeOperator +- Project [x#x, y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -2952,8 +2997,9 @@ values (0, 'abc') tab(x, y) |> order by x -- !query analysis Sort [x#x ASC NULLS FIRST], true -+- SubqueryAlias tab - +- LocalRelation [x#x, y#x] ++- PipeOperator + +- SubqueryAlias tab + +- LocalRelation [x#x, y#x] -- !query @@ -2963,9 +3009,11 @@ table t -- !query analysis GlobalLimit 1 +- LocalLimit 1 - +- Sort [x#x ASC NULLS FIRST], true - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- PipeOperator + +- Sort [x#x ASC NULLS FIRST], true + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -2977,11 +3025,12 @@ table t GlobalLimit 2 +- LocalLimit 2 +- Offset 1 - +- SubqueryAlias __auto_generated_subquery_name + +- PipeOperator +- Project [y#x] +- Filter (x#x = 1) - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -2991,11 +3040,12 @@ table t |> offset 1 -- !query analysis Offset 1 -+- SubqueryAlias __auto_generated_subquery_name ++- PipeOperator +- Project [y#x] +- Filter (x#x = 1) - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -3003,8 +3053,9 @@ table t |> limit all offset 0 -- !query analysis Offset 0 -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -3012,8 +3063,9 @@ table t |> distribute by x -- !query analysis RepartitionByExpression [x#x] -+- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv ++- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -3022,8 +3074,9 @@ table t -- !query analysis Sort [x#x ASC NULLS FIRST], false +- RepartitionByExpression [x#x] - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -3032,8 +3085,9 @@ table t -- !query analysis RepartitionByExpression [x#x] +- Sort [x#x ASC NULLS FIRST], false - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -3043,8 +3097,9 @@ order by y -- !query analysis Sort [y#x ASC NULLS FIRST], true +- Sort [x#x DESC NULLS LAST], true - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- PipeOperator + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query @@ -3312,7 +3367,7 @@ table other |> where a = 1 -- !query analysis Filter (a#x = 1) -+- SubqueryAlias __auto_generated_subquery_name ++- PipeOperator +- Aggregate [a#x], [a#x] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json diff --git a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql index ec4afc6b23721..0cae29d722a8b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql @@ -359,6 +359,18 @@ table t |> extend 1 as z |> set z = first_value(x) over (partition by y); +-- Any prior table aliases remain visible after a SET operator. +values (0), (1) lhs(a) +|> inner join values (1), (2) rhs(a) using (a) +|> extend lhs.a + rhs.a as z1 +|> extend lhs.a - rhs.a as z2 +|> drop z1 +|> where z2 = 0 +|> order by lhs.a, rhs.a, z2 +|> set z2 = 4 +|> limit 2 +|> select lhs.a, rhs.a, z2; + -- SET operators: negative tests. --------------------------------- diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index 3dd212d889f90..0d5ec57b9e479 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -940,6 +940,23 @@ struct 1 def 1 +-- !query +values (0), (1) lhs(a) +|> inner join values (1), (2) rhs(a) using (a) +|> extend lhs.a + rhs.a as z1 +|> extend lhs.a - rhs.a as z2 +|> drop z1 +|> where z2 = 0 +|> order by lhs.a, rhs.a, z2 +|> set z2 = 4 +|> limit 2 +|> select lhs.a, rhs.a, z2 +-- !query schema +struct +-- !query output +1 1 4 + + -- !query table t |> set z = 1 @@ -1481,7 +1498,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1507,7 +1524,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1533,7 +1550,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1559,7 +1576,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `z`" + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", @@ -1583,7 +1600,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42703", "messageParameters" : { "objectName" : "`y`", - "proposal" : "`x`, `sum_len`" + "proposal" : "`sum_len`, `spark_catalog`.`default`.`t`.`x`" }, "queryContext" : [ { "objectType" : "", From ee868b9a827efbb9b4184cac2799aa8e3081c043 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 9 Jan 2025 13:16:02 +0800 Subject: [PATCH 364/438] [SPARK-50561][SQL] Improve type coercion and boundary checking for UNIFORM SQL function ### What changes were proposed in this pull request? This PR improve type coercion and boundary checking for UNIFORM SQL function. srielau found the following issues and wrote them down in SPARK-50561: * TINYINT and BIGINT and DECIMAL types were not supported. * No type coercion from floating-point numbers was implemented. * No explicit error checking for negative numbers was implemented, resulting in weird stacktraces instead. ### Why are the changes needed? This PR fixes the above problems to make the function work in more cases and produce better error messages when it fails. For example: ``` SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), 0.0D) AS result; > 17.605 SELECT uniform(-20L, -10L, 0) AS result > -12 SELECT uniform(0, cast(10 as tinyint), 0) AS result > 7 SELECT uniform(0, cast(10 as bigint), 0) AS result > 7 ``` ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds golden file based test coverage, and updates existing coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49237 from dtenedor/fix-uniform-types. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan --- .../apache/spark/sql/types/UpCastRule.scala | 2 +- .../expressions/randomExpressions.scala | 67 ++-- .../sql-tests/analyzer-results/random.sql.out | 264 +++++++++++++++ .../resources/sql-tests/inputs/random.sql | 25 +- .../sql-tests/results/random.sql.out | 316 +++++++++++++++++- .../spark/sql/DataFrameFunctionsSuite.scala | 2 +- .../ThriftServerQueryTestSuite.scala | 1 + 7 files changed, 629 insertions(+), 48 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala index 1462559625967..97a81f0fe8f35 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala @@ -72,7 +72,7 @@ private[sql] object UpCastRule { case _ => false } - private def legalNumericPrecedence(from: DataType, to: DataType): Boolean = { + def legalNumericPrecedence(from: DataType, to: DataType): Boolean = { val fromPrecedence = numericPrecedence.indexOf(from) val toPrecedence = numericPrecedence.indexOf(to) fromPrecedence >= 0 && fromPrecedence < toPrecedence diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index 7148d3738f7fa..687dd83291be0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -206,7 +206,7 @@ object Randn { since = "4.0.0", group = "math_funcs") case class Uniform(min: Expression, max: Expression, seedExpression: Expression, hideSeed: Boolean) - extends RuntimeReplaceable with TernaryLike[Expression] with RDG { + extends RuntimeReplaceable with TernaryLike[Expression] with RDG with ExpectsInputTypes { def this(min: Expression, max: Expression) = this(min, max, UnresolvedSeed, hideSeed = true) def this(min: Expression, max: Expression, seedExpression: Expression) = @@ -216,63 +216,46 @@ case class Uniform(min: Expression, max: Expression, seedExpression: Expression, override val nodePatterns: Seq[TreePattern] = Seq(RUNTIME_REPLACEABLE, EXPRESSION_WITH_RANDOM_SEED) + override def inputTypes: Seq[AbstractDataType] = { + val randomSeedTypes = TypeCollection(IntegerType, LongType) + Seq(NumericType, NumericType, randomSeedTypes) + } + override def dataType: DataType = { - val first = min.dataType - val second = max.dataType (min.dataType, max.dataType) match { case _ if !seedExpression.resolved || seedExpression.dataType == NullType => NullType - case (_, NullType) | (NullType, _) => NullType - case (_, LongType) | (LongType, _) - if Seq(first, second).forall(integer) => LongType - case (_, IntegerType) | (IntegerType, _) - if Seq(first, second).forall(integer) => IntegerType - case (_, ShortType) | (ShortType, _) - if Seq(first, second).forall(integer) => ShortType - case (_, DoubleType) | (DoubleType, _) => DoubleType - case (_, FloatType) | (FloatType, _) => FloatType + case (left: IntegralType, right: IntegralType) => + if (UpCastRule.legalNumericPrecedence(left, right)) right else left + case (_: NumericType, DoubleType) | (DoubleType, _: NumericType) => DoubleType + case (_: NumericType, FloatType) | (FloatType, _: NumericType) => FloatType + case (lhs: DecimalType, rhs: DecimalType) => if (lhs.isWiderThan(rhs)) lhs else rhs + case (_, d: DecimalType) => d + case (d: DecimalType, _) => d case _ => throw SparkException.internalError( s"Unexpected argument data types: ${min.dataType}, ${max.dataType}") } } - private def integer(t: DataType): Boolean = t match { - case _: ShortType | _: IntegerType | _: LongType => true - case _ => false - } - override def sql: String = { s"uniform(${min.sql}, ${max.sql}${if (hideSeed) "" else s", ${seedExpression.sql}"})" } override def checkInputDataTypes(): TypeCheckResult = { - var result: TypeCheckResult = TypeCheckResult.TypeCheckSuccess + var result: TypeCheckResult = super.checkInputDataTypes() def requiredType = "integer or floating-point" - Seq((min, "min", 0), - (max, "max", 1), - (seedExpression, "seed", 2)).foreach { - case (expr: Expression, name: String, index: Int) => - if (result == TypeCheckResult.TypeCheckSuccess) { - if (!expr.foldable) { - result = DataTypeMismatch( - errorSubClass = "NON_FOLDABLE_INPUT", - messageParameters = Map( - "inputName" -> toSQLId(name), - "inputType" -> requiredType, - "inputExpr" -> toSQLExpr(expr))) - } else expr.dataType match { - case _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | - _: NullType => - case _ => - result = DataTypeMismatch( - errorSubClass = "UNEXPECTED_INPUT_TYPE", - messageParameters = Map( - "paramIndex" -> ordinalNumber(index), - "requiredType" -> requiredType, - "inputSql" -> toSQLExpr(expr), - "inputType" -> toSQLType(expr.dataType))) - } + Seq((min, "min"), + (max, "max"), + (seedExpression, "seed")).foreach { + case (expr: Expression, name: String) => + if (result == TypeCheckResult.TypeCheckSuccess && !expr.foldable) { + result = DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> toSQLId(name), + "inputType" -> requiredType, + "inputExpr" -> toSQLExpr(expr))) } } result diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out index 31919381c99b6..59e903b8682f1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out @@ -119,6 +119,18 @@ SELECT uniform(0, 10L, 0) AS result [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT uniform(0, cast(10 as tinyint), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, cast(10 as smallint), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + -- !query SELECT uniform(0, 10S, 0) AS result -- !query analysis @@ -137,6 +149,30 @@ SELECT uniform(10.0F, 20.0F, 0) AS result [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(11, 4)), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(10, cast(20 as decimal(10, 3)), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), 20, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + -- !query SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result -- !query analysis @@ -161,24 +197,108 @@ SELECT uniform(10, 20.0F) IS NOT NULL AS result [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT uniform(-10L, 10L, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(-20L, -10L, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(-20L, -10L, -10) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + -- !query SELECT uniform(NULL, 1, 0) AS result -- !query analysis [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT uniform(cast(NULL AS int), 1, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(cast(NULL AS float), 1, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + -- !query SELECT uniform(0, NULL, 0) AS result -- !query analysis [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT uniform(0, cast(NULL AS int), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, cast(NULL AS float), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + -- !query SELECT uniform(0, 1, NULL) AS result -- !query analysis [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT uniform(NULL, NULL, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(NULL, NULL, NULL) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, 1, cast(NULL as int)) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT uniform(0, 1, cast(NULL as float)) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(NULL AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(0, 1, CAST(NULL AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "uniform(0, 1, cast(NULL as float))" + } ] +} + + -- !query SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col) -- !query analysis @@ -271,6 +391,150 @@ org.apache.spark.sql.AnalysisException } +-- !query +SELECT uniform(10.0F, 20.0F, 0.0F) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"0.0\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(10.0, 20.0, 0.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "uniform(10.0F, 20.0F, 0.0F)" + } ] +} + + +-- !query +SELECT uniform(10.0F, 20.0F, 0.0D) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"0.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(10.0, 20.0, 0.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "uniform(10.0F, 20.0F, 0.0D)" + } ] +} + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(0 AS DECIMAL(10,3))\"", + "inputType" : "\"DECIMAL(10,3)\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(CAST(10 AS DECIMAL(10,3)), CAST(20 AS DECIMAL(10,3)), CAST(0 AS DECIMAL(10,3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3)))" + } ] +} + + +-- !query +SELECT uniform('abc', 10, 0) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"abc\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"NUMERIC\"", + "sqlExpr" : "\"uniform(abc, 10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "uniform('abc', 10, 0)" + } ] +} + + +-- !query +SELECT uniform(0, 'def', 0) AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"def\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"NUMERIC\"", + "sqlExpr" : "\"uniform(0, def, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "uniform(0, 'def', 0)" + } ] +} + + +-- !query +SELECT uniform(0, 10, 'ghi') AS result +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"ghi\"", + "inputType" : "\"STRING\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(0, 10, ghi)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "uniform(0, 10, 'ghi')" + } ] +} + + -- !query SELECT randstr(1, 0) AS result -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/random.sql b/sql/core/src/test/resources/sql-tests/inputs/random.sql index a71b0293295fc..a17571aa7286f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/random.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/random.sql @@ -22,21 +22,44 @@ SELECT uniform(0, 1, 0) AS result; SELECT uniform(0, 10, 0) AS result; SELECT uniform(0L, 10L, 0) AS result; SELECT uniform(0, 10L, 0) AS result; +SELECT uniform(0, cast(10 as tinyint), 0) AS result; +SELECT uniform(0, cast(10 as smallint), 0) AS result; SELECT uniform(0, 10S, 0) AS result; SELECT uniform(10, 20, 0) AS result; SELECT uniform(10.0F, 20.0F, 0) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), 0) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(11, 4)), 0) AS result; +SELECT uniform(10, cast(20 as decimal(10, 3)), 0) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), 20, 0) AS result; SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result; SELECT uniform(10, 20.0F, 0) AS result; SELECT uniform(10, 20, 0) AS result FROM VALUES (0), (1), (2) tab(col); SELECT uniform(10, 20.0F) IS NOT NULL AS result; --- Negative test cases for the uniform random number generator. +SELECT uniform(-10L, 10L, 0) AS result; +SELECT uniform(-20L, -10L, 0) AS result; +SELECT uniform(-20L, -10L, -10) AS result; SELECT uniform(NULL, 1, 0) AS result; +SELECT uniform(cast(NULL AS int), 1, 0) AS result; +SELECT uniform(cast(NULL AS float), 1, 0) AS result; SELECT uniform(0, NULL, 0) AS result; +SELECT uniform(0, cast(NULL AS int), 0) AS result; +SELECT uniform(0, cast(NULL AS float), 0) AS result; SELECT uniform(0, 1, NULL) AS result; +SELECT uniform(NULL, NULL, 0) AS result; +SELECT uniform(NULL, NULL, NULL) AS result; +-- Negative test cases for the uniform random number generator. +SELECT uniform(0, 1, cast(NULL as int)) AS result; +SELECT uniform(0, 1, cast(NULL as float)) AS result; SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col); SELECT uniform(col, 10, 0) AS result FROM VALUES (0), (1), (2) tab(col); SELECT uniform(10) AS result; SELECT uniform(10, 20, 30, 40) AS result; +SELECT uniform(10.0F, 20.0F, 0.0F) AS result; +SELECT uniform(10.0F, 20.0F, 0.0D) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3))); +SELECT uniform('abc', 10, 0) AS result; +SELECT uniform(0, 'def', 0) AS result; +SELECT uniform(0, 10, 'ghi') AS result; -- The randstr random string generation function supports generating random strings within a -- specified length. We use a seed of zero for these queries to keep tests deterministic. diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out index 01638abdcec6e..eebfac5fc2b19 100644 --- a/sql/core/src/test/resources/sql-tests/results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -147,6 +147,22 @@ struct 7 +-- !query +SELECT uniform(0, cast(10 as tinyint), 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0, cast(10 as smallint), 0) AS result +-- !query schema +struct +-- !query output +7 + + -- !query SELECT uniform(0, 10S, 0) AS result -- !query schema @@ -171,6 +187,38 @@ struct 17.604954 +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), 0) AS result +-- !query schema +struct +-- !query output +17.605 + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(11, 4)), 0) AS result +-- !query schema +struct +-- !query output +17.6050 + + +-- !query +SELECT uniform(10, cast(20 as decimal(10, 3)), 0) AS result +-- !query schema +struct +-- !query output +17.605 + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), 20, 0) AS result +-- !query schema +struct +-- !query output +17.605 + + -- !query SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result -- !query schema @@ -205,10 +253,50 @@ struct true +-- !query +SELECT uniform(-10L, 10L, 0) AS result +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT uniform(-20L, -10L, 0) AS result +-- !query schema +struct +-- !query output +-12 + + +-- !query +SELECT uniform(-20L, -10L, -10) AS result +-- !query schema +struct +-- !query output +-17 + + -- !query SELECT uniform(NULL, 1, 0) AS result -- !query schema -struct +struct +-- !query output +NULL + + +-- !query +SELECT uniform(cast(NULL AS int), 1, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(cast(NULL AS float), 1, 0) AS result +-- !query schema +struct -- !query output NULL @@ -216,7 +304,23 @@ NULL -- !query SELECT uniform(0, NULL, 0) AS result -- !query schema -struct +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, cast(NULL AS int), 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, cast(NULL AS float), 0) AS result +-- !query schema +struct -- !query output NULL @@ -224,11 +328,61 @@ NULL -- !query SELECT uniform(0, 1, NULL) AS result -- !query schema -struct +struct +-- !query output +0 + + +-- !query +SELECT uniform(NULL, NULL, 0) AS result +-- !query schema +struct -- !query output NULL +-- !query +SELECT uniform(NULL, NULL, NULL) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, 1, cast(NULL as int)) AS result +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT uniform(0, 1, cast(NULL as float)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(NULL AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(0, 1, CAST(NULL AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "uniform(0, 1, cast(NULL as float))" + } ] +} + + -- !query SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col) -- !query schema @@ -329,6 +483,162 @@ org.apache.spark.sql.AnalysisException } +-- !query +SELECT uniform(10.0F, 20.0F, 0.0F) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"0.0\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(10.0, 20.0, 0.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "uniform(10.0F, 20.0F, 0.0F)" + } ] +} + + +-- !query +SELECT uniform(10.0F, 20.0F, 0.0D) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"0.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(10.0, 20.0, 0.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "uniform(10.0F, 20.0F, 0.0D)" + } ] +} + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(0 AS DECIMAL(10,3))\"", + "inputType" : "\"DECIMAL(10,3)\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(CAST(10 AS DECIMAL(10,3)), CAST(20 AS DECIMAL(10,3)), CAST(0 AS DECIMAL(10,3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3)))" + } ] +} + + +-- !query +SELECT uniform('abc', 10, 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"abc\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"NUMERIC\"", + "sqlExpr" : "\"uniform(abc, 10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "uniform('abc', 10, 0)" + } ] +} + + +-- !query +SELECT uniform(0, 'def', 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"def\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"NUMERIC\"", + "sqlExpr" : "\"uniform(0, def, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "uniform(0, 'def', 0)" + } ] +} + + +-- !query +SELECT uniform(0, 10, 'ghi') AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"ghi\"", + "inputType" : "\"STRING\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(0, 10, ghi)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "uniform(0, 10, 'ghi')" + } ] +} + + -- !query SELECT randstr(1, 0) AS result -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index ce34db47c6dfa..018aa2159ba1a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -517,7 +517,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { "paramIndex" -> "second", "inputSql" -> "\"a\"", "inputType" -> "\"STRING\"", - "requiredType" -> "integer or floating-point"), + "requiredType" -> "\"NUMERIC\""), context = ExpectedContext( contextType = QueryContextType.DataFrame, fragment = "uniform", diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 283454ad273ed..bc367d0cc856b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -94,6 +94,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ // SPARK-28636 "decimalArithmeticOperations.sql", "literals.sql", + "random.sql", "subquery/scalar-subquery/scalar-subquery-predicate.sql", "subquery/in-subquery/in-limit.sql", "subquery/in-subquery/in-group-by.sql", From f0fb8bf0f81ae47e90d3bad4abd3899f2e3abea3 Mon Sep 17 00:00:00 2001 From: utkarsh39 Date: Thu, 9 Jan 2025 13:40:33 +0800 Subject: [PATCH 365/438] [SPARK-50749][SQL] Fix ordering bug in CommutativeExpression.gatherCommutative method MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? [SPARK-49977](https://issues.apache.org/jira/browse/SPARK-49977) introduced a bug in the `CommutativeExpression.gatherCommutative()` method, changing the function's output order. Consider the following concrete example: ``` val addExpression = Add(   Literal(1),   Add(     Literal(2),     Literal(3)   ) ) val commutativeExpressions = addExpression.gatherCommutative(addExpression,   { case Add(l, r, _) => Seq(l, r)}) ``` Consider the output of the `gatherCommutative` method. [SPARK-49977](https://issues.apache.org/jira/browse/SPARK-49977) introduced a bug that reversed the output order. This PR fixes the bug in `gatherCommutative()` to restore the original correct ordered output. ``` // Prior to [SPARK-49977](https://issues.apache.org/jira/browse/SPARK-49977) and after this fix // commutativeExpressions -> Seq(Literal(1), Literal(2), Literal(3))) // Post [SPARK-49977](https://issues.apache.org/jira/browse/SPARK-49977) and before this fix // commutativeExpressions -> Seq(Literal(3), Literal(2), Literal(1))) ``` ### Why are the changes needed? Fixing a bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a test ### Was this patch authored or co-authored using generative AI tooling? Generated-by: ChatGPT Closes #49392 from utkarsh39/SPARK-50749. Authored-by: utkarsh39 Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/Expression.scala | 17 +++++++++++------ .../expressions/CanonicalizeSuite.scala | 13 +++++++++++++ 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 2090aab3b1f5b..4c83f92509ecd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -1354,19 +1354,24 @@ trait UserDefinedExpression { } trait CommutativeExpression extends Expression { - /** Collects adjacent commutative operations. */ - private def gatherCommutative( + /** + * Collects adjacent commutative operations. + * + * Exposed for testing + */ + private[spark] def gatherCommutative( e: Expression, f: PartialFunction[CommutativeExpression, Seq[Expression]]): Seq[Expression] = { val resultBuffer = scala.collection.mutable.Buffer[Expression]() - val stack = scala.collection.mutable.Stack[Expression](e) + val queue = scala.collection.mutable.Queue[Expression](e) // [SPARK-49977]: Use iterative approach to avoid creating many temporary List objects // for deep expression trees through recursion. - while (stack.nonEmpty) { - stack.pop() match { + while (queue.nonEmpty) { + val current = queue.dequeue() + current match { case c: CommutativeExpression if f.isDefinedAt(c) => - stack.pushAll(f(c)) + queue ++= f(c) case other => resultBuffer += other.canonicalized } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index 7e545d3321054..e0d3a176b1a43 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -479,4 +479,17 @@ class CanonicalizeSuite extends SparkFunSuite { } } } + + test("unit test for gatherCommutative()") { + val addExpression = Add( + Literal(1), + Add( + Literal(2), + Literal(3) + ) + ) + val commutativeExpressions = addExpression.gatherCommutative(addExpression, + { case Add(l, r, _) => Seq(l, r)}) + assert(commutativeExpressions == Seq(Literal(1), Literal(2), Literal(3))) + } } From 0123a5ecbe6d4075b0738e9d2faac354f2cbd008 Mon Sep 17 00:00:00 2001 From: Evan Wu Date: Thu, 9 Jan 2025 15:12:38 +0900 Subject: [PATCH 366/438] [SPARK-50764][PYTHON] Refine the docstring of xpath related methods ### What changes were proposed in this pull request refined docstring for xpath related methods ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? doctests ### Was this patch authored or co-authored using generative AI tooling? No Closes #49422 from drexler-sky/docstring. Authored-by: Evan Wu Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 72 +++++++++++++++++++------ 1 file changed, 56 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 0ff9b17d68a2c..6b74c9dc2dd74 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -11666,10 +11666,15 @@ def xpath(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame( ... [('b1b2b3c1c2',)], ['x']) - >>> df.select(xpath(df.x, lit('a/b/text()')).alias('r')).collect() - [Row(r=['b1', 'b2', 'b3'])] + >>> df.select(sf.xpath(df.x, sf.lit('a/b/text()'))).show() + +--------------------+ + |xpath(x, a/b/text())| + +--------------------+ + | [b1, b2, b3]| + +--------------------+ """ return _invoke_function_over_columns("xpath", xml, path) @@ -11683,9 +11688,14 @@ def xpath_boolean(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('1',)], ['x']) - >>> df.select(xpath_boolean(df.x, lit('a/b')).alias('r')).collect() - [Row(r=True)] + >>> df.select(sf.xpath_boolean(df.x, sf.lit('a/b'))).show() + +---------------------+ + |xpath_boolean(x, a/b)| + +---------------------+ + | true| + +---------------------+ """ return _invoke_function_over_columns("xpath_boolean", xml, path) @@ -11700,9 +11710,14 @@ def xpath_double(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('12',)], ['x']) - >>> df.select(xpath_double(df.x, lit('sum(a/b)')).alias('r')).collect() - [Row(r=3.0)] + >>> df.select(sf.xpath_double(df.x, sf.lit('sum(a/b)'))).show() + +-------------------------+ + |xpath_double(x, sum(a/b))| + +-------------------------+ + | 3.0| + +-------------------------+ """ return _invoke_function_over_columns("xpath_double", xml, path) @@ -11740,9 +11755,14 @@ def xpath_float(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('12',)], ['x']) - >>> df.select(xpath_float(df.x, lit('sum(a/b)')).alias('r')).collect() - [Row(r=3.0)] + >>> df.select(sf.xpath_float(df.x, sf.lit('sum(a/b)'))).show() + +------------------------+ + |xpath_float(x, sum(a/b))| + +------------------------+ + | 3.0| + +------------------------+ """ return _invoke_function_over_columns("xpath_float", xml, path) @@ -11757,9 +11777,14 @@ def xpath_int(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('12',)], ['x']) - >>> df.select(xpath_int(df.x, lit('sum(a/b)')).alias('r')).collect() - [Row(r=3)] + >>> df.select(sf.xpath_int(df.x, sf.lit('sum(a/b)'))).show() + +----------------------+ + |xpath_int(x, sum(a/b))| + +----------------------+ + | 3| + +----------------------+ """ return _invoke_function_over_columns("xpath_int", xml, path) @@ -11774,9 +11799,14 @@ def xpath_long(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('12',)], ['x']) - >>> df.select(xpath_long(df.x, lit('sum(a/b)')).alias('r')).collect() - [Row(r=3)] + >>> df.select(sf.xpath_long(df.x, sf.lit('sum(a/b)'))).show() + +-----------------------+ + |xpath_long(x, sum(a/b))| + +-----------------------+ + | 3| + +-----------------------+ """ return _invoke_function_over_columns("xpath_long", xml, path) @@ -11791,9 +11821,14 @@ def xpath_short(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('12',)], ['x']) - >>> df.select(xpath_short(df.x, lit('sum(a/b)')).alias('r')).collect() - [Row(r=3)] + >>> df.select(sf.xpath_short(df.x, sf.lit('sum(a/b)'))).show() + +------------------------+ + |xpath_short(x, sum(a/b))| + +------------------------+ + | 3| + +------------------------+ """ return _invoke_function_over_columns("xpath_short", xml, path) @@ -11807,9 +11842,14 @@ def xpath_string(xml: "ColumnOrName", path: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('bcc',)], ['x']) - >>> df.select(xpath_string(df.x, lit('a/c')).alias('r')).collect() - [Row(r='cc')] + >>> df.select(sf.xpath_string(df.x, sf.lit('a/c'))).show() + +--------------------+ + |xpath_string(x, a/c)| + +--------------------+ + | cc| + +--------------------+ """ return _invoke_function_over_columns("xpath_string", xml, path) From f8e8fcc835d7916f941948fe64fc54b1c4efdb06 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 9 Jan 2025 17:29:02 +0800 Subject: [PATCH 367/438] [SPARK-50705][SQL] Make `QueryPlan` lock-free ### What changes were proposed in this pull request? Replace a group of `lazy val` in `QueryPlan` with new lock-free helper classes. Not all `lazy val`s are replaced in this PR, we will need to handle remaining `lazy val`s together with its subclasses to make it take effect. ### Why are the changes needed? for the deadlock issues on query plan nodes: - sometimes we want the plan node methods to use a coarse lock (just lock the plan node itself), as these methods (expressions , output , references , deterministic , schema , canonicalized , etc.) may call each other, so using a coarse lock can prevent deadlocks. - sometimes we want these methods to use fine-grained locks, because these methods may call each other of a parent/child plan node. If you traverse the tree with different directions at the same time, it's likely to hit deadlock using coarse lock. the only solution is to not use locks if possible ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit tests Manually test against a deadlock case ### Was this patch authored or co-authored using generative AI tooling? No Closes #49212 from zhengruifeng/query_plan_atom_refs. Lead-authored-by: Ruifeng Zheng Co-authored-by: Josh Rosen Signed-off-by: Wenchen Fan --- ...ientLazy.scala => BestEffortLazyVal.scala} | 42 ++++-- .../util/TransientBestEffortLazyVal.scala | 69 ++++++++++ .../apache/spark/SerializerTestUtils.scala | 44 +++++++ .../spark/util/BestEffortLazyValSuite.scala | 120 ++++++++++++++++++ .../TransientBestEffortLazyValSuite.scala | 116 +++++++++++++++++ .../spark/util/TransientLazySuite.scala | 58 --------- .../spark/sql/catalyst/plans/QueryPlan.scala | 36 ++++-- 7 files changed, 402 insertions(+), 83 deletions(-) rename core/src/main/scala/org/apache/spark/util/{TransientLazy.scala => BestEffortLazyVal.scala} (50%) create mode 100644 core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala create mode 100644 core/src/test/scala/org/apache/spark/SerializerTestUtils.scala create mode 100644 core/src/test/scala/org/apache/spark/util/BestEffortLazyValSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/TransientLazySuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/TransientLazy.scala b/core/src/main/scala/org/apache/spark/util/BestEffortLazyVal.scala similarity index 50% rename from core/src/main/scala/org/apache/spark/util/TransientLazy.scala rename to core/src/main/scala/org/apache/spark/util/BestEffortLazyVal.scala index 2833ef93669a6..83044055fe404 100644 --- a/core/src/main/scala/org/apache/spark/util/TransientLazy.scala +++ b/core/src/main/scala/org/apache/spark/util/BestEffortLazyVal.scala @@ -16,15 +16,21 @@ */ package org.apache.spark.util +import java.util.concurrent.atomic.AtomicReference + /** - * Construct to lazily initialize a variable. - * This may be helpful for avoiding deadlocks in certain scenarios. For example, - * a) Thread 1 entered a synchronized method, grabbing a coarse lock on the parent object. - * b) Thread 2 gets spawned off, and tries to initialize a lazy value on the same parent object - * (in our case, this was the logger). This causes scala to also try to grab a coarse lock on - * the parent object. - * c) If thread 1 waits for thread 2 to join, a deadlock occurs. - * The main difference between this and [[LazyTry]] is that this does not cache failures. + * A lock-free implementation of a lazily-initialized variable. + * If there are concurrent initializations then the `compute()` function may be invoked + * multiple times. However, only a single `compute()` result will be stored and all readers + * will receive the same result object instance. + * + * This may be helpful for avoiding deadlocks in certain scenarios where exactly-once + * value computation is not a hard requirement. + * + * @note + * This helper class has additional requirements on the compute function: + * 1) The compute function MUST not return null; + * 2) The computation failure is not cached. * * @note * Scala 3 uses a different implementation of lazy vals which doesn't have this problem. @@ -32,12 +38,24 @@ package org.apache.spark.util * href="https://docs.scala-lang.org/scala3/reference/changed-features/lazy-vals-init.html">Lazy * Vals Initialization for more details. */ -private[spark] class TransientLazy[T](initializer: => T) extends Serializable { +private[spark] class BestEffortLazyVal[T <: AnyRef]( + @volatile private[this] var compute: () => T) extends Serializable { - @transient - private[this] lazy val value: T = initializer + private[this] val cached: AtomicReference[T] = new AtomicReference(null.asInstanceOf[T]) def apply(): T = { - value + val value = cached.get() + if (value != null) { + value + } else { + val f = compute + if (f != null) { + val newValue = f() + assert(newValue != null, "compute function cannot return null.") + cached.compareAndSet(null.asInstanceOf[T], newValue) + compute = null // allow closure to be GC'd + } + cached.get() + } } } diff --git a/core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala b/core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala new file mode 100644 index 0000000000000..033b783ede40b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/TransientBestEffortLazyVal.scala @@ -0,0 +1,69 @@ +/* + * 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.util + +import java.io.{IOException, ObjectInputStream} +import java.util.concurrent.atomic.AtomicReference + +/** + * A lock-free implementation of a lazily-initialized variable. + * If there are concurrent initializations then the `compute()` function may be invoked + * multiple times. However, only a single `compute()` result will be stored and all readers + * will receive the same result object instance. + * + * This may be helpful for avoiding deadlocks in certain scenarios where exactly-once + * value computation is not a hard requirement. + * + * The main difference between this and [[BestEffortLazyVal]] is that: + * [[BestEffortLazyVal]] serializes the cached value after computation, while + * [[TransientBestEffortLazyVal]] always serializes the compute function. + * + * @note + * This helper class has additional requirements on the compute function: + * 1) The compute function MUST not return null; + * 2) The computation failure is not cached. + * + * @note + * Scala 3 uses a different implementation of lazy vals which doesn't have this problem. + * Please refer to Lazy + * Vals Initialization for more details. + */ +private[spark] class TransientBestEffortLazyVal[T <: AnyRef]( + private[this] val compute: () => T) extends Serializable { + + @transient + private[this] var cached: AtomicReference[T] = new AtomicReference(null.asInstanceOf[T]) + + def apply(): T = { + val value = cached.get() + if (value != null) { + value + } else { + val newValue = compute() + assert(newValue != null, "compute function cannot return null.") + cached.compareAndSet(null.asInstanceOf[T], newValue) + cached.get() + } + } + + @throws(classOf[IOException]) + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { + ois.defaultReadObject() + cached = new AtomicReference(null.asInstanceOf[T]) + } +} diff --git a/core/src/test/scala/org/apache/spark/SerializerTestUtils.scala b/core/src/test/scala/org/apache/spark/SerializerTestUtils.scala new file mode 100644 index 0000000000000..bd81003777317 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SerializerTestUtils.scala @@ -0,0 +1,44 @@ +/* + * 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 + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} + +trait SerializerTestUtils { + + protected def roundtripSerialize[T](obj: T): T = { + deserializeFromBytes(serializeToBytes(obj)) + } + + protected def serializeToBytes[T](o: T): Array[Byte] = { + val baos = new ByteArrayOutputStream + val oos = new ObjectOutputStream(baos) + try { + oos.writeObject(o) + baos.toByteArray + } finally { + oos.close() + } + } + + protected def deserializeFromBytes[T](bytes: Array[Byte]): T = { + val bais = new ByteArrayInputStream(bytes) + val ois = new ObjectInputStream(bais) + ois.readObject().asInstanceOf[T] + } +} diff --git a/core/src/test/scala/org/apache/spark/util/BestEffortLazyValSuite.scala b/core/src/test/scala/org/apache/spark/util/BestEffortLazyValSuite.scala new file mode 100644 index 0000000000000..a6555eca8b859 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/BestEffortLazyValSuite.scala @@ -0,0 +1,120 @@ +/* + * 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.util + +import java.io.NotSerializableException +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicInteger + +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration._ + +import org.apache.spark.{SerializerTestUtils, SparkFunSuite} + +class BestEffortLazyValSuite extends SparkFunSuite with SerializerTestUtils { + + test("BestEffortLazy works") { + val numInitializerCalls = new AtomicInteger(0) + // Simulate a race condition where two threads concurrently + // initialize the lazy value: + val latch = new CountDownLatch(2) + val lazyval = new BestEffortLazyVal(() => { + numInitializerCalls.incrementAndGet() + latch.countDown() + latch.await() + new Object() + }) + + // Ensure no initialization happened before the lazy value was invoked + assert(numInitializerCalls.get() === 0) + + // Two threads concurrently invoke the lazy value + implicit val ec: ExecutionContext = ExecutionContext.global + val future1 = Future { lazyval() } + val future2 = Future { lazyval() } + val value1 = ThreadUtils.awaitResult(future1, 10.seconds) + val value2 = ThreadUtils.awaitResult(future2, 10.seconds) + + // The initializer should have been invoked twice (due to how we set up the + // race condition via the latch): + assert(numInitializerCalls.get() === 2) + + // But the value should only have been computed once: + assert(value1 eq value2) + + // Ensure the subsequent invocation serves the same object + assert(lazyval() eq value1) + assert(numInitializerCalls.get() === 2) + } + + test("BestEffortLazyVal is serializable") { + val lazyval = new BestEffortLazyVal(() => "test") + + // serialize and deserialize before first invocation + val lazyval2 = roundtripSerialize(lazyval) + assert(lazyval2() === "test") + + // first invocation + assert(lazyval() === "test") + + // serialize and deserialize after first invocation + val lazyval3 = roundtripSerialize(lazyval) + assert(lazyval3() === "test") + } + + test("BestEffortLazyVal is serializable: unserializable value") { + val lazyval = new BestEffortLazyVal(() => new Object()) + + // serialize and deserialize before first invocation + val lazyval2 = roundtripSerialize(lazyval) + assert(lazyval2() != null) + + // first invocation + assert(lazyval() != null) + + // serialize and deserialize after first invocation + // try to serialize the cached value and cause NotSerializableException + val e = intercept[NotSerializableException] { + val lazyval3 = roundtripSerialize(lazyval) + } + assert(e.getMessage.contains("java.lang.Object")) + } + + test("BestEffortLazyVal is serializable: initialization failure") { + val lazyval = new BestEffortLazyVal[String](() => throw new RuntimeException("test")) + + // serialize and deserialize before first invocation + val lazyval2 = roundtripSerialize(lazyval) + val e2 = intercept[RuntimeException] { + val v = lazyval2() + } + assert(e2.getMessage.contains("test")) + + // initialization failure + val e = intercept[RuntimeException] { + val v = lazyval() + } + assert(e.getMessage.contains("test")) + + // serialize and deserialize after initialization failure + val lazyval3 = roundtripSerialize(lazyval) + val e3 = intercept[RuntimeException] { + val v = lazyval3() + } + assert(e3.getMessage.contains("test")) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala b/core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala new file mode 100644 index 0000000000000..3ed9f2958fd9c --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/TransientBestEffortLazyValSuite.scala @@ -0,0 +1,116 @@ +/* + * 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.util + +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicInteger + +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration._ + +import org.apache.spark.{SerializerTestUtils, SparkFunSuite} + +class TransientBestEffortLazyValSuite extends SparkFunSuite with SerializerTestUtils { + + test("TransientBestEffortLazyVal works") { + val numInitializerCalls = new AtomicInteger(0) + // Simulate a race condition where two threads concurrently + // initialize the lazy value: + val latch = new CountDownLatch(2) + val lazyval = new TransientBestEffortLazyVal(() => { + numInitializerCalls.incrementAndGet() + latch.countDown() + latch.await() + new Object() + }) + + // Ensure no initialization happened before the lazy value was invoked + assert(numInitializerCalls.get() === 0) + + // Two threads concurrently invoke the lazy value + implicit val ec: ExecutionContext = ExecutionContext.global + val future1 = Future { lazyval() } + val future2 = Future { lazyval() } + val value1 = ThreadUtils.awaitResult(future1, 10.seconds) + val value2 = ThreadUtils.awaitResult(future2, 10.seconds) + + // The initializer should have been invoked twice (due to how we set up the + // race condition via the latch): + assert(numInitializerCalls.get() === 2) + + // But the value should only have been computed once: + assert(value1 eq value2) + + // Ensure the subsequent invocation serves the same object + assert(lazyval() eq value1) + assert(numInitializerCalls.get() === 2) + } + + test("TransientBestEffortLazyVal is serializable") { + val lazyval = new TransientBestEffortLazyVal(() => "test") + + // serialize and deserialize before first invocation + val lazyval2 = roundtripSerialize(lazyval) + assert(lazyval2() === "test") + + // first invocation + assert(lazyval() === "test") + + // serialize and deserialize after first invocation + val lazyval3 = roundtripSerialize(lazyval) + assert(lazyval3() === "test") + } + + test("TransientBestEffortLazyVal is serializable: unserializable value") { + val lazyval = new TransientBestEffortLazyVal(() => new Object()) + + // serialize and deserialize before first invocation + val lazyval2 = roundtripSerialize(lazyval) + assert(lazyval2() != null) + + // first invocation + assert(lazyval() != null) + + // serialize and deserialize after first invocation + val lazyval3 = roundtripSerialize(lazyval) + assert(lazyval3() != null) + } + + test("TransientBestEffortLazyVal is serializable: failure in compute function") { + val lazyval = new TransientBestEffortLazyVal[String](() => throw new RuntimeException("test")) + + // serialize and deserialize before first invocation + val lazyval2 = roundtripSerialize(lazyval) + val e2 = intercept[RuntimeException] { + val v = lazyval2() + } + assert(e2.getMessage.contains("test")) + + // initialization failure + val e = intercept[RuntimeException] { + val v = lazyval() + } + assert(e.getMessage.contains("test")) + + // serialize and deserialize after initialization failure + val lazyval3 = roundtripSerialize(lazyval) + val e3 = intercept[RuntimeException] { + val v = lazyval3() + } + assert(e3.getMessage.contains("test")) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/TransientLazySuite.scala b/core/src/test/scala/org/apache/spark/util/TransientLazySuite.scala deleted file mode 100644 index c0754ee063d67..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/TransientLazySuite.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.util - -import java.io.{ByteArrayOutputStream, ObjectOutputStream} - -import org.apache.spark.SparkFunSuite - -class TransientLazySuite extends SparkFunSuite { - - test("TransientLazy val works") { - var test: Option[Object] = None - - val lazyval = new TransientLazy({ - test = Some(new Object()) - test - }) - - // Ensure no initialization happened before the lazy value was dereferenced - assert(test.isEmpty) - - // Ensure the first invocation creates a new object - assert(lazyval() == test && test.isDefined) - - // Ensure the subsequent invocation serves the same object - assert(lazyval() == test && test.isDefined) - } - - test("TransientLazy val is serializable") { - val lazyval = new TransientLazy({ - new Object() - }) - - // Ensure serializable before the dereference - val oos = new ObjectOutputStream(new ByteArrayOutputStream()) - oos.writeObject(lazyval) - - val dereferenced = lazyval() - - // Ensure serializable after the dereference - val oos2 = new ObjectOutputStream(new ByteArrayOutputStream()) - oos2.writeObject(lazyval) - } -} 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 40244595da57f..9052f6228a9d5 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.plans +import java.lang.{Boolean => JBoolean} import java.util.IdentityHashMap import scala.collection.mutable @@ -32,7 +33,7 @@ import org.apache.spark.sql.catalyst.trees.TreePatternBits import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.util.TransientLazy +import org.apache.spark.util.{BestEffortLazyVal, TransientBestEffortLazyVal} import org.apache.spark.util.collection.BitSet /** @@ -54,8 +55,9 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] /** * Returns the set of attributes that are output by this node. */ - @transient - lazy val outputSet: AttributeSet = AttributeSet(output) + def outputSet: AttributeSet = _outputSet() + + private val _outputSet = new TransientBestEffortLazyVal(() => AttributeSet(output)) /** * Returns the output ordering that this plan generates, although the semantics differ in logical @@ -97,16 +99,17 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] */ def references: AttributeSet = _references() - private val _references = new TransientLazy({ - AttributeSet(expressions) -- producedAttributes - }) + private val _references = new TransientBestEffortLazyVal(() => + AttributeSet(expressions) -- producedAttributes) /** * Returns true when the all the expressions in the current node as well as all of its children * are deterministic */ - lazy val deterministic: Boolean = expressions.forall(_.deterministic) && - children.forall(_.deterministic) + def deterministic: Boolean = _deterministic() + + private val _deterministic = new BestEffortLazyVal[JBoolean](() => + expressions.forall(_.deterministic) && children.forall(_.deterministic)) /** * Attributes that are referenced by expressions but not provided by this node's children. @@ -427,7 +430,10 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] } } - lazy val schema: StructType = DataTypeUtils.fromAttributes(output) + def schema: StructType = _schema() + + private val _schema = new BestEffortLazyVal[StructType](() => + DataTypeUtils.fromAttributes(output)) /** Returns the output schema in the tree format. */ def schemaString: String = schema.treeString @@ -480,11 +486,13 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] /** * All the top-level subqueries of the current plan node. Nested subqueries are not included. */ - @transient lazy val subqueries: Seq[PlanType] = { + def subqueries: Seq[PlanType] = _subqueries() + + private val _subqueries = new TransientBestEffortLazyVal(() => expressions.filter(_.containsPattern(PLAN_EXPRESSION)).flatMap(_.collect { case e: PlanExpression[_] => e.plan.asInstanceOf[PlanType] }) - } + ) /** * All the subqueries of the current plan node and all its children. Nested subqueries are also @@ -620,7 +628,9 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] * Plan nodes that require special canonicalization should override [[doCanonicalize()]]. * They should remove expressions cosmetic variations themselves. */ - @transient final lazy val canonicalized: PlanType = { + def canonicalized: PlanType = _canonicalized() + + private val _canonicalized = new TransientBestEffortLazyVal(() => { var plan = doCanonicalize() // If the plan has not been changed due to canonicalization, make a copy of it so we don't // mutate the original plan's _isCanonicalizedPlan flag. @@ -629,7 +639,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] } plan._isCanonicalizedPlan = true plan - } + }) /** * Defines how the canonicalization should work for the current plan. From a22991700fab447fa9e402e3b853299f88a18ef1 Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Fri, 10 Jan 2025 07:24:49 +0900 Subject: [PATCH 368/438] [SPARK-49883][SS][TESTS][FOLLOWUP] RocksDB Fault Tolerance Test ### What changes were proposed in this pull request? Add a new test that verifies the correct snapshot version is loaded in ckpt v2. ### Why are the changes needed? Test coverage ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Test only addition ### Was this patch authored or co-authored using generative AI tooling? No Closes #49175 from WweiL/test1. Authored-by: Wei Liu Signed-off-by: Jungtaek Lim --- .../streaming/state/RocksDBSuite.scala | 299 ++++++++++++------ 1 file changed, 199 insertions(+), 100 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index fb0a4720f63de..634a3c9de9011 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -553,28 +553,12 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 49) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() if ((version + 1) % 5 == 0) db.doMaintenance() } } - } - - testWithColumnFamilies( - "RocksDB: check changelog and snapshot version", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - val conf = dbConf.copy(minDeltasForSnapshot = 1) - new File(remoteDir).delete() // to make sure that the directory gets created - for (version <- 0 to 49) { - withDB(remoteDir, version = version, conf = conf, - useColumnFamilies = colFamiliesEnabled) { db => - db.put(version.toString, version.toString) - db.commit() - if ((version + 1) % 5 == 0) db.doMaintenance() - } - } if (isChangelogCheckpointingEnabled) { assert(changelogVersionsPresent(remoteDir) === (1 to 50)) @@ -613,7 +597,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => ex = intercept[SparkException] { - db.load(1) + db.load(1, versionToUniqueId.get(1)) } checkError( ex, @@ -728,27 +712,27 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.commit() db.doMaintenance() } // Snapshot should not be created because minDeltasForSnapshot = 3 assert(snapshotVersionsPresent(remoteDir) === Seq.empty) assert(changelogVersionsPresent(remoteDir) == Seq(1, 2)) - db.load(2) + db.load(2, versionToUniqueId.get(2)) db.commit() db.doMaintenance() assert(snapshotVersionsPresent(remoteDir) === Seq(3)) - db.load(3) + db.load(3, versionToUniqueId.get(3)) for (version <- 3 to 7) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.commit() db.doMaintenance() } assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6)) for (version <- 8 to 17) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.commit() } db.doMaintenance() @@ -759,13 +743,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => - db.load(18) + db.load(18, versionToUniqueId.get(18)) db.commit() db.doMaintenance() assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) for (version <- 19 to 20) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.commit() } db.doMaintenance() @@ -785,7 +769,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 2) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } @@ -793,7 +777,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession db.doMaintenance() // Roll back to version 1 and start to process data. for (version <- 1 to 3) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } @@ -805,7 +789,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => // Open the db to verify that the state in 4.zip is no corrupted. - db.load(4) + db.load(4, versionToUniqueId.get(4)) } } @@ -824,7 +808,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { - db.load(version - 1) + db.load(version - 1, versionToUniqueId.get(version - 1)) db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -842,11 +826,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } for (version <- 30 to 60) { - db.load(version - 1) + db.load(version - 1, versionToUniqueId.get(version - 1)) db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -854,13 +838,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(snapshotVersionsPresent(remoteDir) === (1 to 30)) assert(changelogVersionsPresent(remoteDir) === (30 to 60)) for (version <- 1 to 60) { - db.load(version, readOnly = true) + db.load(version, versionToUniqueId.get(version), readOnly = true) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } // recommit 60 to ensure that acquireLock is released for maintenance for (version <- 60 to 60) { - db.load(version - 1) + db.load(version - 1, versionToUniqueId.get(version - 1)) db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -877,12 +861,116 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Verify the content of retained versions. for (version <- 30 to 60) { - db.load(version, readOnly = true) + db.load(version, versionToUniqueId.get(version), readOnly = true) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } } } + testWithChangelogCheckpointingEnabled("RocksDB Fault Tolerance: correctly handle when there " + + "are multiple snapshot files for the same version") { + val enableStateStoreCheckpointIds = true + val useColumnFamily = true + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created + val enableChangelogCheckpointingConf = + dbConf.copy(enableChangelogCheckpointing = true, minVersionsToRetain = 20, + minDeltasForSnapshot = 3) + + // Simulate when there are multiple snapshot files for the same version + // The first DB writes to version 0 with uniqueId + val versionToUniqueId1 = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = enableChangelogCheckpointingConf, + useColumnFamilies = useColumnFamily, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId1) { db => + db.load(0, versionToUniqueId1.get(0)) + db.put("a", "1") // write key a here + db.commit() + + // Add some change log files after the snapshot + for (version <- 2 to 5) { + db.load(version - 1, versionToUniqueId1.get(version - 1)) + db.put(version.toString, version.toString) // update "1" -> "1", "2" -> "2", ... + db.commit() + } + + // doMaintenance uploads the snapshot + db.doMaintenance() + + for (version <- 6 to 10) { + db.load(version - 1, versionToUniqueId1.get(version - 1)) + db.put(version.toString, version.toString) + db.commit() + } + } + + // versionToUniqueId1 should be non-empty, meaning the id is updated from rocksDB to the map + assert(versionToUniqueId1.nonEmpty) + + // The second DB writes to version 0 with another uniqueId + val versionToUniqueId2 = new mutable.HashMap[Long, String]() + withDB(remoteDir, conf = enableChangelogCheckpointingConf, + useColumnFamilies = useColumnFamily, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId2) { db => + db.load(0, versionToUniqueId2.get(0)) + db.put("b", "2") // write key b here + db.commit() + // Add some change log files after the snapshot + for (version <- 2 to 5) { + db.load(version - 1, versionToUniqueId2.get(version - 1)) + db.put(version.toString, (version + 1).toString) // update "1" -> "2", "2" -> "3", ... + db.commit() + } + + // doMaintenance uploads the snapshot + db.doMaintenance() + + for (version <- 6 to 10) { + db.load(version - 1, versionToUniqueId2.get(version - 1)) + db.put(version.toString, (version + 1).toString) + db.commit() + } + } + + // versionToUniqueId2 should be non-empty, meaning the id is updated from rocksDB to the map + assert(versionToUniqueId2.nonEmpty) + + // During a load() with linage from the first rocksDB, + // the DB should load with data in the first db + withDB(remoteDir, conf = enableChangelogCheckpointingConf, + useColumnFamilies = useColumnFamily, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId1) { db => + db.load(10, versionToUniqueId1.get(10)) + assert(toStr(db.get("a")) === "1") + for (version <- 2 to 10) { + // first time we write version -> version + // second time we write version -> version + 1 + // here since we are loading from the first db lineage, we should see version -> version + assert(toStr(db.get(version.toString)) === version.toString) + } + } + + // During a load() with linage from the second rocksDB, + // the DB should load with data in the second db + withDB(remoteDir, conf = enableChangelogCheckpointingConf, + useColumnFamilies = useColumnFamily, + enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, + versionToUniqueId = versionToUniqueId2) { db => + db.load(10, versionToUniqueId2.get(10)) + assert(toStr(db.get("b")) === "2") + for (version <- 2 to 10) { + // first time we write version -> version + // second time we write version -> version + 1 + // here since we are loading from the second db lineage, + // we should see version -> version + 1 + assert(toStr(db.get(version.toString)) === (version + 1).toString) + } + } + } + // A rocksdb instance with changelog checkpointing disabled should be able to load // an existing checkpoint with changelog. testWithStateStoreCheckpointIdsAndColumnFamilies( @@ -899,7 +987,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { - db.load(version - 1) + db.load(version - 1, versionToUniqueId.get(version - 1)) db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -916,11 +1004,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } for (version <- 31 to 60) { - db.load(version - 1) + db.load(version - 1, versionToUniqueId.get(version - 1)) db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -928,7 +1016,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(changelogVersionsPresent(remoteDir) === (1 to 30)) assert(snapshotVersionsPresent(remoteDir) === (31 to 60)) for (version <- 1 to 60) { - db.load(version, readOnly = true) + db.load(version, versionToUniqueId.get(version), readOnly = true) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } // Check that snapshots and changelogs get purged correctly. @@ -937,7 +1025,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(changelogVersionsPresent(remoteDir) === Seq.empty) // Verify the content of retained versions. for (version <- 41 to 60) { - db.load(version, readOnly = true) + db.load(version, versionToUniqueId.get(version), readOnly = true) assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } } @@ -1030,7 +1118,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(toStr(db.get("b")) === "2") assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - db.load(1) + db.load(1, versionToUniqueId.get(1)) assert(toStr(db.get("b")) === null) assert(db.iterator().map(toStr).toSet === Set(("a", "1"))) } @@ -1847,12 +1935,12 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // DB has been loaded so current thread has already // acquired the lock on the RocksDB instance - db.load(0) // Current thread should be able to load again + db.load(0, versionToUniqueId.get(0)) // Current thread should be able to load again // Another thread should not be able to load while current thread is using it var ex = intercept[SparkException] { ThreadUtils.runInNewThread("concurrent-test-thread-1") { - db.load(0) + db.load(0, versionToUniqueId.get(0)) } } checkError( @@ -1872,15 +1960,15 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Commit should release the instance allowing other threads to load new version db.commit() ThreadUtils.runInNewThread("concurrent-test-thread-2") { - db.load(1) + db.load(1, versionToUniqueId.get(1)) db.commit() } // Another thread should not be able to load while current thread is using it - db.load(2) + db.load(2, versionToUniqueId.get(2)) ex = intercept[SparkException] { ThreadUtils.runInNewThread("concurrent-test-thread-2") { - db.load(2) + db.load(2, versionToUniqueId.get(2)) } } checkError( @@ -1900,7 +1988,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Rollback should release the instance allowing other threads to load new version db.rollback() ThreadUtils.runInNewThread("concurrent-test-thread-3") { - db.load(1) + db.load(1, versionToUniqueId.get(1)) db.commit() } } @@ -2354,14 +2442,14 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } // upload snapshot 2.zip db.doMaintenance() for (version <- Seq(2)) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } @@ -2379,16 +2467,16 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } db1.doMaintenance() } - db.load(2) + db.load(2, versionToUniqueId.get(2)) for (version <- Seq(2)) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } // upload snapshot 3.zip db.doMaintenance() // rollback to version 2 - db.load(2) + db.load(2, versionToUniqueId.get(2)) } } @@ -2403,18 +2491,18 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } // upload snapshot 2.zip db.doMaintenance() for (version <- 2 to 3) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } - db.load(0) + db.load(0, versionToUniqueId.get(0)) // simulate db in another executor that override the zip file // In checkpoint V2, reusing the same versionToUniqueId to simulate when two executors // are scheduled with the same uniqueId in the same microbatch @@ -2429,7 +2517,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession db1.doMaintenance() } for (version <- 2 to 3) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } @@ -2450,14 +2538,14 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 2) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } // upload snapshot 2.zip db.doMaintenance() for (version <- 1 to 3) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } @@ -2478,13 +2566,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { - db.load(version) + db.load(version, versionToUniqueId.get(version)) db.put(version.toString, version.toString) db.commit() } // load previous version, and recreate the snapshot - db.load(1) + db.load(1, versionToUniqueId.get(1)) db.put("3", "3") // upload any latest snapshots so far @@ -2514,12 +2602,12 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, hadoopConf = hadoopConf, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => - db.load(0) + db.load(0, versionToUniqueId.get(0)) db.put("a", "1") db.commit() // load previous version, will recreate snapshot on commit - db.load(0) + db.load(0, versionToUniqueId.get(0)) db.put("a", "1") // upload version 1 snapshot created previously @@ -2565,13 +2653,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // This test was accidentally fixed by // SPARK-48931 (https://github.com/apache/spark/pull/47393) - db.load(0) + db.load(0, versionToUniqueId.get(0)) db.put("foo", "bar") // Snapshot checkpoint not needed db.commit() // Continue using local DB - db.load(1) + db.load(1, versionToUniqueId.get(1)) db.put("foo", "bar") // Should create a local RocksDB snapshot db.commit() @@ -2579,19 +2667,19 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession db.doMaintenance() // This will reload Db from the cloud. - db.load(1) + db.load(1, versionToUniqueId.get(1)) db.put("foo", "bar") // Should create another local snapshot db.commit() // Continue using local DB - db.load(2) + db.load(2, versionToUniqueId.get(2)) db.put("foo", "bar") // Snapshot checkpoint not needed db.commit() // Reload DB from the cloud, loading from 2.zip - db.load(2) + db.load(2, versionToUniqueId.get(2)) db.put("foo", "bar") // Snapshot checkpoint not needed db.commit() @@ -2600,14 +2688,14 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession db.doMaintenance() // Reload new 2.zip just uploaded to validate it is not corrupted. - db.load(2) + db.load(2, versionToUniqueId.get(2)) db.put("foo", "bar") db.commit() // Test the maintenance thread is delayed even after the next snapshot is created. // There will be two outstanding snapshots. for (batchVersion <- 3 to 6) { - db.load(batchVersion) + db.load(batchVersion, versionToUniqueId.get(batchVersion)) db.put("foo", "bar") // In batchVersion 3 and 5, it will generate a local snapshot but won't be uploaded. db.commit() @@ -2618,7 +2706,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // maintenance tasks finish quickly. for (batchVersion <- 7 to 10) { for (j <- 0 to 1) { - db.load(batchVersion) + db.load(batchVersion, versionToUniqueId.get(batchVersion)) db.put("foo", "bar") db.commit() db.doMaintenance() @@ -2649,7 +2737,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession val random = new Random(randomSeed) var curVer: Int = 0 for (i <- 1 to 100) { - db.load(curVer) + db.load(curVer, versionToUniqueId.get(curVer)) db.put("foo", "bar") db.commit() // For a one in five chance, maintenance task is executed. The chance is created to @@ -2702,33 +2790,33 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db1 - db1.load(0) + db1.load(0, versionToUniqueId.get(0)) db1.put("a", "1") db1.put("b", "1") db1.commit() // commit version 1 via db2 - db2.load(0) + db2.load(0, versionToUniqueId.get(0)) db2.put("a", "1") db2.put("b", "1") db2.commit() // commit version 2 via db2 - db2.load(1) + db2.load(1, versionToUniqueId.get(1)) db2.put("a", "2") db2.put("b", "2") db2.commit() // reload version 1, this should succeed - db2.load(1) - db1.load(1) + db2.load(1, versionToUniqueId.get(1)) + db1.load(1, versionToUniqueId.get(1)) // reload version 2, this should succeed - db2.load(2) - db1.load(2) + db2.load(2, versionToUniqueId.get(2)) + db1.load(2, versionToUniqueId.get(2)) } } } @@ -2748,33 +2836,33 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db1 - db1.load(0) + db1.load(0, versionToUniqueId.get(0)) db1.put("a", "1") db1.put("b", "1") db1.commit() // commit version 1 via db2 - db2.load(0) + db2.load(0, versionToUniqueId.get(0)) db2.put("a", "1") db2.put("b", "1") db2.commit() // commit version 2 via db2 - db2.load(1) + db2.load(1, versionToUniqueId.get(1)) db2.put("a", "2") db2.put("b", "2") db2.commit() // reload version 1, this should succeed - db2.load(1) - db1.load(1) + db2.load(1, versionToUniqueId.get(1)) + db1.load(1, versionToUniqueId.get(1)) // reload version 2, this should succeed - db2.load(2) - db1.load(2) + db2.load(2, versionToUniqueId.get(2)) + db1.load(2, versionToUniqueId.get(2)) } } } @@ -2798,33 +2886,33 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db2 - db2.load(0) + db2.load(0, versionToUniqueId.get(0)) db2.put("a", "1") db2.put("b", "1") db2.commit() // commit version 1 via db1 - db1.load(0) + db1.load(0, versionToUniqueId.get(0)) db1.put("a", "1") db1.put("b", "1") db1.commit() // commit version 2 via db2 - db2.load(1) + db2.load(1, versionToUniqueId.get(1)) db2.put("a", "2") db2.put("b", "2") db2.commit() // reload version 1, this should succeed - db2.load(1) - db1.load(1) + db2.load(1, versionToUniqueId.get(1)) + db1.load(1, versionToUniqueId.get(1)) // reload version 2, this should succeed - db2.load(2) - db1.load(2) + db2.load(2, versionToUniqueId.get(2)) + db1.load(2, versionToUniqueId.get(2)) } } } @@ -2844,33 +2932,33 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db2 => // commit version 1 via db2 - db2.load(0) + db2.load(0, versionToUniqueId.get(0)) db2.put("a", "1") db2.put("b", "1") db2.commit() // commit version 1 via db1 - db1.load(0) + db1.load(0, versionToUniqueId.get(0)) db1.put("a", "1") db1.put("b", "1") db1.commit() // commit version 2 via db2 - db2.load(1) + db2.load(1, versionToUniqueId.get(1)) db2.put("a", "2") db2.put("b", "2") db2.commit() // reload version 1, this should succeed - db2.load(1) - db1.load(1) + db2.load(1, versionToUniqueId.get(1)) + db1.load(1, versionToUniqueId.get(1)) // reload version 2, this should succeed - db2.load(2) - db1.load(2) + db2.load(2, versionToUniqueId.get(2)) + db1.load(2, versionToUniqueId.get(2)) } } } @@ -3163,7 +3251,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession version: Long, ckptId: Option[String] = None, readOnly: Boolean = false): RocksDB = { - super.load(version, versionToUniqueId.get(version), readOnly) + // When a ckptId is defined, it means the test is explicitly using v2 semantic + // When it is not, it is possible that implicitly uses it. + // So still do a versionToUniqueId.get + ckptId match { + case Some(_) => super.load(version, ckptId, readOnly) + case None => super.load(version, versionToUniqueId.get(version), readOnly) + } } override def commit(): Long = { @@ -3184,6 +3278,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession hadoopConf: Configuration = hadoopConf, useColumnFamilies: Boolean = false, enableStateStoreCheckpointIds: Boolean = false, + // versionToUniqueId is used in checkpoint format v2, it simulates the lineage + // stored in the commit log. The lineage will be automatically updated in db.commit() + // When testing V2, please create a versionToUniqueId map + // and call versionToUniqueId.get(version) in the db.load() function. + // In V1, versionToUniqueId is not used and versionToUniqueId.get(version) returns None. versionToUniqueId : mutable.Map[Long, String] = mutable.Map[Long, String](), localDir: File = Utils.createTempDir())( func: RocksDB => T): T = { @@ -3207,7 +3306,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession loggingId = s"[Thread-${Thread.currentThread.getId}]", useColumnFamilies = useColumnFamilies) } - db.load(version) + db.load(version, versionToUniqueId.get(version)) func(db) } finally { if (db != null) { From 66162360e773f0c37afe1200b4539caff6423143 Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Fri, 10 Jan 2025 09:05:19 +0800 Subject: [PATCH 369/438] [SPARK-48344][SQL] Enhance SQL Script Execution: Replace NOOP with COLLECT for Result DataFrames ### What changes were proposed in this pull request? This pull request proposes replacing the **noop** operation with **collect** for all _result_ DataFrames on the caller side of the SQL Script execution process. This is the 4th PR in the series of introducing SQL Scripting Execution into Spark. ### Why are the changes needed? The proposed change is necessary to maintain a **critical invariant during SQL Script execution**: when `SqlScriptingExecution` returns the next available result statement, it must be executed before proceeding with iteration. ### Implementation details SQL Script execution is based on iterating over interpreted statements and executing them as they are encountered. For certain statement types (_result_ statements), execution is delegated to the caller side (`SparkSession`). To achieve this, the iteration process is divided into two stages: - All Compound Execution Nodes (Begin-End block, control flow structures, loops) implement iterator accessible via the `getTreeIterator` method. - `SqlScriptingExecution` serves as a second-level iterator, iterating over all statements and executing those that **are not** _result_ statements. _Result_ statements are returned to the caller for execution on the caller side. The caller must adhere to the contract of executing the returned statement before continuing iteration. Due to the nature of this contract between the caller and the `SqlScriptingExecution` API, the implementation of the Java Iterator Interface is not feasible. It is expected from caller to call `getNextResult` until it returns `None` We will enforce correct usage of `SqlScriptingExecution` API through the future PR review process. In this approach we collect every DataFrame to eliminate concerns about which one needs to be returned last. This strategy will also be utilized when we introduce multiple-results API functionality. ### Impact and Considerations This change enhances the robustness of our SQL Script execution process and lays the groundwork for future improvements, including the implementation of a multiple-results API. Reviewers should pay particular attention to the handling of DataFrame collection and the maintenance of execution order integrity. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49372 from miland-db/milan-dankovic_data/refactor-execution-3-followup. Authored-by: Milan Dankovic Signed-off-by: Wenchen Fan --- .../logical/SqlScriptingLogicalPlans.scala | 2 ++ .../org/apache/spark/sql/SparkSession.scala | 17 ++++----- .../sql/scripting/SqlScriptingExecution.scala | 36 ++++++++++--------- .../SqlScriptingExecutionContext.scala | 3 +- .../scripting/SqlScriptingExecutionNode.scala | 3 +- .../SqlScriptingExecutionSuite.scala | 12 ++++++- .../SqlScriptingInterpreterSuite.scala | 2 +- 7 files changed, 46 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala index 207c586996fd8..ad00a5216b4c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala @@ -62,6 +62,8 @@ case class SingleStatement(parsedPlan: LogicalPlan) * @param label Label set to CompoundBody by user or UUID otherwise. * It can be None in case when CompoundBody is not part of BeginEndCompoundBlock * for example when CompoundBody is inside loop or conditional block. + * @param isScope Flag indicating if the CompoundBody is a labeled scope. + * Scopes are used for grouping local variables and exception handlers. */ case class CompoundBody( collection: Seq[CompoundPlanStatement], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 878fdc8e267a5..3b36f6b59cb38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -448,16 +448,17 @@ class SparkSession private( val sse = new SqlScriptingExecution(script, this, args) var result: Option[Seq[Row]] = None - while (sse.hasNext) { + // We must execute returned df before calling sse.getNextResult again because sse.hasNext + // advances the script execution and executes all statements until the next result. We must + // collect results immediately to maintain execution order. + // This ensures we respect the contract of SqlScriptingExecution API. + var df: Option[DataFrame] = sse.getNextResult + while (df.isDefined) { sse.withErrorHandling { - val df = sse.next() - if (sse.hasNext) { - df.write.format("noop").mode("overwrite").save() - } else { - // Collect results from the last DataFrame. - result = Some(df.collect().toSeq) - } + // Collect results from the current DataFrame. + result = Some(df.get.collect().toSeq) } + df = sse.getNextResult } if (result.isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala index 71b44cbbd0704..2b15a6c55fa97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecution.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.scripting -import org.apache.spark.SparkException import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, CompoundBody} @@ -25,6 +24,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, CompoundBody} /** * SQL scripting executor - executes script and returns result statements. * This supports returning multiple result statements from a single script. + * The caller of the SqlScriptingExecution API must adhere to the contract of executing + * the returned statement before continuing iteration. Executing the statement needs to be done + * inside withErrorHandling block. * * @param sqlScript CompoundBody which need to be executed. * @param session Spark session that SQL script is executed within. @@ -33,7 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, CompoundBody} class SqlScriptingExecution( sqlScript: CompoundBody, session: SparkSession, - args: Map[String, Expression]) extends Iterator[DataFrame] { + args: Map[String, Expression]) { private val interpreter = SqlScriptingInterpreter(session) @@ -42,7 +44,7 @@ class SqlScriptingExecution( val ctx = new SqlScriptingExecutionContext() val executionPlan = interpreter.buildExecutionPlan(sqlScript, args, ctx) // Add frame which represents SQL Script to the context. - ctx.frames.addOne(new SqlScriptingExecutionFrame(executionPlan.getTreeIterator)) + ctx.frames.append(new SqlScriptingExecutionFrame(executionPlan.getTreeIterator)) // Enter the scope of the top level compound. // We don't need to exit this scope explicitly as it will be done automatically // when the frame is removed during iteration. @@ -50,32 +52,32 @@ class SqlScriptingExecution( ctx } - private var current: Option[DataFrame] = getNextResult - - override def hasNext: Boolean = current.isDefined - - override def next(): DataFrame = { - current match { - case None => throw SparkException.internalError("No more elements to iterate through.") - case Some(result) => - current = getNextResult - result - } - } /** Helper method to iterate get next statements from the first available frame. */ private def getNextStatement: Option[CompoundStatementExec] = { + // Remove frames that are already executed. while (context.frames.nonEmpty && !context.frames.last.hasNext) { context.frames.remove(context.frames.size - 1) } + // If there are still frames available, get the next statement. if (context.frames.nonEmpty) { return Some(context.frames.last.next()) } None } - /** Helper method to iterate through statements until next result statement is encountered. */ - private def getNextResult: Option[DataFrame] = { + /** + * Advances through the script and executes statements until a result statement or + * end of script is encountered. + * + * To know if there is result statement available, the method has to advance through script and + * execute statements until the result statement or end of script is encountered. For that reason + * the returned result must be executed before subsequent calls. Multiple calls without executing + * the intermediate results will lead to incorrect behavior. + * + * @return Result DataFrame if it is available, otherwise None. + */ + def getNextResult: Option[DataFrame] = { var currentStatement = getNextStatement // While we don't have a result statement, execute the statements. while (currentStatement.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala index 5a2ef62e3bb7d..94462ab828f75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionContext.scala @@ -63,7 +63,7 @@ class SqlScriptingExecutionFrame( } def enterScope(label: String): Unit = { - scopes.addOne(new SqlScriptingExecutionScope(label)) + scopes.append(new SqlScriptingExecutionScope(label)) } def exitScope(label: String): Unit = { @@ -76,6 +76,7 @@ class SqlScriptingExecutionFrame( scopes.remove(scopes.length - 1) } + // Remove the scope with the given label. if (scopes.nonEmpty) { scopes.remove(scopes.length - 1) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index ee47491b803bb..58cbfb0feb015 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -181,7 +181,8 @@ class NoOpStatementExec extends LeafStatementExec { * @param label * Label set by user to CompoundBody or None otherwise. * @param isScope - * Flag that indicates whether Compound Body is scope or not. + * Flag indicating if the CompoundBody is a labeled scope. + * Scopes are used for grouping local variables and exception handlers. * @param context * SqlScriptingExecutionContext keeps the execution state of current script. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala index bbeae942f9fe7..5b5285ea13275 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.scripting +import scala.collection.mutable.ListBuffer + import org.apache.spark.SparkConf import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.Expression @@ -43,7 +45,15 @@ class SqlScriptingExecutionSuite extends QueryTest with SharedSparkSession { args: Map[String, Expression] = Map.empty): Seq[Array[Row]] = { val compoundBody = spark.sessionState.sqlParser.parsePlan(sqlText).asInstanceOf[CompoundBody] val sse = new SqlScriptingExecution(compoundBody, spark, args) - sse.map { df => df.collect() }.toList + val result: ListBuffer[Array[Row]] = ListBuffer.empty + + var df = sse.getNextResult + while (df.isDefined) { + // Collect results from the current DataFrame. + result.append(df.get.collect()) + df = sse.getNextResult + } + result.toSeq } private def verifySqlScriptResult(sqlText: String, expected: Seq[Seq[Row]]): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 20997504b15eb..c7439a8934d73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -49,7 +49,7 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { // Initialize context so scopes can be entered correctly. val context = new SqlScriptingExecutionContext() val executionPlan = interpreter.buildExecutionPlan(compoundBody, args, context) - context.frames.addOne(new SqlScriptingExecutionFrame(executionPlan.getTreeIterator)) + context.frames.append(new SqlScriptingExecutionFrame(executionPlan.getTreeIterator)) executionPlan.enterScope() executionPlan.getTreeIterator.flatMap { From 48a81194351e565b881be96d49fbdde0907d4f31 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 10 Jan 2025 10:37:55 +0800 Subject: [PATCH 370/438] [SPARK-50777][CORE] Remove redundant no-op `init/destroy` methods from `Filter` classes ### What changes were proposed in this pull request? This PR aims to remove redundant no-op `init/destroy` methods from `Filter` classes. ### Why are the changes needed? `Filter` interface already provides the default no-op methods for `init` and `destroy`. So, we can clean up them. - https://github.com/jakartaee/servlet/blob/5.0.0-RELEASE/api/src/main/java/jakarta/servlet/Filter.java#L79 - https://github.com/jakartaee/servlet/blob/5.0.0-RELEASE/api/src/main/java/jakarta/servlet/Filter.java#L133 ``` default public void init(FilterConfig filterConfig) throws ServletException {} default public void destroy() {} ``` ### 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 #49429 from dongjoon-hyun/SPARK-50777. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- .../org/apache/spark/deploy/history/ApplicationCache.scala | 7 +------ .../scala/org/apache/spark/ui/HttpSecurityFilter.scala | 4 ---- .../apache/spark/deploy/history/HistoryServerSuite.scala | 5 ----- core/src/test/scala/org/apache/spark/ui/UISuite.scala | 2 -- 4 files changed, 1 insertion(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index 6e0fe69f3bfb6..8caf67ff4680b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import com.codahale.metrics.{Counter, MetricRegistry, Timer} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache, RemovalListener, RemovalNotification} import com.google.common.util.concurrent.UncheckedExecutionException -import jakarta.servlet.{DispatcherType, Filter, FilterChain, FilterConfig, ServletException, ServletRequest, ServletResponse} +import jakarta.servlet.{DispatcherType, Filter, FilterChain, ServletException, ServletRequest, ServletResponse} import jakarta.servlet.http.{HttpServletRequest, HttpServletResponse} import org.eclipse.jetty.servlet.FilterHolder @@ -428,9 +428,4 @@ private[history] class ApplicationCacheCheckFilter( httpResponse.sendRedirect(redirectUrl) } } - - override def init(config: FilterConfig): Unit = { } - - override def destroy(): Unit = { } - } diff --git a/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala index 551f0eb98cb87..cf881b6ea9900 100644 --- a/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala +++ b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala @@ -44,10 +44,6 @@ private class HttpSecurityFilter( conf: SparkConf, securityMgr: SecurityManager) extends Filter { - override def destroy(): Unit = { } - - override def init(config: FilterConfig): Unit = { } - override def doFilter(req: ServletRequest, res: ServletResponse, chain: FilterChain): Unit = { val hreq = req.asInstanceOf[HttpServletRequest] val hres = res.asInstanceOf[HttpServletResponse] diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 6b2bd90cd4314..10092f416f9e1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -794,11 +794,6 @@ object HistoryServerSuite { * A filter used for auth tests; sets the request's user to the value of the "HTTP_USER" header. */ class FakeAuthFilter extends Filter { - - override def destroy(): Unit = { } - - override def init(config: FilterConfig): Unit = { } - override def doFilter(req: ServletRequest, res: ServletResponse, chain: FilterChain): Unit = { val hreq = req.asInstanceOf[HttpServletRequest] val wrapped = new HttpServletRequestWrapper(hreq) { diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 1b68ed301fb92..6d12e88e8efac 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -504,8 +504,6 @@ private[spark] class TestFilter extends Filter { private var rc: Int = HttpServletResponse.SC_OK - override def destroy(): Unit = { } - override def init(config: FilterConfig): Unit = { if (config.getInitParameter("responseCode") != null) { rc = config.getInitParameter("responseCode").toInt From e638f6d20f25cecce91ed907fc04294220abe23c Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Fri, 10 Jan 2025 10:47:37 +0800 Subject: [PATCH 371/438] [SPARK-50778][PYTHON] Add metadataColumn to PySpark DataFrame ### What changes were proposed in this pull request? Add `metadataColumn` to PySpark DataFrame. ### Why are the changes needed? Feature parity: The API is missing in PySpark. ### Does this PR introduce _any_ user-facing change? Yes, the new API will be available. ### How was this patch tested? Added the related test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49430 from ueshin/issues/SPARK-50778/metadata_column. Authored-by: Takuya Ueshin Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/classic/dataframe.py | 9 +++++++++ python/pyspark/sql/connect/dataframe.py | 11 ++++++++++- python/pyspark/sql/connect/expressions.py | 10 +++++++++- python/pyspark/sql/dataframe.py | 22 ++++++++++++++++++++++ python/pyspark/sql/tests/test_dataframe.py | 20 ++++++++++++++++++++ 5 files changed, 70 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py index 8beabda6c135d..84498f1b2294d 100644 --- a/python/pyspark/sql/classic/dataframe.py +++ b/python/pyspark/sql/classic/dataframe.py @@ -651,6 +651,15 @@ def dtypes(self) -> List[Tuple[str, str]]: def columns(self) -> List[str]: return [f.name for f in self.schema.fields] + def metadataColumn(self, colName: str) -> Column: + if not isinstance(colName, str): + raise PySparkTypeError( + errorClass="NOT_STR", + messageParameters={"arg_name": "colName", "arg_type": type(colName).__name__}, + ) + jc = self._jdf.metadataColumn(colName) + return Column(jc) + def colRegex(self, colName: str) -> Column: if not isinstance(colName, str): raise PySparkTypeError( diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 789292bdd56f6..76b7881f234ff 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -274,6 +274,14 @@ def alias(self, alias: str) -> ParentDataFrame: res._cached_schema = self._cached_schema return res + def metadataColumn(self, colName: str) -> Column: + if not isinstance(colName, str): + raise PySparkTypeError( + errorClass="NOT_STR", + messageParameters={"arg_name": "colName", "arg_type": type(colName).__name__}, + ) + return self._col(colName, is_metadata_column=True) + def colRegex(self, colName: str) -> Column: from pyspark.sql.connect.column import Column as ConnectColumn @@ -1750,13 +1758,14 @@ def __getitem__( messageParameters={"arg_name": "item", "arg_type": type(item).__name__}, ) - def _col(self, name: str) -> Column: + def _col(self, name: str, is_metadata_column: bool = False) -> Column: from pyspark.sql.connect.column import Column as ConnectColumn return ConnectColumn( ColumnReference( unparsed_identifier=name, plan_id=self._plan._plan_id, + is_metadata_column=is_metadata_column, ) ) diff --git a/python/pyspark/sql/connect/expressions.py b/python/pyspark/sql/connect/expressions.py index 413a69181683b..c32db14968c6b 100644 --- a/python/pyspark/sql/connect/expressions.py +++ b/python/pyspark/sql/connect/expressions.py @@ -524,7 +524,12 @@ class ColumnReference(Expression): treat it as an unresolved attribute. Attributes that have the same fully qualified name are identical""" - def __init__(self, unparsed_identifier: str, plan_id: Optional[int] = None) -> None: + def __init__( + self, + unparsed_identifier: str, + plan_id: Optional[int] = None, + is_metadata_column: bool = False, + ) -> None: super().__init__() assert isinstance(unparsed_identifier, str) self._unparsed_identifier = unparsed_identifier @@ -532,6 +537,8 @@ def __init__(self, unparsed_identifier: str, plan_id: Optional[int] = None) -> N assert plan_id is None or isinstance(plan_id, int) self._plan_id = plan_id + self._is_metadata_column = is_metadata_column + def name(self) -> str: """Returns the qualified name of the column reference.""" return self._unparsed_identifier @@ -542,6 +549,7 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression: expr.unresolved_attribute.unparsed_identifier = self._unparsed_identifier if self._plan_id is not None: expr.unresolved_attribute.plan_id = self._plan_id + expr.unresolved_attribute.is_metadata_column = self._is_metadata_column return expr def __repr__(self) -> str: diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 394ac6bdb69c9..f2c0bc8155821 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -2277,6 +2277,28 @@ def columns(self) -> List[str]: """ ... + @dispatch_df_method + def metadataColumn(self, colName: str) -> Column: + """ + Selects a metadata column based on its logical column name and returns it as a + :class:`Column`. + + A metadata column can be accessed this way even if the underlying data source defines a data + column with a conflicting name. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + colName : str + string, metadata column name + + Returns + ------- + :class:`Column` + """ + ... + @dispatch_df_method def colRegex(self, colName: str) -> Column: """ diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index a0234a527f639..e85877cc87e09 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -1056,6 +1056,26 @@ def test_transpose_with_invalid_index_columns(self): messageParameters={"reason": "Index column must be an atomic attribute"}, ) + def test_metadata_column(self): + with self.sql_conf( + {"spark.sql.catalog.testcat": "org.apache.spark.sql.connector.catalog.InMemoryCatalog"} + ): + tbl = "testcat.t" + with self.table(tbl): + self.spark.sql( + f""" + CREATE TABLE {tbl} (index bigint, data string) + PARTITIONED BY (bucket(4, index), index) + """ + ) + self.spark.sql(f"""INSERT INTO {tbl} VALUES (1, 'a'), (2, 'b'), (3, 'c')""") + + df = self.spark.sql(f"""SELECT * FROM {tbl}""") + assertDataFrameEqual( + df.select(df.metadataColumn("index")), + [Row(0), Row(0), Row(0)], + ) + class DataFrameTests(DataFrameTestsMixin, ReusedSQLTestCase): def test_query_execution_unsupported_in_classic(self): From a8bec11af9417956f8552d2dfb72c9afc80a7671 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 10 Jan 2025 11:21:50 +0800 Subject: [PATCH 372/438] [SPARK-49565][SQL] Improve auto-generated expression aliases with pipe SQL operators ### What changes were proposed in this pull request? This RP improves auto-generated expression aliases with pipe SQL operators. For example, consider the pipe SQL syntax query: ``` table t |> extend 1 ``` Previously, the analyzed plan was: ``` Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS pipeexpression(1)#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv ``` After this PR, it is: ``` Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS 1#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv ``` Note that the output aliases visible in the resulting DataFrame for the query derive from the `AS ` part of the analyzed plans shown. ### Why are the changes needed? This improves the user experience with pipe SQL syntax. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? Existing golden file tests update to show the improved aliases. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49245 from dtenedor/fix-pipe-output-aliases. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 1 + .../catalyst/expressions/pipeOperators.scala | 67 +++--- .../sql/catalyst/rules/RuleIdCollection.scala | 1 + .../sql/catalyst/trees/TreePatterns.scala | 1 + .../analyzer-results/pipe-operators.sql.out | 203 +++++++++--------- .../sql-tests/results/pipe-operators.sql.out | 26 +-- 6 files changed, 161 insertions(+), 138 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e8839148f51b9..9282e0554a2d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -373,6 +373,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolveProcedures :: BindProcedures :: ResolveTableSpec :: + ValidateAndStripPipeExpressions :: ResolveAliases :: ResolveSubquery :: ResolveSubqueryColumnAliases :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala index 40d7d24263a78..2ee68663ad2fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.TreePattern.{PIPE_OPERATOR, TreePattern} +import org.apache.spark.sql.catalyst.trees.TreePattern.{PIPE_EXPRESSION, PIPE_OPERATOR, TreePattern} import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.types.DataType /** * Represents an expression when used with a SQL pipe operator. @@ -33,31 +34,12 @@ import org.apache.spark.sql.errors.QueryCompilationErrors * @param clause The clause of the pipe operator. This is used to generate error messages. */ case class PipeExpression(child: Expression, isAggregate: Boolean, clause: String) - extends UnaryExpression with RuntimeReplaceable { + extends UnaryExpression with Unevaluable { + final override val nodePatterns = Seq(PIPE_EXPRESSION) + final override lazy val resolved = false override def withNewChildInternal(newChild: Expression): Expression = PipeExpression(newChild, isAggregate, clause) - override lazy val replacement: Expression = { - val firstAggregateFunction: Option[AggregateFunction] = findFirstAggregate(child) - if (isAggregate && firstAggregateFunction.isEmpty) { - throw QueryCompilationErrors.pipeOperatorAggregateExpressionContainsNoAggregateFunction(child) - } else if (!isAggregate) { - firstAggregateFunction.foreach { a => - throw QueryCompilationErrors.pipeOperatorContainsAggregateFunction(a, clause) - } - } - child - } - - /** Returns the first aggregate function in the given expression, or None if not found. */ - private def findFirstAggregate(e: Expression): Option[AggregateFunction] = e match { - case a: AggregateFunction => - Some(a) - case _: WindowExpression => - // Window functions are allowed in these pipe operators, so do not traverse into children. - None - case _ => - e.children.flatMap(findFirstAggregate).headOption - } + override def dataType: DataType = child.dataType } /** @@ -79,6 +61,43 @@ object EliminatePipeOperators extends Rule[LogicalPlan] { } } +/** + * Validates and strips PipeExpression nodes from a logical plan once the child expressions are + * resolved. + */ +object ValidateAndStripPipeExpressions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning( + _.containsPattern(PIPE_EXPRESSION), ruleId) { + case node: LogicalPlan => + node.resolveExpressions { + case p: PipeExpression if p.child.resolved => + // Once the child expression is resolved, we can perform the necessary invariant checks + // and then remove this expression, replacing it with the child expression instead. + val firstAggregateFunction: Option[AggregateFunction] = findFirstAggregate(p.child) + if (p.isAggregate && firstAggregateFunction.isEmpty) { + throw QueryCompilationErrors + .pipeOperatorAggregateExpressionContainsNoAggregateFunction(p.child) + } else if (!p.isAggregate) { + firstAggregateFunction.foreach { a => + throw QueryCompilationErrors.pipeOperatorContainsAggregateFunction(a, p.clause) + } + } + p.child + } + } + + /** Returns the first aggregate function in the given expression, or None if not found. */ + private def findFirstAggregate(e: Expression): Option[AggregateFunction] = e match { + case a: AggregateFunction => + Some(a) + case _: WindowExpression => + // Window functions are allowed in these pipe operators, so do not traverse into children. + None + case _ => + e.children.flatMap(findFirstAggregate).headOption + } +} + object PipeOperators { // These are definitions of query result clauses that can be used with the pipe operator. val aggregateClause = "AGGREGATE" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index 3f79e74b18a45..ee5245054bcca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -109,6 +109,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.UpdateAttributeNullability" :: "org.apache.spark.sql.catalyst.analysis.ResolveUpdateEventTimeWatermarkColumn" :: "org.apache.spark.sql.catalyst.expressions.EliminatePipeOperators" :: + "org.apache.spark.sql.catalyst.expressions.ValidateAndStripPipeExpressions" :: // Catalyst Optimizer rules "org.apache.spark.sql.catalyst.optimizer.BooleanSimplification" :: "org.apache.spark.sql.catalyst.optimizer.CollapseProject" :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 25ef341b8cef3..b56085ecae8d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -79,6 +79,7 @@ object TreePattern extends Enumeration { val OUTER_REFERENCE: Value = Value val PARAMETER: Value = Value val PARAMETERIZED_QUERY: Value = Value + val PIPE_EXPRESSION: Value = Value val PIPE_OPERATOR: Value = Value val PIVOT: Value = Value val PLAN_EXPRESSION: Value = Value diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index ac74fea1dbfba..b231199cc4732 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -283,7 +283,7 @@ SubqueryAlias spark_catalog.default.t from t |> select 1 as x -- !query analysis -Project [pipeexpression(1, false, SELECT) AS x#x] +Project [1 AS x#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -307,7 +307,7 @@ from t as t_alias Project [tx#x] +- Filter (ty#x = def) +- PipeOperator - +- Project [pipeexpression(x#x, false, SELECT) AS tx#x, pipeexpression(y#x, false, SELECT) AS ty#x] + +- Project [x#x AS tx#x, y#x AS ty#x] +- SubqueryAlias t_alias +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -317,7 +317,7 @@ Project [tx#x] from t, other |> select t.x + other.a as z -- !query analysis -Project [pipeexpression((x#x + a#x), false, SELECT) AS z#x] +Project [(x#x + a#x) AS z#x] +- Join Inner :- SubqueryAlias spark_catalog.default.t : +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -329,7 +329,7 @@ Project [pipeexpression((x#x + a#x), false, SELECT) AS z#x] from t join other on (t.x = other.a) |> select t.x + other.a as z -- !query analysis -Project [pipeexpression((x#x + a#x), false, SELECT) AS z#x] +Project [(x#x + a#x) AS z#x] +- Join Inner, (x#x = a#x) :- SubqueryAlias spark_catalog.default.t : +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -341,7 +341,7 @@ Project [pipeexpression((x#x + a#x), false, SELECT) AS z#x] from t lateral view explode(array(100, 101)) as ly |> select t.x + ly as z -- !query analysis -Project [pipeexpression((x#x + ly#x), false, SELECT) AS z#x] +Project [(x#x + ly#x) AS z#x] +- Generate explode(array(100, 101)), false, as, [ly#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -370,7 +370,7 @@ Project [col#x.i1 AS i1#x] from values (0), (1) tab(col) |> select col as x -- !query analysis -Project [pipeexpression(col#x, false, SELECT) AS x#x] +Project [col#x AS x#x] +- SubqueryAlias tab +- LocalRelation [col#x] @@ -394,7 +394,7 @@ org.apache.spark.sql.catalyst.parser.ParseException table t |> select 1 as x -- !query analysis -Project [pipeexpression(1, false, SELECT) AS x#x] +Project [1 AS x#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -413,7 +413,7 @@ table t |> select x, y |> select x + length(y) as z -- !query analysis -Project [pipeexpression((x#x + length(y#x)), false, SELECT) AS z#x] +Project [(x#x + length(y#x)) AS z#x] +- Project [x#x, y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -423,7 +423,7 @@ Project [pipeexpression((x#x + length(y#x)), false, SELECT) AS z#x] values (0), (1) tab(col) |> select col * 2 as result -- !query analysis -Project [pipeexpression((col#x * 2), false, SELECT) AS result#x] +Project [(col#x * 2) AS result#x] +- SubqueryAlias tab +- LocalRelation [col#x] @@ -432,7 +432,7 @@ Project [pipeexpression((col#x * 2), false, SELECT) AS result#x] (select * from t union all select * from t) |> select x + length(y) as result -- !query analysis -Project [pipeexpression((x#x + length(y#x)), false, SELECT) AS result#x] +Project [(x#x + length(y#x)) AS result#x] +- Union false, false :- Project [x#x, y#x] : +- SubqueryAlias spark_catalog.default.t @@ -483,7 +483,7 @@ Project [col#x.i1 AS i1#x] table t |> select (select a from other where x = a limit 1) as result -- !query analysis -Project [pipeexpression(scalar-subquery#x [x#x], false, SELECT) AS result#x] +Project [scalar-subquery#x [x#x] AS result#x] : +- GlobalLimit 1 : +- LocalLimit 1 : +- Project [a#x] @@ -508,7 +508,7 @@ Project [scalar-subquery#x [] AS result#x] table t |> select (select any_value(a) from other where x = a limit 1) as result -- !query analysis -Project [pipeexpression(scalar-subquery#x [x#x], false, SELECT) AS result#x] +Project [scalar-subquery#x [x#x] AS result#x] : +- GlobalLimit 1 : +- LocalLimit 1 : +- Aggregate [any_value(a#x, false) AS any_value(a)#x] @@ -523,8 +523,8 @@ Project [pipeexpression(scalar-subquery#x [x#x], false, SELECT) AS result#x] table t |> select x + length(x) as z, z + 1 as plus_one -- !query analysis -Project [z#x, pipeexpression((z#x + 1), false, SELECT) AS plus_one#x] -+- Project [x#x, y#x, pipeexpression((x#x + length(cast(x#x as string))), false, SELECT) AS z#x] +Project [z#x, (z#x + 1) AS plus_one#x] ++- Project [x#x, y#x, (x#x + length(cast(x#x as string))) AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -534,8 +534,8 @@ table t |> select first_value(x) over (partition by y) as result -- !query analysis Project [result#x] -+- Project [x#x, y#x, _we0#x, pipeexpression(_we0#x, false, SELECT) AS result#x] - +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#x], [y#x] ++- Project [x#x, y#x, result#x, result#x] + +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS result#x], [y#x] +- Project [x#x, y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -551,8 +551,8 @@ select 1 x, 2 y, 3 z -- !query analysis Project [a2#x] +- Project [(1 + sum(x) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, x#x, a2#x] - +- Project [x#x, y#x, _w1#x, z#x, _we0#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, _we2#x, (cast(1 as bigint) + _we0#xL) AS (1 + sum(x) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, pipeexpression(_we2#x, false, SELECT) AS a2#x] - +- Window [avg(_w1#x) windowspecdefinition(y#x, z#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS _we2#x], [y#x], [z#x ASC NULLS FIRST] + +- Project [x#x, y#x, _w1#x, z#x, _we0#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, a2#x, (cast(1 as bigint) + _we0#xL) AS (1 + sum(x) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, a2#x] + +- Window [avg(_w1#x) windowspecdefinition(y#x, z#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS a2#x], [y#x], [z#x ASC NULLS FIRST] +- Window [sum(x#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#xL, avg(y#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] +- Project [x#x, y#x, (x#x + 1) AS _w1#x, z#x] +- Project [1 AS x#x, 2 AS y#x, 3 AS z#x] @@ -680,7 +680,7 @@ org.apache.spark.sql.AnalysisException table t |> extend 1 as z -- !query analysis -Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] +Project [x#x, y#x, 1 AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -689,7 +689,7 @@ Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] table t |> extend 1 -- !query analysis -Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS pipeexpression(1)#x] +Project [x#x, y#x, 1 AS 1#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -698,7 +698,7 @@ Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS pipeexpression(1)#x] table t |> extend x as z -- !query analysis -Project [x#x, y#x, pipeexpression(x#x, false, EXTEND) AS z#x] +Project [x#x, y#x, x#x AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -707,7 +707,7 @@ Project [x#x, y#x, pipeexpression(x#x, false, EXTEND) AS z#x] table t |> extend x + length(y) as z -- !query analysis -Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x] +Project [x#x, y#x, (x#x + length(y#x)) AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -716,7 +716,7 @@ Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x] table t |> extend x + length(y) as z, x + 1 as zz -- !query analysis -Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x, pipeexpression((x#x + 1), false, EXTEND) AS zz#x] +Project [x#x, y#x, (x#x + length(y#x)) AS z#x, (x#x + 1) AS zz#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -726,8 +726,8 @@ table t |> extend x + length(y) as z |> extend z + 1 as zz -- !query analysis -Project [x#x, y#x, z#x, pipeexpression((z#x + 1), false, EXTEND) AS zz#x] -+- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x] +Project [x#x, y#x, z#x, (z#x + 1) AS zz#x] ++- Project [x#x, y#x, (x#x + length(y#x)) AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -736,7 +736,7 @@ Project [x#x, y#x, z#x, pipeexpression((z#x + 1), false, EXTEND) AS zz#x] select col from st |> extend col.i1 as z -- !query analysis -Project [col#x, pipeexpression(col#x.i1, false, EXTEND) AS z#x] +Project [col#x, col#x.i1 AS z#x] +- Project [col#x] +- SubqueryAlias spark_catalog.default.st +- Relation spark_catalog.default.st[x#x,col#x] parquet @@ -746,7 +746,7 @@ Project [col#x, pipeexpression(col#x.i1, false, EXTEND) AS z#x] table t |> extend (select a from other where x = a limit 1) as z -- !query analysis -Project [x#x, y#x, pipeexpression(scalar-subquery#x [x#x], false, EXTEND) AS z#x] +Project [x#x, y#x, scalar-subquery#x [x#x] AS z#x] : +- GlobalLimit 1 : +- LocalLimit 1 : +- Project [a#x] @@ -765,8 +765,8 @@ table t |> select * except (a, b)) -- !query analysis Filter exists#x [x#x] -: +- Project [pipeexpression(outer(spark_catalog.default.t.x))#x] -: +- Project [a#x, b#x, pipeexpression(outer(x#x), false, EXTEND) AS pipeexpression(outer(spark_catalog.default.t.x))#x] +: +- Project [x#x] +: +- Project [a#x, b#x, outer(x#x)] : +- SubqueryAlias spark_catalog.default.other : +- Relation spark_catalog.default.other[a#x,b#x] json +- PipeOperator @@ -778,7 +778,7 @@ Filter exists#x [x#x] table t |> extend 1 as x -- !query analysis -Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS x#x] +Project [x#x, y#x, 1 AS x#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -788,8 +788,8 @@ table t |> extend first_value(x) over (partition by y) as result -- !query analysis Project [x#x, y#x, result#x] -+- Project [x#x, y#x, _we0#x, pipeexpression(_we0#x, false, EXTEND) AS result#x] - +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#x], [y#x] ++- Project [x#x, y#x, result#x, result#x] + +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS result#x], [y#x] +- Project [x#x, y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -799,8 +799,8 @@ Project [x#x, y#x, result#x] table t |> extend x + length(y) as z, z + 1 as plus_one -- !query analysis -Project [x#x, y#x, z#x, pipeexpression((z#x + 1), false, EXTEND) AS plus_one#x] -+- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x] +Project [x#x, y#x, z#x, (z#x + 1) AS plus_one#x] ++- Project [x#x, y#x, (x#x + length(y#x)) AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -861,7 +861,7 @@ org.apache.spark.sql.AnalysisException table t |> set x = 1 -- !query analysis -Project [pipeexpression(1, false, SET) AS x#x, y#x] +Project [1 AS x#x, y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -870,7 +870,7 @@ Project [pipeexpression(1, false, SET) AS x#x, y#x] table t |> set y = x -- !query analysis -Project [x#x, pipeexpression(x#x, false, SET) AS y#x] +Project [x#x, x#x AS y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -880,8 +880,8 @@ table t |> extend 1 as z |> set z = x + length(y) -- !query analysis -Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x] -+- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] +Project [x#x, y#x, (x#x + length(y#x)) AS z#x] ++- Project [x#x, y#x, 1 AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -892,10 +892,10 @@ table t |> extend 2 as zz |> set z = x + length(y), zz = x + 1 -- !query analysis -Project [x#x, y#x, z#x, pipeexpression((x#x + 1), false, SET) AS zz#x] -+- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x, zz#x] - +- Project [x#x, y#x, z#x, pipeexpression(2, false, EXTEND) AS zz#x] - +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] +Project [x#x, y#x, z#x, (x#x + 1) AS zz#x] ++- Project [x#x, y#x, (x#x + length(y#x)) AS z#x, zz#x] + +- Project [x#x, y#x, z#x, 2 AS zz#x] + +- Project [x#x, y#x, 1 AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -905,9 +905,9 @@ table other |> extend 3 as c |> set a = b, b = c -- !query analysis -Project [a#x, pipeexpression(c#x, false, SET) AS b#x, c#x] -+- Project [pipeexpression(b#x, false, SET) AS a#x, b#x, c#x] - +- Project [a#x, b#x, pipeexpression(3, false, EXTEND) AS c#x] +Project [a#x, c#x AS b#x, c#x] ++- Project [b#x AS a#x, b#x, c#x] + +- Project [a#x, b#x, 3 AS c#x] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -918,10 +918,10 @@ table t |> extend 2 as zz |> set z = x + length(y), zz = z + 1 -- !query analysis -Project [x#x, y#x, z#x, pipeexpression((z#x + 1), false, SET) AS zz#x] -+- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x, zz#x] - +- Project [x#x, y#x, z#x, pipeexpression(2, false, EXTEND) AS zz#x] - +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] +Project [x#x, y#x, z#x, (z#x + 1) AS zz#x] ++- Project [x#x, y#x, (x#x + length(y#x)) AS z#x, zz#x] + +- Project [x#x, y#x, z#x, 2 AS zz#x] + +- Project [x#x, y#x, 1 AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -932,9 +932,9 @@ table t |> set z = x + length(y) |> set z = z + 1 -- !query analysis -Project [x#x, y#x, pipeexpression((z#x + 1), false, SET) AS z#x] -+- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x] - +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] +Project [x#x, y#x, (z#x + 1) AS z#x] ++- Project [x#x, y#x, (x#x + length(y#x)) AS z#x] + +- Project [x#x, y#x, 1 AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -944,9 +944,9 @@ table t |> extend 1 as z |> set z = x + length(y), z = z + 1 -- !query analysis -Project [x#x, y#x, pipeexpression((z#x + 1), false, SET) AS z#x] -+- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS z#x] - +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] +Project [x#x, y#x, (z#x + 1) AS z#x] ++- Project [x#x, y#x, (x#x + length(y#x)) AS z#x] + +- Project [x#x, y#x, 1 AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -956,8 +956,8 @@ select col from st |> extend 1 as z |> set z = col.i1 -- !query analysis -Project [col#x, pipeexpression(col#x.i1, false, SET) AS z#x] -+- Project [col#x, pipeexpression(1, false, EXTEND) AS z#x] +Project [col#x, col#x.i1 AS z#x] ++- Project [col#x, 1 AS z#x] +- Project [col#x] +- SubqueryAlias spark_catalog.default.st +- Relation spark_catalog.default.st[x#x,col#x] parquet @@ -967,7 +967,7 @@ Project [col#x, pipeexpression(col#x.i1, false, SET) AS z#x] table t |> set y = (select a from other where x = a limit 1) -- !query analysis -Project [x#x, pipeexpression(scalar-subquery#x [x#x], false, SET) AS y#x] +Project [x#x, scalar-subquery#x [x#x] AS y#x] : +- GlobalLimit 1 : +- LocalLimit 1 : +- Project [a#x] @@ -983,8 +983,8 @@ table t |> extend 1 as `x.y.z` |> set `x.y.z` = x + length(y) -- !query analysis -Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, SET) AS x.y.z#x] -+- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS x.y.z#x] +Project [x#x, y#x, (x#x + length(y#x)) AS x.y.z#x] ++- Project [x#x, y#x, 1 AS x.y.z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -995,10 +995,10 @@ table t |> set z = first_value(x) over (partition by y) -- !query analysis Project [x#x, y#x, z#x] -+- Project [x#x, y#x, _we0#x, pipeexpression(_we0#x, false, SET) AS z#x] - +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#x], [y#x] ++- Project [x#x, y#x, z#x, z#x] + +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS z#x], [y#x] +- Project [x#x, y#x] - +- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] + +- Project [x#x, y#x, 1 AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -1019,20 +1019,21 @@ Project [a#x, a#x, z2#x] +- GlobalLimit 2 +- LocalLimit 2 +- PipeOperator - +- Project [a#x, pipeexpression(4, false, SET) AS z2#x, a#x] - +- Sort [a#x ASC NULLS FIRST, a#x ASC NULLS FIRST, z2#x ASC NULLS FIRST], true - +- PipeOperator - +- Filter (z2#x = 0) - +- PipeOperator - +- Project [a#x, z2#x, a#x] - +- Project [a#x, z1#x, pipeexpression((a#x - a#x), false, EXTEND) AS z2#x, a#x] - +- Project [a#x, pipeexpression((a#x + a#x), false, EXTEND) AS z1#x, a#x] - +- Project [a#x, a#x, a#x] - +- Join Inner, (a#x = a#x) - :- SubqueryAlias lhs - : +- LocalRelation [a#x] - +- SubqueryAlias rhs - +- LocalRelation [a#x] + +- Project [a#x, 4 AS z2#x, a#x] + +- Project [a#x, z2#x, a#x] + +- Sort [a#x ASC NULLS FIRST, a#x ASC NULLS FIRST, z2#x ASC NULLS FIRST], true + +- PipeOperator + +- Filter (z2#x = 0) + +- PipeOperator + +- Project [a#x, z2#x, a#x, a#x] + +- Project [a#x, z1#x, (a#x - a#x) AS z2#x, a#x, a#x] + +- Project [a#x, (a#x + a#x) AS z1#x, a#x, a#x, a#x] + +- Project [a#x, a#x, a#x, a#x, a#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias lhs + : +- LocalRelation [a#x] + +- SubqueryAlias rhs + +- LocalRelation [a#x] -- !query @@ -1137,7 +1138,7 @@ table t |> drop `x.y.z` -- !query analysis Project [x#x, y#x] -+- Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS x.y.z#x] ++- Project [x#x, y#x, 1 AS x.y.z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -1707,7 +1708,7 @@ table courseSales Project [c#x, __pivot_sum(e) AS s AS `sum(e) AS s`#x[0] AS firstYear_s#xL, __pivot_avg(e) AS a AS `avg(e) AS a`#x[0] AS firstYear_a#x, __pivot_sum(e) AS s AS `sum(e) AS s`#x[1] AS secondYear_s#xL, __pivot_avg(e) AS a AS `avg(e) AS a`#x[1] AS secondYear_a#x] +- Aggregate [c#x], [c#x, pivotfirst(y#x, sum(e) AS s#xL, 2012, 2013, 0, 0) AS __pivot_sum(e) AS s AS `sum(e) AS s`#x, pivotfirst(y#x, avg(e) AS a#x, 2012, 2013, 0, 0) AS __pivot_avg(e) AS a AS `avg(e) AS a`#x] +- Aggregate [c#x, y#x], [c#x, y#x, sum(e#x) AS sum(e) AS s#xL, avg(e#x) AS avg(e) AS a#x] - +- Project [pipeexpression(year#x, false, SELECT) AS y#x, pipeexpression(course#x, false, SELECT) AS c#x, pipeexpression(earnings#x, false, SELECT) AS e#x] + +- Project [year#x AS y#x, course#x AS c#x, earnings#x AS e#x] +- SubqueryAlias coursesales +- View (`courseSales`, [course#x, year#x, earnings#x]) +- Project [cast(course#x as string) AS course#x, cast(year#x as int) AS year#x, cast(earnings#x as int) AS earnings#x] @@ -3208,7 +3209,7 @@ org.apache.spark.sql.catalyst.parser.ParseException table other |> aggregate sum(b) as result group by a -- !query analysis -Aggregate [a#x], [a#x, pipeexpression(sum(b#x), true, AGGREGATE) AS result#xL] +Aggregate [a#x], [a#x, sum(b#x) AS result#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -3219,7 +3220,7 @@ table other |> select result -- !query analysis Project [result#xL] -+- Aggregate [a#x], [a#x, pipeexpression(sum(b#x), true, AGGREGATE) AS result#xL] ++- Aggregate [a#x], [a#x, sum(b#x) AS result#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -3230,7 +3231,7 @@ table other |> select gkey -- !query analysis Project [gkey#x] -+- Aggregate [(a#x + 1)], [(a#x + 1) AS gkey#x, pipeexpression(sum(b#x), true, AGGREGATE) AS pipeexpression(sum(b))#xL] ++- Aggregate [(a#x + 1)], [(a#x + 1) AS gkey#x, sum(b#x) AS sum(b)#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -3257,7 +3258,7 @@ Aggregate [x#x, y#x], [x#x, y#x] values (3, 4) as tab(x, y) |> aggregate sum(y) group by 1 -- !query analysis -Aggregate [x#x], [x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [x#x], [x#x, sum(y#x) AS sum(y)#xL] +- SubqueryAlias tab +- LocalRelation [x#x, y#x] @@ -3266,7 +3267,7 @@ Aggregate [x#x], [x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpressi values (3, 4), (5, 4) as tab(x, y) |> aggregate sum(y) group by 1 -- !query analysis -Aggregate [x#x], [x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [x#x], [x#x, sum(y#x) AS sum(y)#xL] +- SubqueryAlias tab +- LocalRelation [x#x, y#x] @@ -3275,7 +3276,7 @@ Aggregate [x#x], [x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpressi select 3 as x, 4 as y |> aggregate sum(y) group by 1, 1 -- !query analysis -Aggregate [x#x, x#x], [x#x, x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [x#x, x#x], [x#x, x#x, sum(y#x) AS sum(y)#xL] +- Project [3 AS x#x, 4 AS y#x] +- OneRowRelation @@ -3284,7 +3285,7 @@ Aggregate [x#x, x#x], [x#x, x#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pi select 1 as `1`, 2 as `2` |> aggregate sum(`2`) group by `1` -- !query analysis -Aggregate [1#x], [1#x, pipeexpression(sum(2#x), true, AGGREGATE) AS pipeexpression(sum(2))#xL] +Aggregate [1#x], [1#x, sum(2#x) AS sum(2)#xL] +- Project [1 AS 1#x, 2 AS 2#x] +- OneRowRelation @@ -3293,7 +3294,7 @@ Aggregate [1#x], [1#x, pipeexpression(sum(2#x), true, AGGREGATE) AS pipeexpressi select 3 as x, 4 as y |> aggregate sum(y) group by 2 -- !query analysis -Aggregate [y#x], [y#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [y#x], [y#x, sum(y#x) AS sum(y)#xL] +- Project [3 AS x#x, 4 AS y#x] +- OneRowRelation @@ -3302,7 +3303,7 @@ Aggregate [y#x], [y#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpressi select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 2 -- !query analysis -Aggregate [y#x], [y#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [y#x], [y#x, sum(y#x) AS sum(y)#xL] +- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] +- OneRowRelation @@ -3311,7 +3312,7 @@ Aggregate [y#x], [y#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpressi select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 3 -- !query analysis -Aggregate [z#x], [z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [z#x], [z#x, sum(y#x) AS sum(y)#xL] +- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] +- OneRowRelation @@ -3320,7 +3321,7 @@ Aggregate [z#x], [z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpressi select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 2, 3 -- !query analysis -Aggregate [y#x, z#x], [y#x, z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [y#x, z#x], [y#x, z#x, sum(y#x) AS sum(y)#xL] +- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] +- OneRowRelation @@ -3329,7 +3330,7 @@ Aggregate [y#x, z#x], [y#x, z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pi select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 1, 2, 3 -- !query analysis -Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, sum(y#x) AS sum(y)#xL] +- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] +- OneRowRelation @@ -3338,7 +3339,7 @@ Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, pipeexpression(sum(y#x), true, AGGREG select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by x, 2, 3 -- !query analysis -Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, pipeexpression(sum(y#x), true, AGGREGATE) AS pipeexpression(sum(y))#xL] +Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, sum(y#x) AS sum(y)#xL] +- Project [3 AS x#x, 4 AS y#x, 5 AS z#x] +- OneRowRelation @@ -3347,7 +3348,7 @@ Aggregate [x#x, y#x, z#x], [x#x, y#x, z#x, pipeexpression(sum(y#x), true, AGGREG table t |> aggregate sum(x) -- !query analysis -Aggregate [pipeexpression(sum(x#x), true, AGGREGATE) AS pipeexpression(sum(x))#xL] +Aggregate [sum(x#x) AS sum(x)#xL] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -3356,7 +3357,7 @@ Aggregate [pipeexpression(sum(x#x), true, AGGREGATE) AS pipeexpression(sum(x))#x table t |> aggregate sum(x) + 1 as result_plus_one -- !query analysis -Aggregate [pipeexpression((sum(x#x) + cast(1 as bigint)), true, AGGREGATE) AS result_plus_one#xL] +Aggregate [(sum(x#x) + cast(1 as bigint)) AS result_plus_one#xL] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -3406,9 +3407,9 @@ select 1 x, 2 y, 3 z |> aggregate avg(z) z group by x |> aggregate count(distinct z) c -- !query analysis -Aggregate [pipeexpression(count(distinct z#x), true, AGGREGATE) AS c#xL] -+- Aggregate [x#x], [x#x, pipeexpression(avg(z#xL), true, AGGREGATE) AS z#x] - +- Aggregate [x#x, y#x], [x#x, y#x, pipeexpression(sum(z#x), true, AGGREGATE) AS z#xL] +Aggregate [count(distinct z#x) AS c#xL] ++- Aggregate [x#x], [x#x, avg(z#xL) AS z#x] + +- Aggregate [x#x, y#x], [x#x, y#x, sum(z#x) AS z#xL] +- Project [1 AS x#x, 2 AS y#x, 3 AS z#x] +- OneRowRelation @@ -3419,7 +3420,7 @@ select 1 x, 3 z |> select x -- !query analysis Project [x#x] -+- Aggregate [x#x, z#x, x#x], [x#x, z#x, x#x, pipeexpression(count(1), true, AGGREGATE) AS pipeexpression(count(1))#xL] ++- Aggregate [x#x, z#x, x#x], [x#x, z#x, x#x, count(1) AS count(1)#xL] +- Project [1 AS x#x, 3 AS z#x] +- OneRowRelation @@ -3428,7 +3429,7 @@ Project [x#x] table other |> aggregate a + count(b) group by a -- !query analysis -Aggregate [a#x], [a#x, pipeexpression((cast(a#x as bigint) + count(b#x)), true, AGGREGATE) AS pipeexpression((a + count(b)))#xL] +Aggregate [a#x], [a#x, (cast(a#x as bigint) + count(b#x)) AS (a + count(b))#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -3807,8 +3808,8 @@ Project [cate#x, val#x, sum_val#xL, first_value(cate) OVER (ORDER BY val ASC NUL +- Window [first_value(cate#x, false) windowspecdefinition(val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value(cate) OVER (ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [val#x ASC NULLS FIRST] +- Project [cate#x, val#x, sum_val#xL] +- Project [cate#x, val#x, sum_val#xL] - +- Project [cate#x, val#x, _we0#xL, pipeexpression(_we0#xL, false, SELECT) AS sum_val#xL] - +- Window [sum(val#x) windowspecdefinition(cate#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#xL], [cate#x] + +- Project [cate#x, val#x, sum_val#xL, sum_val#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum_val#xL], [cate#x] +- Project [cate#x, val#x] +- SubqueryAlias windowtestdata +- View (`windowTestData`, [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x]) diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index 0d5ec57b9e479..fe1a263c06441 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -641,7 +641,7 @@ struct table t |> extend 1 -- !query schema -struct +struct -- !query output 0 abc 1 1 def 1 @@ -2879,7 +2879,7 @@ struct values (3, 4) as tab(x, y) |> aggregate sum(y) group by 1 -- !query schema -struct +struct -- !query output 3 4 @@ -2888,7 +2888,7 @@ struct values (3, 4), (5, 4) as tab(x, y) |> aggregate sum(y) group by 1 -- !query schema -struct +struct -- !query output 3 4 5 4 @@ -2898,7 +2898,7 @@ struct select 3 as x, 4 as y |> aggregate sum(y) group by 1, 1 -- !query schema -struct +struct -- !query output 3 3 4 @@ -2907,7 +2907,7 @@ struct select 1 as `1`, 2 as `2` |> aggregate sum(`2`) group by `1` -- !query schema -struct<1:int,pipeexpression(sum(2)):bigint> +struct<1:int,sum(2):bigint> -- !query output 1 2 @@ -2916,7 +2916,7 @@ struct<1:int,pipeexpression(sum(2)):bigint> select 3 as x, 4 as y |> aggregate sum(y) group by 2 -- !query schema -struct +struct -- !query output 4 4 @@ -2925,7 +2925,7 @@ struct select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 2 -- !query schema -struct +struct -- !query output 4 4 @@ -2934,7 +2934,7 @@ struct select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 3 -- !query schema -struct +struct -- !query output 5 4 @@ -2943,7 +2943,7 @@ struct select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 2, 3 -- !query schema -struct +struct -- !query output 4 5 4 @@ -2952,7 +2952,7 @@ struct select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by 1, 2, 3 -- !query schema -struct +struct -- !query output 3 4 5 4 @@ -2961,7 +2961,7 @@ struct select 3 as x, 4 as y, 5 as z |> aggregate sum(y) group by x, 2, 3 -- !query schema -struct +struct -- !query output 3 4 5 4 @@ -2970,7 +2970,7 @@ struct table t |> aggregate sum(x) -- !query schema -struct +struct -- !query output 1 @@ -3046,7 +3046,7 @@ struct table other |> aggregate a + count(b) group by a -- !query schema -struct +struct -- !query output 1 3 2 3 From 68305acc01fc0614a554f76316b35065f56f6e0f Mon Sep 17 00:00:00 2001 From: jdesjean Date: Fri, 10 Jan 2025 12:19:11 +0800 Subject: [PATCH 373/438] [SPARK-50600][CONNECT][SQL] Set analyzed on analysis failure ### What changes were proposed in this pull request? As part of [SPARK-44145](https://issues.apache.org/jira/browse/SPARK-44145), a callback was added to track completion of analysis and optimization phase of a query. While the analyzed plan is sent when analysis completes successfully it does not when it fail. In that case, we should fallback to the ParsedPlan. ### Why are the changes needed? The purpose of the analyze event is to track when analysis completes, as such it should also be sent on both success & failure. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit ### Was this patch authored or co-authored using generative AI tooling? No Closes #49383 from jdesjean/jdesjean/SPARK-50600. Authored-by: jdesjean Signed-off-by: Wenchen Fan --- .../sql/catalyst/QueryPlanningTracker.scala | 21 +++++++++++++ .../catalyst/QueryPlanningTrackerSuite.scala | 6 ++++ .../service/ExecuteEventsManager.scala | 24 ++++++++++++-- .../spark/sql/execution/QueryExecution.scala | 16 +++++++--- .../sql/execution/QueryExecutionSuite.scala | 31 +++++++++++++++++-- 5 files changed, 87 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala index 2e14c09bc8193..d1007404158f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala @@ -94,6 +94,16 @@ object QueryPlanningTracker { * Callbacks after planning phase completion. */ abstract class QueryPlanningTrackerCallback { + /** + * Called when query fails analysis + * + * @param tracker tracker that triggered the callback. + * @param parsedPlan The plan prior to analysis + * see @org.apache.spark.sql.catalyst.analysis.Analyzer + */ + def analysisFailed(tracker: QueryPlanningTracker, parsedPlan: LogicalPlan): Unit = { + // Noop by default for backward compatibility + } /** * Called when query has been analyzed. * @@ -147,6 +157,17 @@ class QueryPlanningTracker( ret } + /** + * Set when the query has been parsed but failed to be analyzed. + * Can be called multiple times upon plan change. + * + * @param parsedPlan The plan prior analysis + * see @org.apache.spark.sql.catalyst.analysis.Analyzer + */ + private[sql] def setAnalysisFailed(parsedPlan: LogicalPlan): Unit = { + trackerCallback.foreach(_.analysisFailed(this, parsedPlan)) + } + /** * Set when the query has been analysed. * Can be called multiple times upon plan change. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala index 972b98780bcca..500bbef3c89bf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala @@ -95,7 +95,13 @@ class QueryPlanningTrackerSuite extends SparkFunSuite { val mockCallback = mock[QueryPlanningTrackerCallback] val mockPlan1 = mock[LogicalPlan] val mockPlan2 = mock[LogicalPlan] + val mockPlan3 = mock[LogicalPlan] + val mockPlan4 = mock[LogicalPlan] val t = new QueryPlanningTracker(Some(mockCallback)) + t.setAnalysisFailed(mockPlan3) + verify(mockCallback, times(1)).analysisFailed(t, mockPlan3) + t.setAnalysisFailed(mockPlan4) + verify(mockCallback, times(1)).analysisFailed(t, mockPlan4) t.setAnalyzed(mockPlan1) verify(mockCallback, times(1)).analyzed(t, mockPlan1) t.setAnalyzed(mockPlan2) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala index faa7582d169f1..61cd95621d156 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala @@ -145,13 +145,19 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { * * @param analyzedPlan * The analyzed plan generated by the Connect request plan. None when the request does not - * generate a plan. + * generate a Spark plan or analysis fails. + * @param parsedPlan + * The analyzed plan generated by the Connect request plan. None when the request does not + * generate a plan or does not fail analysis. */ - def postAnalyzed(analyzedPlan: Option[LogicalPlan] = None): Unit = { + def postAnalyzed( + analyzedPlan: Option[LogicalPlan] = None, + parsedPlan: Option[LogicalPlan] = None): Unit = { assertStatus(List(ExecuteStatus.Started, ExecuteStatus.Analyzed), ExecuteStatus.Analyzed) val event = SparkListenerConnectOperationAnalyzed(jobTag, operationId, clock.getTimeMillis()) event.analyzedPlan = analyzedPlan + event.parsedPlan = parsedPlan listenerBus.post(event) } @@ -251,6 +257,12 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { postAnalyzed(Some(analyzedPlan)) } + override def analysisFailed( + tracker: QueryPlanningTracker, + parsedPlan: LogicalPlan): Unit = { + postAnalyzed(parsedPlan = Some(parsedPlan)) + } + def readyForExecution(tracker: QueryPlanningTracker): Unit = postReadyForExecution() })) } @@ -341,9 +353,15 @@ case class SparkListenerConnectOperationAnalyzed( extraTags: Map[String, String] = Map.empty) extends SparkListenerEvent { + /** + * Parsed Spark plan generated by the Connect request. None when the Connect request does not + * generate a Spark plan or does not fail analysis. + */ + @JsonIgnore var parsedPlan: Option[LogicalPlan] = None + /** * Analyzed Spark plan generated by the Connect request. None when the Connect request does not - * generate a Spark plan. + * generate a Spark plan or analysis fails. */ @JsonIgnore var analyzedPlan: Option[LogicalPlan] = None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 5695ea57e7fbc..d9b1a2136a5d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -92,12 +92,18 @@ class QueryExecution( } private val lazyAnalyzed = LazyTry { - val plan = executePhase(QueryPlanningTracker.ANALYSIS) { - // We can't clone `logical` here, which will reset the `_analyzed` flag. - sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) + try { + val plan = executePhase(QueryPlanningTracker.ANALYSIS) { + // We can't clone `logical` here, which will reset the `_analyzed` flag. + sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) + } + tracker.setAnalyzed(plan) + plan + } catch { + case NonFatal(e) => + tracker.setAnalysisFailed(logical) + throw e } - tracker.setAnalyzed(plan) - plan } def analyzed: LogicalPlan = lazyAnalyzed.get diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 974be2f627998..d670b3d8c77d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.execution import scala.collection.mutable import scala.io.Source +import scala.util.Try import org.apache.spark.sql.{AnalysisException, Dataset, ExtendedExplainGenerator, FastOperator} -import org.apache.spark.sql.catalyst.{QueryPlanningTracker, QueryPlanningTrackerCallback} -import org.apache.spark.sql.catalyst.analysis.CurrentNamespace -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.{QueryPlanningTracker, QueryPlanningTrackerCallback, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{CurrentNamespace, UnresolvedFunction, UnresolvedRelation} +import org.apache.spark.sql.catalyst.expressions.{Alias, UnsafeRow} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, LogicalPlan, OneRowRelation, Project, ShowTables, SubqueryAlias} import org.apache.spark.sql.catalyst.trees.TreeNodeTag @@ -405,6 +406,21 @@ class QueryExecutionSuite extends SharedSparkSession { } } + test("SPARK-50600: Failed analysis should send analyzed event") { + val mockCallback = MockCallback() + + def table(ref: String): LogicalPlan = UnresolvedRelation(TableIdentifier(ref)) + + val unresolvedUndefinedFunc = UnresolvedFunction("unknown", Seq.empty, isDistinct = false) + val plan = Project(Seq(Alias(unresolvedUndefinedFunc, "call1")()), table("table")) + val dataset = Try { + val df = Dataset.ofRows(spark, plan, new QueryPlanningTracker(Some(mockCallback))) + df.queryExecution.assertAnalyzed() + } + assert(dataset.failed.get.isInstanceOf[AnalysisException]) + mockCallback.assertAnalyzed() + } + case class MockCallbackEagerCommand( var trackerAnalyzed: QueryPlanningTracker = null, var trackerReadyForExecution: QueryPlanningTracker = null) @@ -447,6 +463,15 @@ class QueryExecutionSuite extends SharedSparkSession { var trackerAnalyzed: QueryPlanningTracker = null, var trackerReadyForExecution: QueryPlanningTracker = null) extends QueryPlanningTrackerCallback { + override def analysisFailed( + trackerFromCallback: QueryPlanningTracker, + analyzedPlan: LogicalPlan): Unit = { + trackerAnalyzed = trackerFromCallback + assert(!trackerAnalyzed.phases.keySet.contains(QueryPlanningTracker.ANALYSIS)) + assert(!trackerAnalyzed.phases.keySet.contains(QueryPlanningTracker.OPTIMIZATION)) + assert(!trackerAnalyzed.phases.keySet.contains(QueryPlanningTracker.PLANNING)) + assert(analyzedPlan != null) + } def analyzed(trackerFromCallback: QueryPlanningTracker, plan: LogicalPlan): Unit = { trackerAnalyzed = trackerFromCallback assert(trackerAnalyzed.phases.keySet.contains(QueryPlanningTracker.ANALYSIS)) From a4f2870b08551031ace305a953ace23a6aa6e71a Mon Sep 17 00:00:00 2001 From: Dima Date: Fri, 10 Jan 2025 12:20:37 +0800 Subject: [PATCH 374/438] [SPARK-50525][SQL] Define InsertMapSortInRepartitionExpressions Optimizer Rule ### What changes were proposed in this pull request? In the current version of Spark, its possible to use `MapType` as column for repartitioning. But `MapData` does not implement `equals` and `hashCode` (in according to [SPARK-9415](https://issues.apache.org/jira/browse/SPARK-9415) and [[SPARK-16135][SQL] Remove hashCode and equals in ArrayBasedMapData](https://github.com/apache/spark/pull/13847)). Considering that, hash value for same Maps can be different. In an attempt to run `xxhash64` or `hash` function on `MapType`, ```org.apache.spark.sql.catalyst.ExtendedAnalysisException: [DATATYPE_MISMATCH.HASH_MAP_TYPE] Cannot resolve "xxhash64(value)" due to data type mismatch: Input to the function `xxhash64` cannot contain elements of the "MAP" type. In Spark, same maps may have different hashcode, thus hash expressions are prohibited on "MAP" elements. To restore previous behavior set "spark.sql.legacy.allowHashOnMapType" to "true".;``` will be thrown. Also, when trying to run `ds.distinct(col("value"))`, where `value` has `MapType`, the following exception is thrown: ```org.apache.spark.sql.catalyst.ExtendedAnalysisException: [UNSUPPORTED_FEATURE.SET_OPERATION_ON_MAP_TYPE] The feature is not supported: Cannot have MAP type columns in DataFrame which calls set operations (INTERSECT, EXCEPT, etc.), but the type of column `value` is "MAP".;``` With the above consideration, a new `InsertMapSortInRepartitionExpressions` `Rule[LogicalPlan]` was implemented to insert `mapsort` for every `MapType` in `RepartitionByExpression.partitionExpressions`. ### Why are the changes needed? To keep `repartition` API for MapType consistent. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49144 from ostronaut/features/map_repartition. Authored-by: Dima Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- ...ns.scala => InsertMapSortExpression.scala} | 66 ++++++++++++++----- .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../org/apache/spark/sql/DataFrameSuite.scala | 37 +++++++++-- 4 files changed, 84 insertions(+), 22 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/{InsertMapSortInGroupingExpressions.scala => InsertMapSortExpression.scala} (69%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 6cd394fd79e95..46ca8e793218b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -884,7 +884,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB o.failAnalysis( errorClass = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", messageParameters = Map( - "expr" -> variantExpr.sql, + "expr" -> toSQLExpr(variantExpr), "dataType" -> toSQLType(variantExpr.dataType))) case o if o.expressions.exists(!_.deterministic) && diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertMapSortInGroupingExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertMapSortExpression.scala similarity index 69% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertMapSortInGroupingExpressions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertMapSortExpression.scala index b6ced6c49a36f..9e613c54a49bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertMapSortInGroupingExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertMapSortExpression.scala @@ -20,32 +20,30 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions.{Alias, ArrayTransform, CreateNamedStruct, Expression, GetStructField, If, IsNull, LambdaFunction, Literal, MapFromArrays, MapKeys, MapSort, MapValues, NamedExpression, NamedLambdaVariable} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project, RepartitionByExpression} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.TreePattern +import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE, REPARTITION_OPERATION} import org.apache.spark.sql.types.{ArrayType, MapType, StructType} import org.apache.spark.util.ArrayImplicits.SparkArrayOps /** - * Adds [[MapSort]] to group expressions containing map columns, as the key/value pairs need to be - * in the correct order before grouping: + * Adds [[MapSort]] to [[Aggregate]] expressions containing map columns, + * as the key/value pairs need to be in the correct order before grouping: * - * SELECT map_column, COUNT(*) FROM TABLE GROUP BY map_column => + * SELECT map_column, COUNT(*) FROM TABLE GROUP BY map_column => * SELECT _groupingmapsort as map_column, COUNT(*) FROM ( * SELECT map_sort(map_column) as _groupingmapsort FROM TABLE * ) GROUP BY _groupingmapsort */ object InsertMapSortInGroupingExpressions extends Rule[LogicalPlan] { - private def shouldAddMapSort(expr: Expression): Boolean = { - expr.dataType.existsRecursively(_.isInstanceOf[MapType]) - } + import InsertMapSortExpression._ override def apply(plan: LogicalPlan): LogicalPlan = { - if (!plan.containsPattern(TreePattern.AGGREGATE)) { + if (!plan.containsPattern(AGGREGATE)) { return plan } val shouldRewrite = plan.exists { - case agg: Aggregate if agg.groupingExpressions.exists(shouldAddMapSort) => true + case agg: Aggregate if agg.groupingExpressions.exists(mapTypeExistsRecursively) => true case _ => false } if (!shouldRewrite) { @@ -53,8 +51,7 @@ object InsertMapSortInGroupingExpressions extends Rule[LogicalPlan] { } plan transformUpWithNewOutput { - case agg @ Aggregate(groupingExprs, aggregateExpressions, child, _) - if agg.groupingExpressions.exists(shouldAddMapSort) => + case agg @ Aggregate(groupingExprs, aggregateExpressions, child, hint) => val exprToMapSort = new mutable.HashMap[Expression, NamedExpression] val newGroupingKeys = groupingExprs.map { expr => val inserted = insertMapSortRecursively(expr) @@ -77,15 +74,53 @@ object InsertMapSortInGroupingExpressions extends Rule[LogicalPlan] { }.asInstanceOf[NamedExpression] } val newChild = Project(child.output ++ exprToMapSort.values, child) - val newAgg = Aggregate(newGroupingKeys, newAggregateExprs, newChild) + val newAgg = Aggregate(newGroupingKeys, newAggregateExprs, newChild, hint) newAgg -> agg.output.zip(newAgg.output) } } +} + +/** + * Adds [[MapSort]] to [[RepartitionByExpression]] expressions containing map columns, + * as the key/value pairs need to be in the correct order before repartitioning: + * + * SELECT * FROM TABLE DISTRIBUTE BY map_column => + * SELECT * FROM TABLE DISTRIBUTE BY map_sort(map_column) + */ +object InsertMapSortInRepartitionExpressions extends Rule[LogicalPlan] { + import InsertMapSortExpression._ + + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.transformUpWithPruning(_.containsPattern(REPARTITION_OPERATION)) { + case rep: RepartitionByExpression + if rep.partitionExpressions.exists(mapTypeExistsRecursively) => + val exprToMapSort = new mutable.HashMap[Expression, Expression] + val newPartitionExprs = rep.partitionExpressions.map { expr => + val inserted = insertMapSortRecursively(expr) + if (expr.ne(inserted)) { + exprToMapSort.getOrElseUpdate(expr.canonicalized, inserted) + } else { + expr + } + } + rep.copy(partitionExpressions = newPartitionExprs) + } + } +} + +private[optimizer] object InsertMapSortExpression { /** - * Inserts MapSort recursively taking into account when it is nested inside a struct or array. + * Returns true if the expression contains a [[MapType]] in DataType tree. */ - private def insertMapSortRecursively(e: Expression): Expression = { + def mapTypeExistsRecursively(expr: Expression): Boolean = { + expr.dataType.existsRecursively(_.isInstanceOf[MapType]) + } + + /** + * Inserts [[MapSort]] recursively taking into account when it is nested inside a struct or array. + */ + def insertMapSortRecursively(e: Expression): Expression = { e.dataType match { case m: MapType => // Check if value type of MapType contains MapType (possibly nested) @@ -122,5 +157,4 @@ object InsertMapSortInGroupingExpressions extends Rule[LogicalPlan] { case _ => e } } - } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c0c76dd44ad5e..8ee2226947ec9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -322,6 +322,7 @@ abstract class Optimizer(catalogManager: CatalogManager) // so the grouping keys can only be attribute and literal which makes // `InsertMapSortInGroupingExpressions` easy to insert `MapSort`. InsertMapSortInGroupingExpressions, + InsertMapSortInRepartitionExpressions, ComputeCurrentTime, ReplaceCurrentLike(catalogManager), SpecialDatetimeValues, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 0972a63a2495c..317a88edf8e95 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -316,7 +316,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException](df.repartition(5, col("v"))), condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", parameters = Map( - "expr" -> "v", + "expr" -> "\"v\"", "dataType" -> "\"VARIANT\"") ) // nested variant column @@ -324,7 +324,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException](df.repartition(5, col("s"))), condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", parameters = Map( - "expr" -> "s", + "expr" -> "\"s\"", "dataType" -> "\"STRUCT\"") ) // variant producing expression @@ -333,7 +333,7 @@ class DataFrameSuite extends QueryTest intercept[AnalysisException](df.repartition(5, parse_json(col("id").cast("string")))), condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", parameters = Map( - "expr" -> "parse_json(CAST(id AS STRING))", + "expr" -> "\"parse_json(CAST(id AS STRING))\"", "dataType" -> "\"VARIANT\"") ) // Partitioning by non-variant column works @@ -350,7 +350,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException](sql("SELECT * FROM tv DISTRIBUTE BY v")), condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", parameters = Map( - "expr" -> "tv.v", + "expr" -> "\"v\"", "dataType" -> "\"VARIANT\""), context = ExpectedContext( fragment = "DISTRIBUTE BY v", @@ -361,7 +361,7 @@ class DataFrameSuite extends QueryTest exception = intercept[AnalysisException](sql("SELECT * FROM tv DISTRIBUTE BY s")), condition = "UNSUPPORTED_FEATURE.PARTITION_BY_VARIANT", parameters = Map( - "expr" -> "tv.s", + "expr" -> "\"s\"", "dataType" -> "\"STRUCT\""), context = ExpectedContext( fragment = "DISTRIBUTE BY s", @@ -428,6 +428,33 @@ class DataFrameSuite extends QueryTest } } + test("repartition by MapType") { + Seq("int", "long", "float", "double", "decimal(10, 2)", "string", "varchar(6)").foreach { dt => + val df = spark.range(20) + .withColumn("c1", + when(col("id") % 3 === 1, typedLit(Map(1 -> 1))) + .when(col("id") % 3 === 2, typedLit(Map(1 -> 1, 2 -> 2))) + .otherwise(typedLit(Map(2 -> 2, 1 -> 1))).cast(s"map<$dt, $dt>")) + .withColumn("c2", typedLit(Map(1 -> null)).cast(s"map<$dt, $dt>")) + .withColumn("c3", lit(null).cast(s"map<$dt, $dt>")) + + assertPartitionNumber(df.repartition(4, col("c1")), 2) + assertPartitionNumber(df.repartition(4, col("c2")), 1) + assertPartitionNumber(df.repartition(4, col("c3")), 1) + assertPartitionNumber(df.repartition(4, col("c1"), col("c2")), 2) + assertPartitionNumber(df.repartition(4, col("c1"), col("c3")), 2) + assertPartitionNumber(df.repartition(4, col("c1"), col("c2"), col("c3")), 2) + assertPartitionNumber(df.repartition(4, col("c2"), col("c3")), 2) + } + } + + private def assertPartitionNumber(df: => DataFrame, max: Int): Unit = { + val dfGrouped = df.groupBy(spark_partition_id()).count() + // Result number of partition can be lower or equal to max, + // but no more than that. + assert(dfGrouped.count() <= max, dfGrouped.queryExecution.simpleString) + } + test("coalesce") { intercept[IllegalArgumentException] { testData.select("key").coalesce(0) From fe3d3ae6172b88734c33da30e900ed3bfae1417c Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 10 Jan 2025 12:23:10 +0800 Subject: [PATCH 375/438] [SPARK-50719][PYTHON] Support `interruptOperation` for PySpark ### What changes were proposed in this pull request? This PR proposes to support `interruptOperation` for PySpark ### Why are the changes needed? For feature parity with Spark Connect ### Does this PR introduce _any_ user-facing change? No, this adds a new API ### How was this patch tested? The existing CI should pass ### Was this patch authored or co-authored using generative AI tooling? No Closes #49423 from itholic/interrupt_operation. Authored-by: Haejoon Lee Signed-off-by: Ruifeng Zheng --- .../reference/pyspark.sql/spark_session.rst | 2 +- python/pyspark/sql/session.py | 17 ++++++++++++----- .../sql/tests/test_connect_compatibility.py | 1 - python/pyspark/sql/tests/test_session.py | 1 - 4 files changed, 13 insertions(+), 8 deletions(-) diff --git a/python/docs/source/reference/pyspark.sql/spark_session.rst b/python/docs/source/reference/pyspark.sql/spark_session.rst index a35fccbcffe99..0d6a1bc79b902 100644 --- a/python/docs/source/reference/pyspark.sql/spark_session.rst +++ b/python/docs/source/reference/pyspark.sql/spark_session.rst @@ -53,6 +53,7 @@ See also :class:`SparkSession`. SparkSession.getActiveSession SparkSession.getTags SparkSession.interruptAll + SparkSession.interruptOperation SparkSession.interruptTag SparkSession.newSession SparkSession.profile @@ -88,6 +89,5 @@ Spark Connect Only SparkSession.clearProgressHandlers SparkSession.client SparkSession.copyFromLocalToFs - SparkSession.interruptOperation SparkSession.registerProgressHandler SparkSession.removeProgressHandler diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index fc434cd16bfbd..5ab186b2957e7 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -2253,13 +2253,15 @@ def interruptTag(self, tag: str) -> List[str]: return python_list - @remote_only def interruptOperation(self, op_id: str) -> List[str]: """ Interrupt an operation of this session with the given operationId. .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Returns ------- list of str @@ -2269,10 +2271,15 @@ def interruptOperation(self, op_id: str) -> List[str]: ----- There is still a possibility of operation finishing just as it is interrupted. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.interruptOperation"}, - ) + java_list = self._jsparkSession.interruptOperation(op_id) + python_list = list() + + # Use iterator to manually iterate through Java list + java_iterator = java_list.iterator() + while java_iterator.hasNext(): + python_list.append(str(java_iterator.next())) + + return python_list def addTag(self, tag: str) -> None: """ diff --git a/python/pyspark/sql/tests/test_connect_compatibility.py b/python/pyspark/sql/tests/test_connect_compatibility.py index 25b8be1f9ac7a..4ac68292b4020 100644 --- a/python/pyspark/sql/tests/test_connect_compatibility.py +++ b/python/pyspark/sql/tests/test_connect_compatibility.py @@ -266,7 +266,6 @@ def test_spark_session_compatibility(self): "addArtifacts", "clearProgressHandlers", "copyFromLocalToFs", - "interruptOperation", "newSession", "registerProgressHandler", "removeProgressHandler", diff --git a/python/pyspark/sql/tests/test_session.py b/python/pyspark/sql/tests/test_session.py index a22fe777e3c9a..c21247e3159c0 100644 --- a/python/pyspark/sql/tests/test_session.py +++ b/python/pyspark/sql/tests/test_session.py @@ -227,7 +227,6 @@ def test_unsupported_api(self): (lambda: session.client, "client"), (session.addArtifacts, "addArtifact(s)"), (lambda: session.copyFromLocalToFs("", ""), "copyFromLocalToFs"), - (lambda: session.interruptOperation(""), "interruptOperation"), ] for func, name in unsupported: From 6320b5cdbb1d7058b288f495e10a645c86a8d037 Mon Sep 17 00:00:00 2001 From: Dejan Krakovic Date: Fri, 10 Jan 2025 13:19:10 +0800 Subject: [PATCH 376/438] [SPARK-50779][SQL] Adding feature flag for object level collations ### What changes were proposed in this pull request? As a follow up from https://github.com/apache/spark/pull/49084 and associated JIRA issue https://issues.apache.org/jira/browse/SPARK-50675, adding an internal feature SQL conf flag around object level collations, to be able to enable/disable the feature. ### Why are the changes needed? The object level collations feature is still in development phase, and so it should be disabled by default until it is completed, other than in unit tests that validate the functionality. ### Does this PR introduce _any_ user-facing change? Just explicitly throwing a query compilation error that object level collations are unsupported feature, in case the newly added feature flag is disabled. ### How was this patch tested? Dedicated unit tests are added to verify that when the feature flag is disabled, attempting to set object level collation (on table or view object) results in query compilation error that the feature is not supported. The existing unit tests already added as part of previous PR linked above validate that when the feature flag is enabled, the object level collation could be properly set (flag is enabled by default in testing environment) ### Was this patch authored or co-authored using generative AI tooling? No Closes #49431 from dejankrak-db/conf-flag-object-level-collations. Authored-by: Dejan Krakovic Signed-off-by: Wenchen Fan --- .../main/resources/error/error-conditions.json | 5 +++++ .../spark/sql/catalyst/parser/AstBuilder.scala | 3 +++ .../sql/errors/QueryCompilationErrors.scala | 7 +++++++ .../org/apache/spark/sql/internal/SQLConf.scala | 14 ++++++++++++++ .../errors/QueryCompilationErrorsSuite.scala | 17 +++++++++++++++++ 5 files changed, 46 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index f7ca1eae0ef29..e9b32b5f9cbe7 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5381,6 +5381,11 @@ "The target JDBC server hosting table does not support ALTER TABLE with multiple actions. Split the ALTER TABLE up into individual actions to avoid this error." ] }, + "OBJECT_LEVEL_COLLATIONS" : { + "message" : [ + "Default collation for the specified object." + ] + }, "ORC_TYPE_CAST" : { "message" : [ "Unable to convert of Orc to data type ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f4f6d2b310f49..5858425f66462 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3875,6 +3875,9 @@ class AstBuilder extends DataTypeAstBuilder } override def visitCollationSpec(ctx: CollationSpecContext): String = withOrigin(ctx) { + if (!SQLConf.get.objectLevelCollationsEnabled) { + throw QueryCompilationErrors.objectLevelCollationsNotEnabledError() + } val collationName = ctx.identifier.getText CollationFactory.fetchCollation(collationName).collationName } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 0d5fe7bc14596..afae0565133b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -357,6 +357,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map.empty) } + def objectLevelCollationsNotEnabledError(): Throwable = { + new AnalysisException( + errorClass = "UNSUPPORTED_FEATURE.OBJECT_LEVEL_COLLATIONS", + messageParameters = Map.empty + ) + } + def trimCollationNotEnabledError(): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE.TRIM_COLLATION", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 875e9543c4720..6077e55561e62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -859,6 +859,18 @@ object SQLConf { .booleanConf .createWithDefault(false) + lazy val OBJECT_LEVEL_COLLATIONS_ENABLED = + buildConf("spark.sql.collation.objectLevel.enabled") + .internal() + .doc( + "Object level collations feature is under development and its use should be done " + + "under this feature flag. The feature allows setting default collation for all " + + "underlying columns within that object, except the ones that were previously created." + ) + .version("4.0.0") + .booleanConf + .createWithDefault(Utils.isTesting) + lazy val TRIM_COLLATION_ENABLED = buildConf("spark.sql.collation.trim.enabled") .internal() @@ -5770,6 +5782,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def allowCollationsInMapKeys: Boolean = getConf(ALLOW_COLLATIONS_IN_MAP_KEYS) + def objectLevelCollationsEnabled: Boolean = getConf(OBJECT_LEVEL_COLLATIONS_ENABLED) + def trimCollationEnabled: Boolean = getConf(TRIM_COLLATION_ENABLED) override def defaultStringType: StringType = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 92c175fe2f94a..779b5ba530aa6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -901,6 +901,23 @@ class QueryCompilationErrorsSuite } } + test("SPARK-50779: the object level collations feature is unsupported when flag is disabled") { + withSQLConf(SQLConf.OBJECT_LEVEL_COLLATIONS_ENABLED.key -> "false") { + Seq( + "CREATE TABLE t (c STRING) USING parquet DEFAULT COLLATION UNICODE", + "REPLACE TABLE t (c STRING) USING parquet DEFAULT COLLATION UNICODE_CI", + "ALTER TABLE t DEFAULT COLLATION sr_CI_AI", + "CREATE VIEW v DEFAULT COLLATION UNICODE as SELECT * FROM t", + "CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE as SELECT * FROM t" + ).foreach { sqlText => + checkError( + exception = intercept[AnalysisException](sql(sqlText)), + condition = "UNSUPPORTED_FEATURE.OBJECT_LEVEL_COLLATIONS" + ) + } + } + } + test("UNSUPPORTED_CALL: call the unsupported method update()") { checkError( exception = intercept[SparkUnsupportedOperationException] { From 1208d3cf64836a08edd6cabc8bfcba751681a367 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 10 Jan 2025 13:35:29 +0800 Subject: [PATCH 377/438] [SPARK-50633][INFRA] Fix `codecov/codecov-action@v4` daily scheduling failure MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? The pr aims to fix `Build / Coverage (master, Scala 2.13, Hadoop 3, JDK 17) (`build_coverage.yml`)` `codecov/codecov-actionv4` job daily scheduling failure. ### Why are the changes needed? - 1.https://app.codecov.io/github/apache?search=spark image - 2.After clicking in above https://app.codecov.io/github/apache/spark image The last successful commit trigger was: [78f7c30](https://app.codecov.io/github/apache/spark/commit/78f7c30e140fd8cf4a80b783dd7e9ee4d1b4d7e2) ```shell (base) ➜ spark-community git:(master) ✗ GLA | grep 78f7c30 78f7c30e140fd8cf4a80b783dd7e9ee4d1b4d7e2 - Nikola Mandic, nikola.mandicdatabricks.com, 10 months ago : [SPARK-42328][SQL] Remove _LEGACY_ERROR_TEMP_1175 from error classes ``` - 3.This job should have failed every day for the past 10 months, as shown below: https://github.com/apache/spark/actions/runs/12410482233/job/34646325911 image - 4.Currently, `codecov/codecov-actionv4` no longer supports without a token. https://docs.codecov.com/docs/codecov-tokens#uploading-without-a-token image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual daily continuous observation. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49251 from panbingkun/SPARK-50633. Authored-by: panbingkun Signed-off-by: panbingkun --- .github/workflows/build_and_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 14b9b87dd50a2..a51e5a2e5c21b 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -621,6 +621,7 @@ jobs: if: fromJSON(inputs.envs).PYSPARK_CODECOV == 'true' uses: codecov/codecov-action@v4 with: + token: ${{ secrets.CODECOV_TOKEN }} files: ./python/coverage.xml flags: unittests name: PySpark From a022bf0a348fab5e7af76cb36f326981f13427f1 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 10 Jan 2025 15:16:34 +0800 Subject: [PATCH 378/438] [SPARK-50634][INFRA] Upgrade `codecov/codecov-action` to v5 ### What changes were proposed in this pull request? The pr aims to upgrade `codecov/codecov-action` from `v4` to `v5`. ### Why are the changes needed? https://github.com/codecov/codecov-action/releases/tag/v5.0.0 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual daily continuous observation. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49228 from panbingkun/codecov-action_v5. Authored-by: panbingkun Signed-off-by: panbingkun --- .github/workflows/build_and_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index a51e5a2e5c21b..c3c83c67c140a 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -619,7 +619,7 @@ jobs: fi - name: Upload coverage to Codecov if: fromJSON(inputs.envs).PYSPARK_CODECOV == 'true' - uses: codecov/codecov-action@v4 + uses: codecov/codecov-action@v5 with: token: ${{ secrets.CODECOV_TOKEN }} files: ./python/coverage.xml From 8b209d188f6c3cb8ca59d41625c49f9a10bb16f2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 10 Jan 2025 16:23:01 +0800 Subject: [PATCH 379/438] [SPARK-50124][SQL][FOLLOWUP] InsertSortForLimitAndOffset should propagate missing ordering columns ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/48661 to extend the fix to handle a common query pattern: doing project/filter after the sort. The implementation is a bit complicated as now we need to propagate the ordering columns pre-shuffle, so that we can perform the local sort post-shuffle. ### Why are the changes needed? extend the bug fix to cover more query patterns. ### Does this PR introduce _any_ user-facing change? No, the fix is not released ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49416 from cloud-fan/fix. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../InsertSortForLimitAndOffset.scala | 81 ++++++++++++------- .../InsertSortForLimitAndOffsetSuite.scala | 47 ++++++++--- 2 files changed, 90 insertions(+), 38 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala index fa63e04d91b00..aa29128cda7e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.execution -import scala.annotation.tailrec - import org.apache.spark.sql.catalyst.expressions.SortOrder +import org.apache.spark.sql.catalyst.plans.logical.{Project, Sort} import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.python.EvalPythonExec import org.apache.spark.sql.internal.SQLConf /** @@ -43,33 +42,61 @@ object InsertSortForLimitAndOffset extends Rule[SparkPlan] { plan transform { case l @ GlobalLimitExec( _, - SinglePartitionShuffleWithGlobalOrdering(ordering), - _) => - val newChild = SortExec(ordering, global = false, child = l.child) - l.withNewChildren(Seq(newChild)) - } - } - - object SinglePartitionShuffleWithGlobalOrdering { - @tailrec - def unapply(plan: SparkPlan): Option[Seq[SortOrder]] = plan match { - case ShuffleExchangeExec(SinglePartition, SparkPlanWithGlobalOrdering(ordering), _, _) => - Some(ordering) - case p: AQEShuffleReadExec => unapply(p.child) - case p: ShuffleQueryStageExec => unapply(p.plan) - case _ => None + // Should not match AQE shuffle stage because we only target un-submitted stages which + // we can still rewrite the query plan. + s @ ShuffleExchangeExec(SinglePartition, child, _, _), + _) if child.logicalLink.isDefined => + extractOrderingAndPropagateOrderingColumns(child) match { + case Some((ordering, newChild)) => + val newShuffle = s.withNewChildren(Seq(newChild)) + val sorted = SortExec(ordering, global = false, child = newShuffle) + // We must set the logical plan link to avoid losing the added SortExec and ProjectExec + // during AQE re-optimization, where we turn physical plan back to logical plan. + val logicalSort = Sort(ordering, global = false, child = s.child.logicalLink.get) + sorted.setLogicalLink(logicalSort) + val projected = if (sorted.output == s.output) { + sorted + } else { + val p = ProjectExec(s.output, sorted) + p.setLogicalLink(Project(s.output, logicalSort)) + p + } + l.withNewChildren(Seq(projected)) + case _ => l + } } } // Note: this is not implementing a generalized notion of "global order preservation", but just - // tackles the regular ORDER BY semantics with optional LIMIT (top-K). - object SparkPlanWithGlobalOrdering { - @tailrec - def unapply(plan: SparkPlan): Option[Seq[SortOrder]] = plan match { - case p: SortExec if p.global => Some(p.sortOrder) - case p: LocalLimitExec => unapply(p.child) - case p: WholeStageCodegenExec => unapply(p.child) - case _ => None - } + // a best effort to catch the common query patterns that the data ordering should be preserved. + private def extractOrderingAndPropagateOrderingColumns( + plan: SparkPlan): Option[(Seq[SortOrder], SparkPlan)] = plan match { + case p: SortExec if p.global => Some(p.sortOrder, p) + case p: UnaryExecNode if + p.isInstanceOf[LocalLimitExec] || + p.isInstanceOf[WholeStageCodegenExec] || + p.isInstanceOf[FilterExec] || + p.isInstanceOf[EvalPythonExec] => + extractOrderingAndPropagateOrderingColumns(p.child) match { + case Some((ordering, newChild)) => Some((ordering, p.withNewChildren(Seq(newChild)))) + case _ => None + } + case p: ProjectExec => + extractOrderingAndPropagateOrderingColumns(p.child) match { + case Some((ordering, newChild)) => + val orderingCols = ordering.flatMap(_.references) + if (orderingCols.forall(p.outputSet.contains)) { + Some((ordering, p.withNewChildren(Seq(newChild)))) + } else { + // In order to do the sort after shuffle, we must propagate the ordering columns in the + // pre-shuffle ProjectExec. + val missingCols = orderingCols.filterNot(p.outputSet.contains) + val newProj = p.copy(projectList = p.projectList ++ missingCols, child = newChild) + newProj.copyTagsFrom(p) + Some((ordering, newProj)) + } + case _ => None + } + case _ => None } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffsetSuite.scala index 8d640a1840f4c..d1b11a74cf35f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffsetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffsetSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{Dataset, QueryTest} +import org.apache.spark.sql.IntegratedUDFTestUtils._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.rand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.IntegerType class InsertSortForLimitAndOffsetSuite extends QueryTest with SharedSparkSession @@ -51,6 +54,7 @@ class InsertSortForLimitAndOffsetSuite extends QueryTest private def hasLocalSort(plan: SparkPlan): Boolean = { find(plan) { case GlobalLimitExec(_, s: SortExec, _) => !s.global + case GlobalLimitExec(_, ProjectExec(_, s: SortExec), _) => !s.global case _ => false }.isDefined } @@ -91,12 +95,16 @@ class InsertSortForLimitAndOffsetSuite extends QueryTest // one partition to read the range-partition shuffle and there is only one shuffle block for // the final single-partition shuffle, random fetch order is no longer an issue. SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { - val df = spark.range(10).orderBy($"id" % 8).limit(2).distinct() - df.collect() - val physicalPlan = df.queryExecution.executedPlan - assertHasGlobalLimitExec(physicalPlan) - // Extra local sort is needed for middle LIMIT - assert(hasLocalSort(physicalPlan)) + val df = 1.to(10).map(v => v -> v).toDF("c1", "c2").orderBy($"c1" % 8) + verifySortAdded(df.limit(2)) + verifySortAdded(df.filter($"c2" > rand()).limit(2)) + verifySortAdded(df.select($"c2").limit(2)) + verifySortAdded(df.filter($"c2" > rand()).select($"c2").limit(2)) + + assume(shouldTestPythonUDFs) + val pythonTestUDF = TestPythonUDF(name = "pyUDF", Some(IntegerType)) + verifySortAdded(df.filter(pythonTestUDF($"c2") > rand()).limit(2)) + verifySortAdded(df.select(pythonTestUDF($"c2")).limit(2)) } } @@ -110,11 +118,28 @@ class InsertSortForLimitAndOffsetSuite extends QueryTest } test("middle OFFSET preserves data ordering with the extra sort") { - val df = spark.range(10).orderBy($"id" % 8).offset(2).distinct() - df.collect() - val physicalPlan = df.queryExecution.executedPlan + val df = 1.to(10).map(v => v -> v).toDF("c1", "c2").orderBy($"c1" % 8) + verifySortAdded(df.offset(2)) + verifySortAdded(df.filter($"c2" > rand()).offset(2)) + verifySortAdded(df.select($"c2").offset(2)) + verifySortAdded(df.filter($"c2" > rand()).select($"c2").offset(2)) + + assume(shouldTestPythonUDFs) + val pythonTestUDF = TestPythonUDF(name = "pyUDF", Some(IntegerType)) + verifySortAdded(df.filter(pythonTestUDF($"c2") > rand()).offset(2)) + verifySortAdded(df.select(pythonTestUDF($"c2")).offset(2)) + } + + private def verifySortAdded(df: Dataset[_]): Unit = { + // Do distinct to trigger a shuffle, so that the LIMIT/OFFSET below won't be planned as + // `CollectLimitExec` + val shuffled = df.distinct() + shuffled.collect() + val physicalPlan = shuffled.queryExecution.executedPlan assertHasGlobalLimitExec(physicalPlan) - // Extra local sort is needed for middle OFFSET + // Extra local sort is needed for middle LIMIT/OFFSET assert(hasLocalSort(physicalPlan)) + // Make sure the schema does not change. + assert(physicalPlan.schema == shuffled.schema) } } From 868549e7ac632e3d0fa26f26b22c2738b06caadc Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 10 Jan 2025 16:25:31 +0800 Subject: [PATCH 380/438] [SPARK-50560][SQL] Improve type coercion and boundary checking for RANDSTR SQL function ### What changes were proposed in this pull request? This PR improve type coercion and boundary checking for RANDSTR SQL function. srielau found the following issues and wrote them down in SPARK-50560: * TINYINT and BIGINT were not supported. * No type coercion from floating-point numbers was implemented. * No explicit error checking for negative numbers was implemented, resulting in weird stacktraces instead. ### Why are the changes needed? This PR fixes the above problems to make the function work in more cases and produce better error messages when it fails. For example: ``` SELECT randstr(CAST(10 AS TINYINT), 0) AS result > ceV0PXaR2I SELECT randstr(CAST(10 AS BIGINT), 0) AS result > ceV0PXaR2I SELECT randstr(1.0F, 0) AS result > c SELECT randstr(1.0D, 0) AS result > c SELECT randstr(-1, 0) AS result > Error: The RANDSTR function desired string length must be positive or zero. Please provide an updated string length and then retry the query or command again. ``` ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds golden file based test coverage, and updates existing coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49210 from dtenedor/fix-randstr-types. Lead-authored-by: Daniel Tenedorio Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../analysis/TypeCoercionHelper.scala | 6 + .../expressions/randomExpressions.scala | 65 ++--- .../catalyst/expressions/RandomSuite.scala | 1 - .../sql-tests/analyzer-results/random.sql.out | 194 +++++++-------- .../resources/sql-tests/inputs/random.sql | 11 +- .../sql-tests/results/random.sql.out | 227 ++++++++++-------- .../sql/CollationExpressionWalkerSuite.scala | 3 + .../spark/sql/DataFrameFunctionsSuite.scala | 8 +- 8 files changed, 280 insertions(+), 235 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala index ab2ab50cb33ef..3b3cf748014b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala @@ -44,6 +44,7 @@ import org.apache.spark.sql.catalyst.expressions.{ MapConcat, MapZipWith, NaNvl, + RandStr, RangeFrame, ScalaUDF, Sequence, @@ -400,6 +401,11 @@ abstract class TypeCoercionHelper { NaNvl(Cast(l, DoubleType), r) case NaNvl(l, r) if r.dataType == NullType => NaNvl(l, Cast(r, l.dataType)) + case r: RandStr if r.length.dataType != IntegerType => + implicitCast(r.length, IntegerType).map { casted => + r.copy(length = casted) + }.getOrElse(r) + case other => other } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index 687dd83291be0..50c699ef69bd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -21,12 +21,12 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedSeed} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch -import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes.{ordinalNumber, toSQLExpr, toSQLId, toSQLType} +import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes.{toSQLExpr, toSQLId} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike, UnaryLike} import org.apache.spark.sql.catalyst.trees.TreePattern.{EXPRESSION_WITH_RANDOM_SEED, RUNTIME_REPLACEABLE, TreePattern} -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types._ import org.apache.spark.util.random.XORShiftRandom @@ -313,7 +313,8 @@ object Uniform { group = "string_funcs") case class RandStr( length: Expression, override val seedExpression: Expression, hideSeed: Boolean) - extends ExpressionWithRandomSeed with BinaryLike[Expression] with Nondeterministic { + extends ExpressionWithRandomSeed with BinaryLike[Expression] with Nondeterministic + with ExpectsInputTypes { def this(length: Expression) = this(length, UnresolvedSeed, hideSeed = true) def this(length: Expression, seedExpression: Expression) = @@ -325,6 +326,10 @@ case class RandStr( override def left: Expression = length override def right: Expression = seedExpression + override def inputTypes: Seq[AbstractDataType] = Seq( + IntegerType, + TypeCollection(IntegerType, LongType)) + /** * Record ID within each partition. By being transient, the Random Number Generator is * reset every time we serialize and deserialize and initialize it. @@ -349,52 +354,48 @@ case class RandStr( } override def checkInputDataTypes(): TypeCheckResult = { - var result: TypeCheckResult = TypeCheckResult.TypeCheckSuccess - def requiredType = "INT or SMALLINT" - Seq((length, "length", 0), - (seedExpression, "seed", 1)).foreach { - case (expr: Expression, name: String, index: Int) => - if (result == TypeCheckResult.TypeCheckSuccess) { - if (!expr.foldable) { - result = DataTypeMismatch( - errorSubClass = "NON_FOLDABLE_INPUT", - messageParameters = Map( - "inputName" -> toSQLId(name), - "inputType" -> requiredType, - "inputExpr" -> toSQLExpr(expr))) - } else expr.dataType match { - case _: ShortType | _: IntegerType => - case _: LongType if index == 1 => - case _ => - result = DataTypeMismatch( - errorSubClass = "UNEXPECTED_INPUT_TYPE", - messageParameters = Map( - "paramIndex" -> ordinalNumber(index), - "requiredType" -> requiredType, - "inputSql" -> toSQLExpr(expr), - "inputType" -> toSQLType(expr.dataType))) - } + var result: TypeCheckResult = super.checkInputDataTypes() + Seq((length, "length"), + (seedExpression, "seed")).foreach { + case (expr: Expression, name: String) => + if (result == TypeCheckResult.TypeCheckSuccess && !expr.foldable) { + result = DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> toSQLId(name), + "inputType" -> "integer", + "inputExpr" -> toSQLExpr(expr))) } } result } override def evalInternal(input: InternalRow): Any = { - val numChars = length.eval(input).asInstanceOf[Number].intValue() + val numChars = lengthInteger() ExpressionImplUtils.randStr(rng, numChars) } + private def lengthInteger(): Int = { + // We should have already added a cast to IntegerType (if necessary) in + // FunctionArgumentTypeCoercion. + assert(length.dataType == IntegerType, s"Expected IntegerType, got ${length.dataType}") + val result = length.eval().asInstanceOf[Int] + if (result < 0) { + throw QueryExecutionErrors.unexpectedValueForLengthInFunctionError(prettyName, result) + } + result + } + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val className = classOf[XORShiftRandom].getName val rngTerm = ctx.addMutableState(className, "rng") ctx.addPartitionInitializationStatement( s"$rngTerm = new $className(${seed}L + partitionIndex);") - val eval = length.genCode(ctx) + val numChars = lengthInteger() ev.copy(code = code""" - |${eval.code} |UTF8String ${ev.value} = - | ${classOf[ExpressionImplUtils].getName}.randStr($rngTerm, (int)(${eval.value})); + | ${classOf[ExpressionImplUtils].getName}.randStr($rngTerm, $numChars); |boolean ${ev.isNull} = false; |""".stripMargin, isNull = FalseLiteral) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala index 2d58d9d3136aa..9e6b59b51138d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala @@ -52,7 +52,6 @@ class RandomSuite extends SparkFunSuite with ExpressionEvalHelper { testRandStr(1, "c") testRandStr(5, "ceV0P") testRandStr(10, "ceV0PXaR2I") - testRandStr(10L, "ceV0PXaR2I") def testUniform(first: Any, second: Any, result: Any): Unit = { checkEvaluationWithoutCodegen( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out index 59e903b8682f1..96a4b2ec91c7a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out @@ -559,6 +559,36 @@ SELECT randstr(10S, 0) AS result [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT randstr(CAST(10 AS TINYINT), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(CAST(10 AS BIGINT), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(1.0F, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(1.0D, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(cast(1 AS DECIMAL(10, 2)), 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + -- !query SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col) -- !query analysis @@ -571,124 +601,40 @@ SELECT randstr(10) IS NOT NULL AS result [Analyzer test output redacted due to nondeterminism] +-- !query +SELECT randstr(1, -1) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + -- !query SELECT randstr(10L, 0) AS result -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"10\"", - "inputType" : "\"BIGINT\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(10, 0)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 22, - "fragment" : "randstr(10L, 0)" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query SELECT randstr(10.0F, 0) AS result -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"10.0\"", - "inputType" : "\"FLOAT\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(10.0, 0)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "randstr(10.0F, 0)" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query SELECT randstr(10.0D, 0) AS result -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"10.0\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(10.0, 0)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "randstr(10.0D, 0)" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query SELECT randstr(NULL, 0) AS result -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"NULL\"", - "inputType" : "\"VOID\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(NULL, 0)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 23, - "fragment" : "randstr(NULL, 0)" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query SELECT randstr(0, NULL) AS result -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"NULL\"", - "inputType" : "\"VOID\"", - "paramIndex" : "second", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(0, NULL)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 23, - "fragment" : "randstr(0, NULL)" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -701,7 +647,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "messageParameters" : { "inputExpr" : "\"col\"", "inputName" : "`length`", - "inputType" : "INT or SMALLINT", + "inputType" : "integer", "sqlExpr" : "\"randstr(col, 0)\"" }, "queryContext" : [ { @@ -724,7 +670,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "messageParameters" : { "inputExpr" : "\"col\"", "inputName" : "`seed`", - "inputType" : "INT or SMALLINT", + "inputType" : "integer", "sqlExpr" : "\"randstr(10, col)\"" }, "queryContext" : [ { @@ -758,3 +704,57 @@ org.apache.spark.sql.AnalysisException "fragment" : "randstr(10, 0, 1)" } ] } + + +-- !query +SELECT randstr(-1, 0) AS result +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randstr(10, "a") AS result FROM VALUES (0) tab(a) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"a\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"randstr(10, a)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(10, \"a\")" + } ] +} + + +-- !query +SELECT randstr(10, 1.5) AS result FROM VALUES (0) tab(a) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.5\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"randstr(10, 1.5)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(10, 1.5)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/random.sql b/sql/core/src/test/resources/sql-tests/inputs/random.sql index a17571aa7286f..95be99595cc8c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/random.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/random.sql @@ -62,13 +62,19 @@ SELECT uniform(0, 'def', 0) AS result; SELECT uniform(0, 10, 'ghi') AS result; -- The randstr random string generation function supports generating random strings within a --- specified length. We use a seed of zero for these queries to keep tests deterministic. +-- specified length. We use a seed of zero for most queries to keep tests deterministic. SELECT randstr(1, 0) AS result; SELECT randstr(5, 0) AS result; SELECT randstr(10, 0) AS result; SELECT randstr(10S, 0) AS result; +SELECT randstr(CAST(10 AS TINYINT), 0) AS result; +SELECT randstr(CAST(10 AS BIGINT), 0) AS result; +SELECT randstr(1.0F, 0) AS result; +SELECT randstr(1.0D, 0) AS result; +SELECT randstr(cast(1 AS DECIMAL(10, 2)), 0) AS result; SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col); SELECT randstr(10) IS NOT NULL AS result; +SELECT randstr(1, -1) AS result; -- Negative test cases for the randstr random number generator. SELECT randstr(10L, 0) AS result; SELECT randstr(10.0F, 0) AS result; @@ -78,3 +84,6 @@ SELECT randstr(0, NULL) AS result; SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col); SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col); SELECT randstr(10, 0, 1) AS result; +SELECT randstr(-1, 0) AS result; +SELECT randstr(10, "a") AS result FROM VALUES (0) tab(a); +SELECT randstr(10, 1.5) AS result FROM VALUES (0) tab(a); diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out index eebfac5fc2b19..049d134003cd4 100644 --- a/sql/core/src/test/resources/sql-tests/results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -671,6 +671,46 @@ struct ceV0PXaR2I +-- !query +SELECT randstr(CAST(10 AS TINYINT), 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(CAST(10 AS BIGINT), 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(1.0F, 0) AS result +-- !query schema +struct +-- !query output +c + + +-- !query +SELECT randstr(1.0D, 0) AS result +-- !query schema +struct +-- !query output +c + + +-- !query +SELECT randstr(cast(1 AS DECIMAL(10, 2)), 0) AS result +-- !query schema +struct +-- !query output +c + + -- !query SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col) -- !query schema @@ -689,206 +729,193 @@ struct true +-- !query +SELECT randstr(1, -1) AS result +-- !query schema +struct +-- !query output +S + + -- !query SELECT randstr(10L, 0) AS result -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"10\"", - "inputType" : "\"BIGINT\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(10, 0)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 22, - "fragment" : "randstr(10L, 0)" - } ] -} +ceV0PXaR2I -- !query SELECT randstr(10.0F, 0) AS result -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"10.0\"", - "inputType" : "\"FLOAT\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(10.0, 0)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "randstr(10.0F, 0)" - } ] -} +ceV0PXaR2I -- !query SELECT randstr(10.0D, 0) AS result -- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(NULL, 0) AS result +-- !query schema +struct +-- !query output + + + +-- !query +SELECT randstr(0, NULL) AS result +-- !query schema +struct +-- !query output + + + +-- !query +SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"10.0\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(10.0, 0)\"" + "inputExpr" : "\"col\"", + "inputName" : "`length`", + "inputType" : "integer", + "sqlExpr" : "\"randstr(col, 0)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "randstr(10.0D, 0)" + "stopIndex" : 22, + "fragment" : "randstr(col, 0)" } ] } -- !query -SELECT randstr(NULL, 0) AS result +SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col) -- !query schema struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"NULL\"", - "inputType" : "\"VOID\"", - "paramIndex" : "first", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(NULL, 0)\"" + "inputExpr" : "\"col\"", + "inputName" : "`seed`", + "inputType" : "integer", + "sqlExpr" : "\"randstr(10, col)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, "stopIndex" : 23, - "fragment" : "randstr(NULL, 0)" + "fragment" : "randstr(10, col)" } ] } -- !query -SELECT randstr(0, NULL) AS result +SELECT randstr(10, 0, 1) AS result -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", "messageParameters" : { - "inputSql" : "\"NULL\"", - "inputType" : "\"VOID\"", - "paramIndex" : "second", - "requiredType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(0, NULL)\"" + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2]", + "functionName" : "`randstr`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 23, - "fragment" : "randstr(0, NULL)" + "stopIndex" : 24, + "fragment" : "randstr(10, 0, 1)" } ] } -- !query -SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col) +SELECT randstr(-1, 0) AS result -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkRuntimeException { - "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", - "sqlState" : "42K09", + "errorClass" : "INVALID_PARAMETER_VALUE.LENGTH", + "sqlState" : "22023", "messageParameters" : { - "inputExpr" : "\"col\"", - "inputName" : "`length`", - "inputType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(col, 0)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 22, - "fragment" : "randstr(col, 0)" - } ] + "functionName" : "`randstr`", + "length" : "-1", + "parameter" : "`length`" + } } -- !query -SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col) +SELECT randstr(10, "a") AS result FROM VALUES (0) tab(a) -- !query schema struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputExpr" : "\"col\"", - "inputName" : "`seed`", - "inputType" : "INT or SMALLINT", - "sqlExpr" : "\"randstr(10, col)\"" + "inputSql" : "\"a\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"randstr(10, a)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, "stopIndex" : 23, - "fragment" : "randstr(10, col)" + "fragment" : "randstr(10, \"a\")" } ] } -- !query -SELECT randstr(10, 0, 1) AS result +SELECT randstr(10, 1.5) AS result FROM VALUES (0) tab(a) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", - "sqlState" : "42605", + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", "messageParameters" : { - "actualNum" : "3", - "docroot" : "https://spark.apache.org/docs/latest", - "expectedNum" : "[1, 2]", - "functionName" : "`randstr`" + "inputSql" : "\"1.5\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"randstr(10, 1.5)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "randstr(10, 0, 1)" + "stopIndex" : 23, + "fragment" : "randstr(10, 1.5)" } ] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala index d0581621148a1..1f9589c1c9ce4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala @@ -104,6 +104,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi Literal.create("DuMmY sTrInG".getBytes) } case BooleanType => Literal(true) + case ByteType => Literal(5.toByte) case _: DatetimeType => Literal(Timestamp.valueOf("2009-07-30 12:58:59")) case DecimalType => Literal((new Decimal).set(5)) case _: DecimalType => Literal((new Decimal).set(5)) @@ -183,6 +184,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi case Utf8Lcase => "Cast('DuMmY sTrInG' collate utf8_lcase as BINARY)" } case BooleanType => "True" + case ByteType => "cast(5 as tinyint)" case _: DatetimeType => "date'2016-04-08'" case DecimalType => "5.0" case _: DecimalType => "5.0" @@ -243,6 +245,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi case AnyTimestampType => "TIMESTAMP" case BinaryType => "BINARY" case BooleanType => "BOOLEAN" + case ByteType => "TINYINT" case _: DatetimeType => "DATE" case DecimalType => "DECIMAL(2, 1)" case _: DecimalType => "DECIMAL(2, 1)" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 018aa2159ba1a..fc6d3023ed072 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -24,7 +24,7 @@ import scala.reflect.runtime.universe.runtimeMirror import scala.util.Random import org.apache.spark.{QueryContextType, SPARK_DOC_ROOT, SparkException, SparkRuntimeException} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, InternalRow} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.Cast._ @@ -458,14 +458,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val df = Seq((0)).toDF("a") var expr = randstr(lit(10), lit("a")) checkError( - intercept[AnalysisException](df.select(expr)), + intercept[ExtendedAnalysisException](df.select(expr).collect()), condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( "sqlExpr" -> "\"randstr(10, a)\"", "paramIndex" -> "second", "inputSql" -> "\"a\"", "inputType" -> "\"STRING\"", - "requiredType" -> "INT or SMALLINT"), + "requiredType" -> "(\"INT\" or \"BIGINT\")"), context = ExpectedContext( contextType = QueryContextType.DataFrame, fragment = "randstr", @@ -480,7 +480,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( "inputName" -> "`length`", - "inputType" -> "INT or SMALLINT", + "inputType" -> "integer", "inputExpr" -> "\"a\"", "sqlExpr" -> "\"randstr(a, 10)\""), context = ExpectedContext( From 784586700d9a8d57faa482068e7529393ba8545e Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 10 Jan 2025 17:00:50 +0800 Subject: [PATCH 381/438] [SPARK-50781][SQL] Cache `QueryPlan.expressions` ### What changes were proposed in this pull request? Cache `QueryPlan.expressions` ### Why are the changes needed? We observed that we were spending a significant amount of time recomputing. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing Tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #49435 from zhengruifeng/sql_cache_expressions. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../org/apache/spark/sql/catalyst/plans/QueryPlan.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 9052f6228a9d5..07341f8ca1765 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 @@ -283,7 +283,9 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] } /** Returns all of the expressions present in this query plan operator. */ - final def expressions: Seq[Expression] = { + final def expressions: Seq[Expression] = _expressions() + + private val _expressions = new BestEffortLazyVal[Seq[Expression]](() => { // Recursively find all expressions from a traversable. def seqToExpressions(seq: Iterable[Any]): Iterable[Expression] = seq.flatMap { case e: Expression => e :: Nil @@ -297,7 +299,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] case seq: Iterable[_] => seqToExpressions(seq) case other => Nil }.toSeq - } + }) /** * A variant of `transformUp`, which takes care of the case that the rule replaces a plan node From 793ad7c87b46a8ad5fa77fa17c69dc9184224688 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 10 Jan 2025 17:11:17 +0800 Subject: [PATCH 382/438] [SPARK-50633][FOLLOWUP] Turn on options `verbose` for `codecov/codecov-action` ### What changes were proposed in this pull request? The pr aims to turn on options `verbose` for `codecov/codecov-action` to identify issues. ### Why are the changes needed? The issue is that I have added `token: ${{ secrets.CODECOV_TOKEN }}` in the configuration, but the workflow does not display it like the below: image Furthermore, the workflow below was not successful. https://github.com/apache/spark/actions/runs/12705043568/job/35415465795 image image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49438 from panbingkun/improve_codecov. Authored-by: panbingkun Signed-off-by: panbingkun --- .github/workflows/build_and_test.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index c3c83c67c140a..d133b74da7cd6 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -621,10 +621,11 @@ jobs: if: fromJSON(inputs.envs).PYSPARK_CODECOV == 'true' uses: codecov/codecov-action@v5 with: - token: ${{ secrets.CODECOV_TOKEN }} files: ./python/coverage.xml + token: ${{ secrets.CODECOV_TOKEN }} flags: unittests name: PySpark + verbose: true - name: Upload test results to report env: ${{ fromJSON(inputs.envs) }} if: always() From 8668c1739570138218e99352e81e82c5c6487337 Mon Sep 17 00:00:00 2001 From: camilesing Date: Fri, 10 Jan 2025 15:22:45 +0500 Subject: [PATCH 383/438] [MINOR][DOCS] Fix miss semicolon on describe sql example ### What changes were proposed in this pull request? fix miss semicolon on describe SQL example ### Why are the changes needed? describe SQL example miss semicolon ### Does this PR introduce _any_ user-facing change? Yes. the patch fix docs miss semicolon SQL. ### How was this patch tested? Manually by inspecting generated docs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49316 from camilesing/fix_docs_miss_semicolon. Authored-by: camilesing Signed-off-by: Max Gekk --- docs/sql-ref-syntax-aux-describe-function.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-aux-describe-function.md b/docs/sql-ref-syntax-aux-describe-function.md index a871fb5bfd406..0c5a3d751a564 100644 --- a/docs/sql-ref-syntax-aux-describe-function.md +++ b/docs/sql-ref-syntax-aux-describe-function.md @@ -85,7 +85,7 @@ DESC FUNCTION max; -- Describe a builtin user defined aggregate function -- Returns function name, implementing class and usage and examples. -DESC FUNCTION EXTENDED explode +DESC FUNCTION EXTENDED explode; +---------------------------------------------------------------+ |function_desc | +---------------------------------------------------------------+ From 3c831ebf7cd81c2ed774c692b2da7a35ce0fa252 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 10 Jan 2025 19:41:45 +0800 Subject: [PATCH 384/438] [MINOR][DOCS] Add new dataframe methods to API references ### What changes were proposed in this pull request? Add new dataframe methods to API references ### Why are the changes needed? these new methods are missing in docs ### Does this PR introduce _any_ user-facing change? yes, docs-only changes ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #49439 from zhengruifeng/py_doc_missing_df. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/docs/source/reference/pyspark.sql/dataframe.rst | 6 ++++++ python/pyspark/sql/dataframe.py | 6 +++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/python/docs/source/reference/pyspark.sql/dataframe.rst b/python/docs/source/reference/pyspark.sql/dataframe.rst index 569c5cec69557..5aaea4c32577f 100644 --- a/python/docs/source/reference/pyspark.sql/dataframe.rst +++ b/python/docs/source/reference/pyspark.sql/dataframe.rst @@ -30,6 +30,7 @@ DataFrame DataFrame.agg DataFrame.alias DataFrame.approxQuantile + DataFrame.asTable DataFrame.cache DataFrame.checkpoint DataFrame.coalesce @@ -56,6 +57,7 @@ DataFrame DataFrame.dtypes DataFrame.exceptAll DataFrame.executionInfo + DataFrame.exists DataFrame.explain DataFrame.fillna DataFrame.filter @@ -75,9 +77,11 @@ DataFrame DataFrame.isStreaming DataFrame.join DataFrame.limit + DataFrame.lateralJoin DataFrame.localCheckpoint DataFrame.mapInPandas DataFrame.mapInArrow + DataFrame.metadataColumn DataFrame.melt DataFrame.na DataFrame.observe @@ -96,6 +100,7 @@ DataFrame DataFrame.sameSemantics DataFrame.sample DataFrame.sampleBy + DataFrame.scalar DataFrame.schema DataFrame.select DataFrame.selectExpr @@ -117,6 +122,7 @@ DataFrame DataFrame.toLocalIterator DataFrame.toPandas DataFrame.transform + DataFrame.transpose DataFrame.union DataFrame.unionAll DataFrame.unionByName diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f2c0bc8155821..2d12704485ad2 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -6609,10 +6609,10 @@ def asTable(self) -> TableArg: After obtaining a TableArg from a DataFrame using this method, you can specify partitioning and ordering for the table argument by calling methods such as `partitionBy`, `orderBy`, and `withSinglePartition` on the `TableArg` instance. - - partitionBy(*cols): Partitions the data based on the specified columns. This method cannot + - partitionBy: Partitions the data based on the specified columns. This method cannot be called after withSinglePartition() has been called. - - orderBy(*cols): Orders the data within partitions based on the specified columns. - - withSinglePartition(): Indicates that the data should be treated as a single partition. + - orderBy: Orders the data within partitions based on the specified columns. + - withSinglePartition: Indicates that the data should be treated as a single partition. This method cannot be called after partitionBy() has been called. .. versionadded:: 4.0.0 From 7bc8e99cde424c59b98fe915e3fdaaa30beadb76 Mon Sep 17 00:00:00 2001 From: Jovan Markovic Date: Fri, 10 Jan 2025 21:26:43 +0800 Subject: [PATCH 385/438] [SPARK-50707][SQL] Enable casting to/from char/varchar ### What changes were proposed in this pull request? Enable casting to/from char/varchar. ### Why are the changes needed? Currently `cast` does not work correctly. It always casts to `StringType`. This PR enables casting to `CharType` and `VarcharType`. This only applies when `spark.sql.preserveCharVarcharTypeInfo` is set to `true`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests in: - `CharVarcharTestSuite` - `CastSuiteBase` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49340 from jovanm-db/char_varchar_cast. Lead-authored-by: Jovan Markovic Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/encoders/RowEncoder.scala | 2 +- .../apache/spark/sql/types/StringType.scala | 55 ++++++++++++ .../apache/spark/sql/types/UpCastRule.scala | 8 +- .../AnsiStringPromotionTypeCoercion.scala | 3 +- .../catalyst/analysis/AnsiTypeCoercion.scala | 9 +- .../sql/catalyst/analysis/TypeCoercion.scala | 9 ++ .../spark/sql/catalyst/expressions/Cast.scala | 16 ++-- .../catalyst/expressions/ToStringBase.scala | 39 ++++++++- .../sql/catalyst/expressions/literals.scala | 2 +- .../sql/catalyst/util/CharVarcharUtils.scala | 2 +- .../apache/spark/sql/util/SchemaUtils.scala | 4 +- .../catalyst/expressions/CastSuiteBase.scala | 41 +++++++-- .../spark/sql/CharVarcharTestSuite.scala | 83 +++++++++++++++++++ 13 files changed, 242 insertions(+), 31 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 718d99043abf2..7260ff8f9fefd 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -84,7 +84,7 @@ object RowEncoder extends DataTypeErrorsBase { CharEncoder(length) case VarcharType(length) if SqlApiConf.get.preserveCharVarcharTypeInfo => VarcharEncoder(length) - case s: StringType if s.constraint == NoConstraint => StringEncoder + case s: StringType if StringHelper.isPlainString(s) => StringEncoder case TimestampType if SqlApiConf.get.datetimeJava8ApiEnabled => InstantEncoder(lenient) case TimestampType => TimestampEncoder(lenient) case TimestampNTZType => LocalDateTimeEncoder diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index 02dd139f073d2..cd3182ab2dcde 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -21,6 +21,7 @@ import org.json4s.JsonAST.{JString, JValue} import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.CollationFactory +import org.apache.spark.sql.internal.SqlApiConf /** * The data type representing `String` values. Please use the singleton `DataTypes.StringType`. @@ -129,6 +130,60 @@ case object StringType sealed trait StringConstraint +case object StringHelper extends PartialOrdering[StringConstraint] { + override def tryCompare(x: StringConstraint, y: StringConstraint): Option[Int] = { + (x, y) match { + case (NoConstraint, NoConstraint) => Some(0) + case (NoConstraint, _) => Some(-1) + case (_, NoConstraint) => Some(1) + case (FixedLength(l1), FixedLength(l2)) => Some(l2.compareTo(l1)) + case (FixedLength(l1), MaxLength(l2)) if l1 <= l2 => Some(1) + case (MaxLength(l1), FixedLength(l2)) if l1 >= l2 => Some(-1) + case (MaxLength(l1), MaxLength(l2)) => Some(l2.compareTo(l1)) + case _ => None + } + } + + override def lteq(x: StringConstraint, y: StringConstraint): Boolean = { + tryCompare(x, y).exists(_ <= 0) + } + + override def gteq(x: StringConstraint, y: StringConstraint): Boolean = { + tryCompare(x, y).exists(_ >= 0) + } + + override def equiv(x: StringConstraint, y: StringConstraint): Boolean = { + tryCompare(x, y).contains(0) + } + + def isPlainString(s: StringType): Boolean = s.constraint == NoConstraint + + def isMoreConstrained(a: StringType, b: StringType): Boolean = + gteq(a.constraint, b.constraint) + + def tightestCommonString(s1: StringType, s2: StringType): Option[StringType] = { + if (s1.collationId != s2.collationId) { + return None + } + if (!SqlApiConf.get.preserveCharVarcharTypeInfo) { + return Some(StringType(s1.collationId)) + } + Some((s1.constraint, s2.constraint) match { + case (FixedLength(l1), FixedLength(l2)) => CharType(l1.max(l2)) + case (MaxLength(l1), FixedLength(l2)) => VarcharType(l1.max(l2)) + case (FixedLength(l1), MaxLength(l2)) => VarcharType(l1.max(l2)) + case (MaxLength(l1), MaxLength(l2)) => VarcharType(l1.max(l2)) + case _ => StringType(s1.collationId) + }) + } + + def removeCollation(s: StringType): StringType = s match { + case CharType(length) => CharType(length) + case VarcharType(length) => VarcharType(length) + case _: StringType => StringType + } +} + case object NoConstraint extends StringConstraint case class FixedLength(length: Int) extends StringConstraint diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala index 97a81f0fe8f35..6272cb03bd797 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala @@ -41,10 +41,10 @@ private[sql] object UpCastRule { case (TimestampNTZType, TimestampType) => true case (TimestampType, TimestampNTZType) => true - case (_: AtomicType, CharType(_) | VarcharType(_)) => false - case (_: CalendarIntervalType, CharType(_) | VarcharType(_)) => false - case (_: AtomicType, _: StringType) => true - case (_: CalendarIntervalType, _: StringType) => true + case (s1: StringType, s2: StringType) => StringHelper.isMoreConstrained(s1, s2) + // TODO: allow upcast from int/double/decimal to char/varchar of sufficient length + case (_: AtomicType, s: StringType) => StringHelper.isPlainString(s) + case (_: CalendarIntervalType, s: StringType) => StringHelper.isPlainString(s) case (NullType, _) => true // Spark supports casting between long and timestamp, please see `longToTimestamp` and diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala index 8345a4b9637e2..e7be95bc645ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala @@ -42,6 +42,7 @@ import org.apache.spark.sql.types.{ IntegralType, LongType, NullType, + StringHelper, StringType, StringTypeExpression, TimestampType @@ -99,7 +100,7 @@ object AnsiStringPromotionTypeCoercion { case (_: StringType, _: AnsiIntervalType) => None // [SPARK-50060] If a binary operation contains two collated string types with different // collation IDs, we can't decide which collation ID the result should have. - case (st1: StringType, st2: StringType) if st1.collationId != st2.collationId => None + case (st1: StringType, st2: StringType) => StringHelper.tightestCommonString(st1, st2) case (_: StringType, a: AtomicType) => Some(a) case (other, st: StringType) if !other.isInstanceOf[StringType] => findWiderTypeForString(st, other) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala index 3ba17c8b8e1a3..aa977b240007b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala @@ -102,6 +102,8 @@ object AnsiTypeCoercion extends TypeCoercionBase { case (NullType, t1) => Some(t1) case (t1, NullType) => Some(t1) + case(s1: StringType, s2: StringType) => StringHelper.tightestCommonString(s1, s2) + case (t1: IntegralType, t2: DecimalType) if t2.isWiderThan(t1) => Some(t2) case (t1: DecimalType, t2: IntegralType) if t1.isWiderThan(t2) => @@ -168,7 +170,12 @@ object AnsiTypeCoercion extends TypeCoercionBase { // If a function expects a StringType, no StringType instance should be implicitly cast to // StringType with a collation that's not accepted (aka. lockdown unsupported collations). - case (_: StringType, _: StringType) => None + case (s1: StringType, s2: StringType) => + if (s1.collationId == s2.collationId && StringHelper.isMoreConstrained(s1, s2)) { + Some(s2) + } else { + None + } case (_: StringType, _: AbstractStringType) => None // If a function expects integral type, fractional input is not allowed. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index c30aa9bf91a1d..4769970b51421 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -77,6 +77,8 @@ object TypeCoercion extends TypeCoercionBase { case (NullType, t1) => Some(t1) case (t1, NullType) => Some(t1) + case(s1: StringType, s2: StringType) => StringHelper.tightestCommonString(s1, s2) + case (t1: IntegralType, t2: DecimalType) if t2.isWiderThan(t1) => Some(t2) case (t1: DecimalType, t2: IntegralType) if t1.isWiderThan(t2) => @@ -149,6 +151,7 @@ object TypeCoercion extends TypeCoercionBase { case (DecimalType.Fixed(_, s), _: StringType) if s > 0 => Some(DoubleType) case (_: StringType, DecimalType.Fixed(_, s)) if s > 0 => Some(DoubleType) + case (s1: StringType, s2: StringType) => StringHelper.tightestCommonString(s1, s2) case (l: StringType, r: AtomicType) if canPromoteAsInBinaryComparison(r) => Some(r) case (l: AtomicType, r: StringType) if canPromoteAsInBinaryComparison(l) => Some(l) case (l, r) => None @@ -190,6 +193,12 @@ object TypeCoercion extends TypeCoercionBase { // Cast null type (usually from null literals) into target types case (NullType, target) => target.defaultConcreteType + case (s1: StringType, s2: StringType) => + if (s1.collationId == s2.collationId && StringHelper.isMoreConstrained(s1, s2)) { + s2 + } else { + null + } // If the function accepts any numeric type and the input is a string, we follow the hive // convention and cast that input into a double case (_: StringType, NumericType) => NumericType.defaultConcreteType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index abd635e22f261..8773d7a6a029e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -93,9 +93,7 @@ object Cast extends QueryErrorsBase { case (NullType, _) => true - case (_, CharType(_) | VarcharType(_)) => false case (_, _: StringType) => true - case (CharType(_) | VarcharType(_), _) => false case (_: StringType, _: BinaryType) => true @@ -200,9 +198,7 @@ object Cast extends QueryErrorsBase { case (NullType, _) => true - case (_, CharType(_) | VarcharType(_)) => false case (_, _: StringType) => true - case (CharType(_) | VarcharType(_), _) => false case (_: StringType, BinaryType) => true case (_: IntegralType, BinaryType) => true @@ -318,8 +314,6 @@ object Cast extends QueryErrorsBase { case _ if from == to => true case (NullType, _) => true case (_: NumericType, _: NumericType) => true - case (_: AtomicType, CharType(_) | VarcharType(_)) => false - case (_: CalendarIntervalType, CharType(_) | VarcharType(_)) => false case (_: AtomicType, _: StringType) => true case (_: CalendarIntervalType, _: StringType) => true case (_: DatetimeType, _: DatetimeType) => true @@ -361,10 +355,9 @@ object Cast extends QueryErrorsBase { case (_, _) if from == to => false case (VariantType, _) => true - case (CharType(_) | VarcharType(_), BinaryType | _: StringType) => false case (_: StringType, BinaryType | _: StringType) => false - case (st: StringType, _) if st.constraint == NoConstraint => true - case (_, st: StringType) if st.constraint == NoConstraint => false + case (_: StringType, _) => true + case (_, _: StringType) => false case (TimestampType, ByteType | ShortType | IntegerType) => true case (FloatType | DoubleType, TimestampType) => true @@ -1138,7 +1131,7 @@ case class Cast( to match { case dt if dt == from => identity[Any] case VariantType => input => variant.VariantExpressionEvalUtils.castToVariant(input, from) - case _: StringType => castToString(from) + case s: StringType => castToString(from, s.constraint) case BinaryType => castToBinary(from) case DateType => castToDate(from) case decimal: DecimalType => castToDecimal(from, decimal) @@ -1244,7 +1237,8 @@ case class Cast( val cls = variant.VariantExpressionEvalUtils.getClass.getName.stripSuffix("$") val fromArg = ctx.addReferenceObj("from", from) (c, evPrim, evNull) => code"$evPrim = $cls.castToVariant($c, $fromArg);" - case _: StringType => (c, evPrim, _) => castToStringCode(from, ctx).apply(c, evPrim) + case s: StringType => + (c, evPrim, _) => castToStringCode(from, ctx, s.constraint).apply(c, evPrim) case BinaryType => castToBinaryCode(from) case DateType => castToDateCode(from, ctx) case decimal: DecimalType => castToDecimalCode(from, decimal, ctx) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala index 130b4ee4c8cac..de72b94df3ac5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala @@ -22,7 +22,7 @@ import java.time.ZoneOffset import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.{ArrayData, DateFormatter, IntervalStringStyles, IntervalUtils, MapData, SparkStringUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.{ArrayData, CharVarcharCodegenUtils, DateFormatter, IntervalStringStyles, IntervalUtils, MapData, SparkStringUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.IntervalStringStyles.ANSI_STYLE import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.BinaryOutputStyle @@ -53,7 +53,17 @@ trait ToStringBase { self: UnaryExpression with TimeZoneAwareExpression => i => func(i.asInstanceOf[T]) // Returns a function to convert a value to pretty string. The function assumes input is not null. - protected final def castToString(from: DataType): Any => UTF8String = from match { + protected final def castToString( + from: DataType, to: StringConstraint = NoConstraint): Any => UTF8String = + to match { + case FixedLength(length) => + s => CharVarcharCodegenUtils.charTypeWriteSideCheck(castToString(from)(s), length) + case MaxLength(length) => + s => CharVarcharCodegenUtils.varcharTypeWriteSideCheck(castToString(from)(s), length) + case NoConstraint => castToString(from) + } + + private def castToString(from: DataType): Any => UTF8String = from match { case CalendarIntervalType => acceptAny[CalendarInterval](i => UTF8String.fromString(i.toString)) case BinaryType => acceptAny[Array[Byte]](binaryFormatter.apply) @@ -167,8 +177,31 @@ trait ToStringBase { self: UnaryExpression with TimeZoneAwareExpression => // Returns a function to generate code to convert a value to pretty string. It assumes the input // is not null. - @scala.annotation.tailrec protected final def castToStringCode( + from: DataType, + ctx: CodegenContext, + to: StringConstraint = NoConstraint): (ExprValue, ExprValue) => Block = + (c, evPrim) => { + val tmpVar = ctx.freshVariable("tmp", classOf[UTF8String]) + val castToString = castToStringCode(from, ctx)(c, tmpVar) + val maintainConstraint = to match { + case FixedLength(length) => + code"""$evPrim = org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils + .charTypeWriteSideCheck($tmpVar, $length);""".stripMargin + case MaxLength(length) => + code"""$evPrim = org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils + .varcharTypeWriteSideCheck($tmpVar, $length);""".stripMargin + case NoConstraint => code"$evPrim = $tmpVar;" + } + code""" + UTF8String $tmpVar; + $castToString + $maintainConstraint + """ + } + + @scala.annotation.tailrec + private def castToStringCode( from: DataType, ctx: CodegenContext): (ExprValue, ExprValue) => Block = { from match { case BinaryType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index f3bed39bcb9f0..c1225f9e5b502 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -204,7 +204,7 @@ object Literal { case VarcharType(length) => create(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(UTF8String.fromString(""), length), dataType) - case st: StringType if st.constraint == NoConstraint => Literal(UTF8String.fromString(""), st) + case st: StringType => Literal(UTF8String.fromString(""), st) case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8)) case CalendarIntervalType => Literal(new CalendarInterval(0, 0, 0)) case arr: ArrayType => create(Array(), arr) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala index 3db0f54f1a8f1..6ba7e528ea230 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala @@ -74,7 +74,7 @@ object CharVarcharUtils extends Logging with SparkCharVarcharUtils { def replaceCharVarcharWithStringForCast(dt: DataType): DataType = { if (SQLConf.get.charVarcharAsString) { replaceCharVarcharWithString(dt) - } else if (hasCharVarchar(dt)) { + } else if (hasCharVarchar(dt) && !SQLConf.get.preserveCharVarcharTypeInfo) { logWarning(log"The Spark cast operator does not support char/varchar type and simply treats" + log" them as string type. Please use string type directly to avoid confusion. Otherwise," + log" you can set ${MDC(CONFIG, SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key)} " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 1d9172e2ba0d2..53b82f9e86f77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, NoConstraint, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringHelper, StringType, StructField, StructType} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SparkSchemaUtils @@ -328,7 +328,7 @@ private[spark] object SchemaUtils { StructType(fields.map { field => field.copy(dataType = replaceCollatedStringWithString(field.dataType)) }) - case st: StringType if st.constraint == NoConstraint => StringType + case st: StringType => StringHelper.removeCollation(st) case _ => dt } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index d7c6c94158b18..cec49a5ae1de0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -1015,14 +1015,10 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } } - test("disallow type conversions between calendar interval type and char/varchar types") { + test("allow type conversions between calendar interval type and char/varchar types") { Seq(CharType(10), VarcharType(10)) .foreach { typ => - verifyCastFailure( - cast(Literal.default(CalendarIntervalType), typ), - DataTypeMismatch( - "CAST_WITHOUT_SUGGESTION", - Map("srcType" -> "\"INTERVAL\"", "targetType" -> toSQLType(typ)))) + assert(cast(Literal.default(CalendarIntervalType), typ).checkInputDataTypes().isSuccess) } } @@ -1428,4 +1424,37 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { assert(!Cast(timestampLiteral, StringType).resolved) assert(!Cast(timestampLiteral, StringType("UTF8_LCASE")).resolved) } + + test(s"Casting from char/varchar") { + Seq(CharType(10), VarcharType(10)).foreach { typ => + Seq( + IntegerType -> ("123", 123), + LongType -> ("123 ", 123L), + BooleanType -> ("true ", true), + BooleanType -> ("false", false), + DoubleType -> ("1.2", 1.2) + ).foreach { case (toType, (from, to)) => + checkEvaluation(cast(Literal.create(from, typ), toType), to) + } + } + } + + test("Casting to char/varchar") { + Seq(CharType(10), VarcharType(10)).foreach { typ => + Seq( + IntegerType -> (123, "123"), + LongType -> (123L, "123"), + BooleanType -> (true, "true"), + BooleanType -> (false, "false"), + DoubleType -> (1.2, "1.2") + ).foreach { case (fromType, (from, to)) => + val paddedTo = if (typ.isInstanceOf[CharType]) { + to.padTo(10, ' ') + } else { + to + } + checkEvaluation(cast(Literal.create(from, fromType), typ), paddedTo) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index a5cbeb552dcba..579756038cb52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -695,6 +695,89 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { } } } + + test(s"insert string literal into char/varchar column when " + + s"${SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key} is true") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + withTable("t") { + sql(s"CREATE TABLE t(c1 CHAR(5), c2 VARCHAR(5)) USING $format") + sql("INSERT INTO t VALUES ('1234', '1234')") + checkAnswer(spark.table("t"), Row("1234 ", "1234")) + assertLengthCheckFailure("INSERT INTO t VALUES ('123456', '1')") + assertLengthCheckFailure("INSERT INTO t VALUES ('1', '123456')") + } + } + } + + test(s"insert from string column into char/varchar column when " + + s"${SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key} is true") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + withTable("a", "b") { + sql(s"CREATE TABLE a AS SELECT '1234' as c1, '1234' as c2") + sql(s"CREATE TABLE b(c1 CHAR(5), c2 VARCHAR(5)) USING $format") + sql("INSERT INTO b SELECT * FROM a") + checkAnswer(spark.table("b"), Row("1234 ", "1234")) + spark.table("b").show() + } + } + } + + test(s"cast from char/varchar when ${SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key} is true") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + Seq("char(5)", "varchar(5)").foreach { typ => + Seq( + "int" -> ("123", 123), + "long" -> ("123 ", 123L), + "boolean" -> ("true ", true), + "boolean" -> ("false", false), + "double" -> ("1.2", 1.2) + ).foreach { case (toType, (from, to)) => + assert(sql(s"select cast($from :: $typ as $toType)").collect() === Array(Row(to))) + } + } + } + } + + test(s"cast to char/varchar when ${SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key} is true") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + Seq("char(10)", "varchar(10)").foreach { typ => + Seq( + 123 -> "123", + 123L-> "123", + true -> "true", + false -> "false", + 1.2 -> "1.2" + ).foreach { case (from, to) => + val paddedTo = if (typ == "char(10)") { + to.padTo(10, ' ') + } else { + to + } + sql(s"select cast($from as $typ)").collect() === Array(Row(paddedTo)) + } + } + } + } + + test("implicitly cast char/varchar into atomics") { + Seq("char", "varchar").foreach { typ => + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + checkAnswer(sql( + s""" + |SELECT + |NOT('false'::$typ(5)), + |1 + ('4'::$typ(5)), + |2L + ('4'::$typ(5)), + |3S + ('4'::$typ(5)), + |4Y - ('4'::$typ(5)), + |1.2 / ('0.6'::$typ(5)), + |MINUTE('2009-07-30 12:58:59'::$typ(30)), + |if(true, '0'::$typ(5), 1), + |if(false, '0'::$typ(5), 1) + """.stripMargin), Row(true, 5, 6, 7, 0, 2.0, 58, 0, 1)) + } + } + } } // Some basic char/varchar tests which doesn't rely on table implementation. From 9547a476df1f13fb0c6cd19665247a3fef79bf27 Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Fri, 10 Jan 2025 07:09:13 -0800 Subject: [PATCH 386/438] [SPARK-50776][K8S][TESTS] Fix test assertions on executor service account ### What changes were proposed in this pull request? `ExecutorKubernetesCredentialsFeatureStepSuite` tests that Spark sets the correct service account on executor pods for various configuration combinations. This patch corrects some invalid test assertions and refactors to reduce redundancy across the different test cases. ### Why are the changes needed? `ExecutorKubernetesCredentialsFeatureStepSuite` attempts to check that the Spark code sets the correct service account on the executor pod. However, the current assertions are actually no-ops that check if a variable is equal to itself, which is always true. The test would pass even if the product code had a bug. ### Does this PR introduce _any_ user-facing change? No, this is a change in tests only. ### How was this patch tested? 1. Intentionally introduce a bug in `ExecutorKubernetesCredentialsFeatureStep` by hard-coding a bogus service account (not included in this pull request). 1. `build/mvn -o -Pkubernetes -Pscala-2.12 -pl resource-managers/kubernetes/core -Dsuites='org.apache.spark.deploy.k8s.features.ExecutorKubernetesCredentialsFeatureStepSuite' test` 1. The test succeeds, even though there is a bug. 1. Apply this patch. 1. Rerun the test, and it fails, as it should. 1. Revert the bug, rerun the test, and it succeeds. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49428 from cnauroth/SPARK-50776. Authored-by: Chris Nauroth Signed-off-by: Dongjoon Hyun --- ...ubernetesCredentialsFeatureStepSuite.scala | 58 +++++++------------ 1 file changed, 21 insertions(+), 37 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala index 59cc7ac91d1ab..7525edd0d25fe 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.deploy.k8s.features import org.scalatest.BeforeAndAfter +import io.fabric8.kubernetes.api.model.PodSpec + import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ class ExecutorKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { @@ -30,58 +32,40 @@ class ExecutorKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with B baseConf = new SparkConf(false) } - private def newExecutorConf(environment: Map[String, String] = Map.empty): - KubernetesExecutorConf = { - KubernetesTestConf.createExecutorConf( - sparkConf = baseConf, - environment = environment) - } - test("configure spark pod with executor service account") { baseConf.set(KUBERNETES_EXECUTOR_SERVICE_ACCOUNT_NAME, "executor-name") - val step = new ExecutorKubernetesCredentialsFeatureStep(newExecutorConf()) - val spec = step - .configurePod(SparkPod.initialPod()) - .pod - .getSpec - - val serviceAccountName = spec.getServiceAccountName - val accountName = spec.getServiceAccount - assertSAName(serviceAccountName, accountName) + val spec = evaluateStep() + assertSAName("executor-name", spec) } test("configure spark pod with with driver service account " + "and without executor service account") { baseConf.set(KUBERNETES_DRIVER_SERVICE_ACCOUNT_NAME, "driver-name") - val step = new ExecutorKubernetesCredentialsFeatureStep(newExecutorConf()) - val spec = step - .configurePod(SparkPod.initialPod()) - .pod - .getSpec - - val serviceAccountName = spec.getServiceAccountName - val accountName = spec.getServiceAccount - assertSAName(serviceAccountName, accountName) + val spec = evaluateStep() + assertSAName("driver-name", spec) } test("configure spark pod with with driver service account " + "and with executor service account") { baseConf.set(KUBERNETES_DRIVER_SERVICE_ACCOUNT_NAME, "driver-name") baseConf.set(KUBERNETES_EXECUTOR_SERVICE_ACCOUNT_NAME, "executor-name") + val spec = evaluateStep() + assertSAName("executor-name", spec) + } + + private def assertSAName(expectedServiceAccountName: String, + spec: PodSpec): Unit = { + assert(spec.getServiceAccountName.equals(expectedServiceAccountName)) + assert(spec.getServiceAccount.equals(expectedServiceAccountName)) + } - val step = new ExecutorKubernetesCredentialsFeatureStep(newExecutorConf()) - val spec = step + private def evaluateStep(): PodSpec = { + val executorConf = KubernetesTestConf.createExecutorConf( + sparkConf = baseConf) + val step = new ExecutorKubernetesCredentialsFeatureStep(executorConf) + step .configurePod(SparkPod.initialPod()) .pod .getSpec - - val serviceAccountName = spec.getServiceAccountName - val accountName = spec.getServiceAccount - assertSAName(serviceAccountName, accountName) - } - - def assertSAName(serviceAccountName: String, accountName: String): Unit = { - assert(serviceAccountName.equals(serviceAccountName)) - assert(accountName.equals(accountName)) } } From 62a23dd55a69fd426a36d8fb7067e3114f17e0d8 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 10 Jan 2025 08:28:17 -0800 Subject: [PATCH 387/438] [SPARK-50780][SQL] Use `overrideStdFeatures` instead of `setFeatureMask` in `JacksonParser` ### What changes were proposed in this pull request? In https://github.com/apache/spark/pull/49018, the restoration logic for feature flags was fixed using the `setFeatureMask` method. However, the `setFeatureMask` method has been deprecated since Jackson 2.7, so this pr reimplements the relevant logic using `overrideStdFeatures`. ### Why are the changes needed? Clean up the use of deprecated APIs. https://github.com/FasterXML/jackson-core/blob/0d2b0f39200d466f49f1abb06d9027053d41483d/src/main/java/com/fasterxml/jackson/core/JsonParser.java#L999-L1035 ``` /** * Bulk set method for (re)setting states of all standard {link Feature}s * * param mask Bit mask that defines set of features to enable * * return This parser, to allow call chaining * * since 2.3 * deprecated Since 2.7, use {link #overrideStdFeatures(int, int)} instead */ Deprecated public JsonParser setFeatureMask(int mask) { _features = mask; return this; } /** * Bulk set method for (re)setting states of features specified by mask. * Functionally equivalent to * * int oldState = getFeatureMask(); * int newState = (oldState & ~mask) | (values & mask); * setFeatureMask(newState); * * but preferred as this lets caller more efficiently specify actual changes made. * * param values Bit mask of set/clear state for features to change * param mask Bit mask of features to change * * return This parser, to allow call chaining * * since 2.6 */ public JsonParser overrideStdFeatures(int values, int mask) { int newState = (_features & ~mask) | (values & mask); return setFeatureMask(newState); } ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Specialized tests have already been added in https://github.com/apache/spark/pull/49018: "feature mask should remain unchanged" in `JacksonParserSuite`. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49434 from LuciferYang/setFeatureMask. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/json/JacksonParser.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 19e2c4228236e..1cd4b4cd29bcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -293,7 +293,8 @@ class JacksonParser( case _: StringType => (parser: JsonParser) => { // This must be enabled if we will retrieve the bytes directly from the raw content: val oldFeature = parser.getFeatureMask - parser.setFeatureMask(oldFeature | JsonParser.Feature.INCLUDE_SOURCE_IN_LOCATION.getMask) + val featureToAdd = JsonParser.Feature.INCLUDE_SOURCE_IN_LOCATION.getMask + parser.overrideStdFeatures(oldFeature | featureToAdd, featureToAdd) val result = parseJsonToken[UTF8String](parser, dataType) { case VALUE_STRING => UTF8String.fromString(parser.getText) @@ -338,8 +339,11 @@ class JacksonParser( UTF8String.fromBytes(writer.toByteArray) } } - // Reset back to the original configuration: - parser.setFeatureMask(oldFeature) + // Reset back to the original configuration using `~0` as the mask, + // which is a bitmask with all bits set, effectively allowing all features + // to be reset. This ensures that every feature is restored to its previous + // state as defined by `oldFeature`. + parser.overrideStdFeatures(oldFeature, ~0) result } From d3848ced1aff9db30fb62eb3c223a5cfb87bbc10 Mon Sep 17 00:00:00 2001 From: William Hyun Date: Fri, 10 Jan 2025 08:30:27 -0800 Subject: [PATCH 388/438] [SPARK-50728][BUILD] Update ORC to 2.1.0 ### What changes were proposed in this pull request? This PR aims to upgrade ORC to 2.1.0. ### Why are the changes needed? To use the most recent version. - https://github.com/apache/orc/releases/tag/v2.1.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 #49358 from williamhyun/SPARK-50728. Authored-by: William Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 8 ++++---- pom.xml | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 9be4986101351..875499320b481 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -234,10 +234,10 @@ opencsv/2.3//opencsv-2.3.jar opentracing-api/0.33.0//opentracing-api-0.33.0.jar opentracing-noop/0.33.0//opentracing-noop-0.33.0.jar opentracing-util/0.33.0//opentracing-util-0.33.0.jar -orc-core/2.0.3/shaded-protobuf/orc-core-2.0.3-shaded-protobuf.jar +orc-core/2.1.0/shaded-protobuf/orc-core-2.1.0-shaded-protobuf.jar orc-format/1.0.0/shaded-protobuf/orc-format-1.0.0-shaded-protobuf.jar -orc-mapreduce/2.0.3/shaded-protobuf/orc-mapreduce-2.0.3-shaded-protobuf.jar -orc-shims/2.0.3//orc-shims-2.0.3.jar +orc-mapreduce/2.1.0/shaded-protobuf/orc-mapreduce-2.1.0-shaded-protobuf.jar +orc-shims/2.1.0//orc-shims-2.1.0.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar @@ -269,7 +269,7 @@ 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.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 +threeten-extra/1.8.0//threeten-extra-1.8.0.jar tink/1.16.0//tink-1.16.0.jar transaction-api/1.1//transaction-api-1.1.jar univocity-parsers/2.9.1//univocity-parsers-2.9.1.jar diff --git a/pom.xml b/pom.xml index 3b242b9ea865d..c64e01d4dcb7a 100644 --- a/pom.xml +++ b/pom.xml @@ -138,7 +138,7 @@ 10.16.1.1 1.15.0 - 2.0.3 + 2.1.0 shaded-protobuf 11.0.24 5.0.0 From f9cb80a5947bbbf909b953fdbed13daa6f6c924f Mon Sep 17 00:00:00 2001 From: Jim Baugh Date: Fri, 10 Jan 2025 08:45:11 -0800 Subject: [PATCH 389/438] [SPARK-50616][SQL] Add File Extension Option to CSV DataSource Writer ### What changes were proposed in this pull request? The existing CSV DataSource allows one to set the delimiter/separator but does not allow the changing of the file extension. This means that a file can have values separated by tabs but me marked as a ".csv" file. This change allows one to change the file extension to match the delimiter/separator (e.g. ".tsv" for a tab separated value file). ### Why are the changes needed? This PR adds an additional option to set the fileExtension. The end result is that when a separator is set that is not a comma that the output file has a file extension that matches the separator (e.g. file.tsv, file.psv, etc...). Notes on Previous Pull Request https://github.com/apache/spark/pull/17973 A pull request adding this option was discussed 7 years ago. One reason it wasn't added was: "I would like to suggest to leave this out if there is no better reason for now. Downside of this is, it looks this allows arbitrary name and it does not gurantee the extention is, say, tsv when the delmiter is a tab. It is purely up to the user." I don't believe this is a good reason to not let the user set the extension. If we let them set the delimiter/separator to an arbitrary string/char then why not let the user also set the file extension to specify the separator that the file uses (e.g. tsv, psv, etc...). This addition keeps the "csv" file extension as the default and has the benefit of allowing other separators to match the file extension. ### Does this PR introduce _any_ user-facing change? Yes. This PR adds one row to the options table for the CSV DataSource documentation to include the "fileExtension" option. ### How was this patch tested? One unit test was added to validate a file is written with the new extension. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49233 from jabbaugh/jbaugh-add-csv-file-ext. Authored-by: Jim Baugh Signed-off-by: Dongjoon Hyun --- .../resources/error/error-conditions.json | 5 +++++ docs/sql-data-sources-csv.md | 6 ++++++ .../spark/sql/catalyst/csv/CSVOptions.scala | 11 ++++++++++ .../sql/errors/QueryExecutionErrors.scala | 10 ++++++++++ .../datasources/csv/CSVFileFormat.scala | 2 +- .../datasources/v2/csv/CSVWrite.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 20 ++++++++++++++++++- 7 files changed, 53 insertions(+), 3 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index e9b32b5f9cbe7..8b266e9d6ac11 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2959,6 +2959,11 @@ "Unsupported dtype: . Valid values: float64, float32." ] }, + "EXTENSION" : { + "message" : [ + "Invalid extension: . Extension is limited to exactly 3 letters (e.g. csv, tsv, etc...)" + ] + }, "INTEGER" : { "message" : [ "expects an integer literal, but got ." diff --git a/docs/sql-data-sources-csv.md b/docs/sql-data-sources-csv.md index 97a7065e0598f..8008bc562082c 100644 --- a/docs/sql-data-sources-csv.md +++ b/docs/sql-data-sources-csv.md @@ -60,6 +60,12 @@ Data source options of CSV can be set via: + + + + + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 5a23d6f7a3ccb..6c68bc1aa5890 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -103,6 +103,16 @@ class CSVOptions( val delimiter = CSVExprUtils.toDelimiterStr( parameters.getOrElse(SEP, parameters.getOrElse(DELIMITER, ","))) + + val extension = { + val ext = parameters.getOrElse(EXTENSION, "csv") + if (ext.size != 3 && !ext.forall(_.isLetter)) { + throw QueryExecutionErrors.invalidFileExtensionError(EXTENSION, ext) + } + + ext + } + val parseMode: ParseMode = parameters.get(MODE).map(ParseMode.fromString).getOrElse(PermissiveMode) val charset = parameters.get(ENCODING).orElse(parameters.get(CHARSET)) @@ -385,6 +395,7 @@ object CSVOptions extends DataSourceOptions { val NEGATIVE_INF = newOption("negativeInf") val TIME_ZONE = newOption("timeZone") val UNESCAPED_QUOTE_HANDLING = newOption("unescapedQuoteHandling") + val EXTENSION = newOption("extension") // Options with alternative val ENCODING = "encoding" val CHARSET = "charset" 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 e500f5e3cbd78..1ae2e5445c0c5 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 @@ -2786,6 +2786,16 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE Map.empty ) + def invalidFileExtensionError(functionName: String, extension: String): RuntimeException = { + new SparkIllegalArgumentException( + errorClass = "INVALID_PARAMETER_VALUE.EXTENSION", + messageParameters = Map( + "functionName" -> toSQLId(functionName), + "parameter" -> toSQLId("extension"), + "fileExtension" -> toSQLId(extension), + "acceptable" -> "Extension is limited to exactly 3 letters (e.g. csv, tsv, etc...)")) + } + def invalidCharsetError(functionName: String, charset: String): RuntimeException = { new SparkIllegalArgumentException( errorClass = "INVALID_PARAMETER_VALUE.CHARSET", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 8ef85ee91aa8f..b2b99e2d0f4ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -86,7 +86,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { } override def getFileExtension(context: TaskAttemptContext): String = { - ".csv" + CodecStreams.getCompressionExtension(context) + "." + csvOptions.extension + CodecStreams.getCompressionExtension(context) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVWrite.scala index f38a1d385a39c..7011fea77d888 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVWrite.scala @@ -58,7 +58,7 @@ case class CSVWrite( } override def getFileExtension(context: TaskAttemptContext): String = { - ".csv" + CodecStreams.getCompressionExtension(context) + "." + csvOptions.extension + CodecStreams.getCompressionExtension(context) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 7cacd8ea2dc50..850e887ac8e75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -3078,6 +3078,23 @@ abstract class CSVSuite } } + test("SPARK-50616: We can write with a tsv file extension") { + withTempPath { path => + val input = Seq( + "1423-11-12T23:41:00", + "1765-03-28", + "2016-01-28T20:00:00" + ).toDF().repartition(1) + input.write.option("extension", "tsv").csv(path.getAbsolutePath) + + val files = Files.list(path.toPath) + .iterator().asScala.map(x => x.getFileName.toString) + .toList.filter(x => x.takeRight(3).equals("tsv")) + + assert(files.size == 1) + } + } + test("SPARK-39904: Parse incorrect timestamp values") { withTempPath { path => Seq( @@ -3308,7 +3325,7 @@ abstract class CSVSuite } test("SPARK-40667: validate CSV Options") { - assert(CSVOptions.getAllOptions.size == 39) + assert(CSVOptions.getAllOptions.size == 40) // Please add validation on any new CSV options here assert(CSVOptions.isValidOption("header")) assert(CSVOptions.isValidOption("inferSchema")) @@ -3347,6 +3364,7 @@ abstract class CSVSuite assert(CSVOptions.isValidOption("compression")) assert(CSVOptions.isValidOption("codec")) assert(CSVOptions.isValidOption("sep")) + assert(CSVOptions.isValidOption("extension")) assert(CSVOptions.isValidOption("delimiter")) assert(CSVOptions.isValidOption("columnPruning")) // Please add validation on any new parquet options with alternative here From be834ee7290ff24db6d1fc9c03cb403aab9ce8b0 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 10 Jan 2025 09:06:00 -0800 Subject: [PATCH 390/438] [SPARK-50776][K8S][TESTS][FOLLOWUP] Fix code style ### What changes were proposed in this pull request? The pr aims to fix code style, following up https://github.com/apache/spark/pull/49428/files ### Why are the changes needed? Fix minor issue as follows: image ### 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 #49444 from panbingkun/SPARK-50776_FOLLOWUP. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- .../ExecutorKubernetesCredentialsFeatureStepSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala index 7525edd0d25fe..6a14711071b80 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.deploy.k8s.features -import org.scalatest.BeforeAndAfter - import io.fabric8.kubernetes.api.model.PodSpec +import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} From 3a84dfc776ae1f1ab2cde1f8d4076c9582b69069 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Fri, 10 Jan 2025 09:39:50 -0800 Subject: [PATCH 391/438] [SPARK-50786][SQL] Remove `removeWhitespace helper` for DESCRIBE TABLE ### What changes were proposed in this pull request? When converting from `toJsonLinkedHashMap` result to `DESCRIBE TABLE`, `removeWhitespace` helper is unnecessary. This PR removes the helper util to ensure equivalent behavior of `DESCRIBE TABLE` output. ### Why are the changes needed? Ensure backwards compatibility for `DESCRIBE TABLE`, after introduction of `DESCRIBE TABLE AS JSON` ### Does this PR introduce _any_ user-facing change? No, it maintains the original output of `DESCRIBE TABLE`. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49433 from asl3/asl3/remove-describeremovewhitespace. Authored-by: Amanda Liu Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/catalog/interface.scala | 14 +++++--------- .../sql-tests/results/show-tables.sql.out | 2 ++ 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 2ebfcf781b97b..de828dfd3e280 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -63,29 +63,25 @@ trait MetadataMapSupport { protected def jsonToString( jsonMap: mutable.LinkedHashMap[String, JValue]): mutable.LinkedHashMap[String, String] = { - def removeWhitespace(str: String): String = { - str.replaceAll("\\s+$", "") - } - val map = new mutable.LinkedHashMap[String, String]() jsonMap.foreach { case (key, jValue) => val stringValue = jValue match { - case JString(value) => removeWhitespace(value) + case JString(value) => value case JArray(values) => values.map(_.values) .map { - case str: String => quoteIdentifier(removeWhitespace(str)) - case other => removeWhitespace(other.toString) + case str: String => quoteIdentifier(str) + case other => other.toString } .mkString("[", ", ", "]") case JObject(fields) => fields.map { case (k, v) => - s"$k=${removeWhitespace(v.values.toString)}" + s"$k=${v.values.toString}" } .mkString("[", ", ", "]") case JInt(value) => value.toString case JDouble(value) => value.toString - case _ => removeWhitespace(jValue.values.toString) + case _ => jValue.values.toString } map.put(key, stringValue) } diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index af1bb75aef88d..93c1444777594 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -128,6 +128,7 @@ View Schema Mode: BINDING Schema: root |-- e: integer (nullable = true) + showdb show_t1 false Catalog: spark_catalog Database: showdb Table: show_t1 @@ -145,6 +146,7 @@ Schema: root |-- c: string (nullable = true) |-- d: string (nullable = true) + showdb show_t2 false Catalog: spark_catalog Database: showdb Table: show_t2 From 9d4b7a5dd44494aa7c19d3f4f83d610530f4e569 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 10 Jan 2025 12:13:27 -0800 Subject: [PATCH 392/438] [SPARK-50784][TESTS] Fix `lint-scala` not to ignore `scalastyle` errors ### What changes were proposed in this pull request? This PR aims to fix `lint-scala` script not to ignore `scalastyle` errors. ### Why are the changes needed? This bug was introduced via the following PR at Apache Spark 3.4.0. - #38258 After the above PR, `lint-scala` ignores `scalastyle` error and only considers the exit code of `scalafmt` like the following CI result. - https://github.com/apache/spark/pull/49428#issuecomment-2582935831 ![Screenshot 2025-01-10 at 07 14 31](https://github.com/user-attachments/assets/bdaa3be3-5daf-401b-a46f-7c02b7610158) ### Does this PR introduce _any_ user-facing change? No, this is a dev-only tool change. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49443 from dongjoon-hyun/SPARK-50784. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/lint-scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/lint-scala b/dev/lint-scala index 23df146a8d1b4..30642a550401e 100755 --- a/dev/lint-scala +++ b/dev/lint-scala @@ -20,8 +20,10 @@ SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" +set -e "$SCRIPT_DIR/scalastyle" "$1" +set +e # For Spark Connect, we actively enforce scalafmt and check that the produced diff is empty. ERRORS=$(./build/mvn \ -Pscala-2.13 \ From f0f850dc21ecf928995736daf8b790082e099221 Mon Sep 17 00:00:00 2001 From: Sakthi Date: Sat, 11 Jan 2025 13:42:23 +0500 Subject: [PATCH 393/438] [SPARK-50787][DOCS] Fix typos and add missing semicolons in sql examples ### What changes were proposed in this pull request? - Corrected documentation typos - Added missing semicolons in SQL examples ### Why are the changes needed? To correct documentation typos and fix SQL syntax examples for clarity. ### Does this PR introduce _any_ user-facing change? Yes, it updates documentation with corrections to typos and SQL examples. ### How was this patch tested? Manually, by building and reviewing the updated documentation. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49448 from the-sakthi/fix-docs. Authored-by: Sakthi Signed-off-by: Max Gekk --- docs/configuration.md | 2 +- docs/running-on-kubernetes.md | 2 +- docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md | 6 +++--- docs/sql-ref-syntax-ddl-alter-table.md | 4 ++-- docs/sql-ref-syntax-qry-star.md | 6 +++--- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 4a85c4f256a95..f080a79580683 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2850,7 +2850,7 @@ Apart from these, the following properties are also available, and may be useful If set to "true", prevent Spark from scheduling tasks on executors that have been excluded due to too many task failures. The algorithm used to exclude executors and nodes can be further controlled by the other "spark.excludeOnFailure" configuration options. - This config will be overriden by "spark.excludeOnFailure.application.enabled" and + This config will be overridden by "spark.excludeOnFailure.application.enabled" and "spark.excludeOnFailure.taskAndStage.enabled" to specify exclusion enablement on individual levels. diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index a0c73813612d0..c7f5d67a6cd85 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -394,7 +394,7 @@ spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount. spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.readOnly=false ``` -To enable shuffle data recovery feature via the built-in `KubernetesLocalDiskShuffleDataIO` plugin, we need to have the followings. You may want to enable `spark.kubernetes.driver.waitToReusePersistentVolumeClaim` additionally. +To enable shuffle data recovery feature via the built-in `KubernetesLocalDiskShuffleDataIO` plugin, we need to have the following. You may want to enable `spark.kubernetes.driver.waitToReusePersistentVolumeClaim` additionally. ``` spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.path=/data/spark-x/executor-x diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 7ad56bf0657b2..9a50db992f9c8 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -54,9 +54,9 @@ ADD JAR '/some/other.jar'; ADD JAR "/path with space/abc.jar"; ADD JARS "/path with space/def.jar" '/path with space/ghi.jar'; ADD JAR "ivy://group:module:version"; -ADD JAR "ivy://group:module:version?transitive=false" -ADD JAR "ivy://group:module:version?transitive=true" -ADD JAR "ivy://group:module:version?exclude=group:module&transitive=true" +ADD JAR "ivy://group:module:version?transitive=false"; +ADD JAR "ivy://group:module:version?transitive=true"; +ADD JAR "ivy://group:module:version?exclude=group:module&transitive=true"; ``` ### Related Statements diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index adcfa8db06f12..28ecc44a5bf7e 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -673,12 +673,12 @@ ALTER TABLE loc_orc SET fileformat orc; ALTER TABLE p1 partition (month=2, day=2) SET fileformat parquet; -- Change the file Location -ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways' +ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways'; -- SET SERDE/ SERDE Properties ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; -ALTER TABLE dbx.tab1 SET SERDE 'org.apache.hadoop' WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee') +ALTER TABLE dbx.tab1 SET SERDE 'org.apache.hadoop' WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee'); -- SET TABLE PROPERTIES ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('winner' = 'loser'); diff --git a/docs/sql-ref-syntax-qry-star.md b/docs/sql-ref-syntax-qry-star.md index 3a997dad644b9..c575727e820e8 100644 --- a/docs/sql-ref-syntax-qry-star.md +++ b/docs/sql-ref-syntax-qry-star.md @@ -21,7 +21,7 @@ license: | ### Description -A shorthand to name all the referencable columns in the FROM clause or a specific table reference's columns or fields in the FROM clause. +A shorthand to name all the referenceable columns in the FROM clause or a specific table reference's columns or fields in the FROM clause. The star clause is most frequently used in the SELECT list. Spark also supports its use in function invocation and certain n-ary operations within the SELECT list and WHERE clause. @@ -38,11 +38,11 @@ except_clause * **name** - If present limits the columns or fields to be named to those in the specified referencable field, column, or table. + If present limits the columns or fields to be named to those in the specified referenceable field, column, or table. * **except_clause** - Optionally prunes columns or fields from the referencable set of columns identified in the select_star clause. + Optionally prunes columns or fields from the referenceable set of columns identified in the select_star clause. * **column_name** From 14a3389151df761d6d6464829ec328bf355985c9 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Sun, 12 Jan 2025 10:46:36 -0800 Subject: [PATCH 394/438] [SPARK-50794][BUILD] Upgrade Ivy to 2.5.3 ### What changes were proposed in this pull request? This pr aims to upgrade ivy from 2.5.2 to 2.5.3. ### Why are the changes needed? The new version bring some bug fix: - FIX: trying to set safe XML features causes SAXExceptions when used with certain XML parsers ([IVY-1647](https://issues.apache.org/jira/browse/IVY-1647)) - FIX: cached Ivy files were not valid in some scenarios ([IVY-1649](https://issues.apache.org/jira/browse/IVY-1649), [IVY-1650](https://issues.apache.org/jira/browse/IVY-1650)) The full release notes as follow: - https://ant.apache.org/ivy/history/2.5.3/release-notes.html ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Run maven test on GitHub Action: https://github.com/LuciferYang/spark/runs/35474430380 ![image](https://github.com/user-attachments/assets/079eed98-82c3-4bfe-b42f-358182b7551d) ### Was this patch authored or co-authored using generative AI tooling? No Closes #49454 from LuciferYang/ivy-253. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- 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 875499320b481..ec3ef63dbe874 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -103,7 +103,7 @@ httpcore/4.4.16//httpcore-4.4.16.jar icu4j/76.1//icu4j-76.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 +ivy/2.5.3//ivy-2.5.3.jar j2objc-annotations/3.0.0//j2objc-annotations-3.0.0.jar jackson-annotations/2.18.2//jackson-annotations-2.18.2.jar jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar diff --git a/pom.xml b/pom.xml index c64e01d4dcb7a..a9065170c0c00 100644 --- a/pom.xml +++ b/pom.xml @@ -145,7 +145,7 @@ 4.0.1 0.10.0 - 2.5.2 + 2.5.3 2.0.8 3.4.1 - 4.29.1 + 4.29.3 3.11.4 3.9.3 5.7.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d9bf604b46c7f..4cf512c0f3cd3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -89,7 +89,7 @@ object BuildCommons { // Google Protobuf version used for generating the protobuf. // SPARK-41247: needs to be consistent with `protobuf.version` in `pom.xml`. - val protoVersion = "4.29.1" + val protoVersion = "4.29.3" // GRPC version used for Spark Connect. val grpcVersion = "1.67.1" } From 898f7afc5dd72e1185cb7d53b3185d38ce94d1d2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 13 Jan 2025 10:26:41 +0800 Subject: [PATCH 397/438] [SPARK-50707][SQL][TESTS][FOLLOWUP] Fix `CharVarcharTestSuite` test case assumption ### What changes were proposed in this pull request? This PR is a follow-up to fix `CharVarcharTestSuite` test case assumption. - #49340 ### Why are the changes needed? To recover the broken Apache Spark NON-ANSI CI: - https://github.com/apache/spark/actions/workflows/build_non_ansi.yml ``` org.apache.spark.sql.DSV2CharVarcharTestSuite org.apache.spark.sql.FileSourceCharVarcharTestSuite org.apache.spark.sql.HiveCharVarcharTestSuite ``` Screenshot 2025-01-12 at 12 15 33 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually test with `SPARK_ANSI_SQL_MODE=false`. Previously, the following three commands fail. ``` $ SPARK_ANSI_SQL_MODE=false build/sbt "sql/testOnly *.DSV2CharVarcharTestSuite" $ SPARK_ANSI_SQL_MODE=false build/sbt "sql/testOnly *.FileSourceCharVarcharTestSuite" $ SPARK_ANSI_SQL_MODE=false build/sbt "hive/testOnly *.HiveCharVarcharTestSuite" -Phive ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49458 from dongjoon-hyun/SPARK-50707. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- .../test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index 579756038cb52..47ebd387e89a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -761,7 +761,8 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { test("implicitly cast char/varchar into atomics") { Seq("char", "varchar").foreach { typ => - withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true", + SQLConf.ANSI_ENABLED.key -> "true") { checkAnswer(sql( s""" |SELECT From e284b2ce1ec2f622c62d233b1837e37196fa2e33 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 12 Jan 2025 18:29:40 -0800 Subject: [PATCH 398/438] [SPARK-50525][SQL][TESTS][FOLLOWUP] Fix `DataFrameSuite.repartition by MapType` test assumption ### What changes were proposed in this pull request? This is a follow-up to recover the NON-ANSI mode CI failure by adding a test assumption clearly. - #49144 ### Why are the changes needed? **BEFORE** ``` $ SPARK_ANSI_SQL_MODE=false build/sbt "sql/testOnly *.DataFrameSuite -- -z MapType" [info] *** 1 TEST FAILED *** [error] Failed tests: [error] org.apache.spark.sql.DataFrameSuite ``` **AFTER** ``` $ SPARK_ANSI_SQL_MODE=false build/sbt "sql/testOnly *.DataFrameSuite -- -z MapType" [info] All tests passed. ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually test with `SPARK_ANSI_SQL_MODE=false`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49457 from dongjoon-hyun/SPARK-50525. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/DataFrameSuite.scala | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 317a88edf8e95..71d55b007aa17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -430,21 +430,23 @@ class DataFrameSuite extends QueryTest test("repartition by MapType") { Seq("int", "long", "float", "double", "decimal(10, 2)", "string", "varchar(6)").foreach { dt => - val df = spark.range(20) - .withColumn("c1", - when(col("id") % 3 === 1, typedLit(Map(1 -> 1))) - .when(col("id") % 3 === 2, typedLit(Map(1 -> 1, 2 -> 2))) - .otherwise(typedLit(Map(2 -> 2, 1 -> 1))).cast(s"map<$dt, $dt>")) - .withColumn("c2", typedLit(Map(1 -> null)).cast(s"map<$dt, $dt>")) - .withColumn("c3", lit(null).cast(s"map<$dt, $dt>")) - - assertPartitionNumber(df.repartition(4, col("c1")), 2) - assertPartitionNumber(df.repartition(4, col("c2")), 1) - assertPartitionNumber(df.repartition(4, col("c3")), 1) - assertPartitionNumber(df.repartition(4, col("c1"), col("c2")), 2) - assertPartitionNumber(df.repartition(4, col("c1"), col("c3")), 2) - assertPartitionNumber(df.repartition(4, col("c1"), col("c2"), col("c3")), 2) - assertPartitionNumber(df.repartition(4, col("c2"), col("c3")), 2) + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + val df = spark.range(20) + .withColumn("c1", + when(col("id") % 3 === 1, typedLit(Map(1 -> 1))) + .when(col("id") % 3 === 2, typedLit(Map(1 -> 1, 2 -> 2))) + .otherwise(typedLit(Map(2 -> 2, 1 -> 1))).cast(s"map<$dt, $dt>")) + .withColumn("c2", typedLit(Map(1 -> null)).cast(s"map<$dt, $dt>")) + .withColumn("c3", lit(null).cast(s"map<$dt, $dt>")) + + assertPartitionNumber(df.repartition(4, col("c1")), 2) + assertPartitionNumber(df.repartition(4, col("c2")), 1) + assertPartitionNumber(df.repartition(4, col("c3")), 1) + assertPartitionNumber(df.repartition(4, col("c1"), col("c2")), 2) + assertPartitionNumber(df.repartition(4, col("c1"), col("c3")), 2) + assertPartitionNumber(df.repartition(4, col("c1"), col("c2"), col("c3")), 2) + assertPartitionNumber(df.repartition(4, col("c2"), col("c3")), 2) + } } } From 4cecde625df1afcb209eac84b41e39a9ea1ad776 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 12 Jan 2025 18:30:47 -0800 Subject: [PATCH 399/438] [SPARK-50797][SQL][TESTS] Move `HiveCharVarcharTestSuite` from `o/a/s/sql` to `o/a/s/sql/hive` ### What changes were proposed in this pull request? This PR aims to move `HiveCharVarcharTestSuite` from `o/a/s/sql` to `o/a/s/sql/hive`. ### Why are the changes needed? All source codes of `hive` module should have `org.apache.spark.hive` package prefix. ### Does this PR introduce _any_ user-facing change? No. This is a test class relocation. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49459 from dongjoon-hyun/SPARK-50797. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/{ => hive}/HiveCharVarcharTestSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) rename sql/hive/src/test/scala/org/apache/spark/sql/{ => hive}/HiveCharVarcharTestSuite.scala (98%) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/HiveCharVarcharTestSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCharVarcharTestSuite.scala similarity index 98% rename from sql/hive/src/test/scala/org/apache/spark/sql/HiveCharVarcharTestSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCharVarcharTestSuite.scala index c12d727e59740..90cb5501ee6f6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/HiveCharVarcharTestSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCharVarcharTestSuite.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.hive +import org.apache.spark.sql.{CharVarcharTestSuite, Row} import org.apache.spark.sql.execution.command.CharVarcharDDLTestBase import org.apache.spark.sql.hive.test.TestHiveSingleton From 3f864558aaff93285c86fe841adf9bd500251e1a Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 13 Jan 2025 13:04:54 +0800 Subject: [PATCH 400/438] [SPARK-50782][SQL] Replace the use of reflection in `CodeGenerator.updateAndGetCompilationStats` with direct calls to `CodeAttribute#code` ### What changes were proposed in this pull request? This pr replace the use of reflection in `CodeGenerator.updateAndGetCompilationStats` with direct calls to `CodeAttribute#code` because CodeAttribute#code became public after janino 3.1.7 and Spark use janino 3.1.9 now. https://github.com/janino-compiler/janino/commit/e2531169ab68ee5231c82f5b12d6e52e95b6ffdc#diff-1d1702934919f4e7e80d55eef45f9eed8b7f0b4c39b4ba04f95fe5bea42ea874 ![image](https://github.com/user-attachments/assets/10bd630c-e9e3-4c28-966f-f36e65719590) ### Why are the changes needed? Reduce unnecessary reflection calls. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Existing test is "metrics are recorded on compile" in `CodeGenerationSuite` ### Was this patch authored or co-authored using generative AI tooling? No Closes #49436 from LuciferYang/SPARK-50782. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: yangjie01 --- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 30c00f5bf96b8..de74bb2f8cd21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.{ExecutionError, UncheckedExecutionExce import org.codehaus.commons.compiler.{CompileException, InternalCompilerException} import org.codehaus.janino.ClassBodyEvaluator import org.codehaus.janino.util.ClassFile +import org.codehaus.janino.util.ClassFile.CodeAttribute import org.apache.spark.{SparkException, SparkIllegalArgumentException, TaskContext, TaskKilledException} import org.apache.spark.executor.InputMetrics @@ -1578,9 +1579,6 @@ object CodeGenerator extends Logging { val classes = evaluator.getBytecodes.asScala // Then walk the classes to get at the method bytecode. - val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") - val codeAttrField = codeAttr.getDeclaredField("code") - codeAttrField.setAccessible(true) val codeStats = classes.map { case (_, classBytes) => val classCodeSize = classBytes.length CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classCodeSize) @@ -1588,8 +1586,8 @@ object CodeGenerator extends Logging { val cf = new ClassFile(new ByteArrayInputStream(classBytes)) val constPoolSize = cf.getConstantPoolSize val methodCodeSizes = cf.methodInfos.asScala.flatMap { method => - method.getAttributes().filter(_.getClass eq codeAttr).map { a => - val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length + method.getAttributes.collect { case attr: CodeAttribute => + val byteCodeSize = attr.code.length CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) if (byteCodeSize > DEFAULT_JVM_HUGE_METHOD_LIMIT) { From d7545d0fc0962849f979a29345ac14169688edf0 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 13 Jan 2025 16:10:02 +0800 Subject: [PATCH 401/438] [SPARK-50624][SQL] Add TimestampNTZType to ColumnarRow/MutableColumnarRow ### What changes were proposed in this pull request? Noticed that this was missing when using this in Iceberg. See additional details in https://github.com/apache/iceberg/pull/11815#discussion_r1892693224 ### Why are the changes needed? To be able to read `TimestampNTZType` when using `ColumnarRow` ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Added some unit tests that failed without the fix ### Was this patch authored or co-authored using generative AI tooling? No Closes #49437 from nastra/SPARK-50624. Authored-by: Eduard Tudenhoefner Signed-off-by: Wenchen Fan --- .../spark/sql/vectorized/ColumnarRow.java | 2 ++ .../vectorized/MutableColumnarRow.java | 4 ++++ .../vectorized/ColumnVectorSuite.scala | 19 +++++++++++++++ .../vectorized/ArrowColumnVectorSuite.scala | 24 +++++++++++++++++++ 4 files changed, 49 insertions(+) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java index aaac980bb332a..ac05981da5a24 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java @@ -188,6 +188,8 @@ public Object get(int ordinal, DataType dataType) { return getInt(ordinal); } else if (dataType instanceof TimestampType) { return getLong(ordinal); + } else if (dataType instanceof TimestampNTZType) { + return getLong(ordinal); } else if (dataType instanceof ArrayType) { return getArray(ordinal); } else if (dataType instanceof StructType) { diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java index 0464fe8159898..42d39457330c1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java @@ -82,6 +82,8 @@ public InternalRow copy() { row.setInt(i, getInt(i)); } else if (dt instanceof TimestampType) { row.setLong(i, getLong(i)); + } else if (dt instanceof TimestampNTZType) { + row.setLong(i, getLong(i)); } else if (dt instanceof StructType) { row.update(i, getStruct(i, ((StructType) dt).fields().length).copy()); } else if (dt instanceof ArrayType) { @@ -191,6 +193,8 @@ public Object get(int ordinal, DataType dataType) { return getInt(ordinal); } else if (dataType instanceof TimestampType) { return getLong(ordinal); + } else if (dataType instanceof TimestampNTZType) { + return getLong(ordinal); } else if (dataType instanceof ArrayType) { return getArray(ordinal); } else if (dataType instanceof StructType structType) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index 0cc4f7bf2548e..0edbfd10d8cde 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -274,6 +274,19 @@ class ColumnVectorSuite extends SparkFunSuite with SQLHelper { } } + testVectors("mutable ColumnarRow with TimestampNTZType", 10, TimestampNTZType) { testVector => + val mutableRow = new MutableColumnarRow(Array(testVector)) + (0 until 10).foreach { i => + mutableRow.rowId = i + mutableRow.setLong(0, 10 - i) + } + (0 until 10).foreach { i => + mutableRow.rowId = i + assert(mutableRow.get(0, TimestampNTZType) === (10 - i)) + assert(mutableRow.copy().get(0, TimestampNTZType) === (10 - i)) + } + } + val arrayType: ArrayType = ArrayType(IntegerType, containsNull = true) testVectors("array", 10, arrayType) { testVector => @@ -384,18 +397,24 @@ class ColumnVectorSuite extends SparkFunSuite with SQLHelper { } val structType: StructType = new StructType().add("int", IntegerType).add("double", DoubleType) + .add("ts", TimestampNTZType) testVectors("struct", 10, structType) { testVector => val c1 = testVector.getChild(0) val c2 = testVector.getChild(1) + val c3 = testVector.getChild(2) c1.putInt(0, 123) c2.putDouble(0, 3.45) + c3.putLong(0, 1000L) c1.putInt(1, 456) c2.putDouble(1, 5.67) + c3.putLong(1, 2000L) assert(testVector.getStruct(0).get(0, IntegerType) === 123) assert(testVector.getStruct(0).get(1, DoubleType) === 3.45) + assert(testVector.getStruct(0).get(2, TimestampNTZType) === 1000L) assert(testVector.getStruct(1).get(0, IntegerType) === 456) assert(testVector.getStruct(1).get(1, DoubleType) === 5.67) + assert(testVector.getStruct(1).get(2, TimestampNTZType) === 2000L) } testVectors("SPARK-44805: getInts with dictionary", 3, IntegerType) { testVector => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala index 436cea50ad972..9180ce1aee198 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala @@ -515,4 +515,28 @@ class ArrowColumnVectorSuite extends SparkFunSuite { columnVector.close() allocator.close() } + + test("struct with TimestampNTZType") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("struct", 0, Long.MaxValue) + val schema = new StructType().add("ts", TimestampNTZType) + val vector = ArrowUtils.toArrowField("struct", schema, nullable = true, null) + .createVector(allocator).asInstanceOf[StructVector] + vector.allocateNew() + val timestampVector = vector.getChildByOrdinal(0).asInstanceOf[TimeStampMicroVector] + + vector.setIndexDefined(0) + timestampVector.setSafe(0, 1000L) + + timestampVector.setValueCount(1) + vector.setValueCount(1) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === schema) + + val row0 = columnVector.getStruct(0) + assert(row0.get(0, TimestampNTZType) === 1000L) + + columnVector.close() + allocator.close() + } } From aaff18b98044471840d696c047df25a2149841df Mon Sep 17 00:00:00 2001 From: Mihailo Timotic Date: Mon, 13 Jan 2025 21:07:59 +0800 Subject: [PATCH 402/438] [SPARK-50798][SQL] Improve `NormalizePlan` ### What changes were proposed in this pull request? Improvement: Normalize `InheritAnalysisRules` nodes to avoid non-deterministic behavior when comparing plans. This PR also reverts #49230 as that change is no longer necessary ### Why are the changes needed? To compare single-pass and fixed-point analyzer results. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a new test case to `NormalizePlanSuite`. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49460 from mihailotim-db/mihailotim-db/normalize_inherit. Authored-by: Mihailo Timotic Signed-off-by: Wenchen Fan --- .../sql/catalyst/plans/NormalizePlan.scala | 55 ++++++++++----- .../catalyst/plans/NormalizePlanSuite.scala | 69 ++++++++++++++----- 2 files changed, 87 insertions(+), 37 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala index d7ba596cf3995..13df749c6d584 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala @@ -25,8 +25,34 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical._ object NormalizePlan extends PredicateHelper { - def apply(plan: LogicalPlan): LogicalPlan = - normalizePlan(normalizeExprIds(plan)) + def apply(plan: LogicalPlan): LogicalPlan = { + val withNormalizedInheritAnalysis = normalizeInheritAnalysisRules(plan) + val withNormalizedExprIds = normalizeExprIds(withNormalizedInheritAnalysis) + normalizePlan(withNormalizedExprIds) + } + + /** + * Normalize [[InheritAnalysisRules]] nodes by replacing them with their replacement expressions. + * This is necessary because fixed-point analyzer may produce non-deterministic results when + * resolving original expressions. For example, in a query like: + * + * {{{ SELECT assert_true(1) }}} + * + * Before resolution, we have [[UnresolvedFunction]] whose child is Literal(1). This child will + * first be converted to Cast(Literal(1), BooleanType) by type coercion. Because in this case + * [[Cast]] doesn't require timezone, the expression will be implicitly resolved. Because the + * child of initially unresolved function is resolved, the function can be converted to + * [[AssertTrue]], which is of type [[InheritAnalysisRules]]. However, because the only child of + * [[InheritAnalysisRules]] is the replacement expression, the original expression will be lost + * timezone will never be applied. This causes inconsistencies, because fixed-point semantic is + * to ALWAYS apply timezone, regardless of whether or not the Cast actually needs it. + */ + def normalizeInheritAnalysisRules(plan: LogicalPlan): LogicalPlan = { + plan transformAllExpressions { + case inheritAnalysisRules: InheritAnalysisRules => + inheritAnalysisRules.child + } + } /** * Since attribute references are given globally unique ids during analysis, @@ -102,14 +128,15 @@ object NormalizePlan extends PredicateHelper { .sortBy(_.hashCode()) .reduce(And) Join(left, right, newJoinType, Some(newCondition), hint) - case Project(outerProjectList, innerProject: Project) => - val normalizedInnerProjectList = normalizeProjectList(innerProject.projectList) - val orderedInnerProjectList = normalizedInnerProjectList.sortBy(_.name) - val newInnerProject = - Project(orderedInnerProjectList, innerProject.child) - Project(normalizeProjectList(outerProjectList), newInnerProject) case Project(projectList, child) => - Project(normalizeProjectList(projectList), child) + val projList = projectList + .map { e => + e.transformUp { + case g: GetViewColumnByNameAndOrdinal => g.copy(viewDDL = None) + } + } + .asInstanceOf[Seq[NamedExpression]] + Project(projList, child) case c: KeepAnalyzedQuery => c.storeAnalyzedQuery() case localRelation: LocalRelation if !localRelation.data.isEmpty => /** @@ -127,16 +154,6 @@ object NormalizePlan extends PredicateHelper { } } - private def normalizeProjectList(projectList: Seq[NamedExpression]): Seq[NamedExpression] = { - projectList - .map { e => - e.transformUp { - case g: GetViewColumnByNameAndOrdinal => g.copy(viewDDL = None) - } - } - .asInstanceOf[Seq[NamedExpression]] - } - /** * Rewrite [[BinaryComparison]] operator to keep order. The following cases will be * equivalent: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala index 80cdf19e4b73b..5ff66098107c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala @@ -18,29 +18,62 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.expressions.{AssertTrue, Cast, If, Literal, TimeZoneAwareExpression} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.types.BooleanType -class NormalizePlanSuite extends SparkFunSuite{ +class NormalizePlanSuite extends SparkFunSuite with SQLConfHelper { - test("Normalize Project") { - val baselineCol1 = $"col1".int - val testCol1 = baselineCol1.newInstance() - val baselinePlan = LocalRelation(baselineCol1).select(baselineCol1) - val testPlan = LocalRelation(testCol1).select(testCol1) + test("Normalize InheritAnalysisRules expressions") { + val castWithoutTimezone = + Cast(child = Literal(1), dataType = BooleanType, ansiEnabled = conf.ansiEnabled) + val castWithTimezone = castWithoutTimezone.withTimeZone(conf.sessionLocalTimeZone) - assert(baselinePlan != testPlan) - assert(NormalizePlan(baselinePlan) == NormalizePlan(testPlan)) - } + val baselineExpression = AssertTrue(castWithTimezone) + val baselinePlan = LocalRelation().select(baselineExpression) + + val testExpression = AssertTrue(castWithoutTimezone) + val testPlan = LocalRelation().select(testExpression) + + // Before calling [[setTimezoneForAllExpression]], [[AssertTrue]] node will look like: + // + // AssertTrue(Cast(Literal(1)), message, If(Cast(Literal(1)), Literal(null), error)) + // + // Calling [[setTimezoneForAllExpression]] will only apply timezone to the second Cast node + // because [[InheritAnalysisRules]] only sees replacement expression as its child. This will + // cause the difference when comparing [[resolvedBaselinePlan]] and [[resolvedTestPlan]], + // therefore we need normalization. + + // Before applying timezone, no timezone is set. + testPlan.expressions.foreach { + case _ @ AssertTrue(firstCast: Cast, _, _ @ If(secondCast: Cast, _, _)) => + assert(firstCast.timeZoneId.isEmpty) + assert(secondCast.timeZoneId.isEmpty) + case _ => + } - test("Normalize ordering in a project list of an inner Project") { - val baselinePlan = - LocalRelation($"col1".int, $"col2".string).select($"col1", $"col2").select($"col1") - val testPlan = - LocalRelation($"col1".int, $"col2".string).select($"col2", $"col1").select($"col1") + val resolvedBaselinePlan = setTimezoneForAllExpression(baselinePlan) + val resolvedTestPlan = setTimezoneForAllExpression(testPlan) + + // After applying timezone, only the second cast gets timezone. + resolvedTestPlan.expressions.foreach { + case _ @ AssertTrue(firstCast: Cast, _, _ @ If(secondCast: Cast, _, _)) => + assert(firstCast.timeZoneId.isEmpty) + assert(secondCast.timeZoneId.isDefined) + case _ => + } + + // However, plans are still different. + assert(resolvedBaselinePlan != resolvedTestPlan) + assert(NormalizePlan(resolvedBaselinePlan) == NormalizePlan(resolvedTestPlan)) + } - assert(baselinePlan != testPlan) - assert(NormalizePlan(baselinePlan) == NormalizePlan(testPlan)) + private def setTimezoneForAllExpression(plan: LogicalPlan): LogicalPlan = { + plan.transformAllExpressions { + case e: TimeZoneAwareExpression if e.timeZoneId.isEmpty => + e.withTimeZone(conf.sessionLocalTimeZone) + } } } From 5ef0eb5f4bb9925bf1a7ba6cb26890415c7d3892 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 13 Jan 2025 09:07:23 -0800 Subject: [PATCH 403/438] [SPARK-50734][SQL][FOLLOWUP] Replace `ScalaObjectMapper` with `ClassTagExtensions` to fix compilation warning ### What changes were proposed in this pull request? This pr aims to replace `ScalaObjectMapper` with `ClassTagExtensions` to fix compilation warning: ``` [warn] /Users/yangjie01/SourceCode/git/spark-mine-sbt/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala:121:40: trait ScalaObjectMapper in package scala is deprecated (since 2.12.1): ScalaObjectMapper is deprecated because Manifests are not supported in Scala3, you might want to use ClassTagExtensions as a replacement [warn] Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.sql.catalyst.catalog.UserDefinedFunction.getObjectMapper.mapper, origin=com.fasterxml.jackson.module.scala.ScalaObjectMapper, version=2.12.1 [warn] val mapper = new ObjectMapper with ScalaObjectMapper ``` the change is refer to: https://github.com/FasterXML/jackson-module-scala/blob/ae04d9f16a2524123c2c083bf981cecdbfc7c72f/src/main/scala-2.%2B/com/fasterxml/jackson/module/scala/ScalaObjectMapper.scala#L23-L24 ``` deprecated("ScalaObjectMapper is deprecated because Manifests are not supported in Scala3, you might want to use ClassTagExtensions as a replacement", "2.12.1") trait ScalaObjectMapper { ``` ### Why are the changes needed? Clean up the use of deprecated APIs. ### 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 #49469 from LuciferYang/SPARK-50734-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/catalog/UserDefinedFunction.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala index fe00184e843a3..b00cae22cf9c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.catalog import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} -import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} +import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule} import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.FunctionIdentifier @@ -118,7 +118,7 @@ object UserDefinedFunction { * Get a object mapper to serialize and deserialize function properties. */ private def getObjectMapper: ObjectMapper = { - val mapper = new ObjectMapper with ScalaObjectMapper + val mapper = new ObjectMapper with ClassTagExtensions mapper.setSerializationInclusion(Include.NON_ABSENT) mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) mapper.registerModule(DefaultScalaModule) From 3569e768e657d4e28ee7520808ec910cdff2b099 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Mon, 13 Jan 2025 11:17:05 -0800 Subject: [PATCH 404/438] [SPARK-50789][CONNECT] The inputs for typed aggregations should be analyzed ### What changes were proposed in this pull request? Fixes `SparkConnectPlanner` to analyze the inputs for typed aggregations. ### Why are the changes needed? The inputs for typed aggregations should be analyzed. For example: ```scala val ds = Seq("abc", "xyz", "hello").toDS().select("*").as[String] ds.groupByKey(_.length).reduceGroups(_ + _).show() ``` fails with: ``` org.apache.spark.SparkException: [INTERNAL_ERROR] Invalid call to toAttribute on unresolved object SQLSTATE: XX000 org.apache.spark.sql.catalyst.analysis.Star.toAttribute(unresolved.scala:439) org.apache.spark.sql.catalyst.plans.logical.Project.$anonfun$output$1(basicLogicalOperators.scala:74) scala.collection.immutable.List.map(List.scala:247) scala.collection.immutable.List.map(List.scala:79) org.apache.spark.sql.catalyst.plans.logical.Project.output(basicLogicalOperators.scala:74) org.apache.spark.sql.connect.planner.SparkConnectPlanner.transformExpressionWithTypedReduceExpression(SparkConnectPlanner.scala:2340) org.apache.spark.sql.connect.planner.SparkConnectPlanner.$anonfun$transformKeyValueGroupedAggregate$1(SparkConnectPlanner.scala:2244) scala.collection.immutable.List.map(List.scala:247) scala.collection.immutable.List.map(List.scala:79) org.apache.spark.sql.connect.planner.SparkConnectPlanner.transformKeyValueGroupedAggregate(SparkConnectPlanner.scala:2244) org.apache.spark.sql.connect.planner.SparkConnectPlanner.transformAggregate(SparkConnectPlanner.scala:2232) ... ``` ### Does this PR introduce _any_ user-facing change? The failure will not appear. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49449 from ueshin/issues/SPARK-50789/typed_agg. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../KeyValueGroupedDatasetE2ETestSuite.scala | 8 ++++ .../sql/UserDefinedFunctionE2ETestSuite.scala | 22 +++++++++- .../connect/planner/SparkConnectPlanner.scala | 43 +++++++++++++------ 3 files changed, 59 insertions(+), 14 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala index 6fd664d905408..021b4fea26e2a 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala @@ -460,6 +460,14 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with RemoteSparkSessi (5, "hello")) } + test("SPARK-50789: reduceGroups on unresolved plan") { + val ds = Seq("abc", "xyz", "hello").toDS().select("*").as[String] + checkDatasetUnorderly( + ds.groupByKey(_.length).reduceGroups(_ + _), + (3, "abcxyz"), + (5, "hello")) + } + test("groupby") { val ds = Seq(("a", 1, 10), ("a", 2, 20), ("b", 2, 1), ("b", 1, 2), ("c", 1, 1)) .toDF("key", "seq", "value") diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala index 8415444c10aac..19275326d6421 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala @@ -401,6 +401,13 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession assert(ds.select(aggCol).head() == 135) // 45 + 90 } + test("SPARK-50789: UDAF custom Aggregator - toColumn on unresolved plan") { + val encoder = Encoders.product[UdafTestInput] + val aggCol = new CompleteUdafTestInputAggregator().toColumn + val ds = spark.range(10).withColumn("extra", col("id") * 2).select("*").as(encoder) + assert(ds.select(aggCol).head() == 135) // 45 + 90 + } + test("UDAF custom Aggregator - multiple extends - toColumn") { val encoder = Encoders.product[UdafTestInput] val aggCol = new CompleteGrandChildUdafTestInputAggregator().toColumn @@ -408,11 +415,24 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession assert(ds.select(aggCol).head() == 540) // (45 + 90) * 4 } - test("UDAF custom aggregator - with rows - toColumn") { + test("SPARK-50789: UDAF custom Aggregator - multiple extends - toColumn on unresolved plan") { + val encoder = Encoders.product[UdafTestInput] + val aggCol = new CompleteGrandChildUdafTestInputAggregator().toColumn + val ds = spark.range(10).withColumn("extra", col("id") * 2).select("*").as(encoder) + assert(ds.select(aggCol).head() == 540) // (45 + 90) * 4 + } + + test("UDAF custom Aggregator - with rows - toColumn") { val ds = spark.range(10).withColumn("extra", col("id") * 2) assert(ds.select(RowAggregator.toColumn).head() == 405) assert(ds.agg(RowAggregator.toColumn).head().getLong(0) == 405) } + + test("SPARK-50789: UDAF custom Aggregator - with rows - toColumn on unresolved plan") { + val ds = spark.range(10).withColumn("extra", col("id") * 2).select("*") + assert(ds.select(RowAggregator.toColumn).head() == 405) + assert(ds.agg(RowAggregator.toColumn).head().getLong(0) == 405) + } } case class UdafTestInput(id: Long, extra: Long) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index c0b4384af8b6d..6ab69aea12e5d 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -845,9 +845,10 @@ class SparkConnectPlanner( kEncoder: ExpressionEncoder[_], vEncoder: ExpressionEncoder[_], analyzed: LogicalPlan, - dataAttributes: Seq[Attribute], + analyzedData: LogicalPlan, groupingAttributes: Seq[Attribute], sortOrder: Seq[SortOrder]) { + val dataAttributes: Seq[Attribute] = analyzedData.output val valueDeserializer: Expression = UnresolvedDeserializer(vEncoder.deserializer, dataAttributes) } @@ -900,7 +901,7 @@ class SparkConnectPlanner( dummyFunc.outEnc, dummyFunc.inEnc, qe.analyzed, - analyzed.output, + analyzed, aliasedGroupings, sortOrder) } @@ -924,7 +925,7 @@ class SparkConnectPlanner( kEnc, vEnc, withGroupingKeyAnalyzed, - analyzed.output, + analyzed, withGroupingKey.newColumns, sortOrder) } @@ -1489,11 +1490,19 @@ class SparkConnectPlanner( logical.OneRowRelation() } + val logicalPlan = + if (rel.getExpressionsList.asScala.toSeq.exists( + _.getExprTypeCase == proto.Expression.ExprTypeCase.TYPED_AGGREGATE_EXPRESSION)) { + session.sessionState.executePlan(baseRel).analyzed + } else { + baseRel + } + val projection = rel.getExpressionsList.asScala.toSeq - .map(transformExpression(_, Some(baseRel))) + .map(transformExpression(_, Some(logicalPlan))) .map(toNamedExpression) - logical.Project(projectList = projection, child = baseRel) + logical.Project(projectList = projection, child = logicalPlan) } /** @@ -2241,7 +2250,7 @@ class SparkConnectPlanner( val keyColumn = TypedAggUtils.aggKeyColumn(ds.kEncoder, ds.groupingAttributes) val namedColumns = rel.getAggregateExpressionsList.asScala.toSeq - .map(expr => transformExpressionWithTypedReduceExpression(expr, input)) + .map(expr => transformExpressionWithTypedReduceExpression(expr, ds.analyzedData)) .map(toNamedExpression) logical.Aggregate(ds.groupingAttributes, keyColumn +: namedColumns, ds.analyzed) } @@ -2252,9 +2261,17 @@ class SparkConnectPlanner( } val input = transformRelation(rel.getInput) + val logicalPlan = + if (rel.getAggregateExpressionsList.asScala.toSeq.exists( + _.getExprTypeCase == proto.Expression.ExprTypeCase.TYPED_AGGREGATE_EXPRESSION)) { + session.sessionState.executePlan(input).analyzed + } else { + input + } + val groupingExprs = rel.getGroupingExpressionsList.asScala.toSeq.map(transformExpression) val aggExprs = rel.getAggregateExpressionsList.asScala.toSeq - .map(expr => transformExpressionWithTypedReduceExpression(expr, input)) + .map(expr => transformExpressionWithTypedReduceExpression(expr, logicalPlan)) val aliasedAgg = (groupingExprs ++ aggExprs).map(toNamedExpression) rel.getGroupType match { @@ -2262,19 +2279,19 @@ class SparkConnectPlanner( logical.Aggregate( groupingExpressions = groupingExprs, aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_ROLLUP => logical.Aggregate( groupingExpressions = Seq(Rollup(groupingExprs.map(Seq(_)))), aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_CUBE => logical.Aggregate( groupingExpressions = Seq(Cube(groupingExprs.map(Seq(_)))), aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_PIVOT => if (!rel.hasPivot) { @@ -2286,7 +2303,7 @@ class SparkConnectPlanner( rel.getPivot.getValuesList.asScala.toSeq.map(transformLiteral) } else { RelationalGroupedDataset - .collectPivotValues(Dataset.ofRows(session, input), Column(pivotExpr)) + .collectPivotValues(Dataset.ofRows(session, logicalPlan), Column(pivotExpr)) .map(expressions.Literal.apply) } logical.Pivot( @@ -2294,7 +2311,7 @@ class SparkConnectPlanner( pivotColumn = pivotExpr, pivotValues = valueExprs, aggregates = aggExprs, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_GROUPING_SETS => val groupingSetsExprs = rel.getGroupingSetsList.asScala.toSeq.map { getGroupingSets => @@ -2306,7 +2323,7 @@ class SparkConnectPlanner( groupingSets = groupingSetsExprs, userGivenGroupByExprs = groupingExprs)), aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case other => throw InvalidPlanInput(s"Unknown Group Type $other") } From c1e5f53cbe0fd8b1358d909e5126530abf3ce004 Mon Sep 17 00:00:00 2001 From: Livia Zhu Date: Tue, 14 Jan 2025 08:26:17 +0800 Subject: [PATCH 405/438] [SPARK-50791][SQL] Fix NPE in State Store error handling ### What changes were proposed in this pull request? Directly calling `contains` on a `SparkException.getCondition` may result in a NullPointerException if `getCondition` returns `null`. This change wraps `getCondition` in an option for safe handling. ### Why are the changes needed? Throwing an NPE is a bug. ### Does this PR introduce _any_ user-facing change? Yes. Previously, on SparkException such as OOM that do not have a set condition, users would see an NullPointerException. Now they will correctly see a CANNOT_LOAD_STATE_STORE exception. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49451 from liviazhu-db/liviazhu-db/statestore-npe. Authored-by: Livia Zhu Signed-off-by: yangjie01 --- .../streaming/state/HDFSBackedStateStoreProvider.scala | 3 ++- .../streaming/state/RocksDBStateStoreProvider.scala | 6 ++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index ae06e82335b12..2deccb845fea2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -291,7 +291,8 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with newMap } catch { - case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException + if Option(e.getCondition).exists(_.contains("CANNOT_LOAD_STATE_STORE")) => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 60652367f3351..9fc48a60d7c6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -439,7 +439,8 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException + if Option(e.getCondition).exists(_.contains("CANNOT_LOAD_STATE_STORE")) => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( @@ -462,7 +463,8 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException + if Option(e.getCondition).exists(_.contains("CANNOT_LOAD_STATE_STORE")) => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( From 313e824931fd9b407b650fb1a8c11157dc3fe676 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Mon, 13 Jan 2025 16:36:34 -0800 Subject: [PATCH 406/438] [SPARK-50392][PYTHON][FOLLOWUP] Move `import`s into methods to fix `connect-only` builds ### What changes were proposed in this pull request? Move imports into methods to fix connect-only builds. ### Why are the changes needed? #49055 broke the connect-only builds: https://github.com/apache/spark/pull/49055#pullrequestreview-2545547927 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49472 from ueshin/issues/SPARK-50392/fup. Authored-by: Takuya Ueshin Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/table_arg.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/table_arg.py b/python/pyspark/sql/table_arg.py index d4b5e1653c7a1..cacfd24b2f1ba 100644 --- a/python/pyspark/sql/table_arg.py +++ b/python/pyspark/sql/table_arg.py @@ -17,7 +17,6 @@ from typing import TYPE_CHECKING -from pyspark.sql.classic.column import _to_java_column, _to_seq from pyspark.sql.tvf_argument import TableValuedFunctionArgument from pyspark.sql.utils import get_active_spark_context @@ -32,6 +31,8 @@ def __init__(self, j_table_arg: "JavaObject"): self._j_table_arg = j_table_arg def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": + from pyspark.sql.classic.column import _to_java_column, _to_seq + sc = get_active_spark_context() if len(cols) == 1 and isinstance(cols[0], list): cols = cols[0] @@ -40,6 +41,8 @@ def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": return TableArg(new_j_table_arg) def orderBy(self, *cols: "ColumnOrName") -> "TableArg": + from pyspark.sql.classic.column import _to_java_column, _to_seq + sc = get_active_spark_context() if len(cols) == 1 and isinstance(cols[0], list): cols = cols[0] From c5529d31513de1df4f596670d7aeea2455ae2199 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 13 Jan 2025 16:47:05 -0800 Subject: [PATCH 407/438] [SPARK-50773][CORE] Disable structured logging by default MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR restores the default Spark logging format to plain text instead of JSON. After changes, compared to Spark 3.x releases, uses can optionally enable structured logging by setting configuration `spark.log.structuredLogging.enabled` to `true` (default is `false`). For additional customization, users can copy `log4j2-json-layout.properties.template` to `conf/log4j2.properties` and adjust as needed. ### Why are the changes needed? After discussions on the [dev mailing list](https://lists.apache.org/thread/4fnlnvhsqym72k53jw8cjhcdjbhpmm95), Spark developers decided to revert to the previous plain text logging format for two main reasons: * Readability: JSON logs are verbose and not easily human-readable. * Setup Requirements: Structured logging requires a logging pipeline to collect JSON logs from both drivers and executors. Enabling it by default doesn’t provide much benefit without this infrastructure in place. ### Does this PR introduce _any_ user-facing change? No, Spark 4.0 has not yet been released. ### How was this patch tested? Existing tests. Also, manually tested on enable/disable the configuration `spark.log.structuredLogging.enabled`, as well as verified the logging behavior with different log4j2 templates. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49421 from gengliangwang/disableStructuredLogging. Authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun --- .../apache/spark/log4j2-defaults.properties | 4 ++-- ...operties => log4j2-json-layout.properties} | 4 ++-- .../org/apache/spark/internal/Logging.scala | 6 ++--- .../util/StructuredSparkLoggerSuite.java | 16 +++++++++++++ .../org/apache/spark/util/MDCSuite.scala | 12 +++++++++- .../spark/util/PatternLoggingSuite.scala | 5 +--- .../spark/util/StructuredLoggingSuite.scala | 16 ++++++++++++- ...=> log4j2-json-layout.properties.template} | 10 ++------ conf/log4j2.properties.template | 10 ++++++-- .../spark/internal/config/package.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 2 +- docs/configuration.md | 23 ++++++++----------- docs/core-migration-guide.md | 4 ---- .../org/apache/spark/sql/LogQuerySuite.scala | 6 +++++ 14 files changed, 78 insertions(+), 42 deletions(-) rename common/utils/src/main/resources/org/apache/spark/{log4j2-pattern-layout-defaults.properties => log4j2-json-layout.properties} (94%) rename conf/{log4j2.properties.pattern-layout-template => log4j2-json-layout.properties.template} (80%) diff --git a/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties b/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties index 9be86b650d091..777c5f2b25915 100644 --- a/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties +++ b/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties @@ -22,8 +22,8 @@ rootLogger.appenderRef.stdout.ref = console appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = JsonTemplateLayout -appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex # Settings to quiet third party logs that are too verbose logger.jetty.name = org.sparkproject.jetty diff --git a/common/utils/src/main/resources/org/apache/spark/log4j2-pattern-layout-defaults.properties b/common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties similarity index 94% rename from common/utils/src/main/resources/org/apache/spark/log4j2-pattern-layout-defaults.properties rename to common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties index 777c5f2b25915..9be86b650d091 100644 --- a/common/utils/src/main/resources/org/apache/spark/log4j2-pattern-layout-defaults.properties +++ b/common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties @@ -22,8 +22,8 @@ rootLogger.appenderRef.stdout.ref = console appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex +appender.console.layout.type = JsonTemplateLayout +appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json # Settings to quiet third party logs that are too verbose logger.jetty.name = org.sparkproject.jetty diff --git a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala index 7471b764bd2b3..4b60cb20f0732 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala @@ -337,9 +337,9 @@ trait Logging { if (Logging.defaultSparkLog4jConfig || Logging.islog4j2DefaultConfigured()) { Logging.defaultSparkLog4jConfig = true val defaultLogProps = if (Logging.isStructuredLoggingEnabled) { - "org/apache/spark/log4j2-defaults.properties" + "org/apache/spark/log4j2-json-layout.properties" } else { - "org/apache/spark/log4j2-pattern-layout-defaults.properties" + "org/apache/spark/log4j2-defaults.properties" } Option(SparkClassUtils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => @@ -398,7 +398,7 @@ private[spark] object Logging { @volatile private var initialized = false @volatile private var defaultRootLevel: Level = null @volatile private var defaultSparkLog4jConfig = false - @volatile private var structuredLoggingEnabled = true + @volatile private var structuredLoggingEnabled = false @volatile private[spark] var sparkShellThresholdLevel: Level = null @volatile private[spark] var setLogLevelPrinted: Boolean = false diff --git a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java b/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java index 6959fe11820ff..1fab167adfeb0 100644 --- a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java +++ b/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java @@ -21,11 +21,27 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.logging.log4j.Level; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + +import org.apache.spark.internal.Logging$; import org.apache.spark.internal.SparkLogger; import org.apache.spark.internal.SparkLoggerFactory; public class StructuredSparkLoggerSuite extends SparkLoggerSuiteBase { + // Enable Structured Logging before running the tests + @BeforeAll + public static void setup() { + Logging$.MODULE$.enableStructuredLogging(); + } + + // Disable Structured Logging after running the tests + @AfterAll + public static void teardown() { + Logging$.MODULE$.disableStructuredLogging(); + } + private static final SparkLogger LOGGER = SparkLoggerFactory.getLogger(StructuredSparkLoggerSuite.class); diff --git a/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala index 7631c25662219..9615eb2263636 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import scala.jdk.CollectionConverters._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite import org.apache.spark.internal.{Logging, MDC} @@ -26,7 +27,16 @@ import org.apache.spark.internal.LogKeys.{EXIT_CODE, OFFSET, RANGE} class MDCSuite extends AnyFunSuite // scalastyle:ignore funsuite - with Logging { + with Logging + with BeforeAndAfterAll { + + override def beforeAll(): Unit = { + Logging.enableStructuredLogging() + } + + override def afterAll(): Unit = { + Logging.disableStructuredLogging() + } test("check MDC message") { val log = log"This is a log, exitcode ${MDC(EXIT_CODE, 10086)}" diff --git a/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala index 2ba2b15c49f33..248136798b362 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala @@ -17,19 +17,16 @@ package org.apache.spark.util import org.apache.logging.log4j.Level -import org.scalatest.BeforeAndAfterAll import org.apache.spark.internal.Logging -class PatternLoggingSuite extends LoggingSuiteBase with BeforeAndAfterAll { +class PatternLoggingSuite extends LoggingSuiteBase { override def className: String = classOf[PatternLoggingSuite].getSimpleName override def logFilePath: String = "target/pattern.log" override def beforeAll(): Unit = Logging.disableStructuredLogging() - override def afterAll(): Unit = Logging.enableStructuredLogging() - override def expectedPatternForBasicMsg(level: Level): String = { s""".*$level $className: This is a log message\n""" } diff --git a/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala index 48951c2084f17..0026b696f0695 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala @@ -23,14 +23,21 @@ import java.nio.file.Files import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.apache.logging.log4j.Level +import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite import org.apache.spark.internal.{LogEntry, Logging, LogKey, LogKeys, MDC, MessageWithContext} trait LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuite + with BeforeAndAfterAll with Logging { + override def afterAll(): Unit = { + super.afterAll() + Logging.disableStructuredLogging() + } + def className: String def logFilePath: String @@ -202,7 +209,7 @@ trait LoggingSuiteBase } } - private val customLog = log"${MDC(CustomLogKeys.CUSTOM_LOG_KEY, "Custom log message.")}" + private lazy val customLog = log"${MDC(CustomLogKeys.CUSTOM_LOG_KEY, "Custom log message.")}" test("Logging with custom LogKey") { Seq( (Level.ERROR, () => logError(customLog)), @@ -265,6 +272,13 @@ class StructuredLoggingSuite extends LoggingSuiteBase { override def className: String = classOf[StructuredLoggingSuite].getSimpleName override def logFilePath: String = "target/structured.log" + override def beforeAll(): Unit = { + super.beforeAll() + Logging.enableStructuredLogging() + } + + override def afterAll(): Unit = super.afterAll() + private val jsonMapper = new ObjectMapper().registerModule(DefaultScalaModule) private def compactAndToRegexPattern(json: String): String = { jsonMapper.readTree(json).toString. diff --git a/conf/log4j2.properties.pattern-layout-template b/conf/log4j2-json-layout.properties.template similarity index 80% rename from conf/log4j2.properties.pattern-layout-template rename to conf/log4j2-json-layout.properties.template index 011fca58c9b2a..76499bb6691e7 100644 --- a/conf/log4j2.properties.pattern-layout-template +++ b/conf/log4j2-json-layout.properties.template @@ -19,17 +19,11 @@ rootLogger.level = info rootLogger.appenderRef.stdout.ref = console -# In the pattern layout configuration below, we specify an explicit `%ex` conversion -# pattern for logging Throwables. If this was omitted, then (by default) Log4J would -# implicitly add an `%xEx` conversion pattern which logs stacktraces with additional -# class packaging information. That extra information can sometimes add a substantial -# performance overhead, so we disable it in our default logging config. -# For more information, see SPARK-39361. appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex +appender.console.layout.type = JsonTemplateLayout +appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json # Set the default spark-shell/spark-sql log level to WARN. When running the # spark-shell/spark-sql, the log level for these classes is used to overwrite diff --git a/conf/log4j2.properties.template b/conf/log4j2.properties.template index 76499bb6691e7..011fca58c9b2a 100644 --- a/conf/log4j2.properties.template +++ b/conf/log4j2.properties.template @@ -19,11 +19,17 @@ rootLogger.level = info rootLogger.appenderRef.stdout.ref = console +# In the pattern layout configuration below, we specify an explicit `%ex` conversion +# pattern for logging Throwables. If this was omitted, then (by default) Log4J would +# implicitly add an `%xEx` conversion pattern which logs stacktraces with additional +# class packaging information. That extra information can sometimes add a substantial +# performance overhead, so we disable it in our default logging config. +# For more information, see SPARK-39361. appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = JsonTemplateLayout -appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex # Set the default spark-shell/spark-sql log level to WARN. When running the # spark-shell/spark-sql, the log level for these classes is used to overwrite diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6d51424f0baff..5dda7afc3ebcb 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -162,7 +162,7 @@ package object config { "PySpark shell.") .version("4.0.0") .booleanConf - .createWithDefault(true) + .createWithDefault(false) private[spark] val LEGACY_TASK_NAME_MDC_ENABLED = ConfigBuilder("spark.log.legacyTaskNameMdc.enabled") 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 e7b65bf1a4eff..536c6b4447aac 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2689,7 +2689,7 @@ private[spark] object Utils * loading SparkConf. */ def resetStructuredLogging(sparkConf: SparkConf): Unit = { - if (sparkConf.getBoolean(STRUCTURED_LOGGING_ENABLED.key, defaultValue = true)) { + if (sparkConf.get(STRUCTURED_LOGGING_ENABLED)) { Logging.enableStructuredLogging() } else { Logging.disableStructuredLogging() diff --git a/docs/configuration.md b/docs/configuration.md index f080a79580683..162165ffe68dd 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3751,15 +3751,20 @@ Note: When running Spark on YARN in `cluster` mode, environment variables need t # Configuring Logging -Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a -`log4j2.properties` file in the `conf` directory. One way to start is to copy the existing templates `log4j2.properties.template` or `log4j2.properties.pattern-layout-template` located there. +Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j2.properties` file in the `conf` directory. To get started, copy one of the provided templates: `log4j2.properties.template` (for plain text logging) or `log4j2-json-layout.properties.template` (for structured logging). + +## Plain Text Logging +The default logging format is plain text, using Log4j's [Pattern Layout](https://logging.apache.org/log4j/2.x/manual/pattern-layout.html). + +MDC (Mapped Diagnostic Context) information is not included by default in plain text logs. To include it, update the `PatternLayout` configuration in the `log4j2.properties` file. For example, add `%X{task_name}` to include the task name in logs. Additionally, use `spark.sparkContext.setLocalProperty("key", "value")` to add custom data to the MDC. ## Structured Logging -Starting from version 4.0.0, `spark-submit` has adopted the [JSON Template Layout](https://logging.apache.org/log4j/2.x/manual/json-template-layout.html) for logging, which outputs logs in JSON format. This format facilitates querying logs using Spark SQL with the JSON data source. Additionally, the logs include all Mapped Diagnostic Context (MDC) information for search and debugging purposes. +Starting with version 4.0.0, `spark-submit` supports optional structured logging using the [JSON Template Layout](https://logging.apache.org/log4j/2.x/manual/json-template-layout.html). This format enables efficient querying of logs with Spark SQL using the JSON data source and includes all MDC information for improved searchability and debugging. -To configure the layout of structured logging, start with the `log4j2.properties.template` file. +To enable structured logging and include MDC information, set the configuration `spark.log.structuredLogging.enabled` to `true` (default is `false`). For additional customization, copy `log4j2-json-layout.properties.template` to `conf/log4j2.properties` and adjust as needed. -To query Spark logs using Spark SQL, you can use the following code snippets: +### Querying Structured Logs with Spark SQL +To query structured logs in JSON format, use the following code snippet: **Python:** ```python @@ -3775,14 +3780,6 @@ import org.apache.spark.util.LogUtils.SPARK_LOG_SCHEMA val logDf = spark.read.schema(SPARK_LOG_SCHEMA).json("path/to/logs") ``` **Note**: If you're using the interactive shell (pyspark shell or spark-shell), you can omit the import statement in the code because SPARK_LOG_SCHEMA is already available in the shell's context. -## Plain Text Logging -If you prefer plain text logging, you have two options: -- Disable structured JSON logging by setting the Spark configuration `spark.log.structuredLogging.enabled` to `false`. -- Use a custom log4j configuration file. Rename `conf/log4j2.properties.pattern-layout-template` to `conf/log4j2.properties`. This reverts to the default configuration prior to Spark 4.0, which utilizes [PatternLayout](https://logging.apache.org/log4j/2.x/manual/layouts.html#PatternLayout) for logging all messages in plain text. - -MDC information is not included by default when with plain text logging. In order to print it in the logs, you can update the patternLayout in the file. For example, you can add `%X{task_name}` to print the task name in the logs. -Moreover, you can use `spark.sparkContext.setLocalProperty(s"mdc.$name", "value")` to add user specific data into MDC. -The key in MDC will be the string of `mdc.$name`. # Overriding configuration directory diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 49737392312a7..9dcf4ad8a2984 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -44,10 +44,6 @@ license: | - Since Spark 4.0, Spark uses the external shuffle service for deleting shuffle blocks for deallocated executors when the shuffle is no longer needed. To restore the legacy behavior, you can set `spark.shuffle.service.removeShuffle` to `false`. -- Starting with Spark 4.0, the default logging format for `spark-submit` has changed from plain text to JSON lines to improve log analysis. If you prefer plain text logs, you have two options: - - Set the Spark configuration `spark.log.structuredLogging.enabled` to `false`. For example, you can use `JDK_JAVA_OPTIONS=-Dspark.log.structuredLogging.enabled=false`. - - Use a custom log4j configuration file, such as renaming the template file `conf/log4j2.properties.pattern-layout-template` to `conf/log4j2.properties`. - - Since Spark 4.0, the MDC (Mapped Diagnostic Context) key for Spark task names in Spark logs has been changed from `mdc.taskName` to `task_name`. To use the key `mdc.taskName`, you can set `spark.log.legacyTaskNameMdc.enabled` to `true`. - Since Spark 4.0, Spark performs speculative executions less aggressively with `spark.speculation.multiplier=3` and `spark.speculation.quantile=0.9`. To restore the legacy behavior, you can set `spark.speculation.multiplier=1.5` and `spark.speculation.quantile=0.75`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala index 873337e7a4242..861b0bf0f3945 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala @@ -33,12 +33,18 @@ class LogQuerySuite extends QueryTest with SharedSparkSession with Logging { new File(pwd + "/target/LogQuerySuite.log") } + override def beforeAll(): Unit = { + super.beforeAll() + Logging.enableStructuredLogging() + } + override def afterAll(): Unit = { super.afterAll() // Clear the log file if (logFile.exists()) { logFile.delete() } + Logging.disableStructuredLogging() } private def createTempView(viewName: String): Unit = { From 0a8798f378ca14a32dad8c48ac085f668252d7b8 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 14 Jan 2025 10:00:10 +0900 Subject: [PATCH 408/438] [SPARK-48745][INFRA][PYTHON][TESTS][FOLLOWUP] use `conda-incubator/setup-miniconda` action ### What changes were proposed in this pull request? This PR follows the PR https://github.com/apache/spark/pull/49441, use `conda-incubator/setup-miniconda` instead of manual installation. ### Why are the changes needed? Reduce complexity. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually check. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49465 from panbingkun/SPARK-48745_FOLLOWUP. Authored-by: panbingkun Signed-off-by: Hyukjin Kwon --- .github/workflows/build_and_test.yml | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index b5f7ba7383a0b..292971092b70d 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -600,19 +600,16 @@ jobs: done - name: Install Conda for pip packaging test if: contains(matrix.modules, 'pyspark-errors') - run: | - curl -s -L "https://github.com/conda-forge/miniforge/releases/download/24.11.2-1/Miniforge3-Linux-x86_64.sh" > miniforge3.sh - bash miniforge3.sh -b -p $HOME/miniforge3 - rm miniforge3.sh + uses: conda-incubator/setup-miniconda@v3 + with: + miniforge-version: latest # Run the tests. - name: Run tests env: ${{ fromJSON(inputs.envs) }} shell: 'script -q -e -c "bash {0}"' run: | if [[ "$MODULES_TO_TEST" == *"pyspark-errors"* ]]; then - export PATH=$PATH:$HOME/miniforge3/bin - env - which conda + export PATH=$CONDA/bin:$PATH export SKIP_PACKAGING=false echo "Python Packaging Tests Enabled!" fi From 22731393069a3f180a9e719e57a694347c0ce87b Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Mon, 13 Jan 2025 18:22:16 -0800 Subject: [PATCH 409/438] [SPARK-50601][SQL] Support withColumns / withColumnsRenamed in subqueries ### What changes were proposed in this pull request? Supports `withColumns` / `withColumnsRenamed` in subqueries. ### Why are the changes needed? When the query is used as a subquery by adding `col.outer()`, `withColumns` or `withColumnsRenamed` doesn't work because they need analyzed plans. ### Does this PR introduce _any_ user-facing change? Yes, those APIs are available in subqueries. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49386 from ueshin/issues/SPARK-50601/with_columns. Lead-authored-by: Takuya Ueshin Co-authored-by: Takuya UESHIN Signed-off-by: Takuya Ueshin --- .../spark/sql/DataFrameSubquerySuite.scala | 57 ++++++-- .../sql/tests/connect/test_parity_subquery.py | 4 - python/pyspark/sql/tests/test_subquery.py | 39 +++++- .../sql/catalyst/analysis/unresolved.scala | 132 ++++++++++++++++-- .../connect/planner/SparkConnectPlanner.scala | 33 +++-- .../planner/SparkConnectPlannerSuite.scala | 33 ++--- .../scala/org/apache/spark/sql/Dataset.scala | 56 ++------ .../spark/sql/DataFrameSubquerySuite.scala | 48 ++++++- 8 files changed, 295 insertions(+), 107 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index 4b36d36983a5d..1d2165b668f61 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.{SparkException, SparkRuntimeException} +import org.apache.spark.SparkRuntimeException import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} @@ -665,15 +665,52 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { withView("t1") { val t1 = table1() - // TODO(SPARK-50601): Fix the SparkConnectPlanner to support this case - checkError( - intercept[SparkException] { - t1.withColumn("scalar", spark.range(1).select($"c1".outer() + $"c2".outer()).scalar()) - .collect() - }, - "INTERNAL_ERROR", - parameters = Map("message" -> "Found the unresolved operator: .*"), - matchPVals = true) + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer() + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .withColumn("c1", $"c1".outer()) + .select($"c1" + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1")) + .withColumn("c2", $"c2".outer()) + .select($"c1" + $"c2") + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + } + } + + test("subquery in withColumnsRenamed") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1"), $"c2".outer().as("c2")) + .withColumnsRenamed(Map("c1" -> "x", "c2" -> "y")) + .select($"x" + $"y") + .scalar()), + t1.select($"*", ($"c1".as("x") + $"c2".as("y")).as("scalar"))) } } diff --git a/python/pyspark/sql/tests/connect/test_parity_subquery.py b/python/pyspark/sql/tests/connect/test_parity_subquery.py index dae60a354d20a..f3225fcb7f2dd 100644 --- a/python/pyspark/sql/tests/connect/test_parity_subquery.py +++ b/python/pyspark/sql/tests/connect/test_parity_subquery.py @@ -45,10 +45,6 @@ def test_scalar_subquery_with_missing_outer_reference(self): def test_subquery_in_unpivot(self): self.check_subquery_in_unpivot(None, None) - @unittest.skip("SPARK-50601: Fix the SparkConnectPlanner to support this case") - def test_subquery_in_with_columns(self): - super().test_subquery_in_with_columns() - if __name__ == "__main__": from pyspark.sql.tests.connect.test_parity_subquery import * # noqa: F401 diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 99a22d7c29664..7c63ddb69458e 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -939,7 +939,44 @@ def test_subquery_in_with_columns(self): .select(sf.col("c1").outer() + sf.col("c2").outer()) .scalar(), ), - t1.withColumn("scalar", sf.col("c1") + sf.col("c2")), + t1.select("*", (sf.col("c1") + sf.col("c2")).alias("scalar")), + ) + assertDataFrameEqual( + t1.withColumn( + "scalar", + self.spark.range(1) + .withColumn("c1", sf.col("c1").outer()) + .select(sf.col("c1") + sf.col("c2").outer()) + .scalar(), + ), + t1.select("*", (sf.col("c1") + sf.col("c2")).alias("scalar")), + ) + assertDataFrameEqual( + t1.withColumn( + "scalar", + self.spark.range(1) + .select(sf.col("c1").outer().alias("c1")) + .withColumn("c2", sf.col("c2").outer()) + .select(sf.col("c1") + sf.col("c2")) + .scalar(), + ), + t1.select("*", (sf.col("c1") + sf.col("c2")).alias("scalar")), + ) + + def test_subquery_in_with_columns_renamed(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + t1.withColumn( + "scalar", + self.spark.range(1) + .select(sf.col("c1").outer().alias("c1"), sf.col("c2").outer().alias("c2")) + .withColumnsRenamed({"c1": "x", "c2": "y"}) + .select(sf.col("x") + sf.col("y")) + .scalar(), + ), + t1.select("*", (sf.col("c1").alias("x") + sf.col("c2").alias("y")).alias("scalar")), ) def test_subquery_in_drop(self): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index b47af90c651a6..fabe551d054ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.connector.catalog.TableWritePrivilege import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Metadata, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils} import org.apache.spark.util.ArrayImplicits._ /** @@ -429,7 +429,7 @@ object UnresolvedFunction { * Represents all of the input attributes to a given relational operator, for example in * "SELECT * FROM ...". A [[Star]] gets automatically expanded during analysis. */ -abstract class Star extends LeafExpression with NamedExpression { +trait Star extends NamedExpression { override def name: String = throw new UnresolvedException("name") override def exprId: ExprId = throw new UnresolvedException("exprId") @@ -451,15 +451,20 @@ abstract class Star extends LeafExpression with NamedExpression { * This is also used to expand structs. For example: * "SELECT record.* from (SELECT struct(a,b,c) as record ...) * - * @param target an optional name that should be the target of the expansion. If omitted all - * targets' columns are produced. This can either be a table name or struct name. This - * is a list of identifiers that is the path of the expansion. - * - * This class provides the shared behavior between the classes for SELECT * ([[UnresolvedStar]]) - * and SELECT * EXCEPT ([[UnresolvedStarExceptOrReplace]]). [[UnresolvedStar]] is just a case class - * of this, while [[UnresolvedStarExceptOrReplace]] adds some additional logic to the expand method. + * This trait provides the shared behavior among the classes for SELECT * ([[UnresolvedStar]]) + * and SELECT * EXCEPT ([[UnresolvedStarExceptOrReplace]]), etc. [[UnresolvedStar]] is just a case + * class of this, while [[UnresolvedStarExceptOrReplace]] or other classes add some additional logic + * to the expand method. */ -abstract class UnresolvedStarBase(target: Option[Seq[String]]) extends Star with Unevaluable { +trait UnresolvedStarBase extends Star with Unevaluable { + + /** + * An optional name that should be the target of the expansion. If omitted all + * targets' columns are produced. This can either be a table name or struct name. This + * is a list of identifiers that is the path of the expansion. + */ + def target: Option[Seq[String]] + /** * Returns true if the nameParts is a subset of the last elements of qualifier of the attribute. * @@ -583,7 +588,7 @@ case class UnresolvedStarExceptOrReplace( target: Option[Seq[String]], excepts: Seq[Seq[String]], replacements: Option[Seq[NamedExpression]]) - extends UnresolvedStarBase(target) { + extends LeafExpression with UnresolvedStarBase { /** * We expand the * EXCEPT by the following three steps: @@ -712,6 +717,103 @@ case class UnresolvedStarExceptOrReplace( } } +/** + * Represents some of the input attributes to a given relational operator, for example in + * `df.withColumn`. + * + * @param colNames a list of column names that should be replaced or produced. + * + * @param exprs the corresponding expressions for `colNames`. + * + * @param explicitMetadata an optional list of explicit metadata to associate with the columns. + */ +case class UnresolvedStarWithColumns( + colNames: Seq[String], + exprs: Seq[Expression], + explicitMetadata: Option[Seq[Metadata]] = None) + extends UnresolvedStarBase { + + override def target: Option[Seq[String]] = None + override def children: Seq[Expression] = exprs + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): UnresolvedStarWithColumns = + copy(exprs = newChildren) + + override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { + assert(colNames.size == exprs.size, + s"The size of column names: ${colNames.size} isn't equal to " + + s"the size of expressions: ${exprs.size}") + explicitMetadata.foreach { m => + assert(colNames.size == m.size, + s"The size of column names: ${colNames.size} isn't equal to " + + s"the size of metadata elements: ${m.size}") + } + + SchemaUtils.checkColumnNameDuplication(colNames, resolver) + + val expandedCols = super.expand(input, resolver) + + val columnSeq = explicitMetadata match { + case Some(ms) => colNames.zip(exprs).zip(ms.map(Some(_))) + case _ => colNames.zip(exprs).map((_, None)) + } + + val replacedAndExistingColumns = expandedCols.map { field => + columnSeq.find { case ((colName, _), _) => + resolver(field.name, colName) + } match { + case Some(((colName, expr), m)) => Alias(expr, colName)(explicitMetadata = m) + case _ => field + } + } + + val newColumns = columnSeq.filter { case ((colName, _), _) => + !expandedCols.exists(f => resolver(f.name, colName)) + }.map { + case ((colName, expr), m) => Alias(expr, colName)(explicitMetadata = m) + } + + replacedAndExistingColumns ++ newColumns + } +} + +/** + * Represents some of the input attributes to a given relational operator, for example in + * `df.withColumnRenamed`. + * + * @param existingNames a list of column names that should be replaced. + * If the column does not exist, it is ignored. + * + * @param newNames a list of new column names that should be used to replace the existing columns. + */ +case class UnresolvedStarWithColumnsRenames( + existingNames: Seq[String], + newNames: Seq[String]) + extends LeafExpression with UnresolvedStarBase { + + override def target: Option[Seq[String]] = None + + override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { + assert(existingNames.size == newNames.size, + s"The size of existing column names: ${existingNames.size} isn't equal to " + + s"the size of new column names: ${newNames.size}") + + val expandedCols = super.expand(input, resolver) + + existingNames.zip(newNames).foldLeft(expandedCols) { + case (attrs, (existingName, newName)) => + attrs.map(attr => + if (resolver(attr.name, existingName)) { + Alias(attr, newName)() + } else { + attr + } + ) + } + } +} + /** * Represents all of the input attributes to a given relational operator, for example in * "SELECT * FROM ...". @@ -723,7 +825,8 @@ case class UnresolvedStarExceptOrReplace( * targets' columns are produced. This can either be a table name or struct name. This * is a list of identifiers that is the path of the expansion. */ -case class UnresolvedStar(target: Option[Seq[String]]) extends UnresolvedStarBase(target) +case class UnresolvedStar(target: Option[Seq[String]]) + extends LeafExpression with UnresolvedStarBase /** * Represents all of the input attributes to a given relational operator, for example in @@ -733,7 +836,7 @@ case class UnresolvedStar(target: Option[Seq[String]]) extends UnresolvedStarBas * tables' columns are produced. */ case class UnresolvedRegex(regexPattern: String, table: Option[String], caseSensitive: Boolean) - extends Star with Unevaluable { + extends LeafExpression with Star with Unevaluable { override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { val pattern = if (caseSensitive) regexPattern else s"(?i)$regexPattern" table match { @@ -791,7 +894,8 @@ case class MultiAlias(child: Expression, names: Seq[String]) * * @param expressions Expressions to expand. */ -case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star with Unevaluable { +case class ResolvedStar(expressions: Seq[NamedExpression]) + extends LeafExpression with Star with Unevaluable { override def newInstance(): NamedExpression = throw new UnresolvedException("newInstance") override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = expressions override def toString: String = expressions.mkString("ResolvedStar(", ", ", ")") diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 6ab69aea12e5d..acbbeb49b267b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -45,7 +45,7 @@ import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction, UnresolvedTranspose} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedStarWithColumns, UnresolvedStarWithColumnsRenames, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ @@ -1065,25 +1065,21 @@ class SparkConnectPlanner( } private def transformWithColumnsRenamed(rel: proto.WithColumnsRenamed): LogicalPlan = { - if (rel.getRenamesCount > 0) { - val (colNames, newColNames) = rel.getRenamesList.asScala.toSeq.map { rename => + val (colNames, newColNames) = if (rel.getRenamesCount > 0) { + rel.getRenamesList.asScala.toSeq.map { rename => (rename.getColName, rename.getNewColName) }.unzip - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .withColumnsRenamed(colNames, newColNames) - .logicalPlan } else { // for backward compatibility - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .withColumnsRenamed(rel.getRenameColumnsMapMap) - .logicalPlan + rel.getRenameColumnsMapMap.asScala.toSeq.unzip } + Project( + Seq(UnresolvedStarWithColumnsRenames(existingNames = colNames, newNames = newColNames)), + transformRelation(rel.getInput)) } private def transformWithColumns(rel: proto.WithColumns): LogicalPlan = { - val (colNames, cols, metadata) = + val (colNames, exprs, metadata) = rel.getAliasesList.asScala.toSeq.map { alias => if (alias.getNameCount != 1) { throw InvalidPlanInput(s"""WithColumns require column name only contains one name part, @@ -1096,13 +1092,16 @@ class SparkConnectPlanner( Metadata.empty } - (alias.getName(0), Column(transformExpression(alias.getExpr)), metadata) + (alias.getName(0), transformExpression(alias.getExpr), metadata) }.unzip3 - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .withColumns(colNames, cols, metadata) - .logicalPlan + Project( + Seq( + UnresolvedStarWithColumns( + colNames = colNames, + exprs = exprs, + explicitMetadata = Some(metadata))), + transformRelation(rel.getInput)) } private def transformWithWatermark(rel: proto.WithWatermark): LogicalPlan = { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala index aaeb5d9fe509a..054a32179935d 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala @@ -504,26 +504,27 @@ class SparkConnectPlannerSuite extends SparkFunSuite with SparkConnectPlanTest { } test("Test duplicated names in WithColumns") { - intercept[AnalysisException] { - transform( - proto.Relation - .newBuilder() - .setWithColumns( - proto.WithColumns - .newBuilder() - .setInput(readRel) - .addAliases(proto.Expression.Alias + val logical = transform( + proto.Relation + .newBuilder() + .setWithColumns( + proto.WithColumns + .newBuilder() + .setInput(readRel) + .addAliases( + proto.Expression.Alias .newBuilder() .addName("test") .setExpr(proto.Expression.newBuilder .setLiteral(proto.Expression.Literal.newBuilder.setInteger(32)))) - .addAliases(proto.Expression.Alias - .newBuilder() - .addName("test") - .setExpr(proto.Expression.newBuilder - .setLiteral(proto.Expression.Literal.newBuilder.setInteger(32))))) - .build()) - } + .addAliases(proto.Expression.Alias + .newBuilder() + .addName("test") + .setExpr(proto.Expression.newBuilder + .setLiteral(proto.Expression.Literal.newBuilder.setInteger(32))))) + .build()) + + intercept[AnalysisException](Dataset.ofRows(spark, logical)) } test("Test multi nameparts for column names in WithColumns") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index e4e782a50e3d4..e41521cba533a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1275,29 +1275,14 @@ class Dataset[T] private[sql]( require(colNames.size == cols.size, s"The size of column names: ${colNames.size} isn't equal to " + s"the size of columns: ${cols.size}") - SchemaUtils.checkColumnNameDuplication( - colNames, - sparkSession.sessionState.conf.caseSensitiveAnalysis) - - val resolver = sparkSession.sessionState.analyzer.resolver - val output = queryExecution.analyzed.output - - val columnSeq = colNames.zip(cols) - - val replacedAndExistingColumns = output.map { field => - columnSeq.find { case (colName, _) => - resolver(field.name, colName) - } match { - case Some((colName: String, col: Column)) => col.as(colName) - case _ => Column(field) - } + withPlan { + Project( + Seq( + UnresolvedStarWithColumns( + colNames = colNames, + exprs = cols.map(_.expr))), + logicalPlan) } - - val newColumns = columnSeq.filter { case (colName, col) => - !output.exists(f => resolver(f.name, colName)) - }.map { case (colName, col) => col.as(colName) } - - select(replacedAndExistingColumns ++ newColumns : _*) } /** @inheritdoc */ @@ -1324,26 +1309,13 @@ class Dataset[T] private[sql]( require(colNames.size == newColNames.size, s"The size of existing column names: ${colNames.size} isn't equal to " + s"the size of new column names: ${newColNames.size}") - - val resolver = sparkSession.sessionState.analyzer.resolver - val output: Seq[NamedExpression] = queryExecution.analyzed.output - var shouldRename = false - - val projectList = colNames.zip(newColNames).foldLeft(output) { - case (attrs, (existingName, newName)) => - attrs.map(attr => - if (resolver(attr.name, existingName)) { - shouldRename = true - Alias(attr, newName)() - } else { - attr - } - ) - } - if (shouldRename) { - withPlan(Project(projectList, logicalPlan)) - } else { - toDF() + withPlan { + Project( + Seq( + UnresolvedStarWithColumnsRenames( + existingNames = colNames, + newNames = newColNames)), + logicalPlan) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index fdfb909d9ba73..621d468454d40 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -777,9 +777,51 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { val t1 = table1() checkAnswer( - t1.withColumn("scalar", spark.range(1).select($"c1".outer() + $"c2".outer()).scalar()), - t1.withColumn("scalar", $"c1" + $"c2") - ) + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer() + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .withColumn("c1", $"c1".outer()) + .select($"c1" + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1")) + .withColumn("c2", $"c2".outer()) + .select($"c1" + $"c2") + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + } + } + + test("subquery in withColumnsRenamed") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1"), $"c2".outer().as("c2")) + .withColumnsRenamed(Map("c1" -> "x", "c2" -> "y")) + .select($"x" + $"y") + .scalar()), + t1.select($"*", ($"c1".as("x") + $"c2".as("y")).as("scalar"))) } } From 8858a4c9628f349986e093a93737b2eaed7e8833 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 14 Jan 2025 11:23:22 +0900 Subject: [PATCH 410/438] [SPARK-47081][CONNECT][FOLLOW-UP] Respect `spark.connect.progress.reportInterval` over timeout ### What changes were proposed in this pull request? This PR is a followup that addresses https://github.com/apache/spark/pull/45150#discussion_r1913310090 ### Why are the changes needed? To respect `spark.connect.progress.reportInterval` ### Does this PR introduce _any_ user-facing change? Virtually no. In corner case, it the progress upgrade might take longer than `spark.connect.progress.reportInterval`. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49474 from HyukjinKwon/SPARK-47081-followup3. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../sql/connect/execution/ExecuteGrpcResponseSender.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala index 44b634af95ca9..72c2b0e3f1095 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala @@ -241,14 +241,13 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( // The state of interrupted, response and lastIndex are changed under executionObserver // monitor, and will notify upon state change. if (response.isEmpty) { + val timeout = Math.max(1, deadlineTimeMillis - System.currentTimeMillis()) // Wake up more frequently to send the progress updates. val progressTimeout = executeHolder.sessionHolder.session.sessionState.conf .getConf(CONNECT_PROGRESS_REPORT_INTERVAL) // If the progress feature is disabled, wait for the deadline. - val timeout = if (progressTimeout > 0) { - progressTimeout - } else { - Math.max(1, deadlineTimeMillis - System.currentTimeMillis()) + if (progressTimeout > 0L) { + Math.min(progressTimeout, timeout) } logTrace(s"Wait for response to become available with timeout=$timeout ms.") executionObserver.responseLock.wait(timeout) From e62697fc585ad5f06878d44f859a9f2fefe4d021 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 13 Jan 2025 18:51:23 -0800 Subject: [PATCH 411/438] [SPARK-50805][CORE] Move method `nameForAppAndAttempt` to `o.a.s.u.Utils` ### What changes were proposed in this pull request? Pure refactor, move method `nameForAppAndAttempt` from `EventLogFileWriter` to `o.a.s.u.Utils`. ### Why are the changes needed? The method could be reused in several other places, e.g. https://github.com/apache/spark/pull/49440 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GHA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49476 from pan3793/SPARK-50805. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .../spark/deploy/history/EventLogFileWriters.scala | 7 +------ core/src/main/scala/org/apache/spark/util/Utils.scala | 9 +++++++++ 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index f3bb6d5af3358..990ab680f3aaf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -187,12 +187,7 @@ object EventLogFileWriter { } def nameForAppAndAttempt(appId: String, appAttemptId: Option[String]): String = { - val base = Utils.sanitizeDirName(appId) - if (appAttemptId.isDefined) { - base + "_" + Utils.sanitizeDirName(appAttemptId.get) - } else { - base - } + Utils.nameForAppAndAttempt(appId, appAttemptId) } def codecName(log: Path): Option[String] = { 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 536c6b4447aac..1efe181a8c38a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2954,6 +2954,15 @@ private[spark] object Utils str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase(Locale.ROOT) } + def nameForAppAndAttempt(appId: String, appAttemptId: Option[String]): String = { + val base = sanitizeDirName(appId) + if (appAttemptId.isDefined) { + base + "_" + sanitizeDirName(appAttemptId.get) + } else { + base + } + } + def isClientMode(conf: SparkConf): Boolean = { "client".equals(conf.get(SparkLauncher.DEPLOY_MODE, "client")) } From 0e51dba278000a7cc8010f1852ecb45923d56439 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 14 Jan 2025 11:05:02 +0800 Subject: [PATCH 412/438] [SPARK-50633][FOLLOWUP] Set `CODECOV_TOKEN` with environment variables ### What changes were proposed in this pull request? The pr aims to set `CODECOV_TOKEN` with `environment variables` for `codecov/codecov-action`. ### Why are the changes needed? Based on the suggestions of the `codecov/codecov-action` community, we will try setting it up in a different way to see if `codecov/codecov-action` can succeed. https://github.com/codecov/codecov-action/issues/1738#issuecomment-2588783885 image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually check. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49477 from panbingkun/SPARK-50633_FOLLOWUP_1. Authored-by: panbingkun Signed-off-by: panbingkun --- .github/workflows/build_and_test.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 292971092b70d..8040169fcb2c8 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -622,9 +622,10 @@ jobs: - name: Upload coverage to Codecov if: fromJSON(inputs.envs).PYSPARK_CODECOV == 'true' uses: codecov/codecov-action@v5 + env: + CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }} with: files: ./python/coverage.xml - token: ${{ secrets.CODECOV_TOKEN }} flags: unittests name: PySpark verbose: true From e945a9031d4eb4f9893f9b1a744646c2cfb214c0 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 13 Jan 2025 19:21:34 -0800 Subject: [PATCH 413/438] [SPARK-50783] Canonicalize JVM profiler results file name and layout on DFS ### What changes were proposed in this pull request? This PR canonicalizes the JVM profiler added in SPARK-46094 profiling result files on DFS to ``` dfsDir/{{APP_ID}}/profile-exec-{{EXECUTOR_ID}}.jfr ``` which majorly follows the event logs file name pattern and layout. ### Why are the changes needed? According to https://github.com/apache/spark/pull/44021#issuecomment-1863873954, we can integrate the profiling results with Spark UI (both live and history) in the future, so it's good to follow the event logs file name pattern and layout as much as possible. ### Does this PR introduce _any_ user-facing change? No, it's an unreleased feature. ### How was this patch tested? ``` $ bin/spark-submit run-example \ --master yarn \ --deploy-mode cluster \ --conf spark.plugins=org.apache.spark.executor.profiler.ExecutorProfilerPlugin \ --conf spark.executor.profiling.enabled=true \ --conf spark.executor.profiling.dfsDir=hdfs:///spark-profiling \ --conf spark.executor.profiling.fraction=1 \ SparkPi 100000 ``` ``` hadoopspark-dev1:~/spark$ hadoop fs -ls /spark-profiling/ Found 1 items drwxrwx--- - hadoop supergroup 0 2025-01-13 10:29 /spark-profiling/application_1736320707252_0023_1 ``` ``` hadoopspark-dev1:~/spark$ hadoop fs -ls /spark-profiling/application_1736320707252_0023_1 Found 48 items -rw-rw---- 3 hadoop supergroup 5255028 2025-01-13 10:29 /spark-profiling/application_1736320707252_0023_1/profile-exec-1.jfr -rw-rw---- 3 hadoop supergroup 3840775 2025-01-13 10:29 /spark-profiling/application_1736320707252_0023_1/profile-exec-10.jfr -rw-rw---- 3 hadoop supergroup 3889002 2025-01-13 10:29 /spark-profiling/application_1736320707252_0023_1/profile-exec-11.jfr -rw-rw---- 3 hadoop supergroup 3570697 2025-01-13 10:29 /spark-profiling/application_1736320707252_0023_1/profile-exec-12.jfr ... ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49440 from pan3793/SPARK-50783. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- connector/profiler/README.md | 6 +- .../profiler/ExecutorJVMProfiler.scala | 77 +++++++++++-------- 2 files changed, 50 insertions(+), 33 deletions(-) diff --git a/connector/profiler/README.md b/connector/profiler/README.md index 1326fd55df097..4d97b15eb96ab 100644 --- a/connector/profiler/README.md +++ b/connector/profiler/README.md @@ -16,7 +16,7 @@ The profiler writes the jfr files to the executor's working directory in the exe Code profiling is currently only supported for * Linux (x64) -* Linux (arm 64) +* Linux (arm64) * Linux (musl, x64) * MacOS @@ -54,7 +54,7 @@ Then enable the profiling in the configuration. @@ -72,7 +72,7 @@ Then enable the profiling in the configuration. diff --git a/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala b/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala index 20b6db5221fa9..94e5b46c65881 100644 --- a/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala +++ b/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala @@ -17,17 +17,17 @@ package org.apache.spark.executor.profiler import java.io.{BufferedInputStream, FileInputStream, InputStream, IOException} -import java.net.URI import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import one.profiler.{AsyncProfiler, AsyncProfilerLoader} import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.PATH -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -38,15 +38,26 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex private var running = false private val enableProfiler = conf.get(EXECUTOR_PROFILING_ENABLED) private val profilerOptions = conf.get(EXECUTOR_PROFILING_OPTIONS) - private val profilerDfsDir = conf.get(EXECUTOR_PROFILING_DFS_DIR) + private val profilerDfsDirOpt = conf.get(EXECUTOR_PROFILING_DFS_DIR) private val profilerLocalDir = conf.get(EXECUTOR_PROFILING_LOCAL_DIR) private val writeInterval = conf.get(EXECUTOR_PROFILING_WRITE_INTERVAL) - private val startcmd = s"start,$profilerOptions,file=$profilerLocalDir/profile.jfr" - private val stopcmd = s"stop,$profilerOptions,file=$profilerLocalDir/profile.jfr" - private val dumpcmd = s"dump,$profilerOptions,file=$profilerLocalDir/profile.jfr" - private val resumecmd = s"resume,$profilerOptions,file=$profilerLocalDir/profile.jfr" + private val appId = try { + conf.getAppId + } catch { + case _: NoSuchElementException => "local-" + System.currentTimeMillis + } + private val appAttemptId = conf.getOption("spark.app.attempt.id") + private val baseName = Utils.nameForAppAndAttempt(appId, appAttemptId) + private val profileFile = s"profile-exec-$executorId.jfr" + + private val startcmd = s"start,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val stopcmd = s"stop,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val dumpcmd = s"dump,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val resumecmd = s"resume,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val PROFILER_FOLDER_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) + private val PROFILER_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("660", 8).toShort) private val UPLOAD_SIZE = 8 * 1024 * 1024 // 8 MB private var outputStream: FSDataOutputStream = _ private var inputStream: InputStream = _ @@ -89,28 +100,34 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex } } + private def requireProfilerBaseDirAsDirectory(fs: FileSystem, profilerDfsDir: String): Unit = { + if (!fs.getFileStatus(new Path(profilerDfsDir)).isDirectory) { + throw new IllegalArgumentException( + s"Profiler DFS base directory $profilerDfsDir is not a directory.") + } + } + private def startWriting(): Unit = { - if (profilerDfsDir.isDefined) { - val applicationId = try { - conf.getAppId - } catch { - case _: NoSuchElementException => "local-" + System.currentTimeMillis + profilerDfsDirOpt.foreach { profilerDfsDir => + val profilerDirForApp = s"$profilerDfsDir/$baseName" + val profileOutputFile = s"$profilerDirForApp/$profileFile" + + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val fs = Utils.getHadoopFileSystem(profilerDfsDir, hadoopConf) + + requireProfilerBaseDirAsDirectory(fs, profilerDfsDir) + + val profilerDirForAppPath = new Path(profilerDirForApp) + if (!fs.exists(profilerDirForAppPath)) { + // SPARK-30860: use the class method to avoid the umask causing permission issues + FileSystem.mkdirs(fs, profilerDirForAppPath, PROFILER_FOLDER_PERMISSIONS) } - val config = SparkHadoopUtil.get.newConfiguration(conf) - val appName = conf.get("spark.app.name").replace(" ", "-") - val profilerOutputDirname = profilerDfsDir.get - - val profileOutputFile = - s"$profilerOutputDirname/$applicationId/profile-$appName-exec-$executorId.jfr" - val fs = FileSystem.get(new URI(profileOutputFile), config); - val filenamePath = new Path(profileOutputFile) - outputStream = fs.create(filenamePath) + + outputStream = FileSystem.create(fs, new Path(profileOutputFile), PROFILER_FILE_PERMISSIONS) try { - if (fs.exists(filenamePath)) { - fs.delete(filenamePath, true) - } logInfo(log"Copying executor profiling file to ${MDC(PATH, profileOutputFile)}") - inputStream = new BufferedInputStream(new FileInputStream(s"$profilerLocalDir/profile.jfr")) + inputStream = new BufferedInputStream( + new FileInputStream(s"$profilerLocalDir/$profileFile")) threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("profilerOutputThread") threadpool.scheduleWithFixedDelay( new Runnable() { @@ -158,14 +175,14 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex } catch { case e: IOException => logError("Exception occurred while writing some profiler output: ", e) case e @ (_: IllegalArgumentException | _: IllegalStateException) => - logError("Some profiler output not written." + - " Exception occurred in profiler native code: ", e) + logError("Some profiler output not written. " + + "Exception occurred in profiler native code: ", e) case e: Exception => logError("Some profiler output not written. Unexpected exception: ", e) } } private def finishWriting(): Unit = { - if (profilerDfsDir.isDefined && writing) { + if (profilerDfsDirOpt.isDefined && writing) { try { // shutdown background writer threadpool.shutdown() @@ -177,8 +194,8 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex } catch { case _: InterruptedException => Thread.currentThread().interrupt() case e: IOException => - logWarning("Some profiling output not written." + - "Exception occurred while completing profiler output", e) + logWarning("Some profiling output not written. " + + "Exception occurred while completing profiler output: ", e) } writing = false } From ebe39f6adc8bee841c79946a438d2a9ea191ac03 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 13 Jan 2025 19:23:45 -0800 Subject: [PATCH 414/438] [MINOR][DOCS] Fix the examples of createDataFrame ### What changes were proposed in this pull request? Fix the examples of createDataFrame `collect` -> `show` ### Why are the changes needed? existing examples generate different outputs ### Does this PR introduce _any_ user-facing change? doc only changes ### How was this patch tested? manually test in `bin/pyspark` ### Was this patch authored or co-authored using generative AI tooling? no Closes #49475 from zhengruifeng/py_doc_create_df. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/session.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 5ab186b2957e7..f5bb269c23d6e 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -1467,7 +1467,9 @@ def createDataFrame( # type: ignore[misc] +-----+---+ |Alice| 1| +-----+---+ - >>> spark.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP + + >>> pdf = pandas.DataFrame([[1, 2]]) # doctest: +SKIP + >>> spark.createDataFrame(pdf).show() # doctest: +SKIP +---+---+ | 0| 1| +---+---+ @@ -1482,8 +1484,9 @@ def createDataFrame( # type: ignore[misc] +-----+---+ |Alice| 1| +-----+---+ + >>> table = pyarrow.table({'0': [1], '1': [2]}) # doctest: +SKIP - >>> spark.createDataFrame(table).collect() # doctest: +SKIP + >>> spark.createDataFrame(table).show() # doctest: +SKIP +---+---+ | 0| 1| +---+---+ From b89082b4190d541a2c0084319e5e4a006a332e79 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Tue, 14 Jan 2025 12:32:39 +0900 Subject: [PATCH 415/438] [SPARK-50774][SQL] Centralize collation names in one place ### What changes were proposed in this pull request? Made a new class to put most commonly used collation names and refactor `CollationSupportSuite` which had thousands of literals for the four most used collations (UTF8_BINARY, UTF8_LCASE, UNICODE and UNICODE_CI) ### Why are the changes needed? To be able to have this centralized and not have to create new string literals with "UTF8_BINARY" over and over again. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Should be covered with existing tests. ### Was this patch authored or co-authored using generative AI tooling? Closes #49425 from stefankandic/makeCollationNamesPublic. Authored-by: Stefan Kandic Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/util/CollationFactory.java | 17 +- .../sql/catalyst/util/CollationNames.java | 25 + .../unsafe/types/CollationSupportSuite.java | 5925 +++++++++-------- .../apache/spark/sql/internal/SQLConf.scala | 4 +- 4 files changed, 2997 insertions(+), 2974 deletions(-) create mode 100644 common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationNames.java diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java index ce416293131a1..81448dc95a374 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java @@ -467,9 +467,6 @@ private enum CaseSensitivity { */ private static final int CASE_SENSITIVITY_MASK = 0b1; - private static final String UTF8_BINARY_COLLATION_NAME = "UTF8_BINARY"; - private static final String UTF8_LCASE_COLLATION_NAME = "UTF8_LCASE"; - private static final int UTF8_BINARY_COLLATION_ID = new CollationSpecUTF8(CaseSensitivity.UNSPECIFIED, SpaceTrimming.NONE).collationId; private static final int UTF8_LCASE_COLLATION_ID = @@ -655,9 +652,9 @@ protected CollationMeta buildCollationMeta() { protected String normalizedCollationName() { StringBuilder builder = new StringBuilder(); if(caseSensitivity == CaseSensitivity.UNSPECIFIED){ - builder.append(UTF8_BINARY_COLLATION_NAME); + builder.append(CollationNames.UTF8_BINARY); } else{ - builder.append(UTF8_LCASE_COLLATION_NAME); + builder.append(CollationNames.UTF8_LCASE); } if (spaceTrimming != SpaceTrimming.NONE) { builder.append('_'); @@ -669,12 +666,12 @@ protected String normalizedCollationName() { static List listCollations() { CollationIdentifier UTF8_BINARY_COLLATION_IDENT = new CollationIdentifier( PROVIDER_SPARK, - UTF8_BINARY_COLLATION_NAME, + CollationNames.UTF8_BINARY, CollationSpecICU.ICU_VERSION ); CollationIdentifier UTF8_LCASE_COLLATION_IDENT = new CollationIdentifier( PROVIDER_SPARK, - UTF8_LCASE_COLLATION_NAME, + CollationNames.UTF8_LCASE, CollationSpecICU.ICU_VERSION ); return Arrays.asList(UTF8_BINARY_COLLATION_IDENT, UTF8_LCASE_COLLATION_IDENT); @@ -758,7 +755,7 @@ private enum AccentSensitivity { VersionInfo.ICU_VERSION.getMinor()); static { - ICULocaleMap.put("UNICODE", ULocale.ROOT); + ICULocaleMap.put(CollationNames.UNICODE, ULocale.ROOT); // ICU-implemented `ULocale`s which have corresponding `Collator` installed. ULocale[] locales = Collator.getAvailableULocales(); // Build locale names in format: language["_" optional script]["_" optional country code]. @@ -806,13 +803,13 @@ private enum AccentSensitivity { } private static final int UNICODE_COLLATION_ID = new CollationSpecICU( - "UNICODE", + CollationNames.UNICODE, CaseSensitivity.CS, AccentSensitivity.AS, SpaceTrimming.NONE).collationId; private static final int UNICODE_CI_COLLATION_ID = new CollationSpecICU( - "UNICODE", + CollationNames.UNICODE, CaseSensitivity.CI, AccentSensitivity.AS, SpaceTrimming.NONE).collationId; diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationNames.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationNames.java new file mode 100644 index 0000000000000..11e9e1a87e713 --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationNames.java @@ -0,0 +1,25 @@ +/* + * 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.catalyst.util; + +public class CollationNames { + public static final String UTF8_BINARY = "UTF8_BINARY"; + public static final String UTF8_LCASE = "UTF8_LCASE"; + public static final String UNICODE = "UNICODE"; + public static final String UNICODE_CI = "UNICODE_CI"; +} diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java index a696da8cf45b8..1db163c1c822d 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java @@ -26,6 +26,7 @@ import java.util.Map; import static org.junit.jupiter.api.Assertions.*; +import static org.apache.spark.sql.catalyst.util.CollationNames.*; // checkstyle.off: AvoidEscapedUnicodeCharacters public class CollationSupportSuite { @@ -37,7 +38,7 @@ public class CollationSupportSuite { * the specified collations (as often seen in some pass-through Spark expressions). */ private final String[] testSupportedCollations = - {"UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI"}; + {UTF8_BINARY, UTF8_LCASE, UNICODE, UNICODE_CI}; /** * Collation-aware UTF8String comparison and equality check. @@ -86,82 +87,82 @@ public void testCompare() throws SparkException { assertCompare("a", "ä", collationName, -1); } // Advanced tests. - assertCompare("äü", "bü", "UTF8_BINARY", 1); - assertCompare("bxx", "bü", "UTF8_BINARY", -1); - assertCompare("äü", "bü", "UTF8_LCASE", 1); - assertCompare("bxx", "bü", "UTF8_LCASE", -1); - assertCompare("äü", "bü", "UNICODE", -1); - assertCompare("bxx", "bü", "UNICODE", 1); - assertCompare("äü", "bü", "UNICODE_CI", -1); - assertCompare("bxx", "bü", "UNICODE_CI", 1); + assertCompare("äü", "bü", UTF8_BINARY, 1); + assertCompare("bxx", "bü", UTF8_BINARY, -1); + assertCompare("äü", "bü", UTF8_LCASE, 1); + assertCompare("bxx", "bü", UTF8_LCASE, -1); + assertCompare("äü", "bü", UNICODE, -1); + assertCompare("bxx", "bü", UNICODE, 1); + assertCompare("äü", "bü", UNICODE_CI, -1); + assertCompare("bxx", "bü", UNICODE_CI, 1); assertCompare("cČć", "ČćC", "SR_CI_AI", 0); // Case variation. - assertCompare("AbCd", "aBcD", "UTF8_BINARY", -1); - assertCompare("ABCD", "abcd", "UTF8_LCASE", 0); - assertCompare("AbcD", "aBCd", "UNICODE", 1); - assertCompare("abcd", "ABCD", "UNICODE_CI", 0); + assertCompare("AbCd", "aBcD", UTF8_BINARY, -1); + assertCompare("ABCD", "abcd", UTF8_LCASE, 0); + assertCompare("AbcD", "aBCd", UNICODE, 1); + assertCompare("abcd", "ABCD", UNICODE_CI, 0); // Accent variation. - assertCompare("aBćD", "ABĆD", "UTF8_BINARY", 1); - assertCompare("AbCδ", "ABCΔ", "UTF8_LCASE", 0); - assertCompare("äBCd", "ÄBCD", "UNICODE", -1); - assertCompare("Ab́cD", "AB́CD", "UNICODE_CI", 0); + assertCompare("aBćD", "ABĆD", UTF8_BINARY, 1); + assertCompare("AbCδ", "ABCΔ", UTF8_LCASE, 0); + assertCompare("äBCd", "ÄBCD", UNICODE, -1); + assertCompare("Ab́cD", "AB́CD", UNICODE_CI, 0); assertCompare("ÈÉÊË", "EeEe", "AF_CI_AI", 0); // One-to-many case mapping (e.g. Turkish dotted I). - assertCompare("i\u0307", "İ", "UTF8_BINARY", -1); - assertCompare("İ", "i\u0307", "UTF8_BINARY", 1); - assertCompare("i\u0307", "İ", "UTF8_LCASE", 0); - assertCompare("İ", "i\u0307", "UTF8_LCASE", 0); - assertCompare("i\u0307", "İ", "UNICODE", -1); - assertCompare("İ", "i\u0307", "UNICODE", 1); - assertCompare("i\u0307", "İ", "UNICODE_CI", 0); - assertCompare("İ", "i\u0307", "UNICODE_CI", 0); - assertCompare("i\u0307İ", "i\u0307İ", "UTF8_LCASE", 0); - assertCompare("i\u0307İ", "İi\u0307", "UTF8_LCASE", 0); - assertCompare("İi\u0307", "i\u0307İ", "UTF8_LCASE", 0); - assertCompare("İi\u0307", "İi\u0307", "UTF8_LCASE", 0); - assertCompare("i\u0307İ", "i\u0307İ", "UNICODE_CI", 0); - assertCompare("i\u0307İ", "İi\u0307", "UNICODE_CI", 0); - assertCompare("İi\u0307", "i\u0307İ", "UNICODE_CI", 0); - assertCompare("İi\u0307", "İi\u0307", "UNICODE_CI", 0); + assertCompare("i\u0307", "İ", UTF8_BINARY, -1); + assertCompare("İ", "i\u0307", UTF8_BINARY, 1); + assertCompare("i\u0307", "İ", UTF8_LCASE, 0); + assertCompare("İ", "i\u0307", UTF8_LCASE, 0); + assertCompare("i\u0307", "İ", UNICODE, -1); + assertCompare("İ", "i\u0307", UNICODE, 1); + assertCompare("i\u0307", "İ", UNICODE_CI, 0); + assertCompare("İ", "i\u0307", UNICODE_CI, 0); + assertCompare("i\u0307İ", "i\u0307İ", UTF8_LCASE, 0); + assertCompare("i\u0307İ", "İi\u0307", UTF8_LCASE, 0); + assertCompare("İi\u0307", "i\u0307İ", UTF8_LCASE, 0); + assertCompare("İi\u0307", "İi\u0307", UTF8_LCASE, 0); + assertCompare("i\u0307İ", "i\u0307İ", UNICODE_CI, 0); + assertCompare("i\u0307İ", "İi\u0307", UNICODE_CI, 0); + assertCompare("İi\u0307", "i\u0307İ", UNICODE_CI, 0); + assertCompare("İi\u0307", "İi\u0307", UNICODE_CI, 0); // Conditional case mapping (e.g. Greek sigmas). - assertCompare("ς", "σ", "UTF8_BINARY", -1); - assertCompare("ς", "Σ", "UTF8_BINARY", 1); - assertCompare("σ", "Σ", "UTF8_BINARY", 1); - assertCompare("ς", "σ", "UTF8_LCASE", 0); - assertCompare("ς", "Σ", "UTF8_LCASE", 0); - assertCompare("σ", "Σ", "UTF8_LCASE", 0); - assertCompare("ς", "σ", "UNICODE", 1); - assertCompare("ς", "Σ", "UNICODE", 1); - assertCompare("σ", "Σ", "UNICODE", -1); - assertCompare("ς", "σ", "UNICODE_CI", 0); - assertCompare("ς", "Σ", "UNICODE_CI", 0); - assertCompare("σ", "Σ", "UNICODE_CI", 0); + assertCompare("ς", "σ", UTF8_BINARY, -1); + assertCompare("ς", "Σ", UTF8_BINARY, 1); + assertCompare("σ", "Σ", UTF8_BINARY, 1); + assertCompare("ς", "σ", UTF8_LCASE, 0); + assertCompare("ς", "Σ", UTF8_LCASE, 0); + assertCompare("σ", "Σ", UTF8_LCASE, 0); + assertCompare("ς", "σ", UNICODE, 1); + assertCompare("ς", "Σ", UNICODE, 1); + assertCompare("σ", "Σ", UNICODE, -1); + assertCompare("ς", "σ", UNICODE_CI, 0); + assertCompare("ς", "Σ", UNICODE_CI, 0); + assertCompare("σ", "Σ", UNICODE_CI, 0); // Surrogate pairs. - assertCompare("a🙃b🙃c", "aaaaa", "UTF8_BINARY", 1); - assertCompare("a🙃b🙃c", "aaaaa", "UTF8_LCASE", 1); - assertCompare("a🙃b🙃c", "aaaaa", "UNICODE", -1); // != UTF8_BINARY - assertCompare("a🙃b🙃c", "aaaaa", "UNICODE_CI", -1); // != UTF8_LCASE - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UNICODE", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UTF8_BINARY", -1); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UTF8_LCASE", -1); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UNICODE", -1); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UNICODE_CI", -1); + assertCompare("a🙃b🙃c", "aaaaa", UTF8_BINARY, 1); + assertCompare("a🙃b🙃c", "aaaaa", UTF8_LCASE, 1); + assertCompare("a🙃b🙃c", "aaaaa", UNICODE, -1); // != UTF8_BINARY + assertCompare("a🙃b🙃c", "aaaaa", UNICODE_CI, -1); // != UTF8_LCASE + assertCompare("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃c", UNICODE, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UTF8_BINARY, -1); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UTF8_LCASE, -1); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UNICODE, -1); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UNICODE_CI, -1); // Maximum code point. int maxCodePoint = Character.MAX_CODE_POINT; String maxCodePointStr = new String(Character.toChars(maxCodePoint)); for (int i = 0; i < maxCodePoint && Character.isValidCodePoint(i); ++i) { - assertCompare(new String(Character.toChars(i)), maxCodePointStr, "UTF8_BINARY", -1); - assertCompare(new String(Character.toChars(i)), maxCodePointStr, "UTF8_LCASE", -1); + assertCompare(new String(Character.toChars(i)), maxCodePointStr, UTF8_BINARY, -1); + assertCompare(new String(Character.toChars(i)), maxCodePointStr, UTF8_LCASE, -1); } // Minimum code point. int minCodePoint = Character.MIN_CODE_POINT; String minCodePointStr = new String(Character.toChars(minCodePoint)); for (int i = minCodePoint + 1; i <= maxCodePoint && Character.isValidCodePoint(i); ++i) { - assertCompare(new String(Character.toChars(i)), minCodePointStr, "UTF8_BINARY", 1); - assertCompare(new String(Character.toChars(i)), minCodePointStr, "UTF8_LCASE", 1); + assertCompare(new String(Character.toChars(i)), minCodePointStr, UTF8_BINARY, 1); + assertCompare(new String(Character.toChars(i)), minCodePointStr, UTF8_LCASE, 1); } } @@ -302,201 +303,201 @@ public void testContains() throws SparkException { assertContains("Здраво", "Здраво", collationName, true); } // Advanced tests. - assertContains("abcde", "bcd", "UTF8_BINARY", true); - assertContains("abcde", "bde", "UTF8_BINARY", false); - assertContains("abcde", "fgh", "UTF8_BINARY", false); - assertContains("abcde", "abcde", "UNICODE", true); - assertContains("abcde", "aBcDe", "UNICODE", false); - assertContains("abcde", "fghij", "UNICODE", false); - assertContains("abcde", "C", "UTF8_LCASE", true); - assertContains("abcde", "AbCdE", "UTF8_LCASE", true); - assertContains("abcde", "X", "UTF8_LCASE", false); - assertContains("abcde", "c", "UNICODE_CI", true); - assertContains("abcde", "bCD", "UNICODE_CI", true); - assertContains("abcde", "123", "UNICODE_CI", false); - assertContains("ab世De", "b世D", "UTF8_BINARY", true); - assertContains("ab世De", "B世d", "UTF8_BINARY", false); - assertContains("äbćδe", "bćδ", "UTF8_BINARY", true); - assertContains("äbćδe", "BcΔ", "UTF8_BINARY", false); - assertContains("ab世De", "ab世De", "UNICODE", true); - assertContains("ab世De", "AB世dE", "UNICODE", false); - assertContains("äbćδe", "äbćδe", "UNICODE", true); - assertContains("äbćδe", "ÄBcΔÉ", "UNICODE", false); - assertContains("ab世De", "b世D", "UTF8_LCASE", true); - assertContains("ab世De", "B世d", "UTF8_LCASE", true); - assertContains("äbćδe", "bćδ", "UTF8_LCASE", true); - assertContains("äbćδe", "BcΔ", "UTF8_LCASE", false); - assertContains("ab世De", "ab世De", "UNICODE_CI", true); - assertContains("ab世De", "AB世dE", "UNICODE_CI", true); - assertContains("äbćδe", "ÄbćδE", "UNICODE_CI", true); - assertContains("äbćδe", "ÄBcΔÉ", "UNICODE_CI", false); - assertContains("The Kelvin.", "Kelvin", "UTF8_LCASE", true); - assertContains("The Kelvin.", "Kelvin", "UTF8_LCASE", true); - assertContains("The KKelvin.", "KKelvin", "UTF8_LCASE", true); - assertContains("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); - assertContains("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); - assertContains("The KKelvin.", "KKelvin,", "UTF8_LCASE", false); + assertContains("abcde", "bcd", UTF8_BINARY, true); + assertContains("abcde", "bde", UTF8_BINARY, false); + assertContains("abcde", "fgh", UTF8_BINARY, false); + assertContains("abcde", "abcde", UNICODE, true); + assertContains("abcde", "aBcDe", UNICODE, false); + assertContains("abcde", "fghij", UNICODE, false); + assertContains("abcde", "C", UTF8_LCASE, true); + assertContains("abcde", "AbCdE", UTF8_LCASE, true); + assertContains("abcde", "X", UTF8_LCASE, false); + assertContains("abcde", "c", UNICODE_CI, true); + assertContains("abcde", "bCD", UNICODE_CI, true); + assertContains("abcde", "123", UNICODE_CI, false); + assertContains("ab世De", "b世D", UTF8_BINARY, true); + assertContains("ab世De", "B世d", UTF8_BINARY, false); + assertContains("äbćδe", "bćδ", UTF8_BINARY, true); + assertContains("äbćδe", "BcΔ", UTF8_BINARY, false); + assertContains("ab世De", "ab世De", UNICODE, true); + assertContains("ab世De", "AB世dE", UNICODE, false); + assertContains("äbćδe", "äbćδe", UNICODE, true); + assertContains("äbćδe", "ÄBcΔÉ", UNICODE, false); + assertContains("ab世De", "b世D", UTF8_LCASE, true); + assertContains("ab世De", "B世d", UTF8_LCASE, true); + assertContains("äbćδe", "bćδ", UTF8_LCASE, true); + assertContains("äbćδe", "BcΔ", UTF8_LCASE, false); + assertContains("ab世De", "ab世De", UNICODE_CI, true); + assertContains("ab世De", "AB世dE", UNICODE_CI, true); + assertContains("äbćδe", "ÄbćδE", UNICODE_CI, true); + assertContains("äbćδe", "ÄBcΔÉ", UNICODE_CI, false); + assertContains("The Kelvin.", "Kelvin", UTF8_LCASE, true); + assertContains("The Kelvin.", "Kelvin", UTF8_LCASE, true); + assertContains("The KKelvin.", "KKelvin", UTF8_LCASE, true); + assertContains("2 Kelvin.", "2 Kelvin", UTF8_LCASE, true); + assertContains("2 Kelvin.", "2 Kelvin", UTF8_LCASE, true); + assertContains("The KKelvin.", "KKelvin,", UTF8_LCASE, false); assertContains("abčćd", "ABCCD", "SR_CI_AI", true); // Case variation. - assertContains("aBcDe", "bcd", "UTF8_BINARY", false); - assertContains("aBcDe", "BcD", "UTF8_BINARY", true); - assertContains("aBcDe", "abcde", "UNICODE", false); - assertContains("aBcDe", "aBcDe", "UNICODE", true); - assertContains("aBcDe", "bcd", "UTF8_LCASE", true); - assertContains("aBcDe", "BCD", "UTF8_LCASE", true); - assertContains("aBcDe", "abcde", "UNICODE_CI", true); - assertContains("aBcDe", "AbCdE", "UNICODE_CI", true); + assertContains("aBcDe", "bcd", UTF8_BINARY, false); + assertContains("aBcDe", "BcD", UTF8_BINARY, true); + assertContains("aBcDe", "abcde", UNICODE, false); + assertContains("aBcDe", "aBcDe", UNICODE, true); + assertContains("aBcDe", "bcd", UTF8_LCASE, true); + assertContains("aBcDe", "BCD", UTF8_LCASE, true); + assertContains("aBcDe", "abcde", UNICODE_CI, true); + assertContains("aBcDe", "AbCdE", UNICODE_CI, true); // Accent variation. - assertContains("aBcDe", "bćd", "UTF8_BINARY", false); - assertContains("aBcDe", "BćD", "UTF8_BINARY", false); - assertContains("aBcDe", "abćde", "UNICODE", false); - assertContains("aBcDe", "aBćDe", "UNICODE", false); - assertContains("aBcDe", "bćd", "UTF8_LCASE", false); - assertContains("aBcDe", "BĆD", "UTF8_LCASE", false); - assertContains("aBcDe", "abćde", "UNICODE_CI", false); - assertContains("aBcDe", "AbĆdE", "UNICODE_CI", false); + assertContains("aBcDe", "bćd", UTF8_BINARY, false); + assertContains("aBcDe", "BćD", UTF8_BINARY, false); + assertContains("aBcDe", "abćde", UNICODE, false); + assertContains("aBcDe", "aBćDe", UNICODE, false); + assertContains("aBcDe", "bćd", UTF8_LCASE, false); + assertContains("aBcDe", "BĆD", UTF8_LCASE, false); + assertContains("aBcDe", "abćde", UNICODE_CI, false); + assertContains("aBcDe", "AbĆdE", UNICODE_CI, false); assertContains("abEEE", "Bèêë", "AF_CI_AI", true); // One-to-many case mapping (e.g. Turkish dotted I). - assertContains("i\u0307", "i", "UNICODE_CI", false); - assertContains("i\u0307", "\u0307", "UNICODE_CI", false); - assertContains("i\u0307", "İ", "UNICODE_CI", true); - assertContains("İ", "i", "UNICODE_CI", false); - assertContains("adi̇os", "io", "UNICODE_CI", false); - assertContains("adi̇os", "Io", "UNICODE_CI", false); - assertContains("adi̇os", "i\u0307o", "UNICODE_CI", true); - assertContains("adi̇os", "İo", "UNICODE_CI", true); - assertContains("adİos", "io", "UNICODE_CI", false); - assertContains("adİos", "Io", "UNICODE_CI", false); - assertContains("adİos", "i\u0307o", "UNICODE_CI", true); - assertContains("adİos", "İo", "UNICODE_CI", true); - assertContains("i\u0307", "i", "UTF8_LCASE", true); // != UNICODE_CI - assertContains("İ", "\u0307", "UTF8_LCASE", false); - assertContains("İ", "i", "UTF8_LCASE", false); - assertContains("i\u0307", "\u0307", "UTF8_LCASE", true); // != UNICODE_CI - assertContains("i\u0307", "İ", "UTF8_LCASE", true); - assertContains("İ", "i", "UTF8_LCASE", false); - assertContains("adi̇os", "io", "UTF8_LCASE", false); - assertContains("adi̇os", "Io", "UTF8_LCASE", false); - assertContains("adi̇os", "i\u0307o", "UTF8_LCASE", true); - assertContains("adi̇os", "İo", "UTF8_LCASE", true); - assertContains("adİos", "io", "UTF8_LCASE", false); - assertContains("adİos", "Io", "UTF8_LCASE", false); - assertContains("adİos", "i\u0307o", "UTF8_LCASE", true); - assertContains("adİos", "İo", "UTF8_LCASE", true); + assertContains("i\u0307", "i", UNICODE_CI, false); + assertContains("i\u0307", "\u0307", UNICODE_CI, false); + assertContains("i\u0307", "İ", UNICODE_CI, true); + assertContains("İ", "i", UNICODE_CI, false); + assertContains("adi̇os", "io", UNICODE_CI, false); + assertContains("adi̇os", "Io", UNICODE_CI, false); + assertContains("adi̇os", "i\u0307o", UNICODE_CI, true); + assertContains("adi̇os", "İo", UNICODE_CI, true); + assertContains("adİos", "io", UNICODE_CI, false); + assertContains("adİos", "Io", UNICODE_CI, false); + assertContains("adİos", "i\u0307o", UNICODE_CI, true); + assertContains("adİos", "İo", UNICODE_CI, true); + assertContains("i\u0307", "i", UTF8_LCASE, true); // != UNICODE_CI + assertContains("İ", "\u0307", UTF8_LCASE, false); + assertContains("İ", "i", UTF8_LCASE, false); + assertContains("i\u0307", "\u0307", UTF8_LCASE, true); // != UNICODE_CI + assertContains("i\u0307", "İ", UTF8_LCASE, true); + assertContains("İ", "i", UTF8_LCASE, false); + assertContains("adi̇os", "io", UTF8_LCASE, false); + assertContains("adi̇os", "Io", UTF8_LCASE, false); + assertContains("adi̇os", "i\u0307o", UTF8_LCASE, true); + assertContains("adi̇os", "İo", UTF8_LCASE, true); + assertContains("adİos", "io", UTF8_LCASE, false); + assertContains("adİos", "Io", UTF8_LCASE, false); + assertContains("adİos", "i\u0307o", UTF8_LCASE, true); + assertContains("adİos", "İo", UTF8_LCASE, true); // Conditional case mapping (e.g. Greek sigmas). - assertContains("σ", "σ", "UTF8_BINARY", true); - assertContains("σ", "ς", "UTF8_BINARY", false); - assertContains("σ", "Σ", "UTF8_BINARY", false); - assertContains("ς", "σ", "UTF8_BINARY", false); - assertContains("ς", "ς", "UTF8_BINARY", true); - assertContains("ς", "Σ", "UTF8_BINARY", false); - assertContains("Σ", "σ", "UTF8_BINARY", false); - assertContains("Σ", "ς", "UTF8_BINARY", false); - assertContains("Σ", "Σ", "UTF8_BINARY", true); - assertContains("σ", "σ", "UTF8_LCASE", true); - assertContains("σ", "ς", "UTF8_LCASE", true); - assertContains("σ", "Σ", "UTF8_LCASE", true); - assertContains("ς", "σ", "UTF8_LCASE", true); - assertContains("ς", "ς", "UTF8_LCASE", true); - assertContains("ς", "Σ", "UTF8_LCASE", true); - assertContains("Σ", "σ", "UTF8_LCASE", true); - assertContains("Σ", "ς", "UTF8_LCASE", true); - assertContains("Σ", "Σ", "UTF8_LCASE", true); - assertContains("σ", "σ", "UNICODE", true); - assertContains("σ", "ς", "UNICODE", false); - assertContains("σ", "Σ", "UNICODE", false); - assertContains("ς", "σ", "UNICODE", false); - assertContains("ς", "ς", "UNICODE", true); - assertContains("ς", "Σ", "UNICODE", false); - assertContains("Σ", "σ", "UNICODE", false); - assertContains("Σ", "ς", "UNICODE", false); - assertContains("Σ", "Σ", "UNICODE", true); - assertContains("σ", "σ", "UNICODE_CI", true); - assertContains("σ", "ς", "UNICODE_CI", true); - assertContains("σ", "Σ", "UNICODE_CI", true); - assertContains("ς", "σ", "UNICODE_CI", true); - assertContains("ς", "ς", "UNICODE_CI", true); - assertContains("ς", "Σ", "UNICODE_CI", true); - assertContains("Σ", "σ", "UNICODE_CI", true); - assertContains("Σ", "ς", "UNICODE_CI", true); - assertContains("Σ", "Σ", "UNICODE_CI", true); - assertContains("ΣΑΛΑΤΑ", "Σ", "UTF8_BINARY", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UTF8_BINARY", false); - assertContains("ΣΑΛΑΤΑ", "ς", "UTF8_BINARY", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_BINARY", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_BINARY", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_BINARY", false); - assertContains("ΣΑΛΑΤΑ", "Σ", "UTF8_LCASE", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UTF8_LCASE", true); - assertContains("ΣΑΛΑΤΑ", "ς", "UTF8_LCASE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_LCASE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_LCASE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_LCASE", true); - assertContains("ΣΑΛΑΤΑ", "Σ", "UNICODE", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UNICODE", false); - assertContains("ΣΑΛΑΤΑ", "ς", "UNICODE", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE", false); - assertContains("ΣΑΛΑΤΑ", "Σ", "UNICODE_CI", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UNICODE_CI", true); - assertContains("ΣΑΛΑΤΑ", "ς", "UNICODE_CI", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE_CI", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE_CI", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE_CI", true); + assertContains("σ", "σ", UTF8_BINARY, true); + assertContains("σ", "ς", UTF8_BINARY, false); + assertContains("σ", "Σ", UTF8_BINARY, false); + assertContains("ς", "σ", UTF8_BINARY, false); + assertContains("ς", "ς", UTF8_BINARY, true); + assertContains("ς", "Σ", UTF8_BINARY, false); + assertContains("Σ", "σ", UTF8_BINARY, false); + assertContains("Σ", "ς", UTF8_BINARY, false); + assertContains("Σ", "Σ", UTF8_BINARY, true); + assertContains("σ", "σ", UTF8_LCASE, true); + assertContains("σ", "ς", UTF8_LCASE, true); + assertContains("σ", "Σ", UTF8_LCASE, true); + assertContains("ς", "σ", UTF8_LCASE, true); + assertContains("ς", "ς", UTF8_LCASE, true); + assertContains("ς", "Σ", UTF8_LCASE, true); + assertContains("Σ", "σ", UTF8_LCASE, true); + assertContains("Σ", "ς", UTF8_LCASE, true); + assertContains("Σ", "Σ", UTF8_LCASE, true); + assertContains("σ", "σ", UNICODE, true); + assertContains("σ", "ς", UNICODE, false); + assertContains("σ", "Σ", UNICODE, false); + assertContains("ς", "σ", UNICODE, false); + assertContains("ς", "ς", UNICODE, true); + assertContains("ς", "Σ", UNICODE, false); + assertContains("Σ", "σ", UNICODE, false); + assertContains("Σ", "ς", UNICODE, false); + assertContains("Σ", "Σ", UNICODE, true); + assertContains("σ", "σ", UNICODE_CI, true); + assertContains("σ", "ς", UNICODE_CI, true); + assertContains("σ", "Σ", UNICODE_CI, true); + assertContains("ς", "σ", UNICODE_CI, true); + assertContains("ς", "ς", UNICODE_CI, true); + assertContains("ς", "Σ", UNICODE_CI, true); + assertContains("Σ", "σ", UNICODE_CI, true); + assertContains("Σ", "ς", UNICODE_CI, true); + assertContains("Σ", "Σ", UNICODE_CI, true); + assertContains("ΣΑΛΑΤΑ", "Σ", UTF8_BINARY, true); + assertContains("ΣΑΛΑΤΑ", "σ", UTF8_BINARY, false); + assertContains("ΣΑΛΑΤΑ", "ς", UTF8_BINARY, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_BINARY, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_BINARY, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_BINARY, false); + assertContains("ΣΑΛΑΤΑ", "Σ", UTF8_LCASE, true); + assertContains("ΣΑΛΑΤΑ", "σ", UTF8_LCASE, true); + assertContains("ΣΑΛΑΤΑ", "ς", UTF8_LCASE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_LCASE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_LCASE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_LCASE, true); + assertContains("ΣΑΛΑΤΑ", "Σ", UNICODE, true); + assertContains("ΣΑΛΑΤΑ", "σ", UNICODE, false); + assertContains("ΣΑΛΑΤΑ", "ς", UNICODE, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE, false); + assertContains("ΣΑΛΑΤΑ", "Σ", UNICODE_CI, true); + assertContains("ΣΑΛΑΤΑ", "σ", UNICODE_CI, true); + assertContains("ΣΑΛΑΤΑ", "ς", UNICODE_CI, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE_CI, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE_CI, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE_CI, true); // Surrogate pairs. - assertContains("a🙃b🙃c", "x", "UTF8_BINARY", false); - assertContains("a🙃b🙃c", "x", "UTF8_LCASE", false); - assertContains("a🙃b🙃c", "x", "UNICODE", false); - assertContains("a🙃b🙃c", "x", "UNICODE_CI", false); - assertContains("a🙃b🙃c", "b", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "b", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "b", "UNICODE", true); - assertContains("a🙃b🙃c", "b", "UNICODE_CI", true); - assertContains("a🙃b🙃c", "a🙃b", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "a🙃b", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "a🙃b", "UNICODE", true); - assertContains("a🙃b🙃c", "a🙃b", "UNICODE_CI", true); - assertContains("a🙃b🙃c", "b🙃c", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "b🙃c", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "b🙃c", "UNICODE", true); - assertContains("a🙃b🙃c", "b🙃c", "UNICODE_CI", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UNICODE", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😄😆", "UTF8_BINARY", false); - assertContains("😀😆😃😄", "😄😆", "UTF8_LCASE", false); - assertContains("😀😆😃😄", "😄😆", "UNICODE", false); - assertContains("😀😆😃😄", "😄😆", "UNICODE_CI", false); - assertContains("😀😆😃😄", "😆😃", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😆😃", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😆😃", "UNICODE", true); - assertContains("😀😆😃😄", "😆😃", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😀😆", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😀😆", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😀😆", "UNICODE", true); - assertContains("😀😆😃😄", "😀😆", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😃😄", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😃😄", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😃😄", "UNICODE", true); - assertContains("😀😆😃😄", "😃😄", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UNICODE", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UNICODE_CI", true); - assertContains("𐐅", "𐐅", "UTF8_BINARY", true); - assertContains("𐐅", "𐐅", "UTF8_LCASE", true); - assertContains("𐐅", "𐐅", "UNICODE", true); - assertContains("𐐅", "𐐅", "UNICODE_CI", true); - assertContains("𐐅", "𐐭", "UTF8_BINARY", false); - assertContains("𐐅", "𐐭", "UTF8_LCASE", true); - assertContains("𐐅", "𐐭", "UNICODE", false); - assertContains("𐐅", "𐐭", "UNICODE_CI", true); - assertContains("𝔸", "𝔸", "UTF8_BINARY", true); - assertContains("𝔸", "𝔸", "UTF8_LCASE", true); - assertContains("𝔸", "𝔸", "UNICODE", true); - assertContains("𝔸", "𝔸", "UNICODE_CI", true); + assertContains("a🙃b🙃c", "x", UTF8_BINARY, false); + assertContains("a🙃b🙃c", "x", UTF8_LCASE, false); + assertContains("a🙃b🙃c", "x", UNICODE, false); + assertContains("a🙃b🙃c", "x", UNICODE_CI, false); + assertContains("a🙃b🙃c", "b", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "b", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "b", UNICODE, true); + assertContains("a🙃b🙃c", "b", UNICODE_CI, true); + assertContains("a🙃b🙃c", "a🙃b", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "a🙃b", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "a🙃b", UNICODE, true); + assertContains("a🙃b🙃c", "a🙃b", UNICODE_CI, true); + assertContains("a🙃b🙃c", "b🙃c", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "b🙃c", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "b🙃c", UNICODE, true); + assertContains("a🙃b🙃c", "b🙃c", UNICODE_CI, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UNICODE, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, true); + assertContains("😀😆😃😄", "😄😆", UTF8_BINARY, false); + assertContains("😀😆😃😄", "😄😆", UTF8_LCASE, false); + assertContains("😀😆😃😄", "😄😆", UNICODE, false); + assertContains("😀😆😃😄", "😄😆", UNICODE_CI, false); + assertContains("😀😆😃😄", "😆😃", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😆😃", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😆😃", UNICODE, true); + assertContains("😀😆😃😄", "😆😃", UNICODE_CI, true); + assertContains("😀😆😃😄", "😀😆", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😀😆", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😀😆", UNICODE, true); + assertContains("😀😆😃😄", "😀😆", UNICODE_CI, true); + assertContains("😀😆😃😄", "😃😄", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😃😄", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😃😄", UNICODE, true); + assertContains("😀😆😃😄", "😃😄", UNICODE_CI, true); + assertContains("😀😆😃😄", "😀😆😃😄", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😀😆😃😄", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😀😆😃😄", UNICODE, true); + assertContains("😀😆😃😄", "😀😆😃😄", UNICODE_CI, true); + assertContains("𐐅", "𐐅", UTF8_BINARY, true); + assertContains("𐐅", "𐐅", UTF8_LCASE, true); + assertContains("𐐅", "𐐅", UNICODE, true); + assertContains("𐐅", "𐐅", UNICODE_CI, true); + assertContains("𐐅", "𐐭", UTF8_BINARY, false); + assertContains("𐐅", "𐐭", UTF8_LCASE, true); + assertContains("𐐅", "𐐭", UNICODE, false); + assertContains("𐐅", "𐐭", UNICODE_CI, true); + assertContains("𝔸", "𝔸", UTF8_BINARY, true); + assertContains("𝔸", "𝔸", UTF8_LCASE, true); + assertContains("𝔸", "𝔸", UNICODE, true); + assertContains("𝔸", "𝔸", UNICODE_CI, true); } /** @@ -549,211 +550,211 @@ public void testStartsWith() throws SparkException { assertStartsWith("Здраво", "Здраво", collationName, true); } // Advanced tests. - assertStartsWith("abcde", "abc", "UTF8_BINARY", true); - assertStartsWith("abcde", "abd", "UTF8_BINARY", false); - assertStartsWith("abcde", "fgh", "UTF8_BINARY", false); - assertStartsWith("abcde", "abcde", "UNICODE", true); - assertStartsWith("abcde", "aBcDe", "UNICODE", false); - assertStartsWith("abcde", "fghij", "UNICODE", false); - assertStartsWith("abcde", "A", "UTF8_LCASE", true); - assertStartsWith("abcde", "AbCdE", "UTF8_LCASE", true); - assertStartsWith("abcde", "X", "UTF8_LCASE", false); - assertStartsWith("abcde", "a", "UNICODE_CI", true); - assertStartsWith("abcde", "aBC", "UNICODE_CI", true); - assertStartsWith("abcde", "bcd", "UNICODE_CI", false); - assertStartsWith("abcde", "123", "UNICODE_CI", false); - assertStartsWith("ab世De", "ab世", "UTF8_BINARY", true); - assertStartsWith("ab世De", "aB世", "UTF8_BINARY", false); - assertStartsWith("äbćδe", "äbć", "UTF8_BINARY", true); - assertStartsWith("äbćδe", "äBc", "UTF8_BINARY", false); - assertStartsWith("ab世De", "ab世De", "UNICODE", true); - assertStartsWith("ab世De", "AB世dE", "UNICODE", false); - assertStartsWith("äbćδe", "äbćδe", "UNICODE", true); - assertStartsWith("äbćδe", "ÄBcΔÉ", "UNICODE", false); - assertStartsWith("ab世De", "ab世", "UTF8_LCASE", true); - assertStartsWith("ab世De", "aB世", "UTF8_LCASE", true); - assertStartsWith("äbćδe", "äbć", "UTF8_LCASE", true); - assertStartsWith("äbćδe", "äBc", "UTF8_LCASE", false); - assertStartsWith("ab世De", "ab世De", "UNICODE_CI", true); - assertStartsWith("ab世De", "AB世dE", "UNICODE_CI", true); - assertStartsWith("äbćδe", "ÄbćδE", "UNICODE_CI", true); - assertStartsWith("äbćδe", "ÄBcΔÉ", "UNICODE_CI", false); - assertStartsWith("Kelvin.", "Kelvin", "UTF8_LCASE", true); - assertStartsWith("Kelvin.", "Kelvin", "UTF8_LCASE", true); - assertStartsWith("KKelvin.", "KKelvin", "UTF8_LCASE", true); - assertStartsWith("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); - assertStartsWith("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); - assertStartsWith("KKelvin.", "KKelvin,", "UTF8_LCASE", false); + assertStartsWith("abcde", "abc", UTF8_BINARY, true); + assertStartsWith("abcde", "abd", UTF8_BINARY, false); + assertStartsWith("abcde", "fgh", UTF8_BINARY, false); + assertStartsWith("abcde", "abcde", UNICODE, true); + assertStartsWith("abcde", "aBcDe", UNICODE, false); + assertStartsWith("abcde", "fghij", UNICODE, false); + assertStartsWith("abcde", "A", UTF8_LCASE, true); + assertStartsWith("abcde", "AbCdE", UTF8_LCASE, true); + assertStartsWith("abcde", "X", UTF8_LCASE, false); + assertStartsWith("abcde", "a", UNICODE_CI, true); + assertStartsWith("abcde", "aBC", UNICODE_CI, true); + assertStartsWith("abcde", "bcd", UNICODE_CI, false); + assertStartsWith("abcde", "123", UNICODE_CI, false); + assertStartsWith("ab世De", "ab世", UTF8_BINARY, true); + assertStartsWith("ab世De", "aB世", UTF8_BINARY, false); + assertStartsWith("äbćδe", "äbć", UTF8_BINARY, true); + assertStartsWith("äbćδe", "äBc", UTF8_BINARY, false); + assertStartsWith("ab世De", "ab世De", UNICODE, true); + assertStartsWith("ab世De", "AB世dE", UNICODE, false); + assertStartsWith("äbćδe", "äbćδe", UNICODE, true); + assertStartsWith("äbćδe", "ÄBcΔÉ", UNICODE, false); + assertStartsWith("ab世De", "ab世", UTF8_LCASE, true); + assertStartsWith("ab世De", "aB世", UTF8_LCASE, true); + assertStartsWith("äbćδe", "äbć", UTF8_LCASE, true); + assertStartsWith("äbćδe", "äBc", UTF8_LCASE, false); + assertStartsWith("ab世De", "ab世De", UNICODE_CI, true); + assertStartsWith("ab世De", "AB世dE", UNICODE_CI, true); + assertStartsWith("äbćδe", "ÄbćδE", UNICODE_CI, true); + assertStartsWith("äbćδe", "ÄBcΔÉ", UNICODE_CI, false); + assertStartsWith("Kelvin.", "Kelvin", UTF8_LCASE, true); + assertStartsWith("Kelvin.", "Kelvin", UTF8_LCASE, true); + assertStartsWith("KKelvin.", "KKelvin", UTF8_LCASE, true); + assertStartsWith("2 Kelvin.", "2 Kelvin", UTF8_LCASE, true); + assertStartsWith("2 Kelvin.", "2 Kelvin", UTF8_LCASE, true); + assertStartsWith("KKelvin.", "KKelvin,", UTF8_LCASE, false); assertStartsWith("Ћао", "Ца", "sr_Cyrl_CI_AI", false); assertStartsWith("Ћао", "ћа", "sr_Cyrl_CI_AI", true); assertStartsWith("Ćao", "Ca", "SR_CI", false); assertStartsWith("Ćao", "Ca", "SR_CI_AI", true); assertStartsWith("Ćao", "Ća", "SR", true); // Case variation. - assertStartsWith("aBcDe", "abc", "UTF8_BINARY", false); - assertStartsWith("aBcDe", "aBc", "UTF8_BINARY", true); - assertStartsWith("aBcDe", "abcde", "UNICODE", false); - assertStartsWith("aBcDe", "aBcDe", "UNICODE", true); - assertStartsWith("aBcDe", "abc", "UTF8_LCASE", true); - assertStartsWith("aBcDe", "ABC", "UTF8_LCASE", true); - assertStartsWith("aBcDe", "abcde", "UNICODE_CI", true); - assertStartsWith("aBcDe", "AbCdE", "UNICODE_CI", true); + assertStartsWith("aBcDe", "abc", UTF8_BINARY, false); + assertStartsWith("aBcDe", "aBc", UTF8_BINARY, true); + assertStartsWith("aBcDe", "abcde", UNICODE, false); + assertStartsWith("aBcDe", "aBcDe", UNICODE, true); + assertStartsWith("aBcDe", "abc", UTF8_LCASE, true); + assertStartsWith("aBcDe", "ABC", UTF8_LCASE, true); + assertStartsWith("aBcDe", "abcde", UNICODE_CI, true); + assertStartsWith("aBcDe", "AbCdE", UNICODE_CI, true); // Accent variation. - assertStartsWith("aBcDe", "abć", "UTF8_BINARY", false); - assertStartsWith("aBcDe", "aBć", "UTF8_BINARY", false); - assertStartsWith("aBcDe", "abćde", "UNICODE", false); - assertStartsWith("aBcDe", "aBćDe", "UNICODE", false); - assertStartsWith("aBcDe", "abć", "UTF8_LCASE", false); - assertStartsWith("aBcDe", "ABĆ", "UTF8_LCASE", false); - assertStartsWith("aBcDe", "abćde", "UNICODE_CI", false); - assertStartsWith("aBcDe", "AbĆdE", "UNICODE_CI", false); + assertStartsWith("aBcDe", "abć", UTF8_BINARY, false); + assertStartsWith("aBcDe", "aBć", UTF8_BINARY, false); + assertStartsWith("aBcDe", "abćde", UNICODE, false); + assertStartsWith("aBcDe", "aBćDe", UNICODE, false); + assertStartsWith("aBcDe", "abć", UTF8_LCASE, false); + assertStartsWith("aBcDe", "ABĆ", UTF8_LCASE, false); + assertStartsWith("aBcDe", "abćde", UNICODE_CI, false); + assertStartsWith("aBcDe", "AbĆdE", UNICODE_CI, false); // One-to-many case mapping (e.g. Turkish dotted I). - assertStartsWith("i\u0307", "i", "UNICODE_CI", false); - assertStartsWith("i\u0307", "İ", "UNICODE_CI", true); - assertStartsWith("İ", "i", "UNICODE_CI", false); - assertStartsWith("İİİ", "i̇i̇", "UNICODE_CI", true); - assertStartsWith("İİİ", "i̇i", "UNICODE_CI", false); - assertStartsWith("İi̇İ", "i̇İ", "UNICODE_CI", true); - assertStartsWith("i̇İi̇i̇", "İi̇İi", "UNICODE_CI", false); - assertStartsWith("i̇onic", "io", "UNICODE_CI", false); - assertStartsWith("i̇onic", "Io", "UNICODE_CI", false); - assertStartsWith("i̇onic", "i\u0307o", "UNICODE_CI", true); - assertStartsWith("i̇onic", "İo", "UNICODE_CI", true); - assertStartsWith("İonic", "io", "UNICODE_CI", false); - assertStartsWith("İonic", "Io", "UNICODE_CI", false); - assertStartsWith("İonic", "i\u0307o", "UNICODE_CI", true); - assertStartsWith("İonic", "İo", "UNICODE_CI", true); - assertStartsWith("i\u0307", "i", "UTF8_LCASE", true); // != UNICODE_CI - assertStartsWith("i\u0307", "İ", "UTF8_LCASE", true); - assertStartsWith("İ", "i", "UTF8_LCASE", false); - assertStartsWith("İİİ", "i̇i̇", "UTF8_LCASE", true); - assertStartsWith("İİİ", "i̇i", "UTF8_LCASE", false); - assertStartsWith("İi̇İ", "i̇İ", "UTF8_LCASE", true); - assertStartsWith("i̇İi̇i̇", "İi̇İi", "UTF8_LCASE", true); // != UNICODE_CI - assertStartsWith("i̇onic", "io", "UTF8_LCASE", false); - assertStartsWith("i̇onic", "Io", "UTF8_LCASE", false); - assertStartsWith("i̇onic", "i\u0307o", "UTF8_LCASE", true); - assertStartsWith("i̇onic", "İo", "UTF8_LCASE", true); - assertStartsWith("İonic", "io", "UTF8_LCASE", false); - assertStartsWith("İonic", "Io", "UTF8_LCASE", false); - assertStartsWith("İonic", "i\u0307o", "UTF8_LCASE", true); - assertStartsWith("İonic", "İo", "UTF8_LCASE", true); - assertStartsWith("oİ", "oİ", "UTF8_LCASE", true); - assertStartsWith("oİ", "oi̇", "UTF8_LCASE", true); + assertStartsWith("i\u0307", "i", UNICODE_CI, false); + assertStartsWith("i\u0307", "İ", UNICODE_CI, true); + assertStartsWith("İ", "i", UNICODE_CI, false); + assertStartsWith("İİİ", "i̇i̇", UNICODE_CI, true); + assertStartsWith("İİİ", "i̇i", UNICODE_CI, false); + assertStartsWith("İi̇İ", "i̇İ", UNICODE_CI, true); + assertStartsWith("i̇İi̇i̇", "İi̇İi", UNICODE_CI, false); + assertStartsWith("i̇onic", "io", UNICODE_CI, false); + assertStartsWith("i̇onic", "Io", UNICODE_CI, false); + assertStartsWith("i̇onic", "i\u0307o", UNICODE_CI, true); + assertStartsWith("i̇onic", "İo", UNICODE_CI, true); + assertStartsWith("İonic", "io", UNICODE_CI, false); + assertStartsWith("İonic", "Io", UNICODE_CI, false); + assertStartsWith("İonic", "i\u0307o", UNICODE_CI, true); + assertStartsWith("İonic", "İo", UNICODE_CI, true); + assertStartsWith("i\u0307", "i", UTF8_LCASE, true); // != UNICODE_CI + assertStartsWith("i\u0307", "İ", UTF8_LCASE, true); + assertStartsWith("İ", "i", UTF8_LCASE, false); + assertStartsWith("İİİ", "i̇i̇", UTF8_LCASE, true); + assertStartsWith("İİİ", "i̇i", UTF8_LCASE, false); + assertStartsWith("İi̇İ", "i̇İ", UTF8_LCASE, true); + assertStartsWith("i̇İi̇i̇", "İi̇İi", UTF8_LCASE, true); // != UNICODE_CI + assertStartsWith("i̇onic", "io", UTF8_LCASE, false); + assertStartsWith("i̇onic", "Io", UTF8_LCASE, false); + assertStartsWith("i̇onic", "i\u0307o", UTF8_LCASE, true); + assertStartsWith("i̇onic", "İo", UTF8_LCASE, true); + assertStartsWith("İonic", "io", UTF8_LCASE, false); + assertStartsWith("İonic", "Io", UTF8_LCASE, false); + assertStartsWith("İonic", "i\u0307o", UTF8_LCASE, true); + assertStartsWith("İonic", "İo", UTF8_LCASE, true); + assertStartsWith("oİ", "oİ", UTF8_LCASE, true); + assertStartsWith("oİ", "oi̇", UTF8_LCASE, true); // Conditional case mapping (e.g. Greek sigmas). - assertStartsWith("σ", "σ", "UTF8_BINARY", true); - assertStartsWith("σ", "ς", "UTF8_BINARY", false); - assertStartsWith("σ", "Σ", "UTF8_BINARY", false); - assertStartsWith("ς", "σ", "UTF8_BINARY", false); - assertStartsWith("ς", "ς", "UTF8_BINARY", true); - assertStartsWith("ς", "Σ", "UTF8_BINARY", false); - assertStartsWith("Σ", "σ", "UTF8_BINARY", false); - assertStartsWith("Σ", "ς", "UTF8_BINARY", false); - assertStartsWith("Σ", "Σ", "UTF8_BINARY", true); - assertStartsWith("σ", "σ", "UTF8_LCASE", true); - assertStartsWith("σ", "ς", "UTF8_LCASE", true); - assertStartsWith("σ", "Σ", "UTF8_LCASE", true); - assertStartsWith("ς", "σ", "UTF8_LCASE", true); - assertStartsWith("ς", "ς", "UTF8_LCASE", true); - assertStartsWith("ς", "Σ", "UTF8_LCASE", true); - assertStartsWith("Σ", "σ", "UTF8_LCASE", true); - assertStartsWith("Σ", "ς", "UTF8_LCASE", true); - assertStartsWith("Σ", "Σ", "UTF8_LCASE", true); - assertStartsWith("σ", "σ", "UNICODE", true); - assertStartsWith("σ", "ς", "UNICODE", false); - assertStartsWith("σ", "Σ", "UNICODE", false); - assertStartsWith("ς", "σ", "UNICODE", false); - assertStartsWith("ς", "ς", "UNICODE", true); - assertStartsWith("ς", "Σ", "UNICODE", false); - assertStartsWith("Σ", "σ", "UNICODE", false); - assertStartsWith("Σ", "ς", "UNICODE", false); - assertStartsWith("Σ", "Σ", "UNICODE", true); - assertStartsWith("σ", "σ", "UNICODE_CI", true); - assertStartsWith("σ", "ς", "UNICODE_CI", true); - assertStartsWith("σ", "Σ", "UNICODE_CI", true); - assertStartsWith("ς", "σ", "UNICODE_CI", true); - assertStartsWith("ς", "ς", "UNICODE_CI", true); - assertStartsWith("ς", "Σ", "UNICODE_CI", true); - assertStartsWith("Σ", "σ", "UNICODE_CI", true); - assertStartsWith("Σ", "ς", "UNICODE_CI", true); - assertStartsWith("Σ", "Σ", "UNICODE_CI", true); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_BINARY", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UTF8_BINARY", false); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UTF8_BINARY", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_BINARY", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_BINARY", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_BINARY", false); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_LCASE", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UTF8_LCASE", true); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UTF8_LCASE", true); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_LCASE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_LCASE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_LCASE", false); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UNICODE", false); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UNICODE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE", false); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE_CI", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UNICODE_CI", true); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UNICODE_CI", true); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE_CI", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE_CI", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE_CI", false); + assertStartsWith("σ", "σ", UTF8_BINARY, true); + assertStartsWith("σ", "ς", UTF8_BINARY, false); + assertStartsWith("σ", "Σ", UTF8_BINARY, false); + assertStartsWith("ς", "σ", UTF8_BINARY, false); + assertStartsWith("ς", "ς", UTF8_BINARY, true); + assertStartsWith("ς", "Σ", UTF8_BINARY, false); + assertStartsWith("Σ", "σ", UTF8_BINARY, false); + assertStartsWith("Σ", "ς", UTF8_BINARY, false); + assertStartsWith("Σ", "Σ", UTF8_BINARY, true); + assertStartsWith("σ", "σ", UTF8_LCASE, true); + assertStartsWith("σ", "ς", UTF8_LCASE, true); + assertStartsWith("σ", "Σ", UTF8_LCASE, true); + assertStartsWith("ς", "σ", UTF8_LCASE, true); + assertStartsWith("ς", "ς", UTF8_LCASE, true); + assertStartsWith("ς", "Σ", UTF8_LCASE, true); + assertStartsWith("Σ", "σ", UTF8_LCASE, true); + assertStartsWith("Σ", "ς", UTF8_LCASE, true); + assertStartsWith("Σ", "Σ", UTF8_LCASE, true); + assertStartsWith("σ", "σ", UNICODE, true); + assertStartsWith("σ", "ς", UNICODE, false); + assertStartsWith("σ", "Σ", UNICODE, false); + assertStartsWith("ς", "σ", UNICODE, false); + assertStartsWith("ς", "ς", UNICODE, true); + assertStartsWith("ς", "Σ", UNICODE, false); + assertStartsWith("Σ", "σ", UNICODE, false); + assertStartsWith("Σ", "ς", UNICODE, false); + assertStartsWith("Σ", "Σ", UNICODE, true); + assertStartsWith("σ", "σ", UNICODE_CI, true); + assertStartsWith("σ", "ς", UNICODE_CI, true); + assertStartsWith("σ", "Σ", UNICODE_CI, true); + assertStartsWith("ς", "σ", UNICODE_CI, true); + assertStartsWith("ς", "ς", UNICODE_CI, true); + assertStartsWith("ς", "Σ", UNICODE_CI, true); + assertStartsWith("Σ", "σ", UNICODE_CI, true); + assertStartsWith("Σ", "ς", UNICODE_CI, true); + assertStartsWith("Σ", "Σ", UNICODE_CI, true); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UTF8_BINARY, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UTF8_BINARY, false); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UTF8_BINARY, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_BINARY, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_BINARY, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_BINARY, false); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UTF8_LCASE, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UTF8_LCASE, true); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UTF8_LCASE, true); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_LCASE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_LCASE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_LCASE, false); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UNICODE, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UNICODE, false); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UNICODE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE, false); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UNICODE_CI, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UNICODE_CI, true); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UNICODE_CI, true); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE_CI, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE_CI, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE_CI, false); // Surrogate pairs. - assertStartsWith("a🙃b🙃c", "x", "UTF8_BINARY", false); - assertStartsWith("a🙃b🙃c", "x", "UTF8_LCASE", false); - assertStartsWith("a🙃b🙃c", "x", "UNICODE", false); - assertStartsWith("a🙃b🙃c", "x", "UNICODE_CI", false); - assertStartsWith("a🙃b🙃c", "b", "UTF8_BINARY", false); - assertStartsWith("a🙃b🙃c", "b", "UTF8_LCASE", false); - assertStartsWith("a🙃b🙃c", "b", "UNICODE", false); - assertStartsWith("a🙃b🙃c", "b", "UNICODE_CI", false); - assertStartsWith("a🙃b🙃c", "a🙃b", "UTF8_BINARY", true); - assertStartsWith("a🙃b🙃c", "a🙃b", "UTF8_LCASE", true); - assertStartsWith("a🙃b🙃c", "a🙃b", "UNICODE", true); - assertStartsWith("a🙃b🙃c", "a🙃b", "UNICODE_CI", true); - assertStartsWith("a🙃b🙃c", "b🙃c", "UTF8_BINARY", false); - assertStartsWith("a🙃b🙃c", "b🙃c", "UTF8_LCASE", false); - assertStartsWith("a🙃b🙃c", "b🙃c", "UNICODE", false); - assertStartsWith("a🙃b🙃c", "b🙃c", "UNICODE_CI", false); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", true); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", true); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE", true); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", true); - assertStartsWith("😀😆😃😄", "😄😆", "UTF8_BINARY", false); - assertStartsWith("😀😆😃😄", "😄😆", "UTF8_LCASE", false); - assertStartsWith("😀😆😃😄", "😄😆", "UNICODE", false); - assertStartsWith("😀😆😃😄", "😄😆", "UNICODE_CI", false); - assertStartsWith("😀😆😃😄", "😆😃", "UTF8_BINARY", false); - assertStartsWith("😀😆😃😄", "😆😃", "UTF8_LCASE", false); - assertStartsWith("😀😆😃😄", "😆😃", "UNICODE", false); - assertStartsWith("😀😆😃😄", "😆😃", "UNICODE_CI", false); - assertStartsWith("😀😆😃😄", "😀😆", "UTF8_BINARY", true); - assertStartsWith("😀😆😃😄", "😀😆", "UTF8_LCASE", true); - assertStartsWith("😀😆😃😄", "😀😆", "UNICODE", true); - assertStartsWith("😀😆😃😄", "😀😆", "UNICODE_CI", true); - assertStartsWith("😀😆😃😄", "😃😄", "UTF8_BINARY", false); - assertStartsWith("😀😆😃😄", "😃😄", "UTF8_LCASE", false); - assertStartsWith("😀😆😃😄", "😃😄", "UNICODE", false); - assertStartsWith("😀😆😃😄", "😃😄", "UNICODE_CI", false); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UTF8_BINARY", true); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UTF8_LCASE", true); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UNICODE", true); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UNICODE_CI", true); - assertStartsWith("𐐅", "𐐅", "UTF8_BINARY", true); - assertStartsWith("𐐅", "𐐅", "UTF8_LCASE", true); - assertStartsWith("𐐅", "𐐅", "UNICODE", true); - assertStartsWith("𐐅", "𐐅", "UNICODE_CI", true); - assertStartsWith("𐐅", "𐐭", "UTF8_BINARY", false); - assertStartsWith("𐐅", "𐐭", "UTF8_LCASE", true); - assertStartsWith("𐐅", "𐐭", "UNICODE", false); - assertStartsWith("𐐅", "𐐭", "UNICODE_CI", true); - assertStartsWith("𝔸", "𝔸", "UTF8_BINARY", true); - assertStartsWith("𝔸", "𝔸", "UTF8_LCASE", true); - assertStartsWith("𝔸", "𝔸", "UNICODE", true); - assertStartsWith("𝔸", "𝔸", "UNICODE_CI", true); + assertStartsWith("a🙃b🙃c", "x", UTF8_BINARY, false); + assertStartsWith("a🙃b🙃c", "x", UTF8_LCASE, false); + assertStartsWith("a🙃b🙃c", "x", UNICODE, false); + assertStartsWith("a🙃b🙃c", "x", UNICODE_CI, false); + assertStartsWith("a🙃b🙃c", "b", UTF8_BINARY, false); + assertStartsWith("a🙃b🙃c", "b", UTF8_LCASE, false); + assertStartsWith("a🙃b🙃c", "b", UNICODE, false); + assertStartsWith("a🙃b🙃c", "b", UNICODE_CI, false); + assertStartsWith("a🙃b🙃c", "a🙃b", UTF8_BINARY, true); + assertStartsWith("a🙃b🙃c", "a🙃b", UTF8_LCASE, true); + assertStartsWith("a🙃b🙃c", "a🙃b", UNICODE, true); + assertStartsWith("a🙃b🙃c", "a🙃b", UNICODE_CI, true); + assertStartsWith("a🙃b🙃c", "b🙃c", UTF8_BINARY, false); + assertStartsWith("a🙃b🙃c", "b🙃c", UTF8_LCASE, false); + assertStartsWith("a🙃b🙃c", "b🙃c", UNICODE, false); + assertStartsWith("a🙃b🙃c", "b🙃c", UNICODE_CI, false); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, true); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, true); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE, true); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, true); + assertStartsWith("😀😆😃😄", "😄😆", UTF8_BINARY, false); + assertStartsWith("😀😆😃😄", "😄😆", UTF8_LCASE, false); + assertStartsWith("😀😆😃😄", "😄😆", UNICODE, false); + assertStartsWith("😀😆😃😄", "😄😆", UNICODE_CI, false); + assertStartsWith("😀😆😃😄", "😆😃", UTF8_BINARY, false); + assertStartsWith("😀😆😃😄", "😆😃", UTF8_LCASE, false); + assertStartsWith("😀😆😃😄", "😆😃", UNICODE, false); + assertStartsWith("😀😆😃😄", "😆😃", UNICODE_CI, false); + assertStartsWith("😀😆😃😄", "😀😆", UTF8_BINARY, true); + assertStartsWith("😀😆😃😄", "😀😆", UTF8_LCASE, true); + assertStartsWith("😀😆😃😄", "😀😆", UNICODE, true); + assertStartsWith("😀😆😃😄", "😀😆", UNICODE_CI, true); + assertStartsWith("😀😆😃😄", "😃😄", UTF8_BINARY, false); + assertStartsWith("😀😆😃😄", "😃😄", UTF8_LCASE, false); + assertStartsWith("😀😆😃😄", "😃😄", UNICODE, false); + assertStartsWith("😀😆😃😄", "😃😄", UNICODE_CI, false); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UTF8_BINARY, true); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UTF8_LCASE, true); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UNICODE, true); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UNICODE_CI, true); + assertStartsWith("𐐅", "𐐅", UTF8_BINARY, true); + assertStartsWith("𐐅", "𐐅", UTF8_LCASE, true); + assertStartsWith("𐐅", "𐐅", UNICODE, true); + assertStartsWith("𐐅", "𐐅", UNICODE_CI, true); + assertStartsWith("𐐅", "𐐭", UTF8_BINARY, false); + assertStartsWith("𐐅", "𐐭", UTF8_LCASE, true); + assertStartsWith("𐐅", "𐐭", UNICODE, false); + assertStartsWith("𐐅", "𐐭", UNICODE_CI, true); + assertStartsWith("𝔸", "𝔸", UTF8_BINARY, true); + assertStartsWith("𝔸", "𝔸", UTF8_LCASE, true); + assertStartsWith("𝔸", "𝔸", UNICODE, true); + assertStartsWith("𝔸", "𝔸", UNICODE_CI, true); } /** @@ -806,212 +807,212 @@ public void testEndsWith() throws SparkException { assertEndsWith("Здраво", "Здраво", collationName, true); } // Advanced tests. - assertEndsWith("abcde", "cde", "UTF8_BINARY", true); - assertEndsWith("abcde", "bde", "UTF8_BINARY", false); - assertEndsWith("abcde", "fgh", "UTF8_BINARY", false); - assertEndsWith("abcde", "abcde", "UNICODE", true); - assertEndsWith("abcde", "aBcDe", "UNICODE", false); - assertEndsWith("abcde", "fghij", "UNICODE", false); - assertEndsWith("abcde", "E", "UTF8_LCASE", true); - assertEndsWith("abcde", "AbCdE", "UTF8_LCASE", true); - assertEndsWith("abcde", "X", "UTF8_LCASE", false); - assertEndsWith("abcde", "e", "UNICODE_CI", true); - assertEndsWith("abcde", "CDe", "UNICODE_CI", true); - assertEndsWith("abcde", "bcd", "UNICODE_CI", false); - assertEndsWith("abcde", "123", "UNICODE_CI", false); - assertEndsWith("ab世De", "世De", "UTF8_BINARY", true); - assertEndsWith("ab世De", "世dE", "UTF8_BINARY", false); - assertEndsWith("äbćδe", "ćδe", "UTF8_BINARY", true); - assertEndsWith("äbćδe", "cΔé", "UTF8_BINARY", false); - assertEndsWith("ab世De", "ab世De", "UNICODE", true); - assertEndsWith("ab世De", "AB世dE", "UNICODE", false); - assertEndsWith("äbćδe", "äbćδe", "UNICODE", true); - assertEndsWith("äbćδe", "ÄBcΔÉ", "UNICODE", false); - assertEndsWith("ab世De", "世De", "UTF8_LCASE", true); - assertEndsWith("ab世De", "世dE", "UTF8_LCASE", true); - assertEndsWith("äbćδe", "ćδe", "UTF8_LCASE", true); - assertEndsWith("äbćδe", "cδE", "UTF8_LCASE", false); - assertEndsWith("ab世De", "ab世De", "UNICODE_CI", true); - assertEndsWith("ab世De", "AB世dE", "UNICODE_CI", true); - assertEndsWith("äbćδe", "ÄbćδE", "UNICODE_CI", true); - assertEndsWith("äbćδe", "ÄBcΔÉ", "UNICODE_CI", false); - assertEndsWith("The Kelvin", "Kelvin", "UTF8_LCASE", true); - assertEndsWith("The Kelvin", "Kelvin", "UTF8_LCASE", true); - assertEndsWith("The KKelvin", "KKelvin", "UTF8_LCASE", true); - assertEndsWith("The 2 Kelvin", "2 Kelvin", "UTF8_LCASE", true); - assertEndsWith("The 2 Kelvin", "2 Kelvin", "UTF8_LCASE", true); - assertEndsWith("The KKelvin", "KKelvin,", "UTF8_LCASE", false); + assertEndsWith("abcde", "cde", UTF8_BINARY, true); + assertEndsWith("abcde", "bde", UTF8_BINARY, false); + assertEndsWith("abcde", "fgh", UTF8_BINARY, false); + assertEndsWith("abcde", "abcde", UNICODE, true); + assertEndsWith("abcde", "aBcDe", UNICODE, false); + assertEndsWith("abcde", "fghij", UNICODE, false); + assertEndsWith("abcde", "E", UTF8_LCASE, true); + assertEndsWith("abcde", "AbCdE", UTF8_LCASE, true); + assertEndsWith("abcde", "X", UTF8_LCASE, false); + assertEndsWith("abcde", "e", UNICODE_CI, true); + assertEndsWith("abcde", "CDe", UNICODE_CI, true); + assertEndsWith("abcde", "bcd", UNICODE_CI, false); + assertEndsWith("abcde", "123", UNICODE_CI, false); + assertEndsWith("ab世De", "世De", UTF8_BINARY, true); + assertEndsWith("ab世De", "世dE", UTF8_BINARY, false); + assertEndsWith("äbćδe", "ćδe", UTF8_BINARY, true); + assertEndsWith("äbćδe", "cΔé", UTF8_BINARY, false); + assertEndsWith("ab世De", "ab世De", UNICODE, true); + assertEndsWith("ab世De", "AB世dE", UNICODE, false); + assertEndsWith("äbćδe", "äbćδe", UNICODE, true); + assertEndsWith("äbćδe", "ÄBcΔÉ", UNICODE, false); + assertEndsWith("ab世De", "世De", UTF8_LCASE, true); + assertEndsWith("ab世De", "世dE", UTF8_LCASE, true); + assertEndsWith("äbćδe", "ćδe", UTF8_LCASE, true); + assertEndsWith("äbćδe", "cδE", UTF8_LCASE, false); + assertEndsWith("ab世De", "ab世De", UNICODE_CI, true); + assertEndsWith("ab世De", "AB世dE", UNICODE_CI, true); + assertEndsWith("äbćδe", "ÄbćδE", UNICODE_CI, true); + assertEndsWith("äbćδe", "ÄBcΔÉ", UNICODE_CI, false); + assertEndsWith("The Kelvin", "Kelvin", UTF8_LCASE, true); + assertEndsWith("The Kelvin", "Kelvin", UTF8_LCASE, true); + assertEndsWith("The KKelvin", "KKelvin", UTF8_LCASE, true); + assertEndsWith("The 2 Kelvin", "2 Kelvin", UTF8_LCASE, true); + assertEndsWith("The 2 Kelvin", "2 Kelvin", UTF8_LCASE, true); + assertEndsWith("The KKelvin", "KKelvin,", UTF8_LCASE, false); assertEndsWith("Ћевапчићи", "цици", "sr_Cyrl_CI_AI", false); assertEndsWith("Ћевапчићи", "чИЋи", "sr_Cyrl_CI_AI", true); assertEndsWith("Ćevapčići", "cici", "SR_CI", false); assertEndsWith("Ćevapčići", "cici", "SR_CI_AI", true); assertEndsWith("Ćevapčići", "čići", "SR", true); // Case variation. - assertEndsWith("aBcDe", "cde", "UTF8_BINARY", false); - assertEndsWith("aBcDe", "cDe", "UTF8_BINARY", true); - assertEndsWith("aBcDe", "abcde", "UNICODE", false); - assertEndsWith("aBcDe", "aBcDe", "UNICODE", true); - assertEndsWith("aBcDe", "cde", "UTF8_LCASE", true); - assertEndsWith("aBcDe", "CDE", "UTF8_LCASE", true); - assertEndsWith("aBcDe", "abcde", "UNICODE_CI", true); - assertEndsWith("aBcDe", "AbCdE", "UNICODE_CI", true); + assertEndsWith("aBcDe", "cde", UTF8_BINARY, false); + assertEndsWith("aBcDe", "cDe", UTF8_BINARY, true); + assertEndsWith("aBcDe", "abcde", UNICODE, false); + assertEndsWith("aBcDe", "aBcDe", UNICODE, true); + assertEndsWith("aBcDe", "cde", UTF8_LCASE, true); + assertEndsWith("aBcDe", "CDE", UTF8_LCASE, true); + assertEndsWith("aBcDe", "abcde", UNICODE_CI, true); + assertEndsWith("aBcDe", "AbCdE", UNICODE_CI, true); // Accent variation. - assertEndsWith("aBcDe", "ćde", "UTF8_BINARY", false); - assertEndsWith("aBcDe", "ćDe", "UTF8_BINARY", false); - assertEndsWith("aBcDe", "abćde", "UNICODE", false); - assertEndsWith("aBcDe", "aBćDe", "UNICODE", false); - assertEndsWith("aBcDe", "ćde", "UTF8_LCASE", false); - assertEndsWith("aBcDe", "ĆDE", "UTF8_LCASE", false); - assertEndsWith("aBcDe", "abćde", "UNICODE_CI", false); - assertEndsWith("aBcDe", "AbĆdE", "UNICODE_CI", false); + assertEndsWith("aBcDe", "ćde", UTF8_BINARY, false); + assertEndsWith("aBcDe", "ćDe", UTF8_BINARY, false); + assertEndsWith("aBcDe", "abćde", UNICODE, false); + assertEndsWith("aBcDe", "aBćDe", UNICODE, false); + assertEndsWith("aBcDe", "ćde", UTF8_LCASE, false); + assertEndsWith("aBcDe", "ĆDE", UTF8_LCASE, false); + assertEndsWith("aBcDe", "abćde", UNICODE_CI, false); + assertEndsWith("aBcDe", "AbĆdE", UNICODE_CI, false); // One-to-many case mapping (e.g. Turkish dotted I). - assertEndsWith("i\u0307", "\u0307", "UNICODE_CI", false); - assertEndsWith("i\u0307", "İ", "UNICODE_CI", true); - assertEndsWith("İ", "i", "UNICODE_CI", false); - assertEndsWith("İİİ", "i̇i̇", "UNICODE_CI", true); - assertEndsWith("İİİ", "ii̇", "UNICODE_CI", false); - assertEndsWith("İi̇İ", "İi̇", "UNICODE_CI", true); - assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", "UNICODE_CI", false); - assertEndsWith("the i\u0307o", "io", "UNICODE_CI", false); - assertEndsWith("the i\u0307o", "Io", "UNICODE_CI", false); - assertEndsWith("the i\u0307o", "i\u0307o", "UNICODE_CI", true); - assertEndsWith("the i\u0307o", "İo", "UNICODE_CI", true); - assertEndsWith("the İo", "io", "UNICODE_CI", false); - assertEndsWith("the İo", "Io", "UNICODE_CI", false); - assertEndsWith("the İo", "i\u0307o", "UNICODE_CI", true); - assertEndsWith("the İo", "İo", "UNICODE_CI", true); - assertEndsWith("i\u0307", "\u0307", "UTF8_LCASE", true); // != UNICODE_CI - assertEndsWith("i\u0307", "İ", "UTF8_LCASE", true); - assertEndsWith("İ", "\u0307", "UTF8_LCASE", false); - assertEndsWith("İİİ", "i̇i̇", "UTF8_LCASE", true); - assertEndsWith("İİİ", "ii̇", "UTF8_LCASE", false); - assertEndsWith("İi̇İ", "İi̇", "UTF8_LCASE", true); - assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", "UTF8_LCASE", true); // != UNICODE_CI - assertEndsWith("i̇İi̇i̇", "\u0307İİ", "UTF8_LCASE", false); - assertEndsWith("the i\u0307o", "io", "UTF8_LCASE", false); - assertEndsWith("the i\u0307o", "Io", "UTF8_LCASE", false); - assertEndsWith("the i\u0307o", "i\u0307o", "UTF8_LCASE", true); - assertEndsWith("the i\u0307o", "İo", "UTF8_LCASE", true); - assertEndsWith("the İo", "io", "UTF8_LCASE", false); - assertEndsWith("the İo", "Io", "UTF8_LCASE", false); - assertEndsWith("the İo", "i\u0307o", "UTF8_LCASE", true); - assertEndsWith("the İo", "İo", "UTF8_LCASE", true); - assertEndsWith("İo", "İo", "UTF8_LCASE", true); - assertEndsWith("İo", "i̇o", "UTF8_LCASE", true); + assertEndsWith("i\u0307", "\u0307", UNICODE_CI, false); + assertEndsWith("i\u0307", "İ", UNICODE_CI, true); + assertEndsWith("İ", "i", UNICODE_CI, false); + assertEndsWith("İİİ", "i̇i̇", UNICODE_CI, true); + assertEndsWith("İİİ", "ii̇", UNICODE_CI, false); + assertEndsWith("İi̇İ", "İi̇", UNICODE_CI, true); + assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", UNICODE_CI, false); + assertEndsWith("the i\u0307o", "io", UNICODE_CI, false); + assertEndsWith("the i\u0307o", "Io", UNICODE_CI, false); + assertEndsWith("the i\u0307o", "i\u0307o", UNICODE_CI, true); + assertEndsWith("the i\u0307o", "İo", UNICODE_CI, true); + assertEndsWith("the İo", "io", UNICODE_CI, false); + assertEndsWith("the İo", "Io", UNICODE_CI, false); + assertEndsWith("the İo", "i\u0307o", UNICODE_CI, true); + assertEndsWith("the İo", "İo", UNICODE_CI, true); + assertEndsWith("i\u0307", "\u0307", UTF8_LCASE, true); // != UNICODE_CI + assertEndsWith("i\u0307", "İ", UTF8_LCASE, true); + assertEndsWith("İ", "\u0307", UTF8_LCASE, false); + assertEndsWith("İİİ", "i̇i̇", UTF8_LCASE, true); + assertEndsWith("İİİ", "ii̇", UTF8_LCASE, false); + assertEndsWith("İi̇İ", "İi̇", UTF8_LCASE, true); + assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", UTF8_LCASE, true); // != UNICODE_CI + assertEndsWith("i̇İi̇i̇", "\u0307İİ", UTF8_LCASE, false); + assertEndsWith("the i\u0307o", "io", UTF8_LCASE, false); + assertEndsWith("the i\u0307o", "Io", UTF8_LCASE, false); + assertEndsWith("the i\u0307o", "i\u0307o", UTF8_LCASE, true); + assertEndsWith("the i\u0307o", "İo", UTF8_LCASE, true); + assertEndsWith("the İo", "io", UTF8_LCASE, false); + assertEndsWith("the İo", "Io", UTF8_LCASE, false); + assertEndsWith("the İo", "i\u0307o", UTF8_LCASE, true); + assertEndsWith("the İo", "İo", UTF8_LCASE, true); + assertEndsWith("İo", "İo", UTF8_LCASE, true); + assertEndsWith("İo", "i̇o", UTF8_LCASE, true); // Conditional case mapping (e.g. Greek sigmas). - assertEndsWith("σ", "σ", "UTF8_BINARY", true); - assertEndsWith("σ", "ς", "UTF8_BINARY", false); - assertEndsWith("σ", "Σ", "UTF8_BINARY", false); - assertEndsWith("ς", "σ", "UTF8_BINARY", false); - assertEndsWith("ς", "ς", "UTF8_BINARY", true); - assertEndsWith("ς", "Σ", "UTF8_BINARY", false); - assertEndsWith("Σ", "σ", "UTF8_BINARY", false); - assertEndsWith("Σ", "ς", "UTF8_BINARY", false); - assertEndsWith("Σ", "Σ", "UTF8_BINARY", true); - assertEndsWith("σ", "σ", "UTF8_LCASE", true); - assertEndsWith("σ", "ς", "UTF8_LCASE", true); - assertEndsWith("σ", "Σ", "UTF8_LCASE", true); - assertEndsWith("ς", "σ", "UTF8_LCASE", true); - assertEndsWith("ς", "ς", "UTF8_LCASE", true); - assertEndsWith("ς", "Σ", "UTF8_LCASE", true); - assertEndsWith("Σ", "σ", "UTF8_LCASE", true); - assertEndsWith("Σ", "ς", "UTF8_LCASE", true); - assertEndsWith("Σ", "Σ", "UTF8_LCASE", true); - assertEndsWith("σ", "σ", "UNICODE", true); - assertEndsWith("σ", "ς", "UNICODE", false); - assertEndsWith("σ", "Σ", "UNICODE", false); - assertEndsWith("ς", "σ", "UNICODE", false); - assertEndsWith("ς", "ς", "UNICODE", true); - assertEndsWith("ς", "Σ", "UNICODE", false); - assertEndsWith("Σ", "σ", "UNICODE", false); - assertEndsWith("Σ", "ς", "UNICODE", false); - assertEndsWith("Σ", "Σ", "UNICODE", true); - assertEndsWith("σ", "σ", "UNICODE_CI", true); - assertEndsWith("σ", "ς", "UNICODE_CI", true); - assertEndsWith("σ", "Σ", "UNICODE_CI", true); - assertEndsWith("ς", "σ", "UNICODE_CI", true); - assertEndsWith("ς", "ς", "UNICODE_CI", true); - assertEndsWith("ς", "Σ", "UNICODE_CI", true); - assertEndsWith("Σ", "σ", "UNICODE_CI", true); - assertEndsWith("Σ", "ς", "UNICODE_CI", true); - assertEndsWith("Σ", "Σ", "UNICODE_CI", true); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_BINARY", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UTF8_BINARY", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UTF8_BINARY", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_BINARY", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_BINARY", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_BINARY", false); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_LCASE", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UTF8_LCASE", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UTF8_LCASE", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_LCASE", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_LCASE", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_LCASE", true); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UNICODE", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UNICODE", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE", false); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE_CI", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UNICODE_CI", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UNICODE_CI", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE_CI", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE_CI", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE_CI", true); + assertEndsWith("σ", "σ", UTF8_BINARY, true); + assertEndsWith("σ", "ς", UTF8_BINARY, false); + assertEndsWith("σ", "Σ", UTF8_BINARY, false); + assertEndsWith("ς", "σ", UTF8_BINARY, false); + assertEndsWith("ς", "ς", UTF8_BINARY, true); + assertEndsWith("ς", "Σ", UTF8_BINARY, false); + assertEndsWith("Σ", "σ", UTF8_BINARY, false); + assertEndsWith("Σ", "ς", UTF8_BINARY, false); + assertEndsWith("Σ", "Σ", UTF8_BINARY, true); + assertEndsWith("σ", "σ", UTF8_LCASE, true); + assertEndsWith("σ", "ς", UTF8_LCASE, true); + assertEndsWith("σ", "Σ", UTF8_LCASE, true); + assertEndsWith("ς", "σ", UTF8_LCASE, true); + assertEndsWith("ς", "ς", UTF8_LCASE, true); + assertEndsWith("ς", "Σ", UTF8_LCASE, true); + assertEndsWith("Σ", "σ", UTF8_LCASE, true); + assertEndsWith("Σ", "ς", UTF8_LCASE, true); + assertEndsWith("Σ", "Σ", UTF8_LCASE, true); + assertEndsWith("σ", "σ", UNICODE, true); + assertEndsWith("σ", "ς", UNICODE, false); + assertEndsWith("σ", "Σ", UNICODE, false); + assertEndsWith("ς", "σ", UNICODE, false); + assertEndsWith("ς", "ς", UNICODE, true); + assertEndsWith("ς", "Σ", UNICODE, false); + assertEndsWith("Σ", "σ", UNICODE, false); + assertEndsWith("Σ", "ς", UNICODE, false); + assertEndsWith("Σ", "Σ", UNICODE, true); + assertEndsWith("σ", "σ", UNICODE_CI, true); + assertEndsWith("σ", "ς", UNICODE_CI, true); + assertEndsWith("σ", "Σ", UNICODE_CI, true); + assertEndsWith("ς", "σ", UNICODE_CI, true); + assertEndsWith("ς", "ς", UNICODE_CI, true); + assertEndsWith("ς", "Σ", UNICODE_CI, true); + assertEndsWith("Σ", "σ", UNICODE_CI, true); + assertEndsWith("Σ", "ς", UNICODE_CI, true); + assertEndsWith("Σ", "Σ", UNICODE_CI, true); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UTF8_BINARY, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UTF8_BINARY, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UTF8_BINARY, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_BINARY, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_BINARY, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_BINARY, false); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UTF8_LCASE, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UTF8_LCASE, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UTF8_LCASE, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_LCASE, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_LCASE, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_LCASE, true); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UNICODE, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UNICODE, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UNICODE, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE, false); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UNICODE_CI, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UNICODE_CI, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UNICODE_CI, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE_CI, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE_CI, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE_CI, true); // Surrogate pairs. - assertEndsWith("a🙃b🙃c", "x", "UTF8_BINARY", false); - assertEndsWith("a🙃b🙃c", "x", "UTF8_LCASE", false); - assertEndsWith("a🙃b🙃c", "x", "UNICODE", false); - assertEndsWith("a🙃b🙃c", "x", "UNICODE_CI", false); - assertEndsWith("a🙃b🙃c", "b", "UTF8_BINARY", false); - assertEndsWith("a🙃b🙃c", "b", "UTF8_LCASE", false); - assertEndsWith("a🙃b🙃c", "b", "UNICODE", false); - assertEndsWith("a🙃b🙃c", "b", "UNICODE_CI", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UTF8_BINARY", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UTF8_LCASE", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UNICODE", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UNICODE_CI", false); - assertEndsWith("a🙃b🙃c", "b🙃c", "UTF8_BINARY", true); - assertEndsWith("a🙃b🙃c", "b🙃c", "UTF8_LCASE", true); - assertEndsWith("a🙃b🙃c", "b🙃c", "UNICODE", true); - assertEndsWith("a🙃b🙃c", "b🙃c", "UNICODE_CI", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", true); - assertEndsWith("😀😆😃😄", "😄😆", "UTF8_BINARY", false); - assertEndsWith("😀😆😃😄", "😄😆", "UTF8_LCASE", false); - assertEndsWith("😀😆😃😄", "😄😆", "UNICODE", false); - assertEndsWith("😀😆😃😄", "😄😆", "UNICODE_CI", false); - assertEndsWith("😀😆😃😄", "😆😃", "UTF8_BINARY", false); - assertEndsWith("😀😆😃😄", "😆😃", "UTF8_LCASE", false); - assertEndsWith("😀😆😃😄", "😆😃", "UNICODE", false); - assertEndsWith("😀😆😃😄", "😆😃", "UNICODE_CI", false); - assertEndsWith("😀😆😃😄", "😀😆", "UTF8_BINARY", false); - assertEndsWith("😀😆😃😄", "😀😆", "UTF8_LCASE", false); - assertEndsWith("😀😆😃😄", "😀😆", "UNICODE", false); - assertEndsWith("😀😆😃😄", "😀😆", "UNICODE_CI", false); - assertEndsWith("😀😆😃😄", "😃😄", "UTF8_BINARY", true); - assertEndsWith("😀😆😃😄", "😃😄", "UTF8_LCASE", true); - assertEndsWith("😀😆😃😄", "😃😄", "UNICODE", true); - assertEndsWith("😀😆😃😄", "😃😄", "UNICODE_CI", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UTF8_BINARY", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UTF8_LCASE", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UNICODE", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UNICODE_CI", true); - assertEndsWith("𐐅", "𐐅", "UTF8_BINARY", true); - assertEndsWith("𐐅", "𐐅", "UTF8_LCASE", true); - assertEndsWith("𐐅", "𐐅", "UNICODE", true); - assertEndsWith("𐐅", "𐐅", "UNICODE_CI", true); - assertEndsWith("𐐅", "𐐭", "UTF8_BINARY", false); - assertEndsWith("𐐅", "𐐭", "UTF8_LCASE", true); - assertEndsWith("𐐅", "𐐭", "UNICODE", false); - assertEndsWith("𐐅", "𐐭", "UNICODE_CI", true); - assertEndsWith("𝔸", "𝔸", "UTF8_BINARY", true); - assertEndsWith("𝔸", "𝔸", "UTF8_LCASE", true); - assertEndsWith("𝔸", "𝔸", "UNICODE", true); - assertEndsWith("𝔸", "𝔸", "UNICODE_CI", true); + assertEndsWith("a🙃b🙃c", "x", UTF8_BINARY, false); + assertEndsWith("a🙃b🙃c", "x", UTF8_LCASE, false); + assertEndsWith("a🙃b🙃c", "x", UNICODE, false); + assertEndsWith("a🙃b🙃c", "x", UNICODE_CI, false); + assertEndsWith("a🙃b🙃c", "b", UTF8_BINARY, false); + assertEndsWith("a🙃b🙃c", "b", UTF8_LCASE, false); + assertEndsWith("a🙃b🙃c", "b", UNICODE, false); + assertEndsWith("a🙃b🙃c", "b", UNICODE_CI, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UTF8_BINARY, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UTF8_LCASE, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UNICODE, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UNICODE_CI, false); + assertEndsWith("a🙃b🙃c", "b🙃c", UTF8_BINARY, true); + assertEndsWith("a🙃b🙃c", "b🙃c", UTF8_LCASE, true); + assertEndsWith("a🙃b🙃c", "b🙃c", UNICODE, true); + assertEndsWith("a🙃b🙃c", "b🙃c", UNICODE_CI, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, true); + assertEndsWith("😀😆😃😄", "😄😆", UTF8_BINARY, false); + assertEndsWith("😀😆😃😄", "😄😆", UTF8_LCASE, false); + assertEndsWith("😀😆😃😄", "😄😆", UNICODE, false); + assertEndsWith("😀😆😃😄", "😄😆", UNICODE_CI, false); + assertEndsWith("😀😆😃😄", "😆😃", UTF8_BINARY, false); + assertEndsWith("😀😆😃😄", "😆😃", UTF8_LCASE, false); + assertEndsWith("😀😆😃😄", "😆😃", UNICODE, false); + assertEndsWith("😀😆😃😄", "😆😃", UNICODE_CI, false); + assertEndsWith("😀😆😃😄", "😀😆", UTF8_BINARY, false); + assertEndsWith("😀😆😃😄", "😀😆", UTF8_LCASE, false); + assertEndsWith("😀😆😃😄", "😀😆", UNICODE, false); + assertEndsWith("😀😆😃😄", "😀😆", UNICODE_CI, false); + assertEndsWith("😀😆😃😄", "😃😄", UTF8_BINARY, true); + assertEndsWith("😀😆😃😄", "😃😄", UTF8_LCASE, true); + assertEndsWith("😀😆😃😄", "😃😄", UNICODE, true); + assertEndsWith("😀😆😃😄", "😃😄", UNICODE_CI, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UTF8_BINARY, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UTF8_LCASE, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UNICODE, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UNICODE_CI, true); + assertEndsWith("𐐅", "𐐅", UTF8_BINARY, true); + assertEndsWith("𐐅", "𐐅", UTF8_LCASE, true); + assertEndsWith("𐐅", "𐐅", UNICODE, true); + assertEndsWith("𐐅", "𐐅", UNICODE_CI, true); + assertEndsWith("𐐅", "𐐭", UTF8_BINARY, false); + assertEndsWith("𐐅", "𐐭", UTF8_LCASE, true); + assertEndsWith("𐐅", "𐐭", UNICODE, false); + assertEndsWith("𐐅", "𐐭", UNICODE_CI, true); + assertEndsWith("𝔸", "𝔸", UTF8_BINARY, true); + assertEndsWith("𝔸", "𝔸", UTF8_LCASE, true); + assertEndsWith("𝔸", "𝔸", UNICODE, true); + assertEndsWith("𝔸", "𝔸", UNICODE_CI, true); } /** @@ -1057,158 +1058,158 @@ public void testStringSplitSQL() throws SparkException { var array_AOB = new UTF8String[] { UTF8String.fromString("A𐐅B") }; var array_AoB = new UTF8String[] { UTF8String.fromString("A𐐭B") }; // Empty strings. - assertStringSplitSQL("", "", "UTF8_BINARY", empty_match); - assertStringSplitSQL("abc", "", "UTF8_BINARY", array_abc); - assertStringSplitSQL("", "abc", "UTF8_BINARY", empty_match); - assertStringSplitSQL("", "", "UNICODE", empty_match); - assertStringSplitSQL("abc", "", "UNICODE", array_abc); - assertStringSplitSQL("", "abc", "UNICODE", empty_match); - assertStringSplitSQL("", "", "UTF8_LCASE", empty_match); - assertStringSplitSQL("abc", "", "UTF8_LCASE", array_abc); - assertStringSplitSQL("", "abc", "UTF8_LCASE", empty_match); - assertStringSplitSQL("", "", "UNICODE_CI", empty_match); - assertStringSplitSQL("abc", "", "UNICODE_CI", array_abc); - assertStringSplitSQL("", "abc", "UNICODE_CI", empty_match); + assertStringSplitSQL("", "", UTF8_BINARY, empty_match); + assertStringSplitSQL("abc", "", UTF8_BINARY, array_abc); + assertStringSplitSQL("", "abc", UTF8_BINARY, empty_match); + assertStringSplitSQL("", "", UNICODE, empty_match); + assertStringSplitSQL("abc", "", UNICODE, array_abc); + assertStringSplitSQL("", "abc", UNICODE, empty_match); + assertStringSplitSQL("", "", UTF8_LCASE, empty_match); + assertStringSplitSQL("abc", "", UTF8_LCASE, array_abc); + assertStringSplitSQL("", "abc", UTF8_LCASE, empty_match); + assertStringSplitSQL("", "", UNICODE_CI, empty_match); + assertStringSplitSQL("abc", "", UNICODE_CI, array_abc); + assertStringSplitSQL("", "abc", UNICODE_CI, empty_match); // Basic tests. - assertStringSplitSQL("1a2", "a", "UTF8_BINARY", array_1_2); - assertStringSplitSQL("1a2", "A", "UTF8_BINARY", array_1a2); - assertStringSplitSQL("1a2", "b", "UTF8_BINARY", array_1a2); - assertStringSplitSQL("1a2", "1a2", "UNICODE", full_match); - assertStringSplitSQL("1a2", "1A2", "UNICODE", array_1a2); - assertStringSplitSQL("1a2", "3b4", "UNICODE", array_1a2); - assertStringSplitSQL("1a2", "A", "UTF8_LCASE", array_1_2); - assertStringSplitSQL("1a2", "1A2", "UTF8_LCASE", full_match); - assertStringSplitSQL("1a2", "X", "UTF8_LCASE", array_1a2); - assertStringSplitSQL("1a2", "a", "UNICODE_CI", array_1_2); - assertStringSplitSQL("1a2", "A", "UNICODE_CI", array_1_2); - assertStringSplitSQL("1a2", "1A2", "UNICODE_CI", full_match); - assertStringSplitSQL("1a2", "123", "UNICODE_CI", array_1a2); + assertStringSplitSQL("1a2", "a", UTF8_BINARY, array_1_2); + assertStringSplitSQL("1a2", "A", UTF8_BINARY, array_1a2); + assertStringSplitSQL("1a2", "b", UTF8_BINARY, array_1a2); + assertStringSplitSQL("1a2", "1a2", UNICODE, full_match); + assertStringSplitSQL("1a2", "1A2", UNICODE, array_1a2); + assertStringSplitSQL("1a2", "3b4", UNICODE, array_1a2); + assertStringSplitSQL("1a2", "A", UTF8_LCASE, array_1_2); + assertStringSplitSQL("1a2", "1A2", UTF8_LCASE, full_match); + assertStringSplitSQL("1a2", "X", UTF8_LCASE, array_1a2); + assertStringSplitSQL("1a2", "a", UNICODE_CI, array_1_2); + assertStringSplitSQL("1a2", "A", UNICODE_CI, array_1_2); + assertStringSplitSQL("1a2", "1A2", UNICODE_CI, full_match); + assertStringSplitSQL("1a2", "123", UNICODE_CI, array_1a2); // Advanced tests. - assertStringSplitSQL("äb世De", "b世D", "UTF8_BINARY", array_a_e); - assertStringSplitSQL("äb世De", "B世d", "UTF8_BINARY", array_special); - assertStringSplitSQL("äbćδe", "bćδ", "UTF8_BINARY", array_a_e); - assertStringSplitSQL("äbćδe", "BcΔ", "UTF8_BINARY", array_abcde); - assertStringSplitSQL("äb世De", "äb世De", "UNICODE", full_match); - assertStringSplitSQL("äb世De", "äB世de", "UNICODE", array_special); - assertStringSplitSQL("äbćδe", "äbćδe", "UNICODE", full_match); - assertStringSplitSQL("äbćδe", "ÄBcΔÉ", "UNICODE", array_abcde); - assertStringSplitSQL("äb世De", "b世D", "UTF8_LCASE", array_a_e); - assertStringSplitSQL("äb世De", "B世d", "UTF8_LCASE", array_a_e); - assertStringSplitSQL("äbćδe", "bćδ", "UTF8_LCASE", array_a_e); - assertStringSplitSQL("äbćδe", "BcΔ", "UTF8_LCASE", array_abcde); - assertStringSplitSQL("äb世De", "ab世De", "UNICODE_CI", array_special); - assertStringSplitSQL("äb世De", "AB世dE", "UNICODE_CI", array_special); - assertStringSplitSQL("äbćδe", "ÄbćδE", "UNICODE_CI", full_match); - assertStringSplitSQL("äbćδe", "ÄBcΔÉ", "UNICODE_CI", array_abcde); + assertStringSplitSQL("äb世De", "b世D", UTF8_BINARY, array_a_e); + assertStringSplitSQL("äb世De", "B世d", UTF8_BINARY, array_special); + assertStringSplitSQL("äbćδe", "bćδ", UTF8_BINARY, array_a_e); + assertStringSplitSQL("äbćδe", "BcΔ", UTF8_BINARY, array_abcde); + assertStringSplitSQL("äb世De", "äb世De", UNICODE, full_match); + assertStringSplitSQL("äb世De", "äB世de", UNICODE, array_special); + assertStringSplitSQL("äbćδe", "äbćδe", UNICODE, full_match); + assertStringSplitSQL("äbćδe", "ÄBcΔÉ", UNICODE, array_abcde); + assertStringSplitSQL("äb世De", "b世D", UTF8_LCASE, array_a_e); + assertStringSplitSQL("äb世De", "B世d", UTF8_LCASE, array_a_e); + assertStringSplitSQL("äbćδe", "bćδ", UTF8_LCASE, array_a_e); + assertStringSplitSQL("äbćδe", "BcΔ", UTF8_LCASE, array_abcde); + assertStringSplitSQL("äb世De", "ab世De", UNICODE_CI, array_special); + assertStringSplitSQL("äb世De", "AB世dE", UNICODE_CI, array_special); + assertStringSplitSQL("äbćδe", "ÄbćδE", UNICODE_CI, full_match); + assertStringSplitSQL("äbćδe", "ÄBcΔÉ", UNICODE_CI, array_abcde); // Case variation. - assertStringSplitSQL("AaXbB", "x", "UTF8_BINARY", array_AaXbB); - assertStringSplitSQL("AaXbB", "X", "UTF8_BINARY", array_Aa_bB); - assertStringSplitSQL("AaXbB", "axb", "UNICODE", array_AaXbB); - assertStringSplitSQL("AaXbB", "aXb", "UNICODE", array_A_B); - assertStringSplitSQL("AaXbB", "axb", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("AaXbB", "AXB", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("AaXbB", "axb", "UNICODE_CI", array_A_B); - assertStringSplitSQL("AaXbB", "AxB", "UNICODE_CI", array_A_B); + assertStringSplitSQL("AaXbB", "x", UTF8_BINARY, array_AaXbB); + assertStringSplitSQL("AaXbB", "X", UTF8_BINARY, array_Aa_bB); + assertStringSplitSQL("AaXbB", "axb", UNICODE, array_AaXbB); + assertStringSplitSQL("AaXbB", "aXb", UNICODE, array_A_B); + assertStringSplitSQL("AaXbB", "axb", UTF8_LCASE, array_A_B); + assertStringSplitSQL("AaXbB", "AXB", UTF8_LCASE, array_A_B); + assertStringSplitSQL("AaXbB", "axb", UNICODE_CI, array_A_B); + assertStringSplitSQL("AaXbB", "AxB", UNICODE_CI, array_A_B); // Accent variation. - assertStringSplitSQL("aBcDe", "bćd", "UTF8_BINARY", array_aBcDe); - assertStringSplitSQL("aBcDe", "BćD", "UTF8_BINARY", array_aBcDe); - assertStringSplitSQL("aBcDe", "abćde", "UNICODE", array_aBcDe); - assertStringSplitSQL("aBcDe", "aBćDe", "UNICODE", array_aBcDe); - assertStringSplitSQL("aBcDe", "bćd", "UTF8_LCASE", array_aBcDe); - assertStringSplitSQL("aBcDe", "BĆD", "UTF8_LCASE", array_aBcDe); - assertStringSplitSQL("aBcDe", "abćde", "UNICODE_CI", array_aBcDe); - assertStringSplitSQL("aBcDe", "AbĆdE", "UNICODE_CI", array_aBcDe); + assertStringSplitSQL("aBcDe", "bćd", UTF8_BINARY, array_aBcDe); + assertStringSplitSQL("aBcDe", "BćD", UTF8_BINARY, array_aBcDe); + assertStringSplitSQL("aBcDe", "abćde", UNICODE, array_aBcDe); + assertStringSplitSQL("aBcDe", "aBćDe", UNICODE, array_aBcDe); + assertStringSplitSQL("aBcDe", "bćd", UTF8_LCASE, array_aBcDe); + assertStringSplitSQL("aBcDe", "BĆD", UTF8_LCASE, array_aBcDe); + assertStringSplitSQL("aBcDe", "abćde", UNICODE_CI, array_aBcDe); + assertStringSplitSQL("aBcDe", "AbĆdE", UNICODE_CI, array_aBcDe); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringSplitSQL("İ", "i", "UTF8_BINARY", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "i", "UTF8_LCASE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "i", "UNICODE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "i", "UNICODE_CI", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UTF8_BINARY", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UTF8_LCASE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UNICODE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UNICODE_CI", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("i\u0307", "i", "UTF8_BINARY", array_dot); - assertStringSplitSQL("i\u0307", "i", "UTF8_LCASE", array_dot); - assertStringSplitSQL("i\u0307", "i", "UNICODE", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("i\u0307", "i", "UNICODE_CI", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("i\u0307", "\u0307", "UTF8_BINARY", array_i); - assertStringSplitSQL("i\u0307", "\u0307", "UTF8_LCASE", array_i); - assertStringSplitSQL("i\u0307", "\u0307", "UNICODE", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("i\u0307", "\u0307", "UNICODE_CI", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("AİB", "İ", "UTF8_BINARY", array_A_B); - assertStringSplitSQL("AİB", "İ", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("AİB", "İ", "UNICODE", array_A_B); - assertStringSplitSQL("AİB", "İ", "UNICODE_CI", array_A_B); - assertStringSplitSQL("AİB", "i\u0307", "UTF8_BINARY", array_AIB); - assertStringSplitSQL("AİB", "i\u0307", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("AİB", "i\u0307", "UNICODE", array_AIB); - assertStringSplitSQL("AİB", "i\u0307", "UNICODE_CI", array_A_B); - assertStringSplitSQL("Ai\u0307B", "İ", "UTF8_BINARY", array_AiB); - assertStringSplitSQL("Ai\u0307B", "İ", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("Ai\u0307B", "İ", "UNICODE", array_AiB); - assertStringSplitSQL("Ai\u0307B", "İ", "UNICODE_CI", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UTF8_BINARY", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UNICODE", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UNICODE_CI", array_A_B); + assertStringSplitSQL("İ", "i", UTF8_BINARY, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "i", UTF8_LCASE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "i", UNICODE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "i", UNICODE_CI, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UTF8_BINARY, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UTF8_LCASE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UNICODE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UNICODE_CI, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("i\u0307", "i", UTF8_BINARY, array_dot); + assertStringSplitSQL("i\u0307", "i", UTF8_LCASE, array_dot); + assertStringSplitSQL("i\u0307", "i", UNICODE, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("i\u0307", "i", UNICODE_CI, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("i\u0307", "\u0307", UTF8_BINARY, array_i); + assertStringSplitSQL("i\u0307", "\u0307", UTF8_LCASE, array_i); + assertStringSplitSQL("i\u0307", "\u0307", UNICODE, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("i\u0307", "\u0307", UNICODE_CI, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("AİB", "İ", UTF8_BINARY, array_A_B); + assertStringSplitSQL("AİB", "İ", UTF8_LCASE, array_A_B); + assertStringSplitSQL("AİB", "İ", UNICODE, array_A_B); + assertStringSplitSQL("AİB", "İ", UNICODE_CI, array_A_B); + assertStringSplitSQL("AİB", "i\u0307", UTF8_BINARY, array_AIB); + assertStringSplitSQL("AİB", "i\u0307", UTF8_LCASE, array_A_B); + assertStringSplitSQL("AİB", "i\u0307", UNICODE, array_AIB); + assertStringSplitSQL("AİB", "i\u0307", UNICODE_CI, array_A_B); + assertStringSplitSQL("Ai\u0307B", "İ", UTF8_BINARY, array_AiB); + assertStringSplitSQL("Ai\u0307B", "İ", UTF8_LCASE, array_A_B); + assertStringSplitSQL("Ai\u0307B", "İ", UNICODE, array_AiB); + assertStringSplitSQL("Ai\u0307B", "İ", UNICODE_CI, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UTF8_BINARY, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UTF8_LCASE, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UNICODE, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UNICODE_CI, array_A_B); // Conditional case mapping (e.g. Greek sigmas). - assertStringSplitSQL("σ", "σ", "UTF8_BINARY", full_match); - assertStringSplitSQL("σ", "σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("σ", "σ", "UNICODE", full_match); - assertStringSplitSQL("σ", "σ", "UNICODE_CI", full_match); - assertStringSplitSQL("σ", "ς", "UTF8_BINARY", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "ς", "UTF8_LCASE", full_match); - assertStringSplitSQL("σ", "ς", "UNICODE", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "ς", "UNICODE_CI", full_match); - assertStringSplitSQL("σ", "Σ", "UTF8_BINARY", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "Σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("σ", "Σ", "UNICODE", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "Σ", "UNICODE_CI", full_match); - assertStringSplitSQL("ς", "σ", "UTF8_BINARY", array_small_final_sigma); - assertStringSplitSQL("ς", "σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("ς", "σ", "UNICODE", array_small_final_sigma); - assertStringSplitSQL("ς", "σ", "UNICODE_CI", full_match); - assertStringSplitSQL("ς", "ς", "UTF8_BINARY", full_match); - assertStringSplitSQL("ς", "ς", "UTF8_LCASE", full_match); - assertStringSplitSQL("ς", "ς", "UNICODE", full_match); - assertStringSplitSQL("ς", "ς", "UNICODE_CI", full_match); - assertStringSplitSQL("ς", "Σ", "UTF8_BINARY", array_small_final_sigma); - assertStringSplitSQL("ς", "Σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("ς", "Σ", "UNICODE", array_small_final_sigma); - assertStringSplitSQL("ς", "Σ", "UNICODE_CI", full_match); - assertStringSplitSQL("Σ", "σ", "UTF8_BINARY", array_capital_sigma); - assertStringSplitSQL("Σ", "σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("Σ", "σ", "UNICODE", array_capital_sigma); - assertStringSplitSQL("Σ", "σ", "UNICODE_CI", full_match); - assertStringSplitSQL("Σ", "ς", "UTF8_BINARY", array_capital_sigma); - assertStringSplitSQL("Σ", "ς", "UTF8_LCASE", full_match); - assertStringSplitSQL("Σ", "ς", "UNICODE", array_capital_sigma); - assertStringSplitSQL("Σ", "ς", "UNICODE_CI", full_match); - assertStringSplitSQL("Σ", "Σ", "UTF8_BINARY", full_match); - assertStringSplitSQL("Σ", "Σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("Σ", "Σ", "UNICODE", full_match); - assertStringSplitSQL("Σ", "Σ", "UNICODE_CI", full_match); + assertStringSplitSQL("σ", "σ", UTF8_BINARY, full_match); + assertStringSplitSQL("σ", "σ", UTF8_LCASE, full_match); + assertStringSplitSQL("σ", "σ", UNICODE, full_match); + assertStringSplitSQL("σ", "σ", UNICODE_CI, full_match); + assertStringSplitSQL("σ", "ς", UTF8_BINARY, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "ς", UTF8_LCASE, full_match); + assertStringSplitSQL("σ", "ς", UNICODE, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "ς", UNICODE_CI, full_match); + assertStringSplitSQL("σ", "Σ", UTF8_BINARY, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "Σ", UTF8_LCASE, full_match); + assertStringSplitSQL("σ", "Σ", UNICODE, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "Σ", UNICODE_CI, full_match); + assertStringSplitSQL("ς", "σ", UTF8_BINARY, array_small_final_sigma); + assertStringSplitSQL("ς", "σ", UTF8_LCASE, full_match); + assertStringSplitSQL("ς", "σ", UNICODE, array_small_final_sigma); + assertStringSplitSQL("ς", "σ", UNICODE_CI, full_match); + assertStringSplitSQL("ς", "ς", UTF8_BINARY, full_match); + assertStringSplitSQL("ς", "ς", UTF8_LCASE, full_match); + assertStringSplitSQL("ς", "ς", UNICODE, full_match); + assertStringSplitSQL("ς", "ς", UNICODE_CI, full_match); + assertStringSplitSQL("ς", "Σ", UTF8_BINARY, array_small_final_sigma); + assertStringSplitSQL("ς", "Σ", UTF8_LCASE, full_match); + assertStringSplitSQL("ς", "Σ", UNICODE, array_small_final_sigma); + assertStringSplitSQL("ς", "Σ", UNICODE_CI, full_match); + assertStringSplitSQL("Σ", "σ", UTF8_BINARY, array_capital_sigma); + assertStringSplitSQL("Σ", "σ", UTF8_LCASE, full_match); + assertStringSplitSQL("Σ", "σ", UNICODE, array_capital_sigma); + assertStringSplitSQL("Σ", "σ", UNICODE_CI, full_match); + assertStringSplitSQL("Σ", "ς", UTF8_BINARY, array_capital_sigma); + assertStringSplitSQL("Σ", "ς", UTF8_LCASE, full_match); + assertStringSplitSQL("Σ", "ς", UNICODE, array_capital_sigma); + assertStringSplitSQL("Σ", "ς", UNICODE_CI, full_match); + assertStringSplitSQL("Σ", "Σ", UTF8_BINARY, full_match); + assertStringSplitSQL("Σ", "Σ", UTF8_LCASE, full_match); + assertStringSplitSQL("Σ", "Σ", UNICODE, full_match); + assertStringSplitSQL("Σ", "Σ", UNICODE_CI, full_match); // Surrogate pairs. - assertStringSplitSQL("a🙃b🙃c", "🙃", "UTF8_BINARY", array_a_b_c); - assertStringSplitSQL("a🙃b🙃c", "🙃", "UTF8_LCASE", array_a_b_c); - assertStringSplitSQL("a🙃b🙃c", "🙃", "UNICODE", array_a_b_c); - assertStringSplitSQL("a🙃b🙃c", "🙃", "UNICODE_CI", array_a_b_c); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UTF8_BINARY", array_emojis); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UTF8_LCASE", array_emojis); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UNICODE", array_emojis); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UNICODE_CI", array_emojis); - assertStringSplitSQL("A𐐅B", "𐐅", "UTF8_BINARY", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐅", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐅", "UNICODE", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐅", "UNICODE_CI", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐭", "UTF8_BINARY", array_AOB); - assertStringSplitSQL("A𐐅B", "𐐭", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐭", "UNICODE", array_AOB); - assertStringSplitSQL("A𐐅B", "𐐭", "UNICODE_CI", array_A_B); - assertStringSplitSQL("A𐐭B", "𐐅", "UTF8_BINARY", array_AoB); - assertStringSplitSQL("A𐐭B", "𐐅", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("A𐐭B", "𐐅", "UNICODE", array_AoB); - assertStringSplitSQL("A𐐭B", "𐐅", "UNICODE_CI", array_A_B); + assertStringSplitSQL("a🙃b🙃c", "🙃", UTF8_BINARY, array_a_b_c); + assertStringSplitSQL("a🙃b🙃c", "🙃", UTF8_LCASE, array_a_b_c); + assertStringSplitSQL("a🙃b🙃c", "🙃", UNICODE, array_a_b_c); + assertStringSplitSQL("a🙃b🙃c", "🙃", UNICODE_CI, array_a_b_c); + assertStringSplitSQL("😀😆😃😄", "😆😃", UTF8_BINARY, array_emojis); + assertStringSplitSQL("😀😆😃😄", "😆😃", UTF8_LCASE, array_emojis); + assertStringSplitSQL("😀😆😃😄", "😆😃", UNICODE, array_emojis); + assertStringSplitSQL("😀😆😃😄", "😆😃", UNICODE_CI, array_emojis); + assertStringSplitSQL("A𐐅B", "𐐅", UTF8_BINARY, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐅", UTF8_LCASE, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐅", UNICODE, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐅", UNICODE_CI, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐭", UTF8_BINARY, array_AOB); + assertStringSplitSQL("A𐐅B", "𐐭", UTF8_LCASE, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐭", UNICODE, array_AOB); + assertStringSplitSQL("A𐐅B", "𐐭", UNICODE_CI, array_A_B); + assertStringSplitSQL("A𐐭B", "𐐅", UTF8_BINARY, array_AoB); + assertStringSplitSQL("A𐐭B", "𐐅", UTF8_LCASE, array_A_B); + assertStringSplitSQL("A𐐭B", "𐐅", UNICODE, array_AoB); + assertStringSplitSQL("A𐐭B", "𐐅", UNICODE_CI, array_A_B); } /** @@ -1391,156 +1392,156 @@ public void testInitCap() throws SparkException { assertInitCap("θαλασσινος", collationName, "Θαλασσινος"); } // Advanced tests. - assertInitCap("aBćDe", "UTF8_BINARY", "Abćde"); - assertInitCap("aBćDe", "UTF8_LCASE", "Abćde"); - assertInitCap("aBćDe", "UNICODE", "Abćde"); - assertInitCap("aBćDe", "UNICODE_CI", "Abćde"); - assertInitCap("ab世De", "UTF8_BINARY", "Ab世de"); - assertInitCap("ab世De", "UTF8_LCASE", "Ab世De"); - assertInitCap("ab世De", "UNICODE", "Ab世De"); - assertInitCap("ab世De", "UNICODE_CI", "Ab世De"); - assertInitCap("äbćδe", "UTF8_BINARY", "Äbćδe"); - assertInitCap("äbćδe", "UTF8_LCASE", "Äbćδe"); - assertInitCap("äbćδe", "UNICODE", "Äbćδe"); - assertInitCap("äbćδe", "UNICODE_CI", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UTF8_BINARY", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UTF8_LCASE", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UNICODE", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UNICODE_CI", "Äbćδe"); + assertInitCap("aBćDe", UTF8_BINARY, "Abćde"); + assertInitCap("aBćDe", UTF8_LCASE, "Abćde"); + assertInitCap("aBćDe", UNICODE, "Abćde"); + assertInitCap("aBćDe", UNICODE_CI, "Abćde"); + assertInitCap("ab世De", UTF8_BINARY, "Ab世de"); + assertInitCap("ab世De", UTF8_LCASE, "Ab世De"); + assertInitCap("ab世De", UNICODE, "Ab世De"); + assertInitCap("ab世De", UNICODE_CI, "Ab世De"); + assertInitCap("äbćδe", UTF8_BINARY, "Äbćδe"); + assertInitCap("äbćδe", UTF8_LCASE, "Äbćδe"); + assertInitCap("äbćδe", UNICODE, "Äbćδe"); + assertInitCap("äbćδe", UNICODE_CI, "Äbćδe"); + assertInitCap("ÄBĆΔE", UTF8_BINARY, "Äbćδe"); + assertInitCap("ÄBĆΔE", UTF8_LCASE, "Äbćδe"); + assertInitCap("ÄBĆΔE", UNICODE, "Äbćδe"); + assertInitCap("ÄBĆΔE", UNICODE_CI, "Äbćδe"); assertInitCap("êéfgh", "AF_CI_AI", "Êéfgh"); assertInitCap("öoAÄ", "DE_CI_AI", "Öoaä"); // Case-variable character length - assertInitCap("İo", "UTF8_BINARY", "İo", "I\u0307o"); - assertInitCap("İo", "UTF8_LCASE", "İo"); - assertInitCap("İo", "UNICODE", "İo"); - assertInitCap("İo", "UNICODE_CI", "İo"); - assertInitCap("i\u0307o", "UTF8_BINARY", "I\u0307o"); - assertInitCap("i\u0307o", "UTF8_LCASE", "I\u0307o"); - assertInitCap("i\u0307o", "UNICODE", "I\u0307o"); - assertInitCap("i\u0307o", "UNICODE_CI", "I\u0307o"); + assertInitCap("İo", UTF8_BINARY, "İo", "I\u0307o"); + assertInitCap("İo", UTF8_LCASE, "İo"); + assertInitCap("İo", UNICODE, "İo"); + assertInitCap("İo", UNICODE_CI, "İo"); + assertInitCap("i\u0307o", UTF8_BINARY, "I\u0307o"); + assertInitCap("i\u0307o", UTF8_LCASE, "I\u0307o"); + assertInitCap("i\u0307o", UNICODE, "I\u0307o"); + assertInitCap("i\u0307o", UNICODE_CI, "I\u0307o"); // Different possible word boundaries - assertInitCap("aB 世 de", "UTF8_BINARY", "Ab 世 De"); - assertInitCap("aB 世 de", "UTF8_LCASE", "Ab 世 De"); - assertInitCap("aB 世 de", "UNICODE", "Ab 世 De"); - assertInitCap("aB 世 de", "UNICODE_CI", "Ab 世 De"); + assertInitCap("aB 世 de", UTF8_BINARY, "Ab 世 De"); + assertInitCap("aB 世 de", UTF8_LCASE, "Ab 世 De"); + assertInitCap("aB 世 de", UNICODE, "Ab 世 De"); + assertInitCap("aB 世 de", UNICODE_CI, "Ab 世 De"); // One-to-many case mapping (e.g. Turkish dotted I). - assertInitCap("İ", "UTF8_BINARY", "İ", "I\u0307"); - assertInitCap("İ", "UTF8_LCASE", "İ"); - assertInitCap("İ", "UNICODE", "İ"); - assertInitCap("İ", "UNICODE_CI", "İ"); - assertInitCap("I\u0307", "UTF8_BINARY","I\u0307"); - assertInitCap("I\u0307", "UTF8_LCASE","I\u0307"); - assertInitCap("I\u0307", "UNICODE","I\u0307"); - assertInitCap("I\u0307", "UNICODE_CI","I\u0307"); - assertInitCap("İonic", "UTF8_BINARY", "İonic", "I\u0307onic"); - assertInitCap("İonic", "UTF8_LCASE", "İonic"); - assertInitCap("İonic", "UNICODE", "İonic"); - assertInitCap("İonic", "UNICODE_CI", "İonic"); - assertInitCap("i\u0307onic", "UTF8_BINARY","I\u0307onic"); - assertInitCap("i\u0307onic", "UTF8_LCASE","I\u0307onic"); - assertInitCap("i\u0307onic", "UNICODE","I\u0307onic"); - assertInitCap("i\u0307onic", "UNICODE_CI","I\u0307onic"); - assertInitCap("FIDELİO", "UTF8_BINARY", "Fideli\u0307o"); - assertInitCap("FIDELİO", "UTF8_LCASE", "Fideli\u0307o"); - assertInitCap("FIDELİO", "UNICODE", "Fideli\u0307o"); - assertInitCap("FIDELİO", "UNICODE_CI", "Fideli\u0307o"); + assertInitCap("İ", UTF8_BINARY, "İ", "I\u0307"); + assertInitCap("İ", UTF8_LCASE, "İ"); + assertInitCap("İ", UNICODE, "İ"); + assertInitCap("İ", UNICODE_CI, "İ"); + assertInitCap("I\u0307", UTF8_BINARY,"I\u0307"); + assertInitCap("I\u0307", UTF8_LCASE,"I\u0307"); + assertInitCap("I\u0307", UNICODE,"I\u0307"); + assertInitCap("I\u0307", UNICODE_CI,"I\u0307"); + assertInitCap("İonic", UTF8_BINARY, "İonic", "I\u0307onic"); + assertInitCap("İonic", UTF8_LCASE, "İonic"); + assertInitCap("İonic", UNICODE, "İonic"); + assertInitCap("İonic", UNICODE_CI, "İonic"); + assertInitCap("i\u0307onic", UTF8_BINARY,"I\u0307onic"); + assertInitCap("i\u0307onic", UTF8_LCASE,"I\u0307onic"); + assertInitCap("i\u0307onic", UNICODE,"I\u0307onic"); + assertInitCap("i\u0307onic", UNICODE_CI,"I\u0307onic"); + assertInitCap("FIDELİO", UTF8_BINARY, "Fideli\u0307o"); + assertInitCap("FIDELİO", UTF8_LCASE, "Fideli\u0307o"); + assertInitCap("FIDELİO", UNICODE, "Fideli\u0307o"); + assertInitCap("FIDELİO", UNICODE_CI, "Fideli\u0307o"); // Surrogate pairs. - assertInitCap("a🙃B🙃c", "UTF8_BINARY", "A🙃b🙃c"); - assertInitCap("a🙃B🙃c", "UTF8_LCASE", "A🙃B🙃C"); - assertInitCap("a🙃B🙃c", "UNICODE", "A🙃B🙃C"); - assertInitCap("a🙃B🙃c", "UNICODE_CI", "A🙃B🙃C"); - assertInitCap("😄 😆", "UTF8_BINARY", "😄 😆"); - assertInitCap("😄 😆", "UTF8_LCASE", "😄 😆"); - assertInitCap("😄 😆", "UNICODE", "😄 😆"); - assertInitCap("😄 😆", "UNICODE_CI", "😄 😆"); - assertInitCap("😀😆😃😄", "UTF8_BINARY", "😀😆😃😄"); - assertInitCap("😀😆😃😄", "UTF8_LCASE", "😀😆😃😄"); - assertInitCap("😀😆😃😄", "UNICODE", "😀😆😃😄"); - assertInitCap("😀😆😃😄", "UNICODE_CI", "😀😆😃😄"); - assertInitCap("𝔸", "UTF8_BINARY", "𝔸"); - assertInitCap("𝔸", "UTF8_LCASE", "𝔸"); - assertInitCap("𝔸", "UNICODE", "𝔸"); - assertInitCap("𝔸", "UNICODE_CI", "𝔸"); - assertInitCap("𐐅", "UTF8_BINARY", "\uD801\uDC05", "𐐭"); - assertInitCap("𐐅", "UTF8_LCASE", "𐐅"); - assertInitCap("𐐅", "UNICODE", "𐐅"); - assertInitCap("𐐅", "UNICODE_CI", "𐐅"); - assertInitCap("𐐭", "UTF8_BINARY", "\uD801\uDC05", "𐐭"); - assertInitCap("𐐭", "UTF8_LCASE", "𐐅"); - assertInitCap("𐐭", "UNICODE", "𐐅"); - assertInitCap("𐐭", "UNICODE_CI", "𐐅"); - assertInitCap("𐐭𝔸", "UTF8_BINARY", "\uD801\uDC05\uD835\uDD38", "𐐭𝔸"); - assertInitCap("𐐭𝔸", "UTF8_LCASE", "𐐅𝔸"); - assertInitCap("𐐭𝔸", "UNICODE", "𐐅𝔸"); - assertInitCap("𐐭𝔸", "UNICODE_CI", "𐐅𝔸"); + assertInitCap("a🙃B🙃c", UTF8_BINARY, "A🙃b🙃c"); + assertInitCap("a🙃B🙃c", UTF8_LCASE, "A🙃B🙃C"); + assertInitCap("a🙃B🙃c", UNICODE, "A🙃B🙃C"); + assertInitCap("a🙃B🙃c", UNICODE_CI, "A🙃B🙃C"); + assertInitCap("😄 😆", UTF8_BINARY, "😄 😆"); + assertInitCap("😄 😆", UTF8_LCASE, "😄 😆"); + assertInitCap("😄 😆", UNICODE, "😄 😆"); + assertInitCap("😄 😆", UNICODE_CI, "😄 😆"); + assertInitCap("😀😆😃😄", UTF8_BINARY, "😀😆😃😄"); + assertInitCap("😀😆😃😄", UTF8_LCASE, "😀😆😃😄"); + assertInitCap("😀😆😃😄", UNICODE, "😀😆😃😄"); + assertInitCap("😀😆😃😄", UNICODE_CI, "😀😆😃😄"); + assertInitCap("𝔸", UTF8_BINARY, "𝔸"); + assertInitCap("𝔸", UTF8_LCASE, "𝔸"); + assertInitCap("𝔸", UNICODE, "𝔸"); + assertInitCap("𝔸", UNICODE_CI, "𝔸"); + assertInitCap("𐐅", UTF8_BINARY, "\uD801\uDC05", "𐐭"); + assertInitCap("𐐅", UTF8_LCASE, "𐐅"); + assertInitCap("𐐅", UNICODE, "𐐅"); + assertInitCap("𐐅", UNICODE_CI, "𐐅"); + assertInitCap("𐐭", UTF8_BINARY, "\uD801\uDC05", "𐐭"); + assertInitCap("𐐭", UTF8_LCASE, "𐐅"); + assertInitCap("𐐭", UNICODE, "𐐅"); + assertInitCap("𐐭", UNICODE_CI, "𐐅"); + assertInitCap("𐐭𝔸", UTF8_BINARY, "\uD801\uDC05\uD835\uDD38", "𐐭𝔸"); + assertInitCap("𐐭𝔸", UTF8_LCASE, "𐐅𝔸"); + assertInitCap("𐐭𝔸", UNICODE, "𐐅𝔸"); + assertInitCap("𐐭𝔸", UNICODE_CI, "𐐅𝔸"); // Ligatures. - assertInitCap("ß fi ffi ff st ῗ", "UTF8_BINARY", "Ss Fi Ffi Ff St Ϊ͂", "ß fi ffi ff st ῗ"); - assertInitCap("ß fi ffi ff st ῗ", "UTF8_LCASE", "Ss Fi Ffi Ff St \u0399\u0308\u0342"); - assertInitCap("ß fi ffi ff st ῗ", "UNICODE", "Ss Fi Ffi Ff St \u0399\u0308\u0342"); - assertInitCap("ß fi ffi ff st ῗ", "UNICODE", "Ss Fi Ffi Ff St \u0399\u0308\u0342"); - assertInitCap("œ ǽ", "UTF8_BINARY", "Œ Ǽ", "Œ Ǽ"); + assertInitCap("ß fi ffi ff st ῗ", UTF8_BINARY, "Ss Fi Ffi Ff St Ϊ͂", "ß fi ffi ff st ῗ"); + assertInitCap("ß fi ffi ff st ῗ", UTF8_LCASE, "Ss Fi Ffi Ff St \u0399\u0308\u0342"); + assertInitCap("ß fi ffi ff st ῗ", UNICODE, "Ss Fi Ffi Ff St \u0399\u0308\u0342"); + assertInitCap("ß fi ffi ff st ῗ", UNICODE, "Ss Fi Ffi Ff St \u0399\u0308\u0342"); + assertInitCap("œ ǽ", UTF8_BINARY, "Œ Ǽ", "Œ Ǽ"); // Different possible word boundaries. - assertInitCap("a b c", "UTF8_BINARY", "A B C"); - assertInitCap("a b c", "UNICODE", "A B C"); - assertInitCap("a b c", "UTF8_LCASE", "A B C"); - assertInitCap("a b c", "UNICODE_CI", "A B C"); - assertInitCap("a.b,c", "UTF8_BINARY", "A.b,c"); - assertInitCap("a.b,c", "UNICODE", "A.b,C"); - assertInitCap("a.b,c", "UTF8_LCASE", "A.b,C"); - assertInitCap("a.b,c", "UNICODE_CI", "A.b,C"); - assertInitCap("a. b-c", "UTF8_BINARY", "A. B-c"); - assertInitCap("a. b-c", "UNICODE", "A. B-C"); - assertInitCap("a. b-c", "UTF8_LCASE", "A. B-C"); - assertInitCap("a. b-c", "UNICODE_CI", "A. B-C"); - assertInitCap("a?b世c", "UTF8_BINARY", "A?b世c"); - assertInitCap("a?b世c", "UNICODE", "A?B世C"); - assertInitCap("a?b世c", "UTF8_LCASE", "A?B世C"); - assertInitCap("a?b世c", "UNICODE_CI", "A?B世C"); + assertInitCap("a b c", UTF8_BINARY, "A B C"); + assertInitCap("a b c", UNICODE, "A B C"); + assertInitCap("a b c", UTF8_LCASE, "A B C"); + assertInitCap("a b c", UNICODE_CI, "A B C"); + assertInitCap("a.b,c", UTF8_BINARY, "A.b,c"); + assertInitCap("a.b,c", UNICODE, "A.b,C"); + assertInitCap("a.b,c", UTF8_LCASE, "A.b,C"); + assertInitCap("a.b,c", UNICODE_CI, "A.b,C"); + assertInitCap("a. b-c", UTF8_BINARY, "A. B-c"); + assertInitCap("a. b-c", UNICODE, "A. B-C"); + assertInitCap("a. b-c", UTF8_LCASE, "A. B-C"); + assertInitCap("a. b-c", UNICODE_CI, "A. B-C"); + assertInitCap("a?b世c", UTF8_BINARY, "A?b世c"); + assertInitCap("a?b世c", UNICODE, "A?B世C"); + assertInitCap("a?b世c", UTF8_LCASE, "A?B世C"); + assertInitCap("a?b世c", UNICODE_CI, "A?B世C"); // Titlecase characters that are different from uppercase characters. - assertInitCap("dzDZDz", "UTF8_BINARY", "Dzdzdz"); - assertInitCap("dzDZDz", "UNICODE", "Dzdzdz"); - assertInitCap("dzDZDz", "UTF8_LCASE", "Dzdzdz"); - assertInitCap("dzDZDz", "UNICODE_CI", "Dzdzdz"); - assertInitCap("džaba Ljubav NJegova", "UTF8_BINARY", "Džaba Ljubav Njegova"); - assertInitCap("džaba Ljubav NJegova", "UNICODE", "Džaba Ljubav Njegova"); - assertInitCap("džaba Ljubav NJegova", "UTF8_LCASE", "Džaba Ljubav Njegova"); - assertInitCap("džaba Ljubav NJegova", "UNICODE_CI", "Džaba Ljubav Njegova"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UTF8_BINARY", + assertInitCap("dzDZDz", UTF8_BINARY, "Dzdzdz"); + assertInitCap("dzDZDz", UNICODE, "Dzdzdz"); + assertInitCap("dzDZDz", UTF8_LCASE, "Dzdzdz"); + assertInitCap("dzDZDz", UNICODE_CI, "Dzdzdz"); + assertInitCap("džaba Ljubav NJegova", UTF8_BINARY, "Džaba Ljubav Njegova"); + assertInitCap("džaba Ljubav NJegova", UNICODE, "Džaba Ljubav Njegova"); + assertInitCap("džaba Ljubav NJegova", UTF8_LCASE, "Džaba Ljubav Njegova"); + assertInitCap("džaba Ljubav NJegova", UNICODE_CI, "Džaba Ljubav Njegova"); + assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UTF8_BINARY, "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota","ß fi ffi ff st Σημερινος Ασημενιος I\u0307ota"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UTF8_LCASE", + assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UTF8_LCASE, "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UNICODE", + assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UNICODE, "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡςΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UNICODE_CI", + assertInitCap("ß fi ffi ff st ΣΗΜΕΡςΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UNICODE_CI, "Ss Fi Ffi Ff St Σημερςινος Ασημενιος İota"); // Characters that map to multiple characters when titlecased and lowercased. - assertInitCap("ß fi ffi ff st İOTA", "UTF8_BINARY", "Ss Fi Ffi Ff St İota", "ß fi ffi ff st İota"); - assertInitCap("ß fi ffi ff st OİOTA", "UTF8_BINARY", + assertInitCap("ß fi ffi ff st İOTA", UTF8_BINARY, "Ss Fi Ffi Ff St İota", "ß fi ffi ff st İota"); + assertInitCap("ß fi ffi ff st OİOTA", UTF8_BINARY, "Ss Fi Ffi Ff St Oi\u0307ota", "ß fi ffi ff st Oi̇ota"); // Lowercasing Greek letter sigma ('Σ') when case-ignorable character present. - assertInitCap("`Σ", "UTF8_BINARY", "`σ", "`σ"); - assertInitCap("1`Σ`` AΣ", "UTF8_BINARY", "1`σ`` Aς", "1`σ`` Aς"); - assertInitCap("a1`Σ``", "UTF8_BINARY", "A1`σ``", "A1`σ``"); - assertInitCap("a`Σ``", "UTF8_BINARY", "A`ς``", "A`σ``"); - assertInitCap("a`Σ``1", "UTF8_BINARY", "A`ς``1", "A`σ``1"); - assertInitCap("a`Σ``A", "UTF8_BINARY", "A`σ``a", "A`σ``a"); - assertInitCap("ΘΑ�Σ�ΟΣ�", "UTF8_BINARY", "Θα�σ�ος�", "Θα�σ�ος�"); - assertInitCap("ΘΑᵩΣ�ΟᵩΣᵩ�", "UTF8_BINARY", "Θαᵩς�οᵩςᵩ�", "Θαᵩς�οᵩςᵩ�"); - assertInitCap("ΘΑ�ᵩΣ�ΟᵩΣᵩ�", "UTF8_BINARY", "Θα�ᵩσ�οᵩςᵩ�", "Θα�ᵩσ�οᵩςᵩ�"); - assertInitCap("ΘΑ�ᵩΣᵩ�ΟᵩΣᵩ�", "UTF8_BINARY", "Θα�ᵩσᵩ�οᵩςᵩ�", "Θα�ᵩσᵩ�οᵩςᵩ�"); - assertInitCap("ΘΑ�Σ�Ο�Σ�", "UTF8_BINARY", "Θα�σ�ο�σ�", "Θα�σ�ο�σ�"); + assertInitCap("`Σ", UTF8_BINARY, "`σ", "`σ"); + assertInitCap("1`Σ`` AΣ", UTF8_BINARY, "1`σ`` Aς", "1`σ`` Aς"); + assertInitCap("a1`Σ``", UTF8_BINARY, "A1`σ``", "A1`σ``"); + assertInitCap("a`Σ``", UTF8_BINARY, "A`ς``", "A`σ``"); + assertInitCap("a`Σ``1", UTF8_BINARY, "A`ς``1", "A`σ``1"); + assertInitCap("a`Σ``A", UTF8_BINARY, "A`σ``a", "A`σ``a"); + assertInitCap("ΘΑ�Σ�ΟΣ�", UTF8_BINARY, "Θα�σ�ος�", "Θα�σ�ος�"); + assertInitCap("ΘΑᵩΣ�ΟᵩΣᵩ�", UTF8_BINARY, "Θαᵩς�οᵩςᵩ�", "Θαᵩς�οᵩςᵩ�"); + assertInitCap("ΘΑ�ᵩΣ�ΟᵩΣᵩ�", UTF8_BINARY, "Θα�ᵩσ�οᵩςᵩ�", "Θα�ᵩσ�οᵩςᵩ�"); + assertInitCap("ΘΑ�ᵩΣᵩ�ΟᵩΣᵩ�", UTF8_BINARY, "Θα�ᵩσᵩ�οᵩςᵩ�", "Θα�ᵩσᵩ�οᵩςᵩ�"); + assertInitCap("ΘΑ�Σ�Ο�Σ�", UTF8_BINARY, "Θα�σ�ο�σ�", "Θα�σ�ο�σ�"); // Disallowed bytes and invalid sequences. assertInitCap(UTF8String.fromBytes(new byte[] { (byte)0xC0, (byte)0xC1, (byte)0xF5}).toString(), - "UTF8_BINARY", "���", "���"); + UTF8_BINARY, "���", "���"); assertInitCap(UTF8String.fromBytes( new byte[]{(byte)0xC0, (byte)0xC1, (byte)0xF5, 0x20, 0x61, 0x41, (byte)0xC0}).toString(), - "UTF8_BINARY", + UTF8_BINARY, "��� Aa�", "��� Aa�"); assertInitCap(UTF8String.fromBytes(new byte[]{(byte)0xC2,(byte)0xC2}).toString(), - "UTF8_BINARY", "��", "��"); + UTF8_BINARY, "��", "��"); assertInitCap(UTF8String.fromBytes( new byte[]{0x61, 0x41, (byte)0xC2, (byte)0xC2, 0x41}).toString(), - "UTF8_BINARY", + UTF8_BINARY, "Aa��a", "Aa��a"); } @@ -1559,147 +1560,147 @@ private void assertStringInstr(String string, String substring, @Test public void testStringInstr() throws SparkException { // Empty strings. - assertStringInstr("", "", "UTF8_BINARY", 1); - assertStringInstr("", "", "UTF8_LCASE", 1); - assertStringInstr("", "", "UNICODE_CI", 1); - assertStringInstr("", "", "UNICODE", 1); - assertStringInstr("a", "", "UTF8_BINARY", 1); - assertStringInstr("a", "", "UTF8_LCASE", 1); - assertStringInstr("a", "", "UNICODE", 1); - assertStringInstr("a", "", "UNICODE_CI", 1); - assertStringInstr("", "x", "UTF8_BINARY", 0); - assertStringInstr("", "x", "UTF8_LCASE", 0); - assertStringInstr("", "x", "UNICODE", 0); - assertStringInstr("", "x", "UNICODE_CI", 0); + assertStringInstr("", "", UTF8_BINARY, 1); + assertStringInstr("", "", UTF8_LCASE, 1); + assertStringInstr("", "", UNICODE_CI, 1); + assertStringInstr("", "", UNICODE, 1); + assertStringInstr("a", "", UTF8_BINARY, 1); + assertStringInstr("a", "", UTF8_LCASE, 1); + assertStringInstr("a", "", UNICODE, 1); + assertStringInstr("a", "", UNICODE_CI, 1); + assertStringInstr("", "x", UTF8_BINARY, 0); + assertStringInstr("", "x", UTF8_LCASE, 0); + assertStringInstr("", "x", UNICODE, 0); + assertStringInstr("", "x", UNICODE_CI, 0); // Basic tests. - assertStringInstr("aaads", "aa", "UTF8_BINARY", 1); - assertStringInstr("aaads", "aa", "UTF8_LCASE", 1); - assertStringInstr("aaads", "aa", "UNICODE", 1); - assertStringInstr("aaads", "aa", "UNICODE_CI", 1); - assertStringInstr("aaads", "ds", "UTF8_BINARY", 4); - assertStringInstr("aaads", "ds", "UTF8_LCASE", 4); - assertStringInstr("aaads", "ds", "UNICODE", 4); - assertStringInstr("aaads", "ds", "UNICODE_CI", 4); - assertStringInstr("aaads", "Aa", "UTF8_BINARY", 0); - assertStringInstr("aaads", "Aa", "UTF8_LCASE", 1); - assertStringInstr("aaads", "Aa", "UNICODE", 0); - assertStringInstr("aaads", "Aa", "UNICODE_CI", 1); - assertStringInstr("aaaDs", "de", "UTF8_BINARY", 0); - assertStringInstr("aaaDs", "de", "UTF8_LCASE", 0); - assertStringInstr("aaaDs", "de", "UNICODE", 0); - assertStringInstr("aaaDs", "de", "UNICODE_CI", 0); - assertStringInstr("aaaDs", "ds", "UTF8_BINARY", 0); - assertStringInstr("aaaDs", "ds", "UTF8_LCASE", 4); - assertStringInstr("aaaDs", "ds", "UNICODE", 0); - assertStringInstr("aaaDs", "ds", "UNICODE_CI", 4); - assertStringInstr("aaadS", "Ds", "UTF8_BINARY", 0); - assertStringInstr("aaadS", "Ds", "UTF8_LCASE", 4); - assertStringInstr("aaadS", "Ds", "UNICODE", 0); - assertStringInstr("aaadS", "Ds", "UNICODE_CI", 4); + assertStringInstr("aaads", "aa", UTF8_BINARY, 1); + assertStringInstr("aaads", "aa", UTF8_LCASE, 1); + assertStringInstr("aaads", "aa", UNICODE, 1); + assertStringInstr("aaads", "aa", UNICODE_CI, 1); + assertStringInstr("aaads", "ds", UTF8_BINARY, 4); + assertStringInstr("aaads", "ds", UTF8_LCASE, 4); + assertStringInstr("aaads", "ds", UNICODE, 4); + assertStringInstr("aaads", "ds", UNICODE_CI, 4); + assertStringInstr("aaads", "Aa", UTF8_BINARY, 0); + assertStringInstr("aaads", "Aa", UTF8_LCASE, 1); + assertStringInstr("aaads", "Aa", UNICODE, 0); + assertStringInstr("aaads", "Aa", UNICODE_CI, 1); + assertStringInstr("aaaDs", "de", UTF8_BINARY, 0); + assertStringInstr("aaaDs", "de", UTF8_LCASE, 0); + assertStringInstr("aaaDs", "de", UNICODE, 0); + assertStringInstr("aaaDs", "de", UNICODE_CI, 0); + assertStringInstr("aaaDs", "ds", UTF8_BINARY, 0); + assertStringInstr("aaaDs", "ds", UTF8_LCASE, 4); + assertStringInstr("aaaDs", "ds", UNICODE, 0); + assertStringInstr("aaaDs", "ds", UNICODE_CI, 4); + assertStringInstr("aaadS", "Ds", UTF8_BINARY, 0); + assertStringInstr("aaadS", "Ds", UTF8_LCASE, 4); + assertStringInstr("aaadS", "Ds", UNICODE, 0); + assertStringInstr("aaadS", "Ds", UNICODE_CI, 4); assertStringInstr("aaaČŠčšcs", "cs", "SR", 8); assertStringInstr("aaaČŠčšcs", "cs", "SR_CI_AI", 4); // Advanced tests. - assertStringInstr("test大千世界X大千世界", "大千", "UTF8_BINARY", 5); - assertStringInstr("test大千世界X大千世界", "大千", "UTF8_LCASE", 5); - assertStringInstr("test大千世界X大千世界", "大千", "UNICODE", 5); - assertStringInstr("test大千世界X大千世界", "大千", "UNICODE_CI", 5); - assertStringInstr("test大千世界X大千世界", "界X", "UTF8_BINARY", 8); - assertStringInstr("test大千世界X大千世界", "界X", "UTF8_LCASE", 8); - assertStringInstr("test大千世界X大千世界", "界X", "UNICODE", 8); - assertStringInstr("test大千世界X大千世界", "界X", "UNICODE_CI", 8); - assertStringInstr("test大千世界X大千世界", "界x", "UTF8_BINARY", 0); - assertStringInstr("test大千世界X大千世界", "界x", "UTF8_LCASE", 8); - assertStringInstr("test大千世界X大千世界", "界x", "UNICODE", 0); - assertStringInstr("test大千世界X大千世界", "界x", "UNICODE_CI", 8); - assertStringInstr("test大千世界X大千世界", "界y", "UTF8_BINARY", 0); - assertStringInstr("test大千世界X大千世界", "界y", "UTF8_LCASE", 0); - assertStringInstr("test大千世界X大千世界", "界y", "UNICODE", 0); - assertStringInstr("test大千世界X大千世界", "界y", "UNICODE_CI", 0); + assertStringInstr("test大千世界X大千世界", "大千", UTF8_BINARY, 5); + assertStringInstr("test大千世界X大千世界", "大千", UTF8_LCASE, 5); + assertStringInstr("test大千世界X大千世界", "大千", UNICODE, 5); + assertStringInstr("test大千世界X大千世界", "大千", UNICODE_CI, 5); + assertStringInstr("test大千世界X大千世界", "界X", UTF8_BINARY, 8); + assertStringInstr("test大千世界X大千世界", "界X", UTF8_LCASE, 8); + assertStringInstr("test大千世界X大千世界", "界X", UNICODE, 8); + assertStringInstr("test大千世界X大千世界", "界X", UNICODE_CI, 8); + assertStringInstr("test大千世界X大千世界", "界x", UTF8_BINARY, 0); + assertStringInstr("test大千世界X大千世界", "界x", UTF8_LCASE, 8); + assertStringInstr("test大千世界X大千世界", "界x", UNICODE, 0); + assertStringInstr("test大千世界X大千世界", "界x", UNICODE_CI, 8); + assertStringInstr("test大千世界X大千世界", "界y", UTF8_BINARY, 0); + assertStringInstr("test大千世界X大千世界", "界y", UTF8_LCASE, 0); + assertStringInstr("test大千世界X大千世界", "界y", UNICODE, 0); + assertStringInstr("test大千世界X大千世界", "界y", UNICODE_CI, 0); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringInstr("i\u0307", "i", "UNICODE_CI", 0); - assertStringInstr("i\u0307", "\u0307", "UNICODE_CI", 0); - assertStringInstr("i\u0307", "İ", "UNICODE_CI", 1); - assertStringInstr("İ", "i", "UNICODE_CI", 0); - assertStringInstr("İoi̇o12", "i\u0307o", "UNICODE_CI", 1); - assertStringInstr("i̇oİo12", "İo", "UNICODE_CI", 1); - assertStringInstr("abİoi̇o", "i\u0307o", "UNICODE_CI", 3); - assertStringInstr("abi̇oİo", "İo", "UNICODE_CI", 3); - assertStringInstr("ai̇oxXİo", "Xx", "UNICODE_CI", 5); - assertStringInstr("aİoi̇oxx", "XX", "UNICODE_CI", 7); - assertStringInstr("i\u0307", "i", "UTF8_LCASE", 1); // != UNICODE_CI - assertStringInstr("i\u0307", "\u0307", "UTF8_LCASE", 2); // != UNICODE_CI - assertStringInstr("i\u0307", "İ", "UTF8_LCASE", 1); - assertStringInstr("İ", "i", "UTF8_LCASE", 0); - assertStringInstr("İoi̇o12", "i\u0307o", "UTF8_LCASE", 1); - assertStringInstr("i̇oİo12", "İo", "UTF8_LCASE", 1); - assertStringInstr("abİoi̇o", "i\u0307o", "UTF8_LCASE", 3); - assertStringInstr("abi̇oİo", "İo", "UTF8_LCASE", 3); - assertStringInstr("abI\u0307oi̇o", "İo", "UTF8_LCASE", 3); - assertStringInstr("ai̇oxXİo", "Xx", "UTF8_LCASE", 5); - assertStringInstr("abİoi̇o", "\u0307o", "UTF8_LCASE", 6); - assertStringInstr("aİoi̇oxx", "XX", "UTF8_LCASE", 7); + assertStringInstr("i\u0307", "i", UNICODE_CI, 0); + assertStringInstr("i\u0307", "\u0307", UNICODE_CI, 0); + assertStringInstr("i\u0307", "İ", UNICODE_CI, 1); + assertStringInstr("İ", "i", UNICODE_CI, 0); + assertStringInstr("İoi̇o12", "i\u0307o", UNICODE_CI, 1); + assertStringInstr("i̇oİo12", "İo", UNICODE_CI, 1); + assertStringInstr("abİoi̇o", "i\u0307o", UNICODE_CI, 3); + assertStringInstr("abi̇oİo", "İo", UNICODE_CI, 3); + assertStringInstr("ai̇oxXİo", "Xx", UNICODE_CI, 5); + assertStringInstr("aİoi̇oxx", "XX", UNICODE_CI, 7); + assertStringInstr("i\u0307", "i", UTF8_LCASE, 1); // != UNICODE_CI + assertStringInstr("i\u0307", "\u0307", UTF8_LCASE, 2); // != UNICODE_CI + assertStringInstr("i\u0307", "İ", UTF8_LCASE, 1); + assertStringInstr("İ", "i", UTF8_LCASE, 0); + assertStringInstr("İoi̇o12", "i\u0307o", UTF8_LCASE, 1); + assertStringInstr("i̇oİo12", "İo", UTF8_LCASE, 1); + assertStringInstr("abİoi̇o", "i\u0307o", UTF8_LCASE, 3); + assertStringInstr("abi̇oİo", "İo", UTF8_LCASE, 3); + assertStringInstr("abI\u0307oi̇o", "İo", UTF8_LCASE, 3); + assertStringInstr("ai̇oxXİo", "Xx", UTF8_LCASE, 5); + assertStringInstr("abİoi̇o", "\u0307o", UTF8_LCASE, 6); + assertStringInstr("aİoi̇oxx", "XX", UTF8_LCASE, 7); // Conditional case mapping (e.g. Greek sigmas). - assertStringInstr("σ", "σ", "UTF8_BINARY", 1); - assertStringInstr("σ", "ς", "UTF8_BINARY", 0); - assertStringInstr("σ", "Σ", "UTF8_BINARY", 0); - assertStringInstr("ς", "σ", "UTF8_BINARY", 0); - assertStringInstr("ς", "ς", "UTF8_BINARY", 1); - assertStringInstr("ς", "Σ", "UTF8_BINARY", 0); - assertStringInstr("Σ", "σ", "UTF8_BINARY", 0); - assertStringInstr("Σ", "ς", "UTF8_BINARY", 0); - assertStringInstr("Σ", "Σ", "UTF8_BINARY", 1); - assertStringInstr("σ", "σ", "UTF8_LCASE", 1); - assertStringInstr("σ", "ς", "UTF8_LCASE", 1); - assertStringInstr("σ", "Σ", "UTF8_LCASE", 1); - assertStringInstr("ς", "σ", "UTF8_LCASE", 1); - assertStringInstr("ς", "ς", "UTF8_LCASE", 1); - assertStringInstr("ς", "Σ", "UTF8_LCASE", 1); - assertStringInstr("Σ", "σ", "UTF8_LCASE", 1); - assertStringInstr("Σ", "ς", "UTF8_LCASE", 1); - assertStringInstr("Σ", "Σ", "UTF8_LCASE", 1); - assertStringInstr("σ", "σ", "UNICODE", 1); - assertStringInstr("σ", "ς", "UNICODE", 0); - assertStringInstr("σ", "Σ", "UNICODE", 0); - assertStringInstr("ς", "σ", "UNICODE", 0); - assertStringInstr("ς", "ς", "UNICODE", 1); - assertStringInstr("ς", "Σ", "UNICODE", 0); - assertStringInstr("Σ", "σ", "UNICODE", 0); - assertStringInstr("Σ", "ς", "UNICODE", 0); - assertStringInstr("Σ", "Σ", "UNICODE", 1); - assertStringInstr("σ", "σ", "UNICODE_CI", 1); - assertStringInstr("σ", "ς", "UNICODE_CI", 1); - assertStringInstr("σ", "Σ", "UNICODE_CI", 1); - assertStringInstr("ς", "σ", "UNICODE_CI", 1); - assertStringInstr("ς", "ς", "UNICODE_CI", 1); - assertStringInstr("ς", "Σ", "UNICODE_CI", 1); - assertStringInstr("Σ", "σ", "UNICODE_CI", 1); - assertStringInstr("Σ", "ς", "UNICODE_CI", 1); - assertStringInstr("Σ", "Σ", "UNICODE_CI", 1); + assertStringInstr("σ", "σ", UTF8_BINARY, 1); + assertStringInstr("σ", "ς", UTF8_BINARY, 0); + assertStringInstr("σ", "Σ", UTF8_BINARY, 0); + assertStringInstr("ς", "σ", UTF8_BINARY, 0); + assertStringInstr("ς", "ς", UTF8_BINARY, 1); + assertStringInstr("ς", "Σ", UTF8_BINARY, 0); + assertStringInstr("Σ", "σ", UTF8_BINARY, 0); + assertStringInstr("Σ", "ς", UTF8_BINARY, 0); + assertStringInstr("Σ", "Σ", UTF8_BINARY, 1); + assertStringInstr("σ", "σ", UTF8_LCASE, 1); + assertStringInstr("σ", "ς", UTF8_LCASE, 1); + assertStringInstr("σ", "Σ", UTF8_LCASE, 1); + assertStringInstr("ς", "σ", UTF8_LCASE, 1); + assertStringInstr("ς", "ς", UTF8_LCASE, 1); + assertStringInstr("ς", "Σ", UTF8_LCASE, 1); + assertStringInstr("Σ", "σ", UTF8_LCASE, 1); + assertStringInstr("Σ", "ς", UTF8_LCASE, 1); + assertStringInstr("Σ", "Σ", UTF8_LCASE, 1); + assertStringInstr("σ", "σ", UNICODE, 1); + assertStringInstr("σ", "ς", UNICODE, 0); + assertStringInstr("σ", "Σ", UNICODE, 0); + assertStringInstr("ς", "σ", UNICODE, 0); + assertStringInstr("ς", "ς", UNICODE, 1); + assertStringInstr("ς", "Σ", UNICODE, 0); + assertStringInstr("Σ", "σ", UNICODE, 0); + assertStringInstr("Σ", "ς", UNICODE, 0); + assertStringInstr("Σ", "Σ", UNICODE, 1); + assertStringInstr("σ", "σ", UNICODE_CI, 1); + assertStringInstr("σ", "ς", UNICODE_CI, 1); + assertStringInstr("σ", "Σ", UNICODE_CI, 1); + assertStringInstr("ς", "σ", UNICODE_CI, 1); + assertStringInstr("ς", "ς", UNICODE_CI, 1); + assertStringInstr("ς", "Σ", UNICODE_CI, 1); + assertStringInstr("Σ", "σ", UNICODE_CI, 1); + assertStringInstr("Σ", "ς", UNICODE_CI, 1); + assertStringInstr("Σ", "Σ", UNICODE_CI, 1); // Surrogate pairs. - assertStringInstr("a🙃b", "a", "UTF8_BINARY", 1); - assertStringInstr("a🙃b", "a", "UTF8_LCASE", 1); - assertStringInstr("a🙃b", "a", "UNICODE", 1); - assertStringInstr("a🙃b", "a", "UNICODE_CI", 1); - assertStringInstr("a🙃b", "🙃", "UTF8_BINARY", 2); - assertStringInstr("a🙃b", "🙃", "UTF8_LCASE", 2); - assertStringInstr("a🙃b", "🙃", "UNICODE", 2); - assertStringInstr("a🙃b", "🙃", "UNICODE_CI", 2); - assertStringInstr("a🙃b", "b", "UTF8_BINARY", 3); - assertStringInstr("a🙃b", "b", "UTF8_LCASE", 3); - assertStringInstr("a🙃b", "b", "UNICODE", 3); - assertStringInstr("a🙃b", "b", "UNICODE_CI", 3); - assertStringInstr("a🙃🙃b", "🙃", "UTF8_BINARY", 2); - assertStringInstr("a🙃🙃b", "🙃", "UTF8_LCASE", 2); - assertStringInstr("a🙃🙃b", "🙃", "UNICODE", 2); - assertStringInstr("a🙃🙃b", "🙃", "UNICODE_CI", 2); - assertStringInstr("a🙃🙃b", "b", "UTF8_BINARY", 4); - assertStringInstr("a🙃🙃b", "b", "UTF8_LCASE", 4); - assertStringInstr("a🙃🙃b", "b", "UNICODE", 4); - assertStringInstr("a🙃🙃b", "b", "UNICODE_CI", 4); - assertStringInstr("a🙃x🙃b", "b", "UTF8_BINARY", 5); - assertStringInstr("a🙃x🙃b", "b", "UTF8_LCASE", 5); - assertStringInstr("a🙃x🙃b", "b", "UNICODE", 5); - assertStringInstr("a🙃x🙃b", "b", "UNICODE_CI", 5); + assertStringInstr("a🙃b", "a", UTF8_BINARY, 1); + assertStringInstr("a🙃b", "a", UTF8_LCASE, 1); + assertStringInstr("a🙃b", "a", UNICODE, 1); + assertStringInstr("a🙃b", "a", UNICODE_CI, 1); + assertStringInstr("a🙃b", "🙃", UTF8_BINARY, 2); + assertStringInstr("a🙃b", "🙃", UTF8_LCASE, 2); + assertStringInstr("a🙃b", "🙃", UNICODE, 2); + assertStringInstr("a🙃b", "🙃", UNICODE_CI, 2); + assertStringInstr("a🙃b", "b", UTF8_BINARY, 3); + assertStringInstr("a🙃b", "b", UTF8_LCASE, 3); + assertStringInstr("a🙃b", "b", UNICODE, 3); + assertStringInstr("a🙃b", "b", UNICODE_CI, 3); + assertStringInstr("a🙃🙃b", "🙃", UTF8_BINARY, 2); + assertStringInstr("a🙃🙃b", "🙃", UTF8_LCASE, 2); + assertStringInstr("a🙃🙃b", "🙃", UNICODE, 2); + assertStringInstr("a🙃🙃b", "🙃", UNICODE_CI, 2); + assertStringInstr("a🙃🙃b", "b", UTF8_BINARY, 4); + assertStringInstr("a🙃🙃b", "b", UTF8_LCASE, 4); + assertStringInstr("a🙃🙃b", "b", UNICODE, 4); + assertStringInstr("a🙃🙃b", "b", UNICODE_CI, 4); + assertStringInstr("a🙃x🙃b", "b", UTF8_BINARY, 5); + assertStringInstr("a🙃x🙃b", "b", UTF8_LCASE, 5); + assertStringInstr("a🙃x🙃b", "b", UNICODE, 5); + assertStringInstr("a🙃x🙃b", "b", UNICODE_CI, 5); } /** @@ -1717,256 +1718,256 @@ private void assertFindInSet(String word, UTF8String set, String collationName, @Test public void testFindInSet() throws SparkException { // Empty strings. - assertFindInSet("", UTF8String.fromString(""), "UTF8_BINARY", 1); - assertFindInSet("", UTF8String.fromString(""), "UTF8_LCASE", 1); - assertFindInSet("", UTF8String.fromString(""), "UNICODE", 1); - assertFindInSet("", UTF8String.fromString(""), "UNICODE_CI", 1); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UTF8_BINARY", 1); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UTF8_LCASE", 1); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UNICODE", 1); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UNICODE_CI", 1); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UTF8_BINARY", 6); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UTF8_LCASE", 6); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UNICODE", 6); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UNICODE_CI", 6); - assertFindInSet("", UTF8String.fromString("abc"), "UTF8_BINARY", 0); - assertFindInSet("", UTF8String.fromString("abc"), "UTF8_LCASE", 0); - assertFindInSet("", UTF8String.fromString("abc"), "UNICODE", 0); - assertFindInSet("", UTF8String.fromString("abc"), "UNICODE_CI", 0); + assertFindInSet("", UTF8String.fromString(""), UTF8_BINARY, 1); + assertFindInSet("", UTF8String.fromString(""), UTF8_LCASE, 1); + assertFindInSet("", UTF8String.fromString(""), UNICODE, 1); + assertFindInSet("", UTF8String.fromString(""), UNICODE_CI, 1); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UTF8_BINARY, 1); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UTF8_LCASE, 1); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UNICODE, 1); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UNICODE_CI, 1); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UTF8_BINARY, 6); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UTF8_LCASE, 6); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UNICODE, 6); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UNICODE_CI, 6); + assertFindInSet("", UTF8String.fromString("abc"), UTF8_BINARY, 0); + assertFindInSet("", UTF8String.fromString("abc"), UTF8_LCASE, 0); + assertFindInSet("", UTF8String.fromString("abc"), UNICODE, 0); + assertFindInSet("", UTF8String.fromString("abc"), UNICODE_CI, 0); // Basic tests. - assertFindInSet("xx", UTF8String.fromString("xx"), "UTF8_BINARY", 1); - assertFindInSet("xx", UTF8String.fromString("xx"), "UTF8_LCASE", 1); - assertFindInSet("xx", UTF8String.fromString("xx"), "UNICODE", 1); - assertFindInSet("xx", UTF8String.fromString("xx"), "UNICODE_CI", 1); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 1); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 1); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 1); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 1); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 5); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 5); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 5); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 5); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 3); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 3); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 4); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 4); + assertFindInSet("xx", UTF8String.fromString("xx"), UTF8_BINARY, 1); + assertFindInSet("xx", UTF8String.fromString("xx"), UTF8_LCASE, 1); + assertFindInSet("xx", UTF8String.fromString("xx"), UNICODE, 1); + assertFindInSet("xx", UTF8String.fromString("xx"), UNICODE_CI, 1); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 1); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 1); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 1); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 1); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 5); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 5); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 5); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 5); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 3); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 3); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 4); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 4); // Advanced tests. - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_BINARY", 5); - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_LCASE", 5); - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE", 5); - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE_CI", 5); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_BINARY", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_LCASE", 4); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE_CI", 4); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UTF8_BINARY", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UTF8_LCASE", 5); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UNICODE", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UNICODE_CI", 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_BINARY, 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_LCASE, 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE, 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE_CI, 5); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_BINARY, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_LCASE, 4); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE_CI, 4); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UTF8_BINARY, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UTF8_LCASE, 5); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UNICODE, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UNICODE_CI, 5); // One-to-many case mapping (e.g. Turkish dotted I). - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("İ"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("İ"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("İ"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("İ"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UTF8_BINARY", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UNICODE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("İ,"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("İ,"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("İ,"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("İ,"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UTF8_BINARY", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UNICODE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UTF8_BINARY", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UNICODE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UNICODE_CI", 0); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UTF8_BINARY", 0); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UTF8_LCASE", 2); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UNICODE", 0); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UNICODE_CI", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UTF8_BINARY", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UNICODE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UNICODE_CI", 0); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UNICODE", 0); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UNICODE_CI", 2); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UTF8_BINARY", 0); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UTF8_LCASE", 2); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UNICODE", 0); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UNICODE_CI", 2); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("İ"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("İ"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("İ"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("İ"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UTF8_BINARY, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UNICODE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("İ,"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("İ,"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("İ,"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("İ,"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UTF8_BINARY, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UNICODE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UTF8_BINARY, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UNICODE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UNICODE_CI, 0); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UTF8_BINARY, 0); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UTF8_LCASE, 2); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UNICODE, 0); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UNICODE_CI, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UTF8_BINARY, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UNICODE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UNICODE_CI, 0); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UTF8_BINARY, 0); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UTF8_LCASE, 2); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UNICODE, 0); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UNICODE_CI, 2); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UTF8_BINARY, 0); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UTF8_LCASE, 2); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UNICODE, 0); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UNICODE_CI, 2); // Conditional case mapping (e.g. Greek sigmas). - assertFindInSet("σ", UTF8String.fromString("σ"), "UTF8_BINARY", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UTF8_BINARY", 0); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UTF8_BINARY", 0); - assertFindInSet("ς", UTF8String.fromString("σ"), "UTF8_BINARY", 0); - assertFindInSet("ς", UTF8String.fromString("ς"), "UTF8_BINARY", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UTF8_BINARY", 0); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UTF8_BINARY", 0); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UTF8_BINARY", 0); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UTF8_BINARY", 1); - assertFindInSet("σ", UTF8String.fromString("σ"), "UTF8_LCASE", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UTF8_LCASE", 1); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UTF8_LCASE", 1); - assertFindInSet("ς", UTF8String.fromString("σ"), "UTF8_LCASE", 1); - assertFindInSet("ς", UTF8String.fromString("ς"), "UTF8_LCASE", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UTF8_LCASE", 1); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UTF8_LCASE", 1); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UTF8_LCASE", 1); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UTF8_LCASE", 1); - assertFindInSet("σ", UTF8String.fromString("σ"), "UNICODE", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UNICODE", 0); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UNICODE", 0); - assertFindInSet("ς", UTF8String.fromString("σ"), "UNICODE", 0); - assertFindInSet("ς", UTF8String.fromString("ς"), "UNICODE", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UNICODE", 0); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UNICODE", 0); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UNICODE", 0); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UNICODE", 1); - assertFindInSet("σ", UTF8String.fromString("σ"), "UNICODE_CI", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UNICODE_CI", 1); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UNICODE_CI", 1); - assertFindInSet("ς", UTF8String.fromString("σ"), "UNICODE_CI", 1); - assertFindInSet("ς", UTF8String.fromString("ς"), "UNICODE_CI", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UNICODE_CI", 1); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UNICODE_CI", 1); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UNICODE_CI", 1); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UNICODE_CI", 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UTF8_BINARY, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UTF8_BINARY, 0); + assertFindInSet("σ", UTF8String.fromString("Σ"), UTF8_BINARY, 0); + assertFindInSet("ς", UTF8String.fromString("σ"), UTF8_BINARY, 0); + assertFindInSet("ς", UTF8String.fromString("ς"), UTF8_BINARY, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UTF8_BINARY, 0); + assertFindInSet("Σ", UTF8String.fromString("σ"), UTF8_BINARY, 0); + assertFindInSet("Σ", UTF8String.fromString("ς"), UTF8_BINARY, 0); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UTF8_BINARY, 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UTF8_LCASE, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UTF8_LCASE, 1); + assertFindInSet("σ", UTF8String.fromString("Σ"), UTF8_LCASE, 1); + assertFindInSet("ς", UTF8String.fromString("σ"), UTF8_LCASE, 1); + assertFindInSet("ς", UTF8String.fromString("ς"), UTF8_LCASE, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UTF8_LCASE, 1); + assertFindInSet("Σ", UTF8String.fromString("σ"), UTF8_LCASE, 1); + assertFindInSet("Σ", UTF8String.fromString("ς"), UTF8_LCASE, 1); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UTF8_LCASE, 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UNICODE, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UNICODE, 0); + assertFindInSet("σ", UTF8String.fromString("Σ"), UNICODE, 0); + assertFindInSet("ς", UTF8String.fromString("σ"), UNICODE, 0); + assertFindInSet("ς", UTF8String.fromString("ς"), UNICODE, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UNICODE, 0); + assertFindInSet("Σ", UTF8String.fromString("σ"), UNICODE, 0); + assertFindInSet("Σ", UTF8String.fromString("ς"), UNICODE, 0); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UNICODE, 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UNICODE_CI, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UNICODE_CI, 1); + assertFindInSet("σ", UTF8String.fromString("Σ"), UNICODE_CI, 1); + assertFindInSet("ς", UTF8String.fromString("σ"), UNICODE_CI, 1); + assertFindInSet("ς", UTF8String.fromString("ς"), UNICODE_CI, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UNICODE_CI, 1); + assertFindInSet("Σ", UTF8String.fromString("σ"), UNICODE_CI, 1); + assertFindInSet("Σ", UTF8String.fromString("ς"), UNICODE_CI, 1); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UNICODE_CI, 1); // Surrogate pairs. - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 0); - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 0); - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 0); - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 0); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 1); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 1); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 1); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 1); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 2); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 2); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 2); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 2); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 3); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 3); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 3); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 3); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_BINARY", 0); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_LCASE", 0); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE", 0); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE_CI", 0); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_BINARY", 1); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_LCASE", 1); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE", 1); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE_CI", 1); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UTF8_BINARY", 2); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UTF8_LCASE", 2); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UNICODE", 2); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UNICODE_CI", 2); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 0); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 0); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 0); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 0); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 1); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 1); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 1); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 1); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 0); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 1); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 0); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 1); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 3); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 3); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 3); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 1); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 2); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 2); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 2); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 2); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 0); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 2); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 0); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 2); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 0); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 0); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 0); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 0); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 1); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 1); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 1); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 1); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 2); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 2); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 2); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 2); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 3); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 3); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 3); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 3); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UTF8_BINARY, 0); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UTF8_LCASE, 0); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UNICODE, 0); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UNICODE_CI, 0); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UTF8_BINARY, 1); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UTF8_LCASE, 1); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UNICODE, 1); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UNICODE_CI, 1); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UTF8_BINARY, 2); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UTF8_LCASE, 2); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UNICODE, 2); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UNICODE_CI, 2); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 0); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 0); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 0); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 0); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 1); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 1); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 1); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 1); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 0); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 1); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 0); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 1); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 3); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 3); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 3); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 1); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 2); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 2); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 2); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 2); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 0); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 2); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 0); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 2); // Invalid UTF8 strings assertFindInSet("C", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UTF8_BINARY", 3); + UTF8_BINARY, 3); assertFindInSet("c", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UTF8_LCASE", 2); + UTF8_LCASE, 2); assertFindInSet("C", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UNICODE", 2); + UNICODE, 2); assertFindInSet("c", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UNICODE_CI", 2); + UNICODE_CI, 2); } /** @@ -1986,145 +1987,145 @@ private void assertStringReplace(String source, String search, String replace, @Test public void testStringReplace() throws SparkException { // Empty strings. - assertStringReplace("", "", "", "UTF8_BINARY", ""); - assertStringReplace("", "", "", "UTF8_LCASE", ""); - assertStringReplace("", "", "", "UNICODE", ""); - assertStringReplace("", "", "", "UNICODE_CI", ""); - assertStringReplace("abc", "", "", "UTF8_BINARY", "abc"); - assertStringReplace("abc", "", "", "UTF8_LCASE", "abc"); - assertStringReplace("abc", "", "", "UNICODE", "abc"); - assertStringReplace("abc", "", "", "UNICODE_CI", "abc"); - assertStringReplace("", "x", "", "UTF8_BINARY", ""); - assertStringReplace("", "x", "", "UTF8_LCASE", ""); - assertStringReplace("", "x", "", "UNICODE", ""); - assertStringReplace("", "x", "", "UNICODE_CI", ""); - assertStringReplace("", "", "x", "UTF8_BINARY", ""); - assertStringReplace("", "", "x", "UTF8_LCASE", ""); - assertStringReplace("", "", "x", "UNICODE", ""); - assertStringReplace("", "", "x", "UNICODE_CI", ""); - assertStringReplace("", "b", "x", "UTF8_BINARY", ""); - assertStringReplace("", "b", "x", "UTF8_LCASE", ""); - assertStringReplace("", "b", "x", "UNICODE", ""); - assertStringReplace("", "b", "x", "UNICODE_CI", ""); - assertStringReplace("abc", "b", "", "UTF8_BINARY", "ac"); - assertStringReplace("abc", "b", "", "UTF8_LCASE", "ac"); - assertStringReplace("abc", "b", "", "UNICODE", "ac"); - assertStringReplace("abc", "b", "", "UNICODE_CI", "ac"); - assertStringReplace("abc", "", "x", "UTF8_BINARY", "abc"); - assertStringReplace("abc", "", "x", "UTF8_LCASE", "abc"); - assertStringReplace("abc", "", "x", "UNICODE", "abc"); - assertStringReplace("abc", "", "x", "UNICODE_CI", "abc"); + assertStringReplace("", "", "", UTF8_BINARY, ""); + assertStringReplace("", "", "", UTF8_LCASE, ""); + assertStringReplace("", "", "", UNICODE, ""); + assertStringReplace("", "", "", UNICODE_CI, ""); + assertStringReplace("abc", "", "", UTF8_BINARY, "abc"); + assertStringReplace("abc", "", "", UTF8_LCASE, "abc"); + assertStringReplace("abc", "", "", UNICODE, "abc"); + assertStringReplace("abc", "", "", UNICODE_CI, "abc"); + assertStringReplace("", "x", "", UTF8_BINARY, ""); + assertStringReplace("", "x", "", UTF8_LCASE, ""); + assertStringReplace("", "x", "", UNICODE, ""); + assertStringReplace("", "x", "", UNICODE_CI, ""); + assertStringReplace("", "", "x", UTF8_BINARY, ""); + assertStringReplace("", "", "x", UTF8_LCASE, ""); + assertStringReplace("", "", "x", UNICODE, ""); + assertStringReplace("", "", "x", UNICODE_CI, ""); + assertStringReplace("", "b", "x", UTF8_BINARY, ""); + assertStringReplace("", "b", "x", UTF8_LCASE, ""); + assertStringReplace("", "b", "x", UNICODE, ""); + assertStringReplace("", "b", "x", UNICODE_CI, ""); + assertStringReplace("abc", "b", "", UTF8_BINARY, "ac"); + assertStringReplace("abc", "b", "", UTF8_LCASE, "ac"); + assertStringReplace("abc", "b", "", UNICODE, "ac"); + assertStringReplace("abc", "b", "", UNICODE_CI, "ac"); + assertStringReplace("abc", "", "x", UTF8_BINARY, "abc"); + assertStringReplace("abc", "", "x", UTF8_LCASE, "abc"); + assertStringReplace("abc", "", "x", UNICODE, "abc"); + assertStringReplace("abc", "", "x", UNICODE_CI, "abc"); // Basic tests. - assertStringReplace("replace", "pl", "", "UTF8_BINARY", "reace"); - assertStringReplace("replace", "pl", "", "UTF8_LCASE", "reace"); - assertStringReplace("replace", "pl", "", "UNICODE", "reace"); - assertStringReplace("replace", "pl", "", "UNICODE_CI", "reace"); - assertStringReplace("replace", "", "123", "UTF8_BINARY", "replace"); - assertStringReplace("replace", "", "123", "UTF8_LCASE", "replace"); - assertStringReplace("replace", "", "123", "UNICODE", "replace"); - assertStringReplace("replace", "", "123", "UNICODE_CI", "replace"); - assertStringReplace("abcabc", "b", "12", "UTF8_BINARY", "a12ca12c"); - assertStringReplace("abcabc", "b", "12", "UTF8_LCASE", "a12ca12c"); - assertStringReplace("abcabc", "b", "12", "UNICODE", "a12ca12c"); - assertStringReplace("abcabc", "b", "12", "UNICODE_CI", "a12ca12c"); - assertStringReplace("replace", "plx", "123", "UTF8_BINARY", "replace"); - assertStringReplace("replace", "plx", "123", "UTF8_LCASE", "replace"); - assertStringReplace("replace", "plx", "123", "UNICODE", "replace"); - assertStringReplace("replace", "plx", "123", "UNICODE_CI", "replace"); - assertStringReplace("Replace", "re", "", "UTF8_BINARY", "Replace"); - assertStringReplace("Replace", "re", "", "UTF8_LCASE", "place"); - assertStringReplace("Replace", "re", "", "UNICODE", "Replace"); - assertStringReplace("Replace", "re", "", "UNICODE_CI", "place"); - assertStringReplace("abcdabcd", "Bc", "", "UTF8_BINARY", "abcdabcd"); - assertStringReplace("abcdabcd", "Bc", "", "UTF8_LCASE", "adad"); - assertStringReplace("abcdabcd", "Bc", "", "UNICODE", "abcdabcd"); - assertStringReplace("abcdabcd", "Bc", "", "UNICODE_CI", "adad"); - assertStringReplace("AbcdabCd", "Bc", "", "UTF8_BINARY", "AbcdabCd"); - assertStringReplace("AbcdabCd", "Bc", "", "UTF8_LCASE", "Adad"); - assertStringReplace("AbcdabCd", "Bc", "", "UNICODE", "AbcdabCd"); - assertStringReplace("AbcdabCd", "Bc", "", "UNICODE_CI", "Adad"); + assertStringReplace("replace", "pl", "", UTF8_BINARY, "reace"); + assertStringReplace("replace", "pl", "", UTF8_LCASE, "reace"); + assertStringReplace("replace", "pl", "", UNICODE, "reace"); + assertStringReplace("replace", "pl", "", UNICODE_CI, "reace"); + assertStringReplace("replace", "", "123", UTF8_BINARY, "replace"); + assertStringReplace("replace", "", "123", UTF8_LCASE, "replace"); + assertStringReplace("replace", "", "123", UNICODE, "replace"); + assertStringReplace("replace", "", "123", UNICODE_CI, "replace"); + assertStringReplace("abcabc", "b", "12", UTF8_BINARY, "a12ca12c"); + assertStringReplace("abcabc", "b", "12", UTF8_LCASE, "a12ca12c"); + assertStringReplace("abcabc", "b", "12", UNICODE, "a12ca12c"); + assertStringReplace("abcabc", "b", "12", UNICODE_CI, "a12ca12c"); + assertStringReplace("replace", "plx", "123", UTF8_BINARY, "replace"); + assertStringReplace("replace", "plx", "123", UTF8_LCASE, "replace"); + assertStringReplace("replace", "plx", "123", UNICODE, "replace"); + assertStringReplace("replace", "plx", "123", UNICODE_CI, "replace"); + assertStringReplace("Replace", "re", "", UTF8_BINARY, "Replace"); + assertStringReplace("Replace", "re", "", UTF8_LCASE, "place"); + assertStringReplace("Replace", "re", "", UNICODE, "Replace"); + assertStringReplace("Replace", "re", "", UNICODE_CI, "place"); + assertStringReplace("abcdabcd", "Bc", "", UTF8_BINARY, "abcdabcd"); + assertStringReplace("abcdabcd", "Bc", "", UTF8_LCASE, "adad"); + assertStringReplace("abcdabcd", "Bc", "", UNICODE, "abcdabcd"); + assertStringReplace("abcdabcd", "Bc", "", UNICODE_CI, "adad"); + assertStringReplace("AbcdabCd", "Bc", "", UTF8_BINARY, "AbcdabCd"); + assertStringReplace("AbcdabCd", "Bc", "", UTF8_LCASE, "Adad"); + assertStringReplace("AbcdabCd", "Bc", "", UNICODE, "AbcdabCd"); + assertStringReplace("AbcdabCd", "Bc", "", UNICODE_CI, "Adad"); // Advanced tests. - assertStringReplace("abcdabcd", "bc", "", "UTF8_BINARY", "adad"); - assertStringReplace("r世eplace", "pl", "123", "UTF8_BINARY", "r世e123ace"); - assertStringReplace("世Replace", "re", "", "UTF8_BINARY", "世Replace"); - assertStringReplace("r世eplace", "pl", "xx", "UTF8_LCASE", "r世exxace"); - assertStringReplace("repl世ace", "PL", "AB", "UTF8_LCASE", "reAB世ace"); - assertStringReplace("re世place", "世", "x", "UTF8_LCASE", "rexplace"); - assertStringReplace("re世place", "plx", "123", "UNICODE", "re世place"); - assertStringReplace("replace世", "", "123", "UNICODE", "replace世"); - assertStringReplace("aBc世abc", "b", "12", "UNICODE", "aBc世a12c"); - assertStringReplace("aBc世abc", "b", "12", "UNICODE_CI", "a12c世a12c"); - assertStringReplace("a世Bcdabcd", "bC", "", "UNICODE_CI", "a世dad"); - assertStringReplace("repl世ace", "Pl", "", "UNICODE_CI", "re世ace"); + assertStringReplace("abcdabcd", "bc", "", UTF8_BINARY, "adad"); + assertStringReplace("r世eplace", "pl", "123", UTF8_BINARY, "r世e123ace"); + assertStringReplace("世Replace", "re", "", UTF8_BINARY, "世Replace"); + assertStringReplace("r世eplace", "pl", "xx", UTF8_LCASE, "r世exxace"); + assertStringReplace("repl世ace", "PL", "AB", UTF8_LCASE, "reAB世ace"); + assertStringReplace("re世place", "世", "x", UTF8_LCASE, "rexplace"); + assertStringReplace("re世place", "plx", "123", UNICODE, "re世place"); + assertStringReplace("replace世", "", "123", UNICODE, "replace世"); + assertStringReplace("aBc世abc", "b", "12", UNICODE, "aBc世a12c"); + assertStringReplace("aBc世abc", "b", "12", UNICODE_CI, "a12c世a12c"); + assertStringReplace("a世Bcdabcd", "bC", "", UNICODE_CI, "a世dad"); + assertStringReplace("repl世ace", "Pl", "", UNICODE_CI, "re世ace"); assertStringReplace("abcčšdabĆŠscd", "cs", "", "SR_CI_AI", "abcdabscd"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringReplace("abi̇12", "i", "X", "UNICODE_CI", "abi̇12"); - assertStringReplace("abi̇12", "\u0307", "X", "UNICODE_CI", "abi̇12"); - assertStringReplace("abi̇12", "İ", "X", "UNICODE_CI", "abX12"); - assertStringReplace("abİ12", "i", "X", "UNICODE_CI", "abİ12"); - assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", "UNICODE_CI", "xxxxxx"); - assertStringReplace("İi̇İi̇İi̇", "i", "x", "UNICODE_CI", "İi̇İi̇İi̇"); - assertStringReplace("abİo12i̇o", "i\u0307o", "xx", "UNICODE_CI", "abxx12xx"); - assertStringReplace("abi̇o12i̇o", "İo", "yy", "UNICODE_CI", "abyy12yy"); - assertStringReplace("abi̇12", "i", "X", "UTF8_LCASE", "abX\u030712"); // != UNICODE_CI - assertStringReplace("abi̇12", "\u0307", "X", "UTF8_LCASE", "abiX12"); // != UNICODE_CI - assertStringReplace("abi̇12", "İ", "X", "UTF8_LCASE", "abX12"); - assertStringReplace("abİ12", "i", "X", "UTF8_LCASE", "abİ12"); - assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", "UTF8_LCASE", "xxxxxx"); - assertStringReplace("İi̇İi̇İi̇", "i", "x", "UTF8_LCASE", + assertStringReplace("abi̇12", "i", "X", UNICODE_CI, "abi̇12"); + assertStringReplace("abi̇12", "\u0307", "X", UNICODE_CI, "abi̇12"); + assertStringReplace("abi̇12", "İ", "X", UNICODE_CI, "abX12"); + assertStringReplace("abİ12", "i", "X", UNICODE_CI, "abİ12"); + assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", UNICODE_CI, "xxxxxx"); + assertStringReplace("İi̇İi̇İi̇", "i", "x", UNICODE_CI, "İi̇İi̇İi̇"); + assertStringReplace("abİo12i̇o", "i\u0307o", "xx", UNICODE_CI, "abxx12xx"); + assertStringReplace("abi̇o12i̇o", "İo", "yy", UNICODE_CI, "abyy12yy"); + assertStringReplace("abi̇12", "i", "X", UTF8_LCASE, "abX\u030712"); // != UNICODE_CI + assertStringReplace("abi̇12", "\u0307", "X", UTF8_LCASE, "abiX12"); // != UNICODE_CI + assertStringReplace("abi̇12", "İ", "X", UTF8_LCASE, "abX12"); + assertStringReplace("abİ12", "i", "X", UTF8_LCASE, "abİ12"); + assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", UTF8_LCASE, "xxxxxx"); + assertStringReplace("İi̇İi̇İi̇", "i", "x", UTF8_LCASE, "İx\u0307İx\u0307İx\u0307"); // != UNICODE_CI - assertStringReplace("abİo12i̇o", "i\u0307o", "xx", "UTF8_LCASE", "abxx12xx"); - assertStringReplace("abi̇o12i̇o", "İo", "yy", "UTF8_LCASE", "abyy12yy"); + assertStringReplace("abİo12i̇o", "i\u0307o", "xx", UTF8_LCASE, "abxx12xx"); + assertStringReplace("abi̇o12i̇o", "İo", "yy", UTF8_LCASE, "abyy12yy"); // Conditional case mapping (e.g. Greek sigmas). - assertStringReplace("σ", "σ", "x", "UTF8_BINARY", "x"); - assertStringReplace("σ", "ς", "x", "UTF8_BINARY", "σ"); - assertStringReplace("σ", "Σ", "x", "UTF8_BINARY", "σ"); - assertStringReplace("ς", "σ", "x", "UTF8_BINARY", "ς"); - assertStringReplace("ς", "ς", "x", "UTF8_BINARY", "x"); - assertStringReplace("ς", "Σ", "x", "UTF8_BINARY", "ς"); - assertStringReplace("Σ", "σ", "x", "UTF8_BINARY", "Σ"); - assertStringReplace("Σ", "ς", "x", "UTF8_BINARY", "Σ"); - assertStringReplace("Σ", "Σ", "x", "UTF8_BINARY", "x"); - assertStringReplace("σ", "σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("σ", "ς", "x", "UTF8_LCASE", "x"); - assertStringReplace("σ", "Σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("ς", "σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("ς", "ς", "x", "UTF8_LCASE", "x"); - assertStringReplace("ς", "Σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("Σ", "σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("Σ", "ς", "x", "UTF8_LCASE", "x"); - assertStringReplace("Σ", "Σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("σ", "σ", "x", "UNICODE", "x"); - assertStringReplace("σ", "ς", "x", "UNICODE", "σ"); - assertStringReplace("σ", "Σ", "x", "UNICODE", "σ"); - assertStringReplace("ς", "σ", "x", "UNICODE", "ς"); - assertStringReplace("ς", "ς", "x", "UNICODE", "x"); - assertStringReplace("ς", "Σ", "x", "UNICODE", "ς"); - assertStringReplace("Σ", "σ", "x", "UNICODE", "Σ"); - assertStringReplace("Σ", "ς", "x", "UNICODE", "Σ"); - assertStringReplace("Σ", "Σ", "x", "UNICODE", "x"); - assertStringReplace("σ", "σ", "x", "UNICODE_CI", "x"); - assertStringReplace("σ", "ς", "x", "UNICODE_CI", "x"); - assertStringReplace("σ", "Σ", "x", "UNICODE_CI", "x"); - assertStringReplace("ς", "σ", "x", "UNICODE_CI", "x"); - assertStringReplace("ς", "ς", "x", "UNICODE_CI", "x"); - assertStringReplace("ς", "Σ", "x", "UNICODE_CI", "x"); - assertStringReplace("Σ", "σ", "x", "UNICODE_CI", "x"); - assertStringReplace("Σ", "ς", "x", "UNICODE_CI", "x"); - assertStringReplace("Σ", "Σ", "x", "UNICODE_CI", "x"); + assertStringReplace("σ", "σ", "x", UTF8_BINARY, "x"); + assertStringReplace("σ", "ς", "x", UTF8_BINARY, "σ"); + assertStringReplace("σ", "Σ", "x", UTF8_BINARY, "σ"); + assertStringReplace("ς", "σ", "x", UTF8_BINARY, "ς"); + assertStringReplace("ς", "ς", "x", UTF8_BINARY, "x"); + assertStringReplace("ς", "Σ", "x", UTF8_BINARY, "ς"); + assertStringReplace("Σ", "σ", "x", UTF8_BINARY, "Σ"); + assertStringReplace("Σ", "ς", "x", UTF8_BINARY, "Σ"); + assertStringReplace("Σ", "Σ", "x", UTF8_BINARY, "x"); + assertStringReplace("σ", "σ", "x", UTF8_LCASE, "x"); + assertStringReplace("σ", "ς", "x", UTF8_LCASE, "x"); + assertStringReplace("σ", "Σ", "x", UTF8_LCASE, "x"); + assertStringReplace("ς", "σ", "x", UTF8_LCASE, "x"); + assertStringReplace("ς", "ς", "x", UTF8_LCASE, "x"); + assertStringReplace("ς", "Σ", "x", UTF8_LCASE, "x"); + assertStringReplace("Σ", "σ", "x", UTF8_LCASE, "x"); + assertStringReplace("Σ", "ς", "x", UTF8_LCASE, "x"); + assertStringReplace("Σ", "Σ", "x", UTF8_LCASE, "x"); + assertStringReplace("σ", "σ", "x", UNICODE, "x"); + assertStringReplace("σ", "ς", "x", UNICODE, "σ"); + assertStringReplace("σ", "Σ", "x", UNICODE, "σ"); + assertStringReplace("ς", "σ", "x", UNICODE, "ς"); + assertStringReplace("ς", "ς", "x", UNICODE, "x"); + assertStringReplace("ς", "Σ", "x", UNICODE, "ς"); + assertStringReplace("Σ", "σ", "x", UNICODE, "Σ"); + assertStringReplace("Σ", "ς", "x", UNICODE, "Σ"); + assertStringReplace("Σ", "Σ", "x", UNICODE, "x"); + assertStringReplace("σ", "σ", "x", UNICODE_CI, "x"); + assertStringReplace("σ", "ς", "x", UNICODE_CI, "x"); + assertStringReplace("σ", "Σ", "x", UNICODE_CI, "x"); + assertStringReplace("ς", "σ", "x", UNICODE_CI, "x"); + assertStringReplace("ς", "ς", "x", UNICODE_CI, "x"); + assertStringReplace("ς", "Σ", "x", UNICODE_CI, "x"); + assertStringReplace("Σ", "σ", "x", UNICODE_CI, "x"); + assertStringReplace("Σ", "ς", "x", UNICODE_CI, "x"); + assertStringReplace("Σ", "Σ", "x", UNICODE_CI, "x"); // Surrogate pairs. - assertStringReplace("a🙃b", "a", "x", "UTF8_BINARY", "x🙃b"); - assertStringReplace("a🙃b", "b", "x", "UTF8_BINARY", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UTF8_BINARY", "axb"); - assertStringReplace("a🙃b", "b", "c", "UTF8_LCASE", "a🙃c"); - assertStringReplace("a🙃b", "b", "x", "UTF8_LCASE", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UTF8_LCASE", "axb"); - assertStringReplace("a🙃b", "b", "c", "UNICODE", "a🙃c"); - assertStringReplace("a🙃b", "b", "x", "UNICODE", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UNICODE", "axb"); - assertStringReplace("a🙃b", "b", "c", "UNICODE_CI", "a🙃c"); - assertStringReplace("a🙃b", "b", "x", "UNICODE_CI", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UNICODE_CI", "axb"); + assertStringReplace("a🙃b", "a", "x", UTF8_BINARY, "x🙃b"); + assertStringReplace("a🙃b", "b", "x", UTF8_BINARY, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UTF8_BINARY, "axb"); + assertStringReplace("a🙃b", "b", "c", UTF8_LCASE, "a🙃c"); + assertStringReplace("a🙃b", "b", "x", UTF8_LCASE, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UTF8_LCASE, "axb"); + assertStringReplace("a🙃b", "b", "c", UNICODE, "a🙃c"); + assertStringReplace("a🙃b", "b", "x", UNICODE, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UNICODE, "axb"); + assertStringReplace("a🙃b", "b", "c", UNICODE_CI, "a🙃c"); + assertStringReplace("a🙃b", "b", "x", UNICODE_CI, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UNICODE_CI, "axb"); } /** @@ -2145,293 +2146,293 @@ private void assertStringLocate(String substring, String string, int start, @Test public void testStringLocate() throws SparkException { // Empty strings. - assertStringLocate("", "", -1, "UTF8_BINARY", 1); - assertStringLocate("", "", -1, "UTF8_LCASE", 1); - assertStringLocate("", "", -1, "UNICODE", 1); - assertStringLocate("", "", -1, "UNICODE_CI", 1); - assertStringLocate("", "", 0, "UTF8_BINARY", 1); - assertStringLocate("", "", 0, "UTF8_LCASE", 1); - assertStringLocate("", "", 0, "UNICODE", 1); - assertStringLocate("", "", 0, "UNICODE_CI", 1); - assertStringLocate("", "", 1, "UTF8_BINARY", 1); - assertStringLocate("", "", 1, "UTF8_LCASE", 1); - assertStringLocate("", "", 1, "UNICODE", 1); - assertStringLocate("", "", 1, "UNICODE_CI", 1); - assertStringLocate("a", "", -1, "UTF8_BINARY", 0); - assertStringLocate("a", "", -1, "UTF8_LCASE", 0); - assertStringLocate("a", "", -1, "UNICODE", 0); - assertStringLocate("a", "", -1, "UNICODE_CI", 0); - assertStringLocate("a", "", 0, "UTF8_BINARY", 0); - assertStringLocate("a", "", 0, "UTF8_LCASE", 0); - assertStringLocate("a", "", 0, "UNICODE", 0); - assertStringLocate("a", "", 0, "UNICODE_CI", 0); - assertStringLocate("a", "", 1, "UTF8_BINARY", 0); - assertStringLocate("a", "", 1, "UTF8_LCASE", 0); - assertStringLocate("a", "", 1, "UNICODE", 0); - assertStringLocate("a", "", 1, "UNICODE_CI", 0); - assertStringLocate("", "x", -1, "UTF8_BINARY", 1); - assertStringLocate("", "x", -1, "UTF8_LCASE", 1); - assertStringLocate("", "x", -1, "UNICODE", 1); - assertStringLocate("", "x", -1, "UNICODE_CI", 1); - assertStringLocate("", "x", 0, "UTF8_BINARY", 1); - assertStringLocate("", "x", 0, "UTF8_LCASE", 1); - assertStringLocate("", "x", 0, "UNICODE", 1); - assertStringLocate("", "x", 0, "UNICODE_CI", 1); - assertStringLocate("", "x", 1, "UTF8_BINARY", 1); - assertStringLocate("", "x", 1, "UTF8_LCASE", 1); - assertStringLocate("", "x", 1, "UNICODE", 1); - assertStringLocate("", "x", 1, "UNICODE_CI", 1); + assertStringLocate("", "", -1, UTF8_BINARY, 1); + assertStringLocate("", "", -1, UTF8_LCASE, 1); + assertStringLocate("", "", -1, UNICODE, 1); + assertStringLocate("", "", -1, UNICODE_CI, 1); + assertStringLocate("", "", 0, UTF8_BINARY, 1); + assertStringLocate("", "", 0, UTF8_LCASE, 1); + assertStringLocate("", "", 0, UNICODE, 1); + assertStringLocate("", "", 0, UNICODE_CI, 1); + assertStringLocate("", "", 1, UTF8_BINARY, 1); + assertStringLocate("", "", 1, UTF8_LCASE, 1); + assertStringLocate("", "", 1, UNICODE, 1); + assertStringLocate("", "", 1, UNICODE_CI, 1); + assertStringLocate("a", "", -1, UTF8_BINARY, 0); + assertStringLocate("a", "", -1, UTF8_LCASE, 0); + assertStringLocate("a", "", -1, UNICODE, 0); + assertStringLocate("a", "", -1, UNICODE_CI, 0); + assertStringLocate("a", "", 0, UTF8_BINARY, 0); + assertStringLocate("a", "", 0, UTF8_LCASE, 0); + assertStringLocate("a", "", 0, UNICODE, 0); + assertStringLocate("a", "", 0, UNICODE_CI, 0); + assertStringLocate("a", "", 1, UTF8_BINARY, 0); + assertStringLocate("a", "", 1, UTF8_LCASE, 0); + assertStringLocate("a", "", 1, UNICODE, 0); + assertStringLocate("a", "", 1, UNICODE_CI, 0); + assertStringLocate("", "x", -1, UTF8_BINARY, 1); + assertStringLocate("", "x", -1, UTF8_LCASE, 1); + assertStringLocate("", "x", -1, UNICODE, 1); + assertStringLocate("", "x", -1, UNICODE_CI, 1); + assertStringLocate("", "x", 0, UTF8_BINARY, 1); + assertStringLocate("", "x", 0, UTF8_LCASE, 1); + assertStringLocate("", "x", 0, UNICODE, 1); + assertStringLocate("", "x", 0, UNICODE_CI, 1); + assertStringLocate("", "x", 1, UTF8_BINARY, 1); + assertStringLocate("", "x", 1, UTF8_LCASE, 1); + assertStringLocate("", "x", 1, UNICODE, 1); + assertStringLocate("", "x", 1, UNICODE_CI, 1); // Basic tests. - assertStringLocate("aa", "aaads", 1, "UTF8_BINARY", 1); - assertStringLocate("aa", "aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("aa", "aaads", 1, "UNICODE", 1); - assertStringLocate("aa", "aaads", 1, "UNICODE_CI", 1); - assertStringLocate("aa", "aaads", 2, "UTF8_BINARY", 2); - assertStringLocate("aa", "aaads", 2, "UTF8_LCASE", 2); - assertStringLocate("aa", "aaads", 2, "UNICODE", 2); - assertStringLocate("aa", "aaads", 2, "UNICODE_CI", 2); - assertStringLocate("aa", "aaads", 3, "UTF8_BINARY", 0); - assertStringLocate("aa", "aaads", 3, "UTF8_LCASE", 0); - assertStringLocate("aa", "aaads", 3, "UNICODE", 0); - assertStringLocate("aa", "aaads", 3, "UNICODE_CI", 0); - assertStringLocate("Aa", "aaads", 1, "UTF8_BINARY", 0); - assertStringLocate("Aa", "aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("Aa", "aaads", 1, "UNICODE", 0); - assertStringLocate("Aa", "aaads", 1, "UNICODE_CI", 1); - assertStringLocate("Aa", "aaads", 2, "UTF8_BINARY", 0); - assertStringLocate("Aa", "aaads", 2, "UTF8_LCASE", 2); - assertStringLocate("Aa", "aaads", 2, "UNICODE", 0); - assertStringLocate("Aa", "aaads", 2, "UNICODE_CI", 2); - assertStringLocate("Aa", "aaads", 3, "UTF8_BINARY", 0); - assertStringLocate("Aa", "aaads", 3, "UTF8_LCASE", 0); - assertStringLocate("Aa", "aaads", 3, "UNICODE", 0); - assertStringLocate("Aa", "aaads", 3, "UNICODE_CI", 0); - assertStringLocate("Aa", "aAads", 1, "UTF8_BINARY", 2); - assertStringLocate("Aa", "aAads", 1, "UTF8_LCASE", 1); - assertStringLocate("Aa", "aAads", 1, "UNICODE", 2); - assertStringLocate("Aa", "aAads", 1, "UNICODE_CI", 1); - assertStringLocate("AA", "aaads", 1, "UTF8_BINARY", 0); - assertStringLocate("AA", "aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("AA", "aaads", 1, "UNICODE", 0); - assertStringLocate("AA", "aaads", 1, "UNICODE_CI", 1); - assertStringLocate("aa", "aAads", 2, "UTF8_BINARY", 0); - assertStringLocate("aa", "aAads", 2, "UTF8_LCASE", 2); - assertStringLocate("aa", "aAads", 2, "UNICODE", 0); - assertStringLocate("aa", "aAads", 2, "UNICODE_CI", 2); - assertStringLocate("aa", "aaAds", 3, "UTF8_BINARY", 0); - assertStringLocate("aa", "aaAds", 3, "UTF8_LCASE", 0); - assertStringLocate("aa", "aaAds", 3, "UNICODE", 0); - assertStringLocate("aa", "aaAds", 3, "UNICODE_CI", 0); - assertStringLocate("abC", "abcabc", 1, "UTF8_BINARY", 0); - assertStringLocate("abC", "abcabc", 1, "UTF8_LCASE", 1); - assertStringLocate("abC", "abcabc", 1, "UNICODE", 0); - assertStringLocate("abC", "abcabc", 1, "UNICODE_CI", 1); - assertStringLocate("abC", "abCabc", 2, "UTF8_BINARY", 0); - assertStringLocate("abC", "abCabc", 2, "UTF8_LCASE", 4); - assertStringLocate("abC", "abCabc", 2, "UNICODE", 0); - assertStringLocate("abC", "abCabc", 2, "UNICODE_CI", 4); - assertStringLocate("abc", "abcabc", 1, "UTF8_BINARY", 1); - assertStringLocate("abc", "abcabc", 1, "UTF8_LCASE", 1); - assertStringLocate("abc", "abcabc", 1, "UNICODE", 1); - assertStringLocate("abc", "abcabc", 1, "UNICODE_CI", 1); - assertStringLocate("abc", "abcabc", 2, "UTF8_BINARY", 4); - assertStringLocate("abc", "abcabc", 2, "UTF8_LCASE", 4); - assertStringLocate("abc", "abcabc", 2, "UNICODE", 4); - assertStringLocate("abc", "abcabc", 2, "UNICODE_CI", 4); - assertStringLocate("abc", "abcabc", 3, "UTF8_BINARY", 4); - assertStringLocate("abc", "abcabc", 3, "UTF8_LCASE", 4); - assertStringLocate("abc", "abcabc", 3, "UNICODE", 4); - assertStringLocate("abc", "abcabc", 3, "UNICODE_CI", 4); - assertStringLocate("abc", "abcabc", 4, "UTF8_BINARY", 4); - assertStringLocate("abc", "abcabc", 4, "UTF8_LCASE", 4); - assertStringLocate("abc", "abcabc", 4, "UNICODE", 4); - assertStringLocate("abc", "abcabc", 4, "UNICODE_CI", 4); - assertStringLocate("aa", "Aaads", 1, "UTF8_BINARY", 2); - assertStringLocate("aa", "Aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("aa", "Aaads", 1, "UNICODE", 2); - assertStringLocate("aa", "Aaads", 1, "UNICODE_CI", 1); + assertStringLocate("aa", "aaads", 1, UTF8_BINARY, 1); + assertStringLocate("aa", "aaads", 1, UTF8_LCASE, 1); + assertStringLocate("aa", "aaads", 1, UNICODE, 1); + assertStringLocate("aa", "aaads", 1, UNICODE_CI, 1); + assertStringLocate("aa", "aaads", 2, UTF8_BINARY, 2); + assertStringLocate("aa", "aaads", 2, UTF8_LCASE, 2); + assertStringLocate("aa", "aaads", 2, UNICODE, 2); + assertStringLocate("aa", "aaads", 2, UNICODE_CI, 2); + assertStringLocate("aa", "aaads", 3, UTF8_BINARY, 0); + assertStringLocate("aa", "aaads", 3, UTF8_LCASE, 0); + assertStringLocate("aa", "aaads", 3, UNICODE, 0); + assertStringLocate("aa", "aaads", 3, UNICODE_CI, 0); + assertStringLocate("Aa", "aaads", 1, UTF8_BINARY, 0); + assertStringLocate("Aa", "aaads", 1, UTF8_LCASE, 1); + assertStringLocate("Aa", "aaads", 1, UNICODE, 0); + assertStringLocate("Aa", "aaads", 1, UNICODE_CI, 1); + assertStringLocate("Aa", "aaads", 2, UTF8_BINARY, 0); + assertStringLocate("Aa", "aaads", 2, UTF8_LCASE, 2); + assertStringLocate("Aa", "aaads", 2, UNICODE, 0); + assertStringLocate("Aa", "aaads", 2, UNICODE_CI, 2); + assertStringLocate("Aa", "aaads", 3, UTF8_BINARY, 0); + assertStringLocate("Aa", "aaads", 3, UTF8_LCASE, 0); + assertStringLocate("Aa", "aaads", 3, UNICODE, 0); + assertStringLocate("Aa", "aaads", 3, UNICODE_CI, 0); + assertStringLocate("Aa", "aAads", 1, UTF8_BINARY, 2); + assertStringLocate("Aa", "aAads", 1, UTF8_LCASE, 1); + assertStringLocate("Aa", "aAads", 1, UNICODE, 2); + assertStringLocate("Aa", "aAads", 1, UNICODE_CI, 1); + assertStringLocate("AA", "aaads", 1, UTF8_BINARY, 0); + assertStringLocate("AA", "aaads", 1, UTF8_LCASE, 1); + assertStringLocate("AA", "aaads", 1, UNICODE, 0); + assertStringLocate("AA", "aaads", 1, UNICODE_CI, 1); + assertStringLocate("aa", "aAads", 2, UTF8_BINARY, 0); + assertStringLocate("aa", "aAads", 2, UTF8_LCASE, 2); + assertStringLocate("aa", "aAads", 2, UNICODE, 0); + assertStringLocate("aa", "aAads", 2, UNICODE_CI, 2); + assertStringLocate("aa", "aaAds", 3, UTF8_BINARY, 0); + assertStringLocate("aa", "aaAds", 3, UTF8_LCASE, 0); + assertStringLocate("aa", "aaAds", 3, UNICODE, 0); + assertStringLocate("aa", "aaAds", 3, UNICODE_CI, 0); + assertStringLocate("abC", "abcabc", 1, UTF8_BINARY, 0); + assertStringLocate("abC", "abcabc", 1, UTF8_LCASE, 1); + assertStringLocate("abC", "abcabc", 1, UNICODE, 0); + assertStringLocate("abC", "abcabc", 1, UNICODE_CI, 1); + assertStringLocate("abC", "abCabc", 2, UTF8_BINARY, 0); + assertStringLocate("abC", "abCabc", 2, UTF8_LCASE, 4); + assertStringLocate("abC", "abCabc", 2, UNICODE, 0); + assertStringLocate("abC", "abCabc", 2, UNICODE_CI, 4); + assertStringLocate("abc", "abcabc", 1, UTF8_BINARY, 1); + assertStringLocate("abc", "abcabc", 1, UTF8_LCASE, 1); + assertStringLocate("abc", "abcabc", 1, UNICODE, 1); + assertStringLocate("abc", "abcabc", 1, UNICODE_CI, 1); + assertStringLocate("abc", "abcabc", 2, UTF8_BINARY, 4); + assertStringLocate("abc", "abcabc", 2, UTF8_LCASE, 4); + assertStringLocate("abc", "abcabc", 2, UNICODE, 4); + assertStringLocate("abc", "abcabc", 2, UNICODE_CI, 4); + assertStringLocate("abc", "abcabc", 3, UTF8_BINARY, 4); + assertStringLocate("abc", "abcabc", 3, UTF8_LCASE, 4); + assertStringLocate("abc", "abcabc", 3, UNICODE, 4); + assertStringLocate("abc", "abcabc", 3, UNICODE_CI, 4); + assertStringLocate("abc", "abcabc", 4, UTF8_BINARY, 4); + assertStringLocate("abc", "abcabc", 4, UTF8_LCASE, 4); + assertStringLocate("abc", "abcabc", 4, UNICODE, 4); + assertStringLocate("abc", "abcabc", 4, UNICODE_CI, 4); + assertStringLocate("aa", "Aaads", 1, UTF8_BINARY, 2); + assertStringLocate("aa", "Aaads", 1, UTF8_LCASE, 1); + assertStringLocate("aa", "Aaads", 1, UNICODE, 2); + assertStringLocate("aa", "Aaads", 1, UNICODE_CI, 1); assertStringLocate("ćČ", "CćČČćCČĆČcČcććČč", 3, "SR", 14); assertStringLocate("ćČ", "CćČČćCČĆČcČcććČč", 3, "SR_CI_AI", 3); // Advanced tests. - assertStringLocate("界x", "test大千世界X大千世界", 1, "UTF8_BINARY", 0); - assertStringLocate("界X", "test大千世界X大千世界", 1, "UTF8_BINARY", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UTF8_BINARY", 13); - assertStringLocate("界x", "test大千世界X大千世界", 1, "UTF8_LCASE", 8); - assertStringLocate("界X", "test大千世界Xtest大千世界", 1, "UTF8_LCASE", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UTF8_LCASE", 13); - assertStringLocate("大千", "test大千世界大千世界", 1, "UTF8_LCASE", 5); - assertStringLocate("大千", "test大千世界大千世界", 9, "UTF8_LCASE", 9); - assertStringLocate("大千", "大千世界大千世界", 1, "UTF8_LCASE", 1); - assertStringLocate("界x", "test大千世界X大千世界", 1, "UNICODE", 0); - assertStringLocate("界X", "test大千世界X大千世界", 1, "UNICODE", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UNICODE", 13); - assertStringLocate("界x", "test大千世界X大千世界", 1, "UNICODE_CI", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UNICODE_CI", 13); - assertStringLocate("大千", "test大千世界大千世界", 1, "UNICODE_CI", 5); - assertStringLocate("大千", "test大千世界大千世界", 9, "UNICODE_CI", 9); - assertStringLocate("大千", "大千世界大千世界", 1, "UNICODE_CI", 1); + assertStringLocate("界x", "test大千世界X大千世界", 1, UTF8_BINARY, 0); + assertStringLocate("界X", "test大千世界X大千世界", 1, UTF8_BINARY, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UTF8_BINARY, 13); + assertStringLocate("界x", "test大千世界X大千世界", 1, UTF8_LCASE, 8); + assertStringLocate("界X", "test大千世界Xtest大千世界", 1, UTF8_LCASE, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UTF8_LCASE, 13); + assertStringLocate("大千", "test大千世界大千世界", 1, UTF8_LCASE, 5); + assertStringLocate("大千", "test大千世界大千世界", 9, UTF8_LCASE, 9); + assertStringLocate("大千", "大千世界大千世界", 1, UTF8_LCASE, 1); + assertStringLocate("界x", "test大千世界X大千世界", 1, UNICODE, 0); + assertStringLocate("界X", "test大千世界X大千世界", 1, UNICODE, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UNICODE, 13); + assertStringLocate("界x", "test大千世界X大千世界", 1, UNICODE_CI, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UNICODE_CI, 13); + assertStringLocate("大千", "test大千世界大千世界", 1, UNICODE_CI, 5); + assertStringLocate("大千", "test大千世界大千世界", 9, UNICODE_CI, 9); + assertStringLocate("大千", "大千世界大千世界", 1, UNICODE_CI, 1); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringLocate("\u0307", "i\u0307", 1, "UTF8_BINARY", 2); - assertStringLocate("\u0307", "İ", 1, "UTF8_LCASE", 0); // != UTF8_BINARY - assertStringLocate("i", "i\u0307", 1, "UNICODE_CI", 0); - assertStringLocate("\u0307", "i\u0307", 1, "UNICODE_CI", 0); - assertStringLocate("i\u0307", "i", 1, "UNICODE_CI", 0); - assertStringLocate("İ", "i\u0307", 1, "UNICODE_CI", 1); - assertStringLocate("İ", "i", 1, "UNICODE_CI", 0); - assertStringLocate("i", "i\u0307", 1, "UTF8_LCASE", 1); // != UNICODE_CI - assertStringLocate("\u0307", "i\u0307", 1, "UTF8_LCASE", 2); // != UNICODE_CI - assertStringLocate("i\u0307", "i", 1, "UTF8_LCASE", 0); - assertStringLocate("İ", "i\u0307", 1, "UTF8_LCASE", 1); - assertStringLocate("İ", "i", 1, "UTF8_LCASE", 0); - assertStringLocate("i\u0307o", "İo世界大千世界", 1, "UNICODE_CI", 1); - assertStringLocate("i\u0307o", "大千İo世界大千世界", 1, "UNICODE_CI", 3); - assertStringLocate("i\u0307o", "世界İo大千世界大千İo", 4, "UNICODE_CI", 11); - assertStringLocate("İo", "i̇o世界大千世界", 1, "UNICODE_CI", 1); - assertStringLocate("İo", "大千i̇o世界大千世界", 1, "UNICODE_CI", 3); - assertStringLocate("İo", "世界i̇o大千世界大千i̇o", 4, "UNICODE_CI", 12); + assertStringLocate("\u0307", "i\u0307", 1, UTF8_BINARY, 2); + assertStringLocate("\u0307", "İ", 1, UTF8_LCASE, 0); // != UTF8_BINARY + assertStringLocate("i", "i\u0307", 1, UNICODE_CI, 0); + assertStringLocate("\u0307", "i\u0307", 1, UNICODE_CI, 0); + assertStringLocate("i\u0307", "i", 1, UNICODE_CI, 0); + assertStringLocate("İ", "i\u0307", 1, UNICODE_CI, 1); + assertStringLocate("İ", "i", 1, UNICODE_CI, 0); + assertStringLocate("i", "i\u0307", 1, UTF8_LCASE, 1); // != UNICODE_CI + assertStringLocate("\u0307", "i\u0307", 1, UTF8_LCASE, 2); // != UNICODE_CI + assertStringLocate("i\u0307", "i", 1, UTF8_LCASE, 0); + assertStringLocate("İ", "i\u0307", 1, UTF8_LCASE, 1); + assertStringLocate("İ", "i", 1, UTF8_LCASE, 0); + assertStringLocate("i\u0307o", "İo世界大千世界", 1, UNICODE_CI, 1); + assertStringLocate("i\u0307o", "大千İo世界大千世界", 1, UNICODE_CI, 3); + assertStringLocate("i\u0307o", "世界İo大千世界大千İo", 4, UNICODE_CI, 11); + assertStringLocate("İo", "i̇o世界大千世界", 1, UNICODE_CI, 1); + assertStringLocate("İo", "大千i̇o世界大千世界", 1, UNICODE_CI, 3); + assertStringLocate("İo", "世界i̇o大千世界大千i̇o", 4, UNICODE_CI, 12); // Conditional case mapping (e.g. Greek sigmas). - assertStringLocate("σ", "σ", 1, "UTF8_BINARY", 1); - assertStringLocate("σ", "ς", 1, "UTF8_BINARY", 0); - assertStringLocate("σ", "Σ", 1, "UTF8_BINARY", 0); - assertStringLocate("ς", "σ", 1, "UTF8_BINARY", 0); - assertStringLocate("ς", "ς", 1, "UTF8_BINARY", 1); - assertStringLocate("ς", "Σ", 1, "UTF8_BINARY", 0); - assertStringLocate("Σ", "σ", 1, "UTF8_BINARY", 0); - assertStringLocate("Σ", "ς", 1, "UTF8_BINARY", 0); - assertStringLocate("Σ", "Σ", 1, "UTF8_BINARY", 1); - assertStringLocate("σ", "σ", 1, "UTF8_LCASE", 1); - assertStringLocate("σ", "ς", 1, "UTF8_LCASE", 1); - assertStringLocate("σ", "Σ", 1, "UTF8_LCASE", 1); - assertStringLocate("ς", "σ", 1, "UTF8_LCASE", 1); - assertStringLocate("ς", "ς", 1, "UTF8_LCASE", 1); - assertStringLocate("ς", "Σ", 1, "UTF8_LCASE", 1); - assertStringLocate("Σ", "σ", 1, "UTF8_LCASE", 1); - assertStringLocate("Σ", "ς", 1, "UTF8_LCASE", 1); - assertStringLocate("Σ", "Σ", 1, "UTF8_LCASE", 1); - assertStringLocate("σ", "σ", 1, "UNICODE", 1); - assertStringLocate("σ", "ς", 1, "UNICODE", 0); - assertStringLocate("σ", "Σ", 1, "UNICODE", 0); - assertStringLocate("ς", "σ", 1, "UNICODE", 0); - assertStringLocate("ς", "ς", 1, "UNICODE", 1); - assertStringLocate("ς", "Σ", 1, "UNICODE", 0); - assertStringLocate("Σ", "σ", 1, "UNICODE", 0); - assertStringLocate("Σ", "ς", 1, "UNICODE", 0); - assertStringLocate("Σ", "Σ", 1, "UNICODE", 1); - assertStringLocate("σ", "σ", 1, "UNICODE_CI", 1); - assertStringLocate("σ", "ς", 1, "UNICODE_CI", 1); - assertStringLocate("σ", "Σ", 1, "UNICODE_CI", 1); - assertStringLocate("ς", "σ", 1, "UNICODE_CI", 1); - assertStringLocate("ς", "ς", 1, "UNICODE_CI", 1); - assertStringLocate("ς", "Σ", 1, "UNICODE_CI", 1); - assertStringLocate("Σ", "σ", 1, "UNICODE_CI", 1); - assertStringLocate("Σ", "ς", 1, "UNICODE_CI", 1); - assertStringLocate("Σ", "Σ", 1, "UNICODE_CI", 1); + assertStringLocate("σ", "σ", 1, UTF8_BINARY, 1); + assertStringLocate("σ", "ς", 1, UTF8_BINARY, 0); + assertStringLocate("σ", "Σ", 1, UTF8_BINARY, 0); + assertStringLocate("ς", "σ", 1, UTF8_BINARY, 0); + assertStringLocate("ς", "ς", 1, UTF8_BINARY, 1); + assertStringLocate("ς", "Σ", 1, UTF8_BINARY, 0); + assertStringLocate("Σ", "σ", 1, UTF8_BINARY, 0); + assertStringLocate("Σ", "ς", 1, UTF8_BINARY, 0); + assertStringLocate("Σ", "Σ", 1, UTF8_BINARY, 1); + assertStringLocate("σ", "σ", 1, UTF8_LCASE, 1); + assertStringLocate("σ", "ς", 1, UTF8_LCASE, 1); + assertStringLocate("σ", "Σ", 1, UTF8_LCASE, 1); + assertStringLocate("ς", "σ", 1, UTF8_LCASE, 1); + assertStringLocate("ς", "ς", 1, UTF8_LCASE, 1); + assertStringLocate("ς", "Σ", 1, UTF8_LCASE, 1); + assertStringLocate("Σ", "σ", 1, UTF8_LCASE, 1); + assertStringLocate("Σ", "ς", 1, UTF8_LCASE, 1); + assertStringLocate("Σ", "Σ", 1, UTF8_LCASE, 1); + assertStringLocate("σ", "σ", 1, UNICODE, 1); + assertStringLocate("σ", "ς", 1, UNICODE, 0); + assertStringLocate("σ", "Σ", 1, UNICODE, 0); + assertStringLocate("ς", "σ", 1, UNICODE, 0); + assertStringLocate("ς", "ς", 1, UNICODE, 1); + assertStringLocate("ς", "Σ", 1, UNICODE, 0); + assertStringLocate("Σ", "σ", 1, UNICODE, 0); + assertStringLocate("Σ", "ς", 1, UNICODE, 0); + assertStringLocate("Σ", "Σ", 1, UNICODE, 1); + assertStringLocate("σ", "σ", 1, UNICODE_CI, 1); + assertStringLocate("σ", "ς", 1, UNICODE_CI, 1); + assertStringLocate("σ", "Σ", 1, UNICODE_CI, 1); + assertStringLocate("ς", "σ", 1, UNICODE_CI, 1); + assertStringLocate("ς", "ς", 1, UNICODE_CI, 1); + assertStringLocate("ς", "Σ", 1, UNICODE_CI, 1); + assertStringLocate("Σ", "σ", 1, UNICODE_CI, 1); + assertStringLocate("Σ", "ς", 1, UNICODE_CI, 1); + assertStringLocate("Σ", "Σ", 1, UNICODE_CI, 1); // Surrogate pairs. - assertStringLocate("a", "a🙃b", 1, "UTF8_BINARY", 1); - assertStringLocate("a", "a🙃b", 1, "UTF8_LCASE", 1); - assertStringLocate("a", "a🙃b", 1, "UNICODE", 1); - assertStringLocate("a", "a🙃b", 1, "UNICODE_CI", 1); - assertStringLocate("a", "a🙃b", 2, "UTF8_BINARY", 0); - assertStringLocate("a", "a🙃b", 2, "UTF8_LCASE", 0); - assertStringLocate("a", "a🙃b", 2, "UNICODE", 0); - assertStringLocate("a", "a🙃b", 2, "UNICODE_CI", 0); - assertStringLocate("a", "a🙃b", 3, "UTF8_BINARY", 0); - assertStringLocate("a", "a🙃b", 3, "UTF8_LCASE", 0); - assertStringLocate("a", "a🙃b", 3, "UNICODE", 0); - assertStringLocate("a", "a🙃b", 3, "UNICODE_CI", 0); - assertStringLocate("🙃", "a🙃b", 1, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃b", 1, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃b", 1, "UNICODE", 2); - assertStringLocate("🙃", "a🙃b", 1, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃b", 2, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃b", 2, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃b", 2, "UNICODE", 2); - assertStringLocate("🙃", "a🙃b", 2, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃b", 3, "UTF8_BINARY", 0); - assertStringLocate("🙃", "a🙃b", 3, "UTF8_LCASE", 0); - assertStringLocate("🙃", "a🙃b", 3, "UNICODE", 0); - assertStringLocate("🙃", "a🙃b", 3, "UNICODE_CI", 0); - assertStringLocate("b", "a🙃b", 1, "UTF8_BINARY", 3); - assertStringLocate("b", "a🙃b", 1, "UTF8_LCASE", 3); - assertStringLocate("b", "a🙃b", 1, "UNICODE", 3); - assertStringLocate("b", "a🙃b", 1, "UNICODE_CI", 3); - assertStringLocate("b", "a🙃b", 2, "UTF8_BINARY", 3); - assertStringLocate("b", "a🙃b", 2, "UTF8_LCASE", 3); - assertStringLocate("b", "a🙃b", 2, "UNICODE", 3); - assertStringLocate("b", "a🙃b", 2, "UNICODE_CI", 3); - assertStringLocate("b", "a🙃b", 3, "UTF8_BINARY", 3); - assertStringLocate("b", "a🙃b", 3, "UTF8_LCASE", 3); - assertStringLocate("b", "a🙃b", 3, "UNICODE", 3); - assertStringLocate("b", "a🙃b", 3, "UNICODE_CI", 3); - assertStringLocate("🙃", "a🙃🙃b", 1, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃🙃b", 1, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃🙃b", 1, "UNICODE", 2); - assertStringLocate("🙃", "a🙃🙃b", 1, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UNICODE", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃🙃b", 3, "UTF8_BINARY", 3); - assertStringLocate("🙃", "a🙃🙃b", 3, "UTF8_LCASE", 3); - assertStringLocate("🙃", "a🙃🙃b", 3, "UNICODE", 3); - assertStringLocate("🙃", "a🙃🙃b", 3, "UNICODE_CI", 3); - assertStringLocate("🙃", "a🙃🙃b", 4, "UTF8_BINARY", 0); - assertStringLocate("🙃", "a🙃🙃b", 4, "UTF8_LCASE", 0); - assertStringLocate("🙃", "a🙃🙃b", 4, "UNICODE", 0); - assertStringLocate("🙃", "a🙃🙃b", 4, "UNICODE_CI", 0); - assertStringLocate("b", "a🙃🙃b", 1, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 1, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 1, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 1, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃x🙃b", 1, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 1, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 1, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 1, "UNICODE_CI", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UNICODE_CI", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UNICODE_CI", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UNICODE_CI", 5); + assertStringLocate("a", "a🙃b", 1, UTF8_BINARY, 1); + assertStringLocate("a", "a🙃b", 1, UTF8_LCASE, 1); + assertStringLocate("a", "a🙃b", 1, UNICODE, 1); + assertStringLocate("a", "a🙃b", 1, UNICODE_CI, 1); + assertStringLocate("a", "a🙃b", 2, UTF8_BINARY, 0); + assertStringLocate("a", "a🙃b", 2, UTF8_LCASE, 0); + assertStringLocate("a", "a🙃b", 2, UNICODE, 0); + assertStringLocate("a", "a🙃b", 2, UNICODE_CI, 0); + assertStringLocate("a", "a🙃b", 3, UTF8_BINARY, 0); + assertStringLocate("a", "a🙃b", 3, UTF8_LCASE, 0); + assertStringLocate("a", "a🙃b", 3, UNICODE, 0); + assertStringLocate("a", "a🙃b", 3, UNICODE_CI, 0); + assertStringLocate("🙃", "a🙃b", 1, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃b", 1, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃b", 1, UNICODE, 2); + assertStringLocate("🙃", "a🙃b", 1, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃b", 2, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃b", 2, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃b", 2, UNICODE, 2); + assertStringLocate("🙃", "a🙃b", 2, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃b", 3, UTF8_BINARY, 0); + assertStringLocate("🙃", "a🙃b", 3, UTF8_LCASE, 0); + assertStringLocate("🙃", "a🙃b", 3, UNICODE, 0); + assertStringLocate("🙃", "a🙃b", 3, UNICODE_CI, 0); + assertStringLocate("b", "a🙃b", 1, UTF8_BINARY, 3); + assertStringLocate("b", "a🙃b", 1, UTF8_LCASE, 3); + assertStringLocate("b", "a🙃b", 1, UNICODE, 3); + assertStringLocate("b", "a🙃b", 1, UNICODE_CI, 3); + assertStringLocate("b", "a🙃b", 2, UTF8_BINARY, 3); + assertStringLocate("b", "a🙃b", 2, UTF8_LCASE, 3); + assertStringLocate("b", "a🙃b", 2, UNICODE, 3); + assertStringLocate("b", "a🙃b", 2, UNICODE_CI, 3); + assertStringLocate("b", "a🙃b", 3, UTF8_BINARY, 3); + assertStringLocate("b", "a🙃b", 3, UTF8_LCASE, 3); + assertStringLocate("b", "a🙃b", 3, UNICODE, 3); + assertStringLocate("b", "a🙃b", 3, UNICODE_CI, 3); + assertStringLocate("🙃", "a🙃🙃b", 1, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃🙃b", 1, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃🙃b", 1, UNICODE, 2); + assertStringLocate("🙃", "a🙃🙃b", 1, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UNICODE, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃🙃b", 3, UTF8_BINARY, 3); + assertStringLocate("🙃", "a🙃🙃b", 3, UTF8_LCASE, 3); + assertStringLocate("🙃", "a🙃🙃b", 3, UNICODE, 3); + assertStringLocate("🙃", "a🙃🙃b", 3, UNICODE_CI, 3); + assertStringLocate("🙃", "a🙃🙃b", 4, UTF8_BINARY, 0); + assertStringLocate("🙃", "a🙃🙃b", 4, UTF8_LCASE, 0); + assertStringLocate("🙃", "a🙃🙃b", 4, UNICODE, 0); + assertStringLocate("🙃", "a🙃🙃b", 4, UNICODE_CI, 0); + assertStringLocate("b", "a🙃🙃b", 1, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 1, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 1, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 1, UNICODE_CI, 4); + assertStringLocate("b", "a🙃🙃b", 2, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 2, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 2, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 2, UNICODE_CI, 4); + assertStringLocate("b", "a🙃🙃b", 3, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 3, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 3, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 3, UNICODE_CI, 4); + assertStringLocate("b", "a🙃🙃b", 4, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 4, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 4, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 4, UNICODE_CI, 4); + assertStringLocate("b", "a🙃x🙃b", 1, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 1, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 1, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 1, UNICODE_CI, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UNICODE_CI, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UNICODE_CI, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UNICODE_CI, 5); // Out of bounds test cases. - assertStringLocate("a", "asd", 4, "UTF8_BINARY", 0); - assertStringLocate("a", "asd", 4, "UTF8_LCASE", 0); - assertStringLocate("a", "asd", 4, "UNICODE", 0); - assertStringLocate("a", "asd", 4, "UNICODE_CI", 0); - assertStringLocate("a", "asd", 100, "UTF8_BINARY", 0); - assertStringLocate("a", "asd", 100, "UTF8_LCASE", 0); - assertStringLocate("a", "asd", 100, "UNICODE", 0); - assertStringLocate("a", "asd", 100, "UNICODE_CI", 0); - assertStringLocate("a", "🙃🙃", 4, "UTF8_BINARY", 0); - assertStringLocate("a", "🙃🙃", 4, "UTF8_LCASE", 0); - assertStringLocate("a", "🙃🙃", 4, "UNICODE", 0); - assertStringLocate("a", "🙃🙃", 4, "UNICODE_CI", 0); - assertStringLocate("", "asd", 100, "UTF8_BINARY", 1); - assertStringLocate("", "asd", 100, "UTF8_LCASE", 1); - assertStringLocate("", "asd", 100, "UNICODE", 1); - assertStringLocate("", "asd", 100, "UNICODE_CI", 1); - assertStringLocate("asd", "", 100, "UTF8_BINARY", 0); - assertStringLocate("asd", "", 100, "UTF8_LCASE", 0); - assertStringLocate("asd", "", 100, "UNICODE", 0); - assertStringLocate("asd", "", 100, "UNICODE_CI", 0); + assertStringLocate("a", "asd", 4, UTF8_BINARY, 0); + assertStringLocate("a", "asd", 4, UTF8_LCASE, 0); + assertStringLocate("a", "asd", 4, UNICODE, 0); + assertStringLocate("a", "asd", 4, UNICODE_CI, 0); + assertStringLocate("a", "asd", 100, UTF8_BINARY, 0); + assertStringLocate("a", "asd", 100, UTF8_LCASE, 0); + assertStringLocate("a", "asd", 100, UNICODE, 0); + assertStringLocate("a", "asd", 100, UNICODE_CI, 0); + assertStringLocate("a", "🙃🙃", 4, UTF8_BINARY, 0); + assertStringLocate("a", "🙃🙃", 4, UTF8_LCASE, 0); + assertStringLocate("a", "🙃🙃", 4, UNICODE, 0); + assertStringLocate("a", "🙃🙃", 4, UNICODE_CI, 0); + assertStringLocate("", "asd", 100, UTF8_BINARY, 1); + assertStringLocate("", "asd", 100, UTF8_LCASE, 1); + assertStringLocate("", "asd", 100, UNICODE, 1); + assertStringLocate("", "asd", 100, UNICODE_CI, 1); + assertStringLocate("asd", "", 100, UTF8_BINARY, 0); + assertStringLocate("asd", "", 100, UTF8_LCASE, 0); + assertStringLocate("asd", "", 100, UNICODE, 0); + assertStringLocate("asd", "", 100, UNICODE_CI, 0); } /** @@ -2450,292 +2451,292 @@ private void assertSubstringIndex(String string, String delimiter, int count, @Test public void testSubstringIndex() throws SparkException { // Empty strings. - assertSubstringIndex("", "", 0, "UTF8_BINARY", ""); - assertSubstringIndex("", "", 0, "UTF8_LCASE", ""); - assertSubstringIndex("", "", 0, "UNICODE", ""); - assertSubstringIndex("", "", 0, "UNICODE_CI", ""); - assertSubstringIndex("", "", 1, "UTF8_BINARY", ""); - assertSubstringIndex("", "", 1, "UTF8_LCASE", ""); - assertSubstringIndex("", "", 1, "UNICODE", ""); - assertSubstringIndex("", "", 1, "UNICODE_CI", ""); - assertSubstringIndex("", "", -1, "UTF8_BINARY", ""); - assertSubstringIndex("", "", -1, "UTF8_LCASE", ""); - assertSubstringIndex("", "", -1, "UNICODE", ""); - assertSubstringIndex("", "", -1, "UNICODE_CI", ""); - assertSubstringIndex("", "x", 0, "UTF8_BINARY", ""); - assertSubstringIndex("", "x", 0, "UTF8_LCASE", ""); - assertSubstringIndex("", "x", 0, "UNICODE", ""); - assertSubstringIndex("", "x", 0, "UNICODE_CI", ""); - assertSubstringIndex("", "x", 1, "UTF8_BINARY", ""); - assertSubstringIndex("", "x", 1, "UTF8_LCASE", ""); - assertSubstringIndex("", "x", 1, "UNICODE", ""); - assertSubstringIndex("", "x", 1, "UNICODE_CI", ""); - assertSubstringIndex("", "x", -1, "UTF8_BINARY", ""); - assertSubstringIndex("", "x", -1, "UTF8_LCASE", ""); - assertSubstringIndex("", "x", -1, "UNICODE", ""); - assertSubstringIndex("", "x", -1, "UNICODE_CI", ""); - assertSubstringIndex("abc", "", 0, "UTF8_BINARY", ""); - assertSubstringIndex("abc", "", 0, "UTF8_LCASE", ""); - assertSubstringIndex("abc", "", 0, "UNICODE", ""); - assertSubstringIndex("abc", "", 0, "UNICODE_CI", ""); - assertSubstringIndex("abc", "", 1, "UTF8_BINARY", ""); - assertSubstringIndex("abc", "", 1, "UTF8_LCASE", ""); - assertSubstringIndex("abc", "", 1, "UNICODE", ""); - assertSubstringIndex("abc", "", 1, "UNICODE_CI", ""); - assertSubstringIndex("abc", "", -1, "UTF8_BINARY", ""); - assertSubstringIndex("abc", "", -1, "UTF8_LCASE", ""); - assertSubstringIndex("abc", "", -1, "UNICODE", ""); - assertSubstringIndex("abc", "", -1, "UNICODE_CI", ""); + assertSubstringIndex("", "", 0, UTF8_BINARY, ""); + assertSubstringIndex("", "", 0, UTF8_LCASE, ""); + assertSubstringIndex("", "", 0, UNICODE, ""); + assertSubstringIndex("", "", 0, UNICODE_CI, ""); + assertSubstringIndex("", "", 1, UTF8_BINARY, ""); + assertSubstringIndex("", "", 1, UTF8_LCASE, ""); + assertSubstringIndex("", "", 1, UNICODE, ""); + assertSubstringIndex("", "", 1, UNICODE_CI, ""); + assertSubstringIndex("", "", -1, UTF8_BINARY, ""); + assertSubstringIndex("", "", -1, UTF8_LCASE, ""); + assertSubstringIndex("", "", -1, UNICODE, ""); + assertSubstringIndex("", "", -1, UNICODE_CI, ""); + assertSubstringIndex("", "x", 0, UTF8_BINARY, ""); + assertSubstringIndex("", "x", 0, UTF8_LCASE, ""); + assertSubstringIndex("", "x", 0, UNICODE, ""); + assertSubstringIndex("", "x", 0, UNICODE_CI, ""); + assertSubstringIndex("", "x", 1, UTF8_BINARY, ""); + assertSubstringIndex("", "x", 1, UTF8_LCASE, ""); + assertSubstringIndex("", "x", 1, UNICODE, ""); + assertSubstringIndex("", "x", 1, UNICODE_CI, ""); + assertSubstringIndex("", "x", -1, UTF8_BINARY, ""); + assertSubstringIndex("", "x", -1, UTF8_LCASE, ""); + assertSubstringIndex("", "x", -1, UNICODE, ""); + assertSubstringIndex("", "x", -1, UNICODE_CI, ""); + assertSubstringIndex("abc", "", 0, UTF8_BINARY, ""); + assertSubstringIndex("abc", "", 0, UTF8_LCASE, ""); + assertSubstringIndex("abc", "", 0, UNICODE, ""); + assertSubstringIndex("abc", "", 0, UNICODE_CI, ""); + assertSubstringIndex("abc", "", 1, UTF8_BINARY, ""); + assertSubstringIndex("abc", "", 1, UTF8_LCASE, ""); + assertSubstringIndex("abc", "", 1, UNICODE, ""); + assertSubstringIndex("abc", "", 1, UNICODE_CI, ""); + assertSubstringIndex("abc", "", -1, UTF8_BINARY, ""); + assertSubstringIndex("abc", "", -1, UTF8_LCASE, ""); + assertSubstringIndex("abc", "", -1, UNICODE, ""); + assertSubstringIndex("abc", "", -1, UNICODE_CI, ""); // Basic tests. - assertSubstringIndex("axbxc", "a", 1, "UTF8_BINARY", ""); - assertSubstringIndex("axbxc", "a", 1, "UTF8_LCASE", ""); - assertSubstringIndex("axbxc", "a", 1, "UNICODE", ""); - assertSubstringIndex("axbxc", "a", 1, "UNICODE_CI", ""); - assertSubstringIndex("axbxc", "x", 1, "UTF8_BINARY", "a"); - assertSubstringIndex("axbxc", "x", 1, "UTF8_LCASE", "a"); - assertSubstringIndex("axbxc", "x", 1, "UNICODE", "a"); - assertSubstringIndex("axbxc", "x", 1, "UNICODE_CI", "a"); - assertSubstringIndex("axbxc", "b", 1, "UTF8_BINARY", "ax"); - assertSubstringIndex("axbxc", "b", 1, "UTF8_LCASE", "ax"); - assertSubstringIndex("axbxc", "b", 1, "UNICODE", "ax"); - assertSubstringIndex("axbxc", "b", 1, "UNICODE_CI", "ax"); - assertSubstringIndex("axbxc", "x", 2, "UTF8_BINARY", "axb"); - assertSubstringIndex("axbxc", "x", 2, "UTF8_LCASE", "axb"); - assertSubstringIndex("axbxc", "x", 2, "UNICODE", "axb"); - assertSubstringIndex("axbxc", "x", 2, "UNICODE_CI", "axb"); - assertSubstringIndex("axbxc", "c", 1, "UTF8_BINARY", "axbx"); - assertSubstringIndex("axbxc", "c", 1, "UTF8_LCASE", "axbx"); - assertSubstringIndex("axbxc", "c", 1, "UNICODE", "axbx"); - assertSubstringIndex("axbxc", "c", 1, "UNICODE_CI", "axbx"); - assertSubstringIndex("axbxc", "x", 3, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "x", 3, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "x", 3, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "x", 3, "UNICODE_CI", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UNICODE_CI", "axbxc"); - assertSubstringIndex("axbxc", "c", -1, "UTF8_BINARY", ""); - assertSubstringIndex("axbxc", "c", -1, "UTF8_LCASE", ""); - assertSubstringIndex("axbxc", "c", -1, "UNICODE", ""); - assertSubstringIndex("axbxc", "c", -1, "UNICODE_CI", ""); - assertSubstringIndex("axbxc", "x", -1, "UTF8_BINARY", "c"); - assertSubstringIndex("axbxc", "x", -1, "UTF8_LCASE", "c"); - assertSubstringIndex("axbxc", "x", -1, "UNICODE", "c"); - assertSubstringIndex("axbxc", "x", -1, "UNICODE_CI", "c"); - assertSubstringIndex("axbxc", "b", -1, "UTF8_BINARY", "xc"); - assertSubstringIndex("axbxc", "b", -1, "UTF8_LCASE", "xc"); - assertSubstringIndex("axbxc", "b", -1, "UNICODE", "xc"); - assertSubstringIndex("axbxc", "b", -1, "UNICODE_CI", "xc"); - assertSubstringIndex("axbxc", "x", -2, "UTF8_BINARY", "bxc"); - assertSubstringIndex("axbxc", "x", -2, "UTF8_LCASE", "bxc"); - assertSubstringIndex("axbxc", "x", -2, "UNICODE", "bxc"); - assertSubstringIndex("axbxc", "x", -2, "UNICODE_CI", "bxc"); - assertSubstringIndex("axbxc", "a", -1, "UTF8_BINARY", "xbxc"); - assertSubstringIndex("axbxc", "a", -1, "UTF8_LCASE", "xbxc"); - assertSubstringIndex("axbxc", "a", -1, "UNICODE", "xbxc"); - assertSubstringIndex("axbxc", "a", -1, "UNICODE_CI", "xbxc"); - assertSubstringIndex("axbxc", "x", -3, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "x", -3, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "x", -3, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "x", -3, "UNICODE_CI", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UNICODE_CI", "axbxc"); + assertSubstringIndex("axbxc", "a", 1, UTF8_BINARY, ""); + assertSubstringIndex("axbxc", "a", 1, UTF8_LCASE, ""); + assertSubstringIndex("axbxc", "a", 1, UNICODE, ""); + assertSubstringIndex("axbxc", "a", 1, UNICODE_CI, ""); + assertSubstringIndex("axbxc", "x", 1, UTF8_BINARY, "a"); + assertSubstringIndex("axbxc", "x", 1, UTF8_LCASE, "a"); + assertSubstringIndex("axbxc", "x", 1, UNICODE, "a"); + assertSubstringIndex("axbxc", "x", 1, UNICODE_CI, "a"); + assertSubstringIndex("axbxc", "b", 1, UTF8_BINARY, "ax"); + assertSubstringIndex("axbxc", "b", 1, UTF8_LCASE, "ax"); + assertSubstringIndex("axbxc", "b", 1, UNICODE, "ax"); + assertSubstringIndex("axbxc", "b", 1, UNICODE_CI, "ax"); + assertSubstringIndex("axbxc", "x", 2, UTF8_BINARY, "axb"); + assertSubstringIndex("axbxc", "x", 2, UTF8_LCASE, "axb"); + assertSubstringIndex("axbxc", "x", 2, UNICODE, "axb"); + assertSubstringIndex("axbxc", "x", 2, UNICODE_CI, "axb"); + assertSubstringIndex("axbxc", "c", 1, UTF8_BINARY, "axbx"); + assertSubstringIndex("axbxc", "c", 1, UTF8_LCASE, "axbx"); + assertSubstringIndex("axbxc", "c", 1, UNICODE, "axbx"); + assertSubstringIndex("axbxc", "c", 1, UNICODE_CI, "axbx"); + assertSubstringIndex("axbxc", "x", 3, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "x", 3, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "x", 3, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "x", 3, UNICODE_CI, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UNICODE_CI, "axbxc"); + assertSubstringIndex("axbxc", "c", -1, UTF8_BINARY, ""); + assertSubstringIndex("axbxc", "c", -1, UTF8_LCASE, ""); + assertSubstringIndex("axbxc", "c", -1, UNICODE, ""); + assertSubstringIndex("axbxc", "c", -1, UNICODE_CI, ""); + assertSubstringIndex("axbxc", "x", -1, UTF8_BINARY, "c"); + assertSubstringIndex("axbxc", "x", -1, UTF8_LCASE, "c"); + assertSubstringIndex("axbxc", "x", -1, UNICODE, "c"); + assertSubstringIndex("axbxc", "x", -1, UNICODE_CI, "c"); + assertSubstringIndex("axbxc", "b", -1, UTF8_BINARY, "xc"); + assertSubstringIndex("axbxc", "b", -1, UTF8_LCASE, "xc"); + assertSubstringIndex("axbxc", "b", -1, UNICODE, "xc"); + assertSubstringIndex("axbxc", "b", -1, UNICODE_CI, "xc"); + assertSubstringIndex("axbxc", "x", -2, UTF8_BINARY, "bxc"); + assertSubstringIndex("axbxc", "x", -2, UTF8_LCASE, "bxc"); + assertSubstringIndex("axbxc", "x", -2, UNICODE, "bxc"); + assertSubstringIndex("axbxc", "x", -2, UNICODE_CI, "bxc"); + assertSubstringIndex("axbxc", "a", -1, UTF8_BINARY, "xbxc"); + assertSubstringIndex("axbxc", "a", -1, UTF8_LCASE, "xbxc"); + assertSubstringIndex("axbxc", "a", -1, UNICODE, "xbxc"); + assertSubstringIndex("axbxc", "a", -1, UNICODE_CI, "xbxc"); + assertSubstringIndex("axbxc", "x", -3, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "x", -3, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "x", -3, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "x", -3, UNICODE_CI, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UNICODE_CI, "axbxc"); // Advanced tests. - assertSubstringIndex("wwwgapachegorg", "g", -3, "UTF8_BINARY", "apachegorg"); - assertSubstringIndex("www||apache||org", "||", 2, "UTF8_BINARY", "www||apache"); - assertSubstringIndex("aaaaaaaaaa", "aa", 2, "UTF8_BINARY", "a"); - assertSubstringIndex("AaAaAaAaAa", "aa", 2, "UTF8_LCASE", "A"); - assertSubstringIndex("www.apache.org", ".", 3, "UTF8_LCASE", "www.apache.org"); - assertSubstringIndex("wwwXapacheXorg", "x", 2, "UTF8_LCASE", "wwwXapache"); - assertSubstringIndex("wwwxapachexorg", "X", 1, "UTF8_LCASE", "www"); - assertSubstringIndex("www.apache.org", ".", 0, "UTF8_LCASE", ""); - assertSubstringIndex("www.apache.ORG", ".", -3, "UTF8_LCASE", "www.apache.ORG"); - assertSubstringIndex("wwwGapacheGorg", "g", 1, "UTF8_LCASE", "www"); - assertSubstringIndex("wwwGapacheGorg", "g", 3, "UTF8_LCASE", "wwwGapacheGor"); - assertSubstringIndex("gwwwGapacheGorg", "g", 3, "UTF8_LCASE", "gwwwGapache"); - assertSubstringIndex("wwwGapacheGorg", "g", -3, "UTF8_LCASE", "apacheGorg"); - assertSubstringIndex("wwwmapacheMorg", "M", -2, "UTF8_LCASE", "apacheMorg"); - assertSubstringIndex("www.apache.org", ".", -1, "UTF8_LCASE", "org"); - assertSubstringIndex("www.apache.org.", ".", -1, "UTF8_LCASE", ""); - assertSubstringIndex("", ".", -2, "UTF8_LCASE", ""); - assertSubstringIndex("test大千世界X大千世界", "x", -1, "UTF8_LCASE", "大千世界"); - assertSubstringIndex("test大千世界X大千世界", "X", 1, "UTF8_LCASE", "test大千世界"); - assertSubstringIndex("test大千世界大千世界", "千", 2, "UTF8_LCASE", "test大千世界大"); - assertSubstringIndex("www||APACHE||org", "||", 2, "UTF8_LCASE", "www||APACHE"); - assertSubstringIndex("www||APACHE||org", "||", -1, "UTF8_LCASE", "org"); - assertSubstringIndex("AaAaAaAaAa", "Aa", 2, "UNICODE", "Aa"); - assertSubstringIndex("wwwYapacheyorg", "y", 3, "UNICODE", "wwwYapacheyorg"); - assertSubstringIndex("www.apache.org", ".", 2, "UNICODE", "www.apache"); - assertSubstringIndex("wwwYapacheYorg", "Y", 1, "UNICODE", "www"); - assertSubstringIndex("wwwYapacheYorg", "y", 1, "UNICODE", "wwwYapacheYorg"); - assertSubstringIndex("wwwGapacheGorg", "g", 1, "UNICODE", "wwwGapacheGor"); - assertSubstringIndex("GwwwGapacheGorG", "G", 3, "UNICODE", "GwwwGapache"); - assertSubstringIndex("wwwGapacheGorG", "G", -3, "UNICODE", "apacheGorG"); - assertSubstringIndex("www.apache.org", ".", 0, "UNICODE", ""); - assertSubstringIndex("www.apache.org", ".", -3, "UNICODE", "www.apache.org"); - assertSubstringIndex("www.apache.org", ".", -2, "UNICODE", "apache.org"); - assertSubstringIndex("www.apache.org", ".", -1, "UNICODE", "org"); - assertSubstringIndex("", ".", -2, "UNICODE", ""); - assertSubstringIndex("test大千世界X大千世界", "X", -1, "UNICODE", "大千世界"); - assertSubstringIndex("test大千世界X大千世界", "X", 1, "UNICODE", "test大千世界"); - assertSubstringIndex("大x千世界大千世x界", "x", 1, "UNICODE", "大"); - assertSubstringIndex("大x千世界大千世x界", "x", -1, "UNICODE", "界"); - assertSubstringIndex("大x千世界大千世x界", "x", -2, "UNICODE", "千世界大千世x界"); - assertSubstringIndex("大千世界大千世界", "千", 2, "UNICODE", "大千世界大"); - assertSubstringIndex("www||apache||org", "||", 2, "UNICODE", "www||apache"); - assertSubstringIndex("AaAaAaAaAa", "aa", 2, "UNICODE_CI", "A"); - assertSubstringIndex("www.apache.org", ".", 3, "UNICODE_CI", "www.apache.org"); - assertSubstringIndex("wwwXapacheXorg", "x", 2, "UNICODE_CI", "wwwXapache"); - assertSubstringIndex("wwwxapacheXorg", "X", 1, "UNICODE_CI", "www"); - assertSubstringIndex("www.apache.org", ".", 0, "UNICODE_CI", ""); - assertSubstringIndex("wwwGapacheGorg", "G", 3, "UNICODE_CI", "wwwGapacheGor"); - assertSubstringIndex("gwwwGapacheGorg", "g", 3, "UNICODE_CI", "gwwwGapache"); - assertSubstringIndex("gwwwGapacheGorg", "g", -3, "UNICODE_CI", "apacheGorg"); - assertSubstringIndex("www.apache.ORG", ".", -3, "UNICODE_CI", "www.apache.ORG"); - assertSubstringIndex("wwwmapacheMorg", "M", -2, "UNICODE_CI", "apacheMorg"); - assertSubstringIndex("www.apache.org", ".", -1, "UNICODE_CI", "org"); - assertSubstringIndex("", ".", -2, "UNICODE_CI", ""); - assertSubstringIndex("test大千世界X大千世界", "X", -1, "UNICODE_CI", "大千世界"); - assertSubstringIndex("test大千世界X大千世界", "X", 1, "UNICODE_CI", "test大千世界"); - assertSubstringIndex("test大千世界大千世界", "千", 2, "UNICODE_CI", "test大千世界大"); - assertSubstringIndex("www||APACHE||org", "||", 2, "UNICODE_CI", "www||APACHE"); + assertSubstringIndex("wwwgapachegorg", "g", -3, UTF8_BINARY, "apachegorg"); + assertSubstringIndex("www||apache||org", "||", 2, UTF8_BINARY, "www||apache"); + assertSubstringIndex("aaaaaaaaaa", "aa", 2, UTF8_BINARY, "a"); + assertSubstringIndex("AaAaAaAaAa", "aa", 2, UTF8_LCASE, "A"); + assertSubstringIndex("www.apache.org", ".", 3, UTF8_LCASE, "www.apache.org"); + assertSubstringIndex("wwwXapacheXorg", "x", 2, UTF8_LCASE, "wwwXapache"); + assertSubstringIndex("wwwxapachexorg", "X", 1, UTF8_LCASE, "www"); + assertSubstringIndex("www.apache.org", ".", 0, UTF8_LCASE, ""); + assertSubstringIndex("www.apache.ORG", ".", -3, UTF8_LCASE, "www.apache.ORG"); + assertSubstringIndex("wwwGapacheGorg", "g", 1, UTF8_LCASE, "www"); + assertSubstringIndex("wwwGapacheGorg", "g", 3, UTF8_LCASE, "wwwGapacheGor"); + assertSubstringIndex("gwwwGapacheGorg", "g", 3, UTF8_LCASE, "gwwwGapache"); + assertSubstringIndex("wwwGapacheGorg", "g", -3, UTF8_LCASE, "apacheGorg"); + assertSubstringIndex("wwwmapacheMorg", "M", -2, UTF8_LCASE, "apacheMorg"); + assertSubstringIndex("www.apache.org", ".", -1, UTF8_LCASE, "org"); + assertSubstringIndex("www.apache.org.", ".", -1, UTF8_LCASE, ""); + assertSubstringIndex("", ".", -2, UTF8_LCASE, ""); + assertSubstringIndex("test大千世界X大千世界", "x", -1, UTF8_LCASE, "大千世界"); + assertSubstringIndex("test大千世界X大千世界", "X", 1, UTF8_LCASE, "test大千世界"); + assertSubstringIndex("test大千世界大千世界", "千", 2, UTF8_LCASE, "test大千世界大"); + assertSubstringIndex("www||APACHE||org", "||", 2, UTF8_LCASE, "www||APACHE"); + assertSubstringIndex("www||APACHE||org", "||", -1, UTF8_LCASE, "org"); + assertSubstringIndex("AaAaAaAaAa", "Aa", 2, UNICODE, "Aa"); + assertSubstringIndex("wwwYapacheyorg", "y", 3, UNICODE, "wwwYapacheyorg"); + assertSubstringIndex("www.apache.org", ".", 2, UNICODE, "www.apache"); + assertSubstringIndex("wwwYapacheYorg", "Y", 1, UNICODE, "www"); + assertSubstringIndex("wwwYapacheYorg", "y", 1, UNICODE, "wwwYapacheYorg"); + assertSubstringIndex("wwwGapacheGorg", "g", 1, UNICODE, "wwwGapacheGor"); + assertSubstringIndex("GwwwGapacheGorG", "G", 3, UNICODE, "GwwwGapache"); + assertSubstringIndex("wwwGapacheGorG", "G", -3, UNICODE, "apacheGorG"); + assertSubstringIndex("www.apache.org", ".", 0, UNICODE, ""); + assertSubstringIndex("www.apache.org", ".", -3, UNICODE, "www.apache.org"); + assertSubstringIndex("www.apache.org", ".", -2, UNICODE, "apache.org"); + assertSubstringIndex("www.apache.org", ".", -1, UNICODE, "org"); + assertSubstringIndex("", ".", -2, UNICODE, ""); + assertSubstringIndex("test大千世界X大千世界", "X", -1, UNICODE, "大千世界"); + assertSubstringIndex("test大千世界X大千世界", "X", 1, UNICODE, "test大千世界"); + assertSubstringIndex("大x千世界大千世x界", "x", 1, UNICODE, "大"); + assertSubstringIndex("大x千世界大千世x界", "x", -1, UNICODE, "界"); + assertSubstringIndex("大x千世界大千世x界", "x", -2, UNICODE, "千世界大千世x界"); + assertSubstringIndex("大千世界大千世界", "千", 2, UNICODE, "大千世界大"); + assertSubstringIndex("www||apache||org", "||", 2, UNICODE, "www||apache"); + assertSubstringIndex("AaAaAaAaAa", "aa", 2, UNICODE_CI, "A"); + assertSubstringIndex("www.apache.org", ".", 3, UNICODE_CI, "www.apache.org"); + assertSubstringIndex("wwwXapacheXorg", "x", 2, UNICODE_CI, "wwwXapache"); + assertSubstringIndex("wwwxapacheXorg", "X", 1, UNICODE_CI, "www"); + assertSubstringIndex("www.apache.org", ".", 0, UNICODE_CI, ""); + assertSubstringIndex("wwwGapacheGorg", "G", 3, UNICODE_CI, "wwwGapacheGor"); + assertSubstringIndex("gwwwGapacheGorg", "g", 3, UNICODE_CI, "gwwwGapache"); + assertSubstringIndex("gwwwGapacheGorg", "g", -3, UNICODE_CI, "apacheGorg"); + assertSubstringIndex("www.apache.ORG", ".", -3, UNICODE_CI, "www.apache.ORG"); + assertSubstringIndex("wwwmapacheMorg", "M", -2, UNICODE_CI, "apacheMorg"); + assertSubstringIndex("www.apache.org", ".", -1, UNICODE_CI, "org"); + assertSubstringIndex("", ".", -2, UNICODE_CI, ""); + assertSubstringIndex("test大千世界X大千世界", "X", -1, UNICODE_CI, "大千世界"); + assertSubstringIndex("test大千世界X大千世界", "X", 1, UNICODE_CI, "test大千世界"); + assertSubstringIndex("test大千世界大千世界", "千", 2, UNICODE_CI, "test大千世界大"); + assertSubstringIndex("www||APACHE||org", "||", 2, UNICODE_CI, "www||APACHE"); assertSubstringIndex("wwwèapacheËorg", "Ê", -3, "AF_CI_AI", "apacheËorg"); // One-to-many case mapping (e.g. Turkish dotted I). - assertSubstringIndex("abİo12", "i\u0307o", 1, "UNICODE_CI", "ab"); - assertSubstringIndex("abİo12", "i\u0307o", -1, "UNICODE_CI", "12"); - assertSubstringIndex("abi̇o12", "İo", 1, "UNICODE_CI", "ab"); - assertSubstringIndex("abi̇o12", "İo", -1, "UNICODE_CI", "12"); - assertSubstringIndex("ai̇bi̇o12", "İo", 1, "UNICODE_CI", "ai̇b"); - assertSubstringIndex("ai̇bi̇o12i̇o", "İo", 2, "UNICODE_CI", "ai̇bi̇o12"); - assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -1, "UNICODE_CI", ""); - assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -2, "UNICODE_CI", "12i̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("abi̇12", "i", 1, "UNICODE_CI", "abi̇12"); - assertSubstringIndex("abi̇12", "\u0307", 1, "UNICODE_CI", "abi̇12"); - assertSubstringIndex("abi̇12", "İ", 1, "UNICODE_CI", "ab"); - assertSubstringIndex("abİ12", "i", 1, "UNICODE_CI", "abİ12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, "UNICODE_CI", "ai̇bi̇oİo12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", 3, "UNICODE_CI", "ai̇bi̇oİo12"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", 3, "UNICODE_CI", "ai̇bİoi̇o12"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", 3, "UNICODE_CI", "ai̇bİoi̇o12"); - assertSubstringIndex("abi̇12", "i", 1, "UTF8_LCASE", "ab"); // != UNICODE_CI - assertSubstringIndex("abi̇12", "\u0307", 1, "UTF8_LCASE", "abi"); // != UNICODE_CI - assertSubstringIndex("abi̇12", "İ", 1, "UTF8_LCASE", "ab"); - assertSubstringIndex("abİ12", "i", 1, "UTF8_LCASE", "abİ12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, "UTF8_LCASE", "İo12İoi̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, "UTF8_LCASE", "İo12İoi̇o"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, "UTF8_LCASE", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, "UTF8_LCASE", "i̇o12i̇oİo"); - assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, "UTF8_LCASE", "bİoi̇o12i̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, "UTF8_LCASE", "ai̇bi̇oİo12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", 3, "UTF8_LCASE", "ai̇bi̇oİo12"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", 3, "UTF8_LCASE", "ai̇bİoi̇o12"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", 3, "UTF8_LCASE", "ai̇bİoi̇o12"); - assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, "UTF8_LCASE", "bİoi̇o12i̇o"); + assertSubstringIndex("abİo12", "i\u0307o", 1, UNICODE_CI, "ab"); + assertSubstringIndex("abİo12", "i\u0307o", -1, UNICODE_CI, "12"); + assertSubstringIndex("abi̇o12", "İo", 1, UNICODE_CI, "ab"); + assertSubstringIndex("abi̇o12", "İo", -1, UNICODE_CI, "12"); + assertSubstringIndex("ai̇bi̇o12", "İo", 1, UNICODE_CI, "ai̇b"); + assertSubstringIndex("ai̇bi̇o12i̇o", "İo", 2, UNICODE_CI, "ai̇bi̇o12"); + assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -1, UNICODE_CI, ""); + assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -2, UNICODE_CI, "12i̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("abi̇12", "i", 1, UNICODE_CI, "abi̇12"); + assertSubstringIndex("abi̇12", "\u0307", 1, UNICODE_CI, "abi̇12"); + assertSubstringIndex("abi̇12", "İ", 1, UNICODE_CI, "ab"); + assertSubstringIndex("abİ12", "i", 1, UNICODE_CI, "abİ12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, UNICODE_CI, "ai̇bi̇oİo12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", 3, UNICODE_CI, "ai̇bi̇oİo12"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", 3, UNICODE_CI, "ai̇bİoi̇o12"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", 3, UNICODE_CI, "ai̇bİoi̇o12"); + assertSubstringIndex("abi̇12", "i", 1, UTF8_LCASE, "ab"); // != UNICODE_CI + assertSubstringIndex("abi̇12", "\u0307", 1, UTF8_LCASE, "abi"); // != UNICODE_CI + assertSubstringIndex("abi̇12", "İ", 1, UTF8_LCASE, "ab"); + assertSubstringIndex("abİ12", "i", 1, UTF8_LCASE, "abİ12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, UTF8_LCASE, "İo12İoi̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, UTF8_LCASE, "İo12İoi̇o"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, UTF8_LCASE, "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, UTF8_LCASE, "i̇o12i̇oİo"); + assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, UTF8_LCASE, "bİoi̇o12i̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, UTF8_LCASE, "ai̇bi̇oİo12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", 3, UTF8_LCASE, "ai̇bi̇oİo12"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", 3, UTF8_LCASE, "ai̇bİoi̇o12"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", 3, UTF8_LCASE, "ai̇bİoi̇o12"); + assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, UTF8_LCASE, "bİoi̇o12i̇o"); // Conditional case mapping (e.g. Greek sigmas). - assertSubstringIndex("σ", "σ", 1, "UTF8_BINARY", ""); - assertSubstringIndex("σ", "ς", 1, "UTF8_BINARY", "σ"); - assertSubstringIndex("σ", "Σ", 1, "UTF8_BINARY", "σ"); - assertSubstringIndex("ς", "σ", 1, "UTF8_BINARY", "ς"); - assertSubstringIndex("ς", "ς", 1, "UTF8_BINARY", ""); - assertSubstringIndex("ς", "Σ", 1, "UTF8_BINARY", "ς"); - assertSubstringIndex("Σ", "σ", 1, "UTF8_BINARY", "Σ"); - assertSubstringIndex("Σ", "ς", 1, "UTF8_BINARY", "Σ"); - assertSubstringIndex("Σ", "Σ", 1, "UTF8_BINARY", ""); - assertSubstringIndex("σ", "σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("σ", "ς", 1, "UTF8_LCASE", ""); - assertSubstringIndex("σ", "Σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("ς", "σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("ς", "ς", 1, "UTF8_LCASE", ""); - assertSubstringIndex("ς", "Σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("Σ", "σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("Σ", "ς", 1, "UTF8_LCASE", ""); - assertSubstringIndex("Σ", "Σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("σ", "σ", 1, "UNICODE", ""); - assertSubstringIndex("σ", "ς", 1, "UNICODE", "σ"); - assertSubstringIndex("σ", "Σ", 1, "UNICODE", "σ"); - assertSubstringIndex("ς", "σ", 1, "UNICODE", "ς"); - assertSubstringIndex("ς", "ς", 1, "UNICODE", ""); - assertSubstringIndex("ς", "Σ", 1, "UNICODE", "ς"); - assertSubstringIndex("Σ", "σ", 1, "UNICODE", "Σ"); - assertSubstringIndex("Σ", "ς", 1, "UNICODE", "Σ"); - assertSubstringIndex("Σ", "Σ", 1, "UNICODE", ""); - assertSubstringIndex("σ", "σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("σ", "ς", 1, "UNICODE_CI", ""); - assertSubstringIndex("σ", "Σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("ς", "σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("ς", "ς", 1, "UNICODE_CI", ""); - assertSubstringIndex("ς", "Σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("Σ", "σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("Σ", "ς", 1, "UNICODE_CI", ""); - assertSubstringIndex("Σ", "Σ", 1, "UNICODE_CI", ""); + assertSubstringIndex("σ", "σ", 1, UTF8_BINARY, ""); + assertSubstringIndex("σ", "ς", 1, UTF8_BINARY, "σ"); + assertSubstringIndex("σ", "Σ", 1, UTF8_BINARY, "σ"); + assertSubstringIndex("ς", "σ", 1, UTF8_BINARY, "ς"); + assertSubstringIndex("ς", "ς", 1, UTF8_BINARY, ""); + assertSubstringIndex("ς", "Σ", 1, UTF8_BINARY, "ς"); + assertSubstringIndex("Σ", "σ", 1, UTF8_BINARY, "Σ"); + assertSubstringIndex("Σ", "ς", 1, UTF8_BINARY, "Σ"); + assertSubstringIndex("Σ", "Σ", 1, UTF8_BINARY, ""); + assertSubstringIndex("σ", "σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("σ", "ς", 1, UTF8_LCASE, ""); + assertSubstringIndex("σ", "Σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("ς", "σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("ς", "ς", 1, UTF8_LCASE, ""); + assertSubstringIndex("ς", "Σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("Σ", "σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("Σ", "ς", 1, UTF8_LCASE, ""); + assertSubstringIndex("Σ", "Σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("σ", "σ", 1, UNICODE, ""); + assertSubstringIndex("σ", "ς", 1, UNICODE, "σ"); + assertSubstringIndex("σ", "Σ", 1, UNICODE, "σ"); + assertSubstringIndex("ς", "σ", 1, UNICODE, "ς"); + assertSubstringIndex("ς", "ς", 1, UNICODE, ""); + assertSubstringIndex("ς", "Σ", 1, UNICODE, "ς"); + assertSubstringIndex("Σ", "σ", 1, UNICODE, "Σ"); + assertSubstringIndex("Σ", "ς", 1, UNICODE, "Σ"); + assertSubstringIndex("Σ", "Σ", 1, UNICODE, ""); + assertSubstringIndex("σ", "σ", 1, UNICODE_CI, ""); + assertSubstringIndex("σ", "ς", 1, UNICODE_CI, ""); + assertSubstringIndex("σ", "Σ", 1, UNICODE_CI, ""); + assertSubstringIndex("ς", "σ", 1, UNICODE_CI, ""); + assertSubstringIndex("ς", "ς", 1, UNICODE_CI, ""); + assertSubstringIndex("ς", "Σ", 1, UNICODE_CI, ""); + assertSubstringIndex("Σ", "σ", 1, UNICODE_CI, ""); + assertSubstringIndex("Σ", "ς", 1, UNICODE_CI, ""); + assertSubstringIndex("Σ", "Σ", 1, UNICODE_CI, ""); // Surrogate pairs. - assertSubstringIndex("a🙃b🙃c", "a", 1, "UTF8_BINARY", ""); - assertSubstringIndex("a🙃b🙃c", "a", 1, "UTF8_LCASE", ""); - assertSubstringIndex("a🙃b🙃c", "a", 1, "UNICODE", ""); - assertSubstringIndex("a🙃b🙃c", "a", 1, "UNICODE_CI", ""); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UTF8_BINARY", "a"); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UTF8_LCASE", "a"); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UNICODE", "a"); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UNICODE_CI", "a"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UTF8_BINARY", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UTF8_LCASE", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UNICODE", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UNICODE_CI", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UTF8_BINARY", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UTF8_LCASE", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UNICODE", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UNICODE_CI", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UTF8_BINARY", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UTF8_LCASE", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UNICODE", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UNICODE_CI", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UNICODE_CI", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UNICODE_CI", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UTF8_BINARY", ""); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UTF8_LCASE", ""); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UNICODE", ""); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UNICODE_CI", ""); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UTF8_BINARY", "c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UTF8_LCASE", "c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UNICODE", "c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UNICODE_CI", "c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UTF8_BINARY", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UTF8_LCASE", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UNICODE", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UNICODE_CI", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UTF8_BINARY", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UTF8_LCASE", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UNICODE", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UNICODE_CI", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UTF8_BINARY", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UTF8_LCASE", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UNICODE", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UNICODE_CI", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UNICODE_CI", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UNICODE_CI", "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", 1, UTF8_BINARY, ""); + assertSubstringIndex("a🙃b🙃c", "a", 1, UTF8_LCASE, ""); + assertSubstringIndex("a🙃b🙃c", "a", 1, UNICODE, ""); + assertSubstringIndex("a🙃b🙃c", "a", 1, UNICODE_CI, ""); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UTF8_BINARY, "a"); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UTF8_LCASE, "a"); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UNICODE, "a"); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UNICODE_CI, "a"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UTF8_BINARY, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UTF8_LCASE, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UNICODE, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UNICODE_CI, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UTF8_BINARY, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UTF8_LCASE, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UNICODE, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UNICODE_CI, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UTF8_BINARY, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UTF8_LCASE, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UNICODE, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UNICODE_CI, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UNICODE_CI, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UNICODE_CI, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "c", -1, UTF8_BINARY, ""); + assertSubstringIndex("a🙃b🙃c", "c", -1, UTF8_LCASE, ""); + assertSubstringIndex("a🙃b🙃c", "c", -1, UNICODE, ""); + assertSubstringIndex("a🙃b🙃c", "c", -1, UNICODE_CI, ""); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UTF8_BINARY, "c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UTF8_LCASE, "c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UNICODE, "c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UNICODE_CI, "c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UTF8_BINARY, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UTF8_LCASE, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UNICODE, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UNICODE_CI, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UTF8_BINARY, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UTF8_LCASE, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UNICODE, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UNICODE_CI, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UTF8_BINARY, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UTF8_LCASE, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UNICODE, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UNICODE_CI, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UNICODE_CI, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UNICODE_CI, "a🙃b🙃c"); } /** @@ -2776,279 +2777,279 @@ private void assertStringTrim(String collationName, String sourceString, String @Test public void testStringTrim() throws SparkException { // Basic tests. - assertStringTrim("UTF8_BINARY", "", "", ""); - assertStringTrim("UTF8_BINARY", "", "xyz", ""); - assertStringTrim("UTF8_BINARY", "asd", "", "asd"); - assertStringTrim("UTF8_BINARY", "asd", null, "asd"); - assertStringTrim("UTF8_BINARY", " asd ", null, "asd"); - assertStringTrim("UTF8_BINARY", " a世a ", null, "a世a"); - assertStringTrim("UTF8_BINARY", "asd", "x", "asd"); - assertStringTrim("UTF8_BINARY", "xxasdxx", "x", "asd"); - assertStringTrim("UTF8_BINARY", "xa世ax", "x", "a世a"); - assertStringTrim("UTF8_LCASE", "", "", ""); - assertStringTrim("UTF8_LCASE", "", "xyz", ""); - assertStringTrim("UTF8_LCASE", "asd", "", "asd"); - assertStringTrim("UTF8_LCASE", "asd", null, "asd"); - assertStringTrim("UTF8_LCASE", " asd ", null, "asd"); - assertStringTrim("UTF8_LCASE", " a世a ", null, "a世a"); - assertStringTrim("UTF8_LCASE", "asd", "x", "asd"); - assertStringTrim("UTF8_LCASE", "xxasdxx", "x", "asd"); - assertStringTrim("UTF8_LCASE", "xa世ax", "x", "a世a"); - assertStringTrim("UNICODE", "", "", ""); - assertStringTrim("UNICODE", "", "xyz", ""); - assertStringTrim("UNICODE", "asd", "", "asd"); - assertStringTrim("UNICODE", "asd", null, "asd"); - assertStringTrim("UNICODE", " asd ", null, "asd"); - assertStringTrim("UNICODE", " a世a ", null, "a世a"); - assertStringTrim("UNICODE", "asd", "x", "asd"); - assertStringTrim("UNICODE", "xxasdxx", "x", "asd"); - assertStringTrim("UNICODE", "xa世ax", "x", "a世a"); - assertStringTrim("UNICODE_CI", "", "", ""); - assertStringTrim("UNICODE_CI", "", "xyz", ""); - assertStringTrim("UNICODE_CI", "asd", "", "asd"); - assertStringTrim("UNICODE_CI", "asd", null, "asd"); - assertStringTrim("UNICODE_CI", " asd ", null, "asd"); - assertStringTrim("UNICODE_CI", " a世a ", null, "a世a"); - assertStringTrim("UNICODE_CI", "asd", "x", "asd"); - assertStringTrim("UNICODE_CI", "xxasdxx", "x", "asd"); - assertStringTrim("UNICODE_CI", "xa世ax", "x", "a世a"); + assertStringTrim(UTF8_BINARY, "", "", ""); + assertStringTrim(UTF8_BINARY, "", "xyz", ""); + assertStringTrim(UTF8_BINARY, "asd", "", "asd"); + assertStringTrim(UTF8_BINARY, "asd", null, "asd"); + assertStringTrim(UTF8_BINARY, " asd ", null, "asd"); + assertStringTrim(UTF8_BINARY, " a世a ", null, "a世a"); + assertStringTrim(UTF8_BINARY, "asd", "x", "asd"); + assertStringTrim(UTF8_BINARY, "xxasdxx", "x", "asd"); + assertStringTrim(UTF8_BINARY, "xa世ax", "x", "a世a"); + assertStringTrim(UTF8_LCASE, "", "", ""); + assertStringTrim(UTF8_LCASE, "", "xyz", ""); + assertStringTrim(UTF8_LCASE, "asd", "", "asd"); + assertStringTrim(UTF8_LCASE, "asd", null, "asd"); + assertStringTrim(UTF8_LCASE, " asd ", null, "asd"); + assertStringTrim(UTF8_LCASE, " a世a ", null, "a世a"); + assertStringTrim(UTF8_LCASE, "asd", "x", "asd"); + assertStringTrim(UTF8_LCASE, "xxasdxx", "x", "asd"); + assertStringTrim(UTF8_LCASE, "xa世ax", "x", "a世a"); + assertStringTrim(UNICODE, "", "", ""); + assertStringTrim(UNICODE, "", "xyz", ""); + assertStringTrim(UNICODE, "asd", "", "asd"); + assertStringTrim(UNICODE, "asd", null, "asd"); + assertStringTrim(UNICODE, " asd ", null, "asd"); + assertStringTrim(UNICODE, " a世a ", null, "a世a"); + assertStringTrim(UNICODE, "asd", "x", "asd"); + assertStringTrim(UNICODE, "xxasdxx", "x", "asd"); + assertStringTrim(UNICODE, "xa世ax", "x", "a世a"); + assertStringTrim(UNICODE_CI, "", "", ""); + assertStringTrim(UNICODE_CI, "", "xyz", ""); + assertStringTrim(UNICODE_CI, "asd", "", "asd"); + assertStringTrim(UNICODE_CI, "asd", null, "asd"); + assertStringTrim(UNICODE_CI, " asd ", null, "asd"); + assertStringTrim(UNICODE_CI, " a世a ", null, "a世a"); + assertStringTrim(UNICODE_CI, "asd", "x", "asd"); + assertStringTrim(UNICODE_CI, "xxasdxx", "x", "asd"); + assertStringTrim(UNICODE_CI, "xa世ax", "x", "a世a"); // Case variation. - assertStringTrim("UTF8_BINARY", "asd", "A", "asd"); - assertStringTrim("UTF8_BINARY", "ddsXXXaa", "asd", "XXX"); - assertStringTrim("UTF8_BINARY", "ASD", "a", "ASD"); - assertStringTrim("UTF8_LCASE", "asd", "A", "sd"); - assertStringTrim("UTF8_LCASE", "ASD", "a", "SD"); - assertStringTrim("UTF8_LCASE", "ddsXXXaa", "ASD", "XXX"); - assertStringTrim("UNICODE", "asd", "A", "asd"); - assertStringTrim("UNICODE", "ASD", "a", "ASD"); - assertStringTrim("UNICODE", "ddsXXXaa", "asd", "XXX"); - assertStringTrim("UNICODE_CI", "asd", "A", "sd"); - assertStringTrim("UNICODE_CI", "ASD", "a", "SD"); - assertStringTrim("UNICODE_CI", "ddsXXXaa", "ASD", "XXX"); + assertStringTrim(UTF8_BINARY, "asd", "A", "asd"); + assertStringTrim(UTF8_BINARY, "ddsXXXaa", "asd", "XXX"); + assertStringTrim(UTF8_BINARY, "ASD", "a", "ASD"); + assertStringTrim(UTF8_LCASE, "asd", "A", "sd"); + assertStringTrim(UTF8_LCASE, "ASD", "a", "SD"); + assertStringTrim(UTF8_LCASE, "ddsXXXaa", "ASD", "XXX"); + assertStringTrim(UNICODE, "asd", "A", "asd"); + assertStringTrim(UNICODE, "ASD", "a", "ASD"); + assertStringTrim(UNICODE, "ddsXXXaa", "asd", "XXX"); + assertStringTrim(UNICODE_CI, "asd", "A", "sd"); + assertStringTrim(UNICODE_CI, "ASD", "a", "SD"); + assertStringTrim(UNICODE_CI, "ddsXXXaa", "ASD", "XXX"); assertStringTrim("SR_CI_AI", "cSCšćČXXXsčšČŠsć", "čš", "XXX"); // One-to-many case mapping (e.g. Turkish dotted I).. - assertStringTrim("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrim("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrim("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "aaa"); - assertStringTrim("UTF8_LCASE", "ẞaaaẞ", "ß", "aaa"); - assertStringTrim("UTF8_LCASE", "ßaaaß", "ẞ", "aaa"); - assertStringTrim("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "aaa"); - assertStringTrim("UNICODE", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrim("UNICODE", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrim("UNICODE", "Ëaaaẞ", "Ëẞ", "aaa"); - assertStringTrim("UNICODE_CI", "ẞaaaẞ", "ß", "aaa"); - assertStringTrim("UNICODE_CI", "ßaaaß", "ẞ", "aaa"); - assertStringTrim("UNICODE_CI", "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UTF8_BINARY, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrim(UTF8_BINARY, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrim(UTF8_BINARY, "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UTF8_LCASE, "ẞaaaẞ", "ß", "aaa"); + assertStringTrim(UTF8_LCASE, "ßaaaß", "ẞ", "aaa"); + assertStringTrim(UTF8_LCASE, "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UNICODE, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrim(UNICODE, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrim(UNICODE, "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UNICODE_CI, "ẞaaaẞ", "ß", "aaa"); + assertStringTrim(UNICODE_CI, "ßaaaß", "ẞ", "aaa"); + assertStringTrim(UNICODE_CI, "Ëaaaẞ", "Ëẞ", "aaa"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTrim("UTF8_BINARY", "i", "i", ""); - assertStringTrim("UTF8_BINARY", "iii", "I", "iii"); - assertStringTrim("UTF8_BINARY", "I", "iii", "I"); - assertStringTrim("UTF8_BINARY", "ixi", "i", "x"); - assertStringTrim("UTF8_BINARY", "i", "İ", "i"); - assertStringTrim("UTF8_BINARY", "i\u0307", "İ", "i\u0307"); - assertStringTrim("UTF8_BINARY", "ii\u0307", "İi", "\u0307"); - assertStringTrim("UTF8_BINARY", "iii\u0307", "İi", "\u0307"); - assertStringTrim("UTF8_BINARY", "iiii\u0307", "iİ", "\u0307"); - assertStringTrim("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); - assertStringTrim("UTF8_BINARY", "i\u0307", "i", "\u0307"); - assertStringTrim("UTF8_BINARY", "i\u0307", "\u0307", "i"); - assertStringTrim("UTF8_BINARY", "i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307\u0307", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307i", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307i", "İ", "i\u0307i"); - assertStringTrim("UTF8_BINARY", "i\u0307İ", "i\u0307", "İ"); - assertStringTrim("UTF8_BINARY", "i\u0307İ", "İ", "i\u0307"); - assertStringTrim("UTF8_BINARY", "İ", "İ", ""); - assertStringTrim("UTF8_BINARY", "IXi", "İ", "IXi"); - assertStringTrim("UTF8_BINARY", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrim("UTF8_BINARY", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrim("UTF8_BINARY", "i\u0307x", "ix\u0307İ", ""); - assertStringTrim("UTF8_BINARY", "İ", "i", "İ"); - assertStringTrim("UTF8_BINARY", "İ", "\u0307", "İ"); - assertStringTrim("UTF8_BINARY", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrim("UTF8_BINARY", "IXİ", "ix\u0307", "IXİ"); - assertStringTrim("UTF8_BINARY", "xi\u0307", "\u0307IX", "xi"); - assertStringTrim("UTF8_LCASE", "i", "i", ""); - assertStringTrim("UTF8_LCASE", "iii", "I", ""); - assertStringTrim("UTF8_LCASE", "I", "iii", ""); - assertStringTrim("UTF8_LCASE", "ixi", "i", "x"); - assertStringTrim("UTF8_LCASE", "i", "İ", "i"); - assertStringTrim("UTF8_LCASE", "i\u0307", "İ", ""); - assertStringTrim("UTF8_LCASE", "ii\u0307", "İi", ""); - assertStringTrim("UTF8_LCASE", "iii\u0307", "İi", ""); - assertStringTrim("UTF8_LCASE", "iiii\u0307", "iİ", ""); - assertStringTrim("UTF8_LCASE", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrim("UTF8_LCASE", "i\u0307", "i", "\u0307"); - assertStringTrim("UTF8_LCASE", "i\u0307", "\u0307", "i"); - assertStringTrim("UTF8_LCASE", "i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307\u0307", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307i", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307i", "İ", "i"); - assertStringTrim("UTF8_LCASE", "i\u0307İ", "i\u0307", "İ"); - assertStringTrim("UTF8_LCASE", "i\u0307İ", "İ", ""); - assertStringTrim("UTF8_LCASE", "İ", "İ", ""); - assertStringTrim("UTF8_LCASE", "IXi", "İ", "IXi"); - assertStringTrim("UTF8_LCASE", "ix\u0307", "Ixİ", "\u0307"); - assertStringTrim("UTF8_LCASE", "i\u0307x", "IXİ", ""); - assertStringTrim("UTF8_LCASE", "i\u0307x", "I\u0307xİ", ""); - assertStringTrim("UTF8_LCASE", "İ", "i", "İ"); - assertStringTrim("UTF8_LCASE", "İ", "\u0307", "İ"); - assertStringTrim("UTF8_LCASE", "Ixİ", "i\u0307", "xİ"); - assertStringTrim("UTF8_LCASE", "IXİ", "ix\u0307", "İ"); - assertStringTrim("UTF8_LCASE", "xi\u0307", "\u0307IX", ""); - assertStringTrim("UNICODE", "i", "i", ""); - assertStringTrim("UNICODE", "iii", "I", "iii"); - assertStringTrim("UNICODE", "I", "iii", "I"); - assertStringTrim("UNICODE", "ixi", "i", "x"); - assertStringTrim("UNICODE", "i", "İ", "i"); - assertStringTrim("UNICODE", "i\u0307", "İ", "i\u0307"); - assertStringTrim("UNICODE", "ii\u0307", "İi", "i\u0307"); - assertStringTrim("UNICODE", "iii\u0307", "İi", "i\u0307"); - assertStringTrim("UNICODE", "iiii\u0307", "iİ", "i\u0307"); - assertStringTrim("UNICODE", "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); - assertStringTrim("UNICODE", "i\u0307", "i", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrim("UNICODE", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrim("UNICODE", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrim("UNICODE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrim("UNICODE", "i\u0307İ", "İ", "i\u0307"); - assertStringTrim("UNICODE", "İ", "İ", ""); - assertStringTrim("UNICODE", "IXi", "İ", "IXi"); - assertStringTrim("UNICODE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrim("UNICODE", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrim("UNICODE", "i\u0307x", "ix\u0307İ", "i\u0307"); - assertStringTrim("UNICODE", "İ", "i", "İ"); - assertStringTrim("UNICODE", "İ", "\u0307", "İ"); - assertStringTrim("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrim("UNICODE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrim("UNICODE", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrim("UNICODE_CI", "i", "i", ""); - assertStringTrim("UNICODE_CI", "iii", "I", ""); - assertStringTrim("UNICODE_CI", "I", "iii", ""); - assertStringTrim("UNICODE_CI", "ixi", "i", "x"); - assertStringTrim("UNICODE_CI", "i", "İ", "i"); - assertStringTrim("UNICODE_CI", "i\u0307", "İ", ""); - assertStringTrim("UNICODE_CI", "ii\u0307", "İi", ""); - assertStringTrim("UNICODE_CI", "iii\u0307", "İi", ""); - assertStringTrim("UNICODE_CI", "iiii\u0307", "iİ", ""); - assertStringTrim("UNICODE_CI", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrim("UNICODE_CI", "i\u0307", "i", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307", "\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307i", "İ", "i"); - assertStringTrim("UNICODE_CI", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrim("UNICODE_CI", "i\u0307İ", "İ", ""); - assertStringTrim("UNICODE_CI", "İ", "İ", ""); - assertStringTrim("UNICODE_CI", "IXi", "İ", "IXi"); - assertStringTrim("UNICODE_CI", "ix\u0307", "Ixİ", "x\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307x", "IXİ", ""); - assertStringTrim("UNICODE_CI", "i\u0307x", "I\u0307xİ", ""); - assertStringTrim("UNICODE_CI", "İ", "i", "İ"); - assertStringTrim("UNICODE_CI", "İ", "\u0307", "İ"); - assertStringTrim("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "Ixİ", "i\u0307", "xİ"); - assertStringTrim("UNICODE_CI", "IXİ", "ix\u0307", "İ"); - assertStringTrim("UNICODE_CI", "xi\u0307", "\u0307IX", "i\u0307"); + assertStringTrim(UTF8_BINARY, "i", "i", ""); + assertStringTrim(UTF8_BINARY, "iii", "I", "iii"); + assertStringTrim(UTF8_BINARY, "I", "iii", "I"); + assertStringTrim(UTF8_BINARY, "ixi", "i", "x"); + assertStringTrim(UTF8_BINARY, "i", "İ", "i"); + assertStringTrim(UTF8_BINARY, "i\u0307", "İ", "i\u0307"); + assertStringTrim(UTF8_BINARY, "ii\u0307", "İi", "\u0307"); + assertStringTrim(UTF8_BINARY, "iii\u0307", "İi", "\u0307"); + assertStringTrim(UTF8_BINARY, "iiii\u0307", "iİ", "\u0307"); + assertStringTrim(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); + assertStringTrim(UTF8_BINARY, "i\u0307", "i", "\u0307"); + assertStringTrim(UTF8_BINARY, "i\u0307", "\u0307", "i"); + assertStringTrim(UTF8_BINARY, "i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307\u0307", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307i", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307i", "İ", "i\u0307i"); + assertStringTrim(UTF8_BINARY, "i\u0307İ", "i\u0307", "İ"); + assertStringTrim(UTF8_BINARY, "i\u0307İ", "İ", "i\u0307"); + assertStringTrim(UTF8_BINARY, "İ", "İ", ""); + assertStringTrim(UTF8_BINARY, "IXi", "İ", "IXi"); + assertStringTrim(UTF8_BINARY, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrim(UTF8_BINARY, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrim(UTF8_BINARY, "i\u0307x", "ix\u0307İ", ""); + assertStringTrim(UTF8_BINARY, "İ", "i", "İ"); + assertStringTrim(UTF8_BINARY, "İ", "\u0307", "İ"); + assertStringTrim(UTF8_BINARY, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrim(UTF8_BINARY, "IXİ", "ix\u0307", "IXİ"); + assertStringTrim(UTF8_BINARY, "xi\u0307", "\u0307IX", "xi"); + assertStringTrim(UTF8_LCASE, "i", "i", ""); + assertStringTrim(UTF8_LCASE, "iii", "I", ""); + assertStringTrim(UTF8_LCASE, "I", "iii", ""); + assertStringTrim(UTF8_LCASE, "ixi", "i", "x"); + assertStringTrim(UTF8_LCASE, "i", "İ", "i"); + assertStringTrim(UTF8_LCASE, "i\u0307", "İ", ""); + assertStringTrim(UTF8_LCASE, "ii\u0307", "İi", ""); + assertStringTrim(UTF8_LCASE, "iii\u0307", "İi", ""); + assertStringTrim(UTF8_LCASE, "iiii\u0307", "iİ", ""); + assertStringTrim(UTF8_LCASE, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrim(UTF8_LCASE, "i\u0307", "i", "\u0307"); + assertStringTrim(UTF8_LCASE, "i\u0307", "\u0307", "i"); + assertStringTrim(UTF8_LCASE, "i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307\u0307", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307i", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307i", "İ", "i"); + assertStringTrim(UTF8_LCASE, "i\u0307İ", "i\u0307", "İ"); + assertStringTrim(UTF8_LCASE, "i\u0307İ", "İ", ""); + assertStringTrim(UTF8_LCASE, "İ", "İ", ""); + assertStringTrim(UTF8_LCASE, "IXi", "İ", "IXi"); + assertStringTrim(UTF8_LCASE, "ix\u0307", "Ixİ", "\u0307"); + assertStringTrim(UTF8_LCASE, "i\u0307x", "IXİ", ""); + assertStringTrim(UTF8_LCASE, "i\u0307x", "I\u0307xİ", ""); + assertStringTrim(UTF8_LCASE, "İ", "i", "İ"); + assertStringTrim(UTF8_LCASE, "İ", "\u0307", "İ"); + assertStringTrim(UTF8_LCASE, "Ixİ", "i\u0307", "xİ"); + assertStringTrim(UTF8_LCASE, "IXİ", "ix\u0307", "İ"); + assertStringTrim(UTF8_LCASE, "xi\u0307", "\u0307IX", ""); + assertStringTrim(UNICODE, "i", "i", ""); + assertStringTrim(UNICODE, "iii", "I", "iii"); + assertStringTrim(UNICODE, "I", "iii", "I"); + assertStringTrim(UNICODE, "ixi", "i", "x"); + assertStringTrim(UNICODE, "i", "İ", "i"); + assertStringTrim(UNICODE, "i\u0307", "İ", "i\u0307"); + assertStringTrim(UNICODE, "ii\u0307", "İi", "i\u0307"); + assertStringTrim(UNICODE, "iii\u0307", "İi", "i\u0307"); + assertStringTrim(UNICODE, "iiii\u0307", "iİ", "i\u0307"); + assertStringTrim(UNICODE, "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); + assertStringTrim(UNICODE, "i\u0307", "i", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrim(UNICODE, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrim(UNICODE, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrim(UNICODE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrim(UNICODE, "i\u0307İ", "İ", "i\u0307"); + assertStringTrim(UNICODE, "İ", "İ", ""); + assertStringTrim(UNICODE, "IXi", "İ", "IXi"); + assertStringTrim(UNICODE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrim(UNICODE, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrim(UNICODE, "i\u0307x", "ix\u0307İ", "i\u0307"); + assertStringTrim(UNICODE, "İ", "i", "İ"); + assertStringTrim(UNICODE, "İ", "\u0307", "İ"); + assertStringTrim(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrim(UNICODE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrim(UNICODE, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrim(UNICODE_CI, "i", "i", ""); + assertStringTrim(UNICODE_CI, "iii", "I", ""); + assertStringTrim(UNICODE_CI, "I", "iii", ""); + assertStringTrim(UNICODE_CI, "ixi", "i", "x"); + assertStringTrim(UNICODE_CI, "i", "İ", "i"); + assertStringTrim(UNICODE_CI, "i\u0307", "İ", ""); + assertStringTrim(UNICODE_CI, "ii\u0307", "İi", ""); + assertStringTrim(UNICODE_CI, "iii\u0307", "İi", ""); + assertStringTrim(UNICODE_CI, "iiii\u0307", "iİ", ""); + assertStringTrim(UNICODE_CI, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrim(UNICODE_CI, "i\u0307", "i", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307", "\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307i", "İ", "i"); + assertStringTrim(UNICODE_CI, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrim(UNICODE_CI, "i\u0307İ", "İ", ""); + assertStringTrim(UNICODE_CI, "İ", "İ", ""); + assertStringTrim(UNICODE_CI, "IXi", "İ", "IXi"); + assertStringTrim(UNICODE_CI, "ix\u0307", "Ixİ", "x\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307x", "IXİ", ""); + assertStringTrim(UNICODE_CI, "i\u0307x", "I\u0307xİ", ""); + assertStringTrim(UNICODE_CI, "İ", "i", "İ"); + assertStringTrim(UNICODE_CI, "İ", "\u0307", "İ"); + assertStringTrim(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "Ixİ", "i\u0307", "xİ"); + assertStringTrim(UNICODE_CI, "IXİ", "ix\u0307", "İ"); + assertStringTrim(UNICODE_CI, "xi\u0307", "\u0307IX", "i\u0307"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTrim("UTF8_BINARY", "ςxς", "σ", "ςxς"); - assertStringTrim("UTF8_BINARY", "ςxς", "ς", "x"); - assertStringTrim("UTF8_BINARY", "ςxς", "Σ", "ςxς"); - assertStringTrim("UTF8_BINARY", "σxσ", "σ", "x"); - assertStringTrim("UTF8_BINARY", "σxσ", "ς", "σxσ"); - assertStringTrim("UTF8_BINARY", "σxσ", "Σ", "σxσ"); - assertStringTrim("UTF8_BINARY", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrim("UTF8_BINARY", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrim("UTF8_BINARY", "ΣxΣ", "Σ", "x"); - assertStringTrim("UTF8_LCASE", "ςxς", "σ", "x"); - assertStringTrim("UTF8_LCASE", "ςxς", "ς", "x"); - assertStringTrim("UTF8_LCASE", "ςxς", "Σ", "x"); - assertStringTrim("UTF8_LCASE", "σxσ", "σ", "x"); - assertStringTrim("UTF8_LCASE", "σxσ", "ς", "x"); - assertStringTrim("UTF8_LCASE", "σxσ", "Σ", "x"); - assertStringTrim("UTF8_LCASE", "ΣxΣ", "σ", "x"); - assertStringTrim("UTF8_LCASE", "ΣxΣ", "ς", "x"); - assertStringTrim("UTF8_LCASE", "ΣxΣ", "Σ", "x"); - assertStringTrim("UNICODE", "ςxς", "σ", "ςxς"); - assertStringTrim("UNICODE", "ςxς", "ς", "x"); - assertStringTrim("UNICODE", "ςxς", "Σ", "ςxς"); - assertStringTrim("UNICODE", "σxσ", "σ", "x"); - assertStringTrim("UNICODE", "σxσ", "ς", "σxσ"); - assertStringTrim("UNICODE", "σxσ", "Σ", "σxσ"); - assertStringTrim("UNICODE", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrim("UNICODE", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrim("UNICODE", "ΣxΣ", "Σ", "x"); - assertStringTrim("UNICODE_CI", "ςxς", "σ", "x"); - assertStringTrim("UNICODE_CI", "ςxς", "ς", "x"); - assertStringTrim("UNICODE_CI", "ςxς", "Σ", "x"); - assertStringTrim("UNICODE_CI", "σxσ", "σ", "x"); - assertStringTrim("UNICODE_CI", "σxσ", "ς", "x"); - assertStringTrim("UNICODE_CI", "σxσ", "Σ", "x"); - assertStringTrim("UNICODE_CI", "ΣxΣ", "σ", "x"); - assertStringTrim("UNICODE_CI", "ΣxΣ", "ς", "x"); - assertStringTrim("UNICODE_CI", "ΣxΣ", "Σ", "x"); + assertStringTrim(UTF8_BINARY, "ςxς", "σ", "ςxς"); + assertStringTrim(UTF8_BINARY, "ςxς", "ς", "x"); + assertStringTrim(UTF8_BINARY, "ςxς", "Σ", "ςxς"); + assertStringTrim(UTF8_BINARY, "σxσ", "σ", "x"); + assertStringTrim(UTF8_BINARY, "σxσ", "ς", "σxσ"); + assertStringTrim(UTF8_BINARY, "σxσ", "Σ", "σxσ"); + assertStringTrim(UTF8_BINARY, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrim(UTF8_BINARY, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrim(UTF8_BINARY, "ΣxΣ", "Σ", "x"); + assertStringTrim(UTF8_LCASE, "ςxς", "σ", "x"); + assertStringTrim(UTF8_LCASE, "ςxς", "ς", "x"); + assertStringTrim(UTF8_LCASE, "ςxς", "Σ", "x"); + assertStringTrim(UTF8_LCASE, "σxσ", "σ", "x"); + assertStringTrim(UTF8_LCASE, "σxσ", "ς", "x"); + assertStringTrim(UTF8_LCASE, "σxσ", "Σ", "x"); + assertStringTrim(UTF8_LCASE, "ΣxΣ", "σ", "x"); + assertStringTrim(UTF8_LCASE, "ΣxΣ", "ς", "x"); + assertStringTrim(UTF8_LCASE, "ΣxΣ", "Σ", "x"); + assertStringTrim(UNICODE, "ςxς", "σ", "ςxς"); + assertStringTrim(UNICODE, "ςxς", "ς", "x"); + assertStringTrim(UNICODE, "ςxς", "Σ", "ςxς"); + assertStringTrim(UNICODE, "σxσ", "σ", "x"); + assertStringTrim(UNICODE, "σxσ", "ς", "σxσ"); + assertStringTrim(UNICODE, "σxσ", "Σ", "σxσ"); + assertStringTrim(UNICODE, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrim(UNICODE, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrim(UNICODE, "ΣxΣ", "Σ", "x"); + assertStringTrim(UNICODE_CI, "ςxς", "σ", "x"); + assertStringTrim(UNICODE_CI, "ςxς", "ς", "x"); + assertStringTrim(UNICODE_CI, "ςxς", "Σ", "x"); + assertStringTrim(UNICODE_CI, "σxσ", "σ", "x"); + assertStringTrim(UNICODE_CI, "σxσ", "ς", "x"); + assertStringTrim(UNICODE_CI, "σxσ", "Σ", "x"); + assertStringTrim(UNICODE_CI, "ΣxΣ", "σ", "x"); + assertStringTrim(UNICODE_CI, "ΣxΣ", "ς", "x"); + assertStringTrim(UNICODE_CI, "ΣxΣ", "Σ", "x"); // Unicode normalization. - assertStringTrim("UTF8_BINARY", "åβγδa\u030A", "å", "βγδa\u030A"); - assertStringTrim("UTF8_LCASE", "åβγδa\u030A", "Å", "βγδa\u030A"); - assertStringTrim("UNICODE", "åβγδa\u030A", "å", "βγδ"); - assertStringTrim("UNICODE_CI", "åβγδa\u030A", "Å", "βγδ"); + assertStringTrim(UTF8_BINARY, "åβγδa\u030A", "å", "βγδa\u030A"); + assertStringTrim(UTF8_LCASE, "åβγδa\u030A", "Å", "βγδa\u030A"); + assertStringTrim(UNICODE, "åβγδa\u030A", "å", "βγδ"); + assertStringTrim(UNICODE_CI, "åβγδa\u030A", "Å", "βγδ"); // Surrogate pairs. - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UNICODE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UNICODE", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UNICODE", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UNICODE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UTF8_BINARY", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UTF8_LCASE", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UNICODE", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UNICODE_CI", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UTF8_BINARY", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UTF8_LCASE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UNICODE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UNICODE_CI", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UTF8_BINARY", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UTF8_LCASE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UNICODE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UNICODE_CI", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UTF8_BINARY", "𐐅", "𐐅", ""); - assertStringTrim("UTF8_LCASE", "𐐅", "𐐅", ""); - assertStringTrim("UNICODE", "𐐅", "𐐅", ""); - assertStringTrim("UNICODE_CI", "𐐅", "𐐅", ""); - assertStringTrim("UTF8_BINARY", "𐐅", "𐐭", "𐐅"); - assertStringTrim("UTF8_LCASE", "𐐅", "𐐭", ""); - assertStringTrim("UNICODE", "𐐅", "𐐭", "𐐅"); - assertStringTrim("UNICODE_CI", "𐐅", "𐐭", ""); - assertStringTrim("UTF8_BINARY", "𝔸", "𝔸", ""); - assertStringTrim("UTF8_LCASE", "𝔸", "𝔸", ""); - assertStringTrim("UNICODE", "𝔸", "𝔸", ""); - assertStringTrim("UNICODE_CI", "𝔸", "𝔸", ""); - assertStringTrim("UTF8_BINARY", "𝔸", "A", "𝔸"); - assertStringTrim("UTF8_LCASE", "𝔸", "A", "𝔸"); - assertStringTrim("UNICODE", "𝔸", "A", "𝔸"); - assertStringTrim("UNICODE_CI", "𝔸", "A", ""); - assertStringTrim("UTF8_BINARY", "𝔸", "a", "𝔸"); - assertStringTrim("UTF8_LCASE", "𝔸", "a", "𝔸"); - assertStringTrim("UNICODE", "𝔸", "a", "𝔸"); - assertStringTrim("UNICODE_CI", "𝔸", "a", ""); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UNICODE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UNICODE, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UNICODE, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UNICODE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UTF8_BINARY, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UTF8_LCASE, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UNICODE, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UNICODE_CI, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UTF8_BINARY, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UTF8_LCASE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UNICODE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UNICODE_CI, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UTF8_BINARY, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UTF8_LCASE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UNICODE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UNICODE_CI, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UTF8_BINARY, "𐐅", "𐐅", ""); + assertStringTrim(UTF8_LCASE, "𐐅", "𐐅", ""); + assertStringTrim(UNICODE, "𐐅", "𐐅", ""); + assertStringTrim(UNICODE_CI, "𐐅", "𐐅", ""); + assertStringTrim(UTF8_BINARY, "𐐅", "𐐭", "𐐅"); + assertStringTrim(UTF8_LCASE, "𐐅", "𐐭", ""); + assertStringTrim(UNICODE, "𐐅", "𐐭", "𐐅"); + assertStringTrim(UNICODE_CI, "𐐅", "𐐭", ""); + assertStringTrim(UTF8_BINARY, "𝔸", "𝔸", ""); + assertStringTrim(UTF8_LCASE, "𝔸", "𝔸", ""); + assertStringTrim(UNICODE, "𝔸", "𝔸", ""); + assertStringTrim(UNICODE_CI, "𝔸", "𝔸", ""); + assertStringTrim(UTF8_BINARY, "𝔸", "A", "𝔸"); + assertStringTrim(UTF8_LCASE, "𝔸", "A", "𝔸"); + assertStringTrim(UNICODE, "𝔸", "A", "𝔸"); + assertStringTrim(UNICODE_CI, "𝔸", "A", ""); + assertStringTrim(UTF8_BINARY, "𝔸", "a", "𝔸"); + assertStringTrim(UTF8_LCASE, "𝔸", "a", "𝔸"); + assertStringTrim(UNICODE, "𝔸", "a", "𝔸"); + assertStringTrim(UNICODE_CI, "𝔸", "a", ""); } /** @@ -3078,277 +3079,277 @@ private void assertStringTrimLeft(String collationName, String sourceString, Str @Test public void testStringTrimLeft() throws SparkException { // Basic tests - UTF8_BINARY. - assertStringTrimLeft("UTF8_BINARY", "", "", ""); - assertStringTrimLeft("UTF8_BINARY", "", "xyz", ""); - assertStringTrimLeft("UTF8_BINARY", "asd", "", "asd"); - assertStringTrimLeft("UTF8_BINARY", "asd", null, "asd"); - assertStringTrimLeft("UTF8_BINARY", " asd ", null, "asd "); - assertStringTrimLeft("UTF8_BINARY", " a世a ", null, "a世a "); - assertStringTrimLeft("UTF8_BINARY", "asd", "x", "asd"); - assertStringTrimLeft("UTF8_BINARY", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UTF8_BINARY", "xa世ax", "x", "a世ax"); + assertStringTrimLeft(UTF8_BINARY, "", "", ""); + assertStringTrimLeft(UTF8_BINARY, "", "xyz", ""); + assertStringTrimLeft(UTF8_BINARY, "asd", "", "asd"); + assertStringTrimLeft(UTF8_BINARY, "asd", null, "asd"); + assertStringTrimLeft(UTF8_BINARY, " asd ", null, "asd "); + assertStringTrimLeft(UTF8_BINARY, " a世a ", null, "a世a "); + assertStringTrimLeft(UTF8_BINARY, "asd", "x", "asd"); + assertStringTrimLeft(UTF8_BINARY, "xxasdxx", "x", "asdxx"); + assertStringTrimLeft(UTF8_BINARY, "xa世ax", "x", "a世ax"); // Basic tests - UTF8_LCASE. - assertStringTrimLeft("UTF8_LCASE", "", "", ""); - assertStringTrimLeft("UTF8_LCASE", "", "xyz", ""); - assertStringTrimLeft("UTF8_LCASE", "asd", "", "asd"); - assertStringTrimLeft("UTF8_LCASE", "asd", null, "asd"); - assertStringTrimLeft("UTF8_LCASE", " asd ", null, "asd "); - assertStringTrimLeft("UTF8_LCASE", " a世a ", null, "a世a "); - assertStringTrimLeft("UTF8_LCASE", "asd", "x", "asd"); - assertStringTrimLeft("UTF8_LCASE", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UTF8_LCASE", "xa世ax", "x", "a世ax"); + assertStringTrimLeft(UTF8_LCASE, "", "", ""); + assertStringTrimLeft(UTF8_LCASE, "", "xyz", ""); + assertStringTrimLeft(UTF8_LCASE, "asd", "", "asd"); + assertStringTrimLeft(UTF8_LCASE, "asd", null, "asd"); + assertStringTrimLeft(UTF8_LCASE, " asd ", null, "asd "); + assertStringTrimLeft(UTF8_LCASE, " a世a ", null, "a世a "); + assertStringTrimLeft(UTF8_LCASE, "asd", "x", "asd"); + assertStringTrimLeft(UTF8_LCASE, "xxasdxx", "x", "asdxx"); + assertStringTrimLeft(UTF8_LCASE, "xa世ax", "x", "a世ax"); // Basic tests - UNICODE. - assertStringTrimLeft("UNICODE", "", "", ""); - assertStringTrimLeft("UNICODE", "", "xyz", ""); - assertStringTrimLeft("UNICODE", "asd", "", "asd"); - assertStringTrimLeft("UNICODE", "asd", null, "asd"); - assertStringTrimLeft("UNICODE", " asd ", null, "asd "); - assertStringTrimLeft("UNICODE", " a世a ", null, "a世a "); - assertStringTrimLeft("UNICODE", "asd", "x", "asd"); - assertStringTrimLeft("UNICODE", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UNICODE", "xa世ax", "x", "a世ax"); + assertStringTrimLeft(UNICODE, "", "", ""); + assertStringTrimLeft(UNICODE, "", "xyz", ""); + assertStringTrimLeft(UNICODE, "asd", "", "asd"); + assertStringTrimLeft(UNICODE, "asd", null, "asd"); + assertStringTrimLeft(UNICODE, " asd ", null, "asd "); + assertStringTrimLeft(UNICODE, " a世a ", null, "a世a "); + assertStringTrimLeft(UNICODE, "asd", "x", "asd"); + assertStringTrimLeft(UNICODE, "xxasdxx", "x", "asdxx"); + assertStringTrimLeft(UNICODE, "xa世ax", "x", "a世ax"); // Basic tests - UNICODE_CI. - assertStringTrimLeft("UNICODE_CI", "", "", ""); - assertStringTrimLeft("UNICODE_CI", "", "xyz", ""); - assertStringTrimLeft("UNICODE_CI", "asd", "", "asd"); - assertStringTrimLeft("UNICODE_CI", "asd", null, "asd"); - assertStringTrimLeft("UNICODE_CI", " asd ", null, "asd "); - assertStringTrimLeft("UNICODE_CI", " a世a ", null, "a世a "); - assertStringTrimLeft("UNICODE_CI", "asd", "x", "asd"); - assertStringTrimLeft("UNICODE_CI", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UNICODE_CI", "xa世ax", "x", "a世ax"); + assertStringTrimLeft(UNICODE_CI, "", "", ""); + assertStringTrimLeft(UNICODE_CI, "", "xyz", ""); + assertStringTrimLeft(UNICODE_CI, "asd", "", "asd"); + assertStringTrimLeft(UNICODE_CI, "asd", null, "asd"); + assertStringTrimLeft(UNICODE_CI, " asd ", null, "asd "); + assertStringTrimLeft(UNICODE_CI, " a世a ", null, "a世a "); + assertStringTrimLeft(UNICODE_CI, "asd", "x", "asd"); + assertStringTrimLeft(UNICODE_CI, "xxasdxx", "x", "asdxx"); + assertStringTrimLeft(UNICODE_CI, "xa世ax", "x", "a世ax"); // Case variation. - assertStringTrimLeft("UTF8_BINARY", "ddsXXXaa", "asd", "XXXaa"); - assertStringTrimLeft("UTF8_LCASE", "ddsXXXaa", "aSd", "XXXaa"); - assertStringTrimLeft("UNICODE", "ddsXXXaa", "asd", "XXXaa"); - assertStringTrimLeft("UNICODE_CI", "ddsXXXaa", "aSd", "XXXaa"); + assertStringTrimLeft(UTF8_BINARY, "ddsXXXaa", "asd", "XXXaa"); + assertStringTrimLeft(UTF8_LCASE, "ddsXXXaa", "aSd", "XXXaa"); + assertStringTrimLeft(UNICODE, "ddsXXXaa", "asd", "XXXaa"); + assertStringTrimLeft(UNICODE_CI, "ddsXXXaa", "aSd", "XXXaa"); // One-to-many case mapping (e.g. Turkish dotted I).. - assertStringTrimLeft("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimLeft("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimLeft("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "aaaẞ"); - assertStringTrimLeft("UTF8_LCASE", "ẞaaaẞ", "ß", "aaaẞ"); - assertStringTrimLeft("UTF8_LCASE", "ßaaaß", "ẞ", "aaaß"); - assertStringTrimLeft("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "aaaẞ"); - assertStringTrimLeft("UNICODE", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimLeft("UNICODE", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimLeft("UNICODE", "Ëaaaẞ", "Ëẞ", "aaaẞ"); - assertStringTrimLeft("UNICODE_CI", "ẞaaaẞ", "ß", "aaaẞ"); - assertStringTrimLeft("UNICODE_CI", "ßaaaß", "ẞ", "aaaß"); - assertStringTrimLeft("UNICODE_CI", "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UTF8_BINARY, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimLeft(UTF8_BINARY, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimLeft(UTF8_BINARY, "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UTF8_LCASE, "ẞaaaẞ", "ß", "aaaẞ"); + assertStringTrimLeft(UTF8_LCASE, "ßaaaß", "ẞ", "aaaß"); + assertStringTrimLeft(UTF8_LCASE, "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UNICODE, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimLeft(UNICODE, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimLeft(UNICODE, "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UNICODE_CI, "ẞaaaẞ", "ß", "aaaẞ"); + assertStringTrimLeft(UNICODE_CI, "ßaaaß", "ẞ", "aaaß"); + assertStringTrimLeft(UNICODE_CI, "Ëaaaẞ", "Ëẞ", "aaaẞ"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTrimLeft("UTF8_BINARY", "i", "i", ""); - assertStringTrimLeft("UTF8_BINARY", "iii", "I", "iii"); - assertStringTrimLeft("UTF8_BINARY", "I", "iii", "I"); - assertStringTrimLeft("UTF8_BINARY", "ixi", "i", "xi"); - assertStringTrimLeft("UTF8_BINARY", "i", "İ", "i"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "İ", "i\u0307"); - assertStringTrimLeft("UTF8_BINARY", "ii\u0307", "İi", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "iii\u0307", "İi", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "iiii\u0307", "iİ", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "i", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307i", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307İ", "i\u0307", "İ"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307İ", "İ", "i\u0307İ"); - assertStringTrimLeft("UTF8_BINARY", "İ", "İ", ""); - assertStringTrimLeft("UTF8_BINARY", "IXi", "İ", "IXi"); - assertStringTrimLeft("UTF8_BINARY", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307x", "ix\u0307İ", ""); - assertStringTrimLeft("UTF8_BINARY", "İ", "i", "İ"); - assertStringTrimLeft("UTF8_BINARY", "İ", "\u0307", "İ"); - assertStringTrimLeft("UTF8_BINARY", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimLeft("UTF8_BINARY", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimLeft("UTF8_BINARY", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i", "i", ""); - assertStringTrimLeft("UTF8_LCASE", "iii", "I", ""); - assertStringTrimLeft("UTF8_LCASE", "I", "iii", ""); - assertStringTrimLeft("UTF8_LCASE", "ixi", "i", "xi"); - assertStringTrimLeft("UTF8_LCASE", "i", "İ", "i"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "İ", ""); - assertStringTrimLeft("UTF8_LCASE", "ii\u0307", "İi", ""); - assertStringTrimLeft("UTF8_LCASE", "iii\u0307", "İi", ""); - assertStringTrimLeft("UTF8_LCASE", "iiii\u0307", "iİ", ""); - assertStringTrimLeft("UTF8_LCASE", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "i", "\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307i", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307i", "İ", "i"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307İ", "i\u0307", "İ"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307İ", "İ", ""); - assertStringTrimLeft("UTF8_LCASE", "İ", "İ", ""); - assertStringTrimLeft("UTF8_LCASE", "IXi", "İ", "IXi"); - assertStringTrimLeft("UTF8_LCASE", "ix\u0307", "Ixİ", "\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307x", "IXİ", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimLeft("UTF8_LCASE", "İ", "i", "İ"); - assertStringTrimLeft("UTF8_LCASE", "İ", "\u0307", "İ"); - assertStringTrimLeft("UTF8_LCASE", "Ixİ", "i\u0307", "xİ"); - assertStringTrimLeft("UTF8_LCASE", "IXİ", "ix\u0307", "İ"); - assertStringTrimLeft("UTF8_LCASE", "xi\u0307", "\u0307IX", ""); - assertStringTrimLeft("UNICODE", "i", "i", ""); - assertStringTrimLeft("UNICODE", "iii", "I", "iii"); - assertStringTrimLeft("UNICODE", "I", "iii", "I"); - assertStringTrimLeft("UNICODE", "ixi", "i", "xi"); - assertStringTrimLeft("UNICODE", "i", "İ", "i"); - assertStringTrimLeft("UNICODE", "i\u0307", "İ", "i\u0307"); - assertStringTrimLeft("UNICODE", "ii\u0307", "İi", "i\u0307"); - assertStringTrimLeft("UNICODE", "iii\u0307", "İi", "i\u0307"); - assertStringTrimLeft("UNICODE", "iiii\u0307", "iİ", "i\u0307"); - assertStringTrimLeft("UNICODE", "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307", "i", "i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307i", "i\u0307", "i\u0307i"); - assertStringTrimLeft("UNICODE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimLeft("UNICODE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimLeft("UNICODE", "i\u0307İ", "İ", "i\u0307İ"); - assertStringTrimLeft("UNICODE", "İ", "İ", ""); - assertStringTrimLeft("UNICODE", "IXi", "İ", "IXi"); - assertStringTrimLeft("UNICODE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimLeft("UNICODE", "i\u0307x", "ix\u0307İ", "i\u0307x"); - assertStringTrimLeft("UNICODE", "İ", "i", "İ"); - assertStringTrimLeft("UNICODE", "İ", "\u0307", "İ"); - assertStringTrimLeft("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimLeft("UNICODE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimLeft("UNICODE", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrimLeft("UNICODE_CI", "i", "i", ""); - assertStringTrimLeft("UNICODE_CI", "iii", "I", ""); - assertStringTrimLeft("UNICODE_CI", "I", "iii", ""); - assertStringTrimLeft("UNICODE_CI", "ixi", "i", "xi"); - assertStringTrimLeft("UNICODE_CI", "i", "İ", "i"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "İ", ""); - assertStringTrimLeft("UNICODE_CI", "ii\u0307", "İi", ""); - assertStringTrimLeft("UNICODE_CI", "iii\u0307", "İi", ""); - assertStringTrimLeft("UNICODE_CI", "iiii\u0307", "iİ", ""); - assertStringTrimLeft("UNICODE_CI", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "i", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307i", "i\u0307", "i\u0307i"); - assertStringTrimLeft("UNICODE_CI", "i\u0307i", "İ", "i"); - assertStringTrimLeft("UNICODE_CI", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimLeft("UNICODE_CI", "i\u0307İ", "İ", ""); - assertStringTrimLeft("UNICODE_CI", "İ", "İ", ""); - assertStringTrimLeft("UNICODE_CI", "IXi", "İ", "IXi"); - assertStringTrimLeft("UNICODE_CI", "ix\u0307", "Ixİ", "x\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307x", "IXİ", ""); - assertStringTrimLeft("UNICODE_CI", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimLeft("UNICODE_CI", "İ", "i", "İ"); - assertStringTrimLeft("UNICODE_CI", "İ", "\u0307", "İ"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "Ixİ", "i\u0307", "xİ"); - assertStringTrimLeft("UNICODE_CI", "IXİ", "ix\u0307", "İ"); - assertStringTrimLeft("UNICODE_CI", "xi\u0307", "\u0307IX", "i\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i", "i", ""); + assertStringTrimLeft(UTF8_BINARY, "iii", "I", "iii"); + assertStringTrimLeft(UTF8_BINARY, "I", "iii", "I"); + assertStringTrimLeft(UTF8_BINARY, "ixi", "i", "xi"); + assertStringTrimLeft(UTF8_BINARY, "i", "İ", "i"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "İ", "i\u0307"); + assertStringTrimLeft(UTF8_BINARY, "ii\u0307", "İi", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "iii\u0307", "İi", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "iiii\u0307", "iİ", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "i", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307i", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307İ", "i\u0307", "İ"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307İ", "İ", "i\u0307İ"); + assertStringTrimLeft(UTF8_BINARY, "İ", "İ", ""); + assertStringTrimLeft(UTF8_BINARY, "IXi", "İ", "IXi"); + assertStringTrimLeft(UTF8_BINARY, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307x", "ix\u0307İ", ""); + assertStringTrimLeft(UTF8_BINARY, "İ", "i", "İ"); + assertStringTrimLeft(UTF8_BINARY, "İ", "\u0307", "İ"); + assertStringTrimLeft(UTF8_BINARY, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimLeft(UTF8_BINARY, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimLeft(UTF8_BINARY, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i", "i", ""); + assertStringTrimLeft(UTF8_LCASE, "iii", "I", ""); + assertStringTrimLeft(UTF8_LCASE, "I", "iii", ""); + assertStringTrimLeft(UTF8_LCASE, "ixi", "i", "xi"); + assertStringTrimLeft(UTF8_LCASE, "i", "İ", "i"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "İ", ""); + assertStringTrimLeft(UTF8_LCASE, "ii\u0307", "İi", ""); + assertStringTrimLeft(UTF8_LCASE, "iii\u0307", "İi", ""); + assertStringTrimLeft(UTF8_LCASE, "iiii\u0307", "iİ", ""); + assertStringTrimLeft(UTF8_LCASE, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "i", "\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307i", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307i", "İ", "i"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307İ", "i\u0307", "İ"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307İ", "İ", ""); + assertStringTrimLeft(UTF8_LCASE, "İ", "İ", ""); + assertStringTrimLeft(UTF8_LCASE, "IXi", "İ", "IXi"); + assertStringTrimLeft(UTF8_LCASE, "ix\u0307", "Ixİ", "\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307x", "IXİ", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimLeft(UTF8_LCASE, "İ", "i", "İ"); + assertStringTrimLeft(UTF8_LCASE, "İ", "\u0307", "İ"); + assertStringTrimLeft(UTF8_LCASE, "Ixİ", "i\u0307", "xİ"); + assertStringTrimLeft(UTF8_LCASE, "IXİ", "ix\u0307", "İ"); + assertStringTrimLeft(UTF8_LCASE, "xi\u0307", "\u0307IX", ""); + assertStringTrimLeft(UNICODE, "i", "i", ""); + assertStringTrimLeft(UNICODE, "iii", "I", "iii"); + assertStringTrimLeft(UNICODE, "I", "iii", "I"); + assertStringTrimLeft(UNICODE, "ixi", "i", "xi"); + assertStringTrimLeft(UNICODE, "i", "İ", "i"); + assertStringTrimLeft(UNICODE, "i\u0307", "İ", "i\u0307"); + assertStringTrimLeft(UNICODE, "ii\u0307", "İi", "i\u0307"); + assertStringTrimLeft(UNICODE, "iii\u0307", "İi", "i\u0307"); + assertStringTrimLeft(UNICODE, "iiii\u0307", "iİ", "i\u0307"); + assertStringTrimLeft(UNICODE, "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307", "i", "i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307i", "i\u0307", "i\u0307i"); + assertStringTrimLeft(UNICODE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimLeft(UNICODE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimLeft(UNICODE, "i\u0307İ", "İ", "i\u0307İ"); + assertStringTrimLeft(UNICODE, "İ", "İ", ""); + assertStringTrimLeft(UNICODE, "IXi", "İ", "IXi"); + assertStringTrimLeft(UNICODE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimLeft(UNICODE, "i\u0307x", "ix\u0307İ", "i\u0307x"); + assertStringTrimLeft(UNICODE, "İ", "i", "İ"); + assertStringTrimLeft(UNICODE, "İ", "\u0307", "İ"); + assertStringTrimLeft(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimLeft(UNICODE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimLeft(UNICODE, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimLeft(UNICODE_CI, "i", "i", ""); + assertStringTrimLeft(UNICODE_CI, "iii", "I", ""); + assertStringTrimLeft(UNICODE_CI, "I", "iii", ""); + assertStringTrimLeft(UNICODE_CI, "ixi", "i", "xi"); + assertStringTrimLeft(UNICODE_CI, "i", "İ", "i"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "İ", ""); + assertStringTrimLeft(UNICODE_CI, "ii\u0307", "İi", ""); + assertStringTrimLeft(UNICODE_CI, "iii\u0307", "İi", ""); + assertStringTrimLeft(UNICODE_CI, "iiii\u0307", "iİ", ""); + assertStringTrimLeft(UNICODE_CI, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "i", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307i", "i\u0307", "i\u0307i"); + assertStringTrimLeft(UNICODE_CI, "i\u0307i", "İ", "i"); + assertStringTrimLeft(UNICODE_CI, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimLeft(UNICODE_CI, "i\u0307İ", "İ", ""); + assertStringTrimLeft(UNICODE_CI, "İ", "İ", ""); + assertStringTrimLeft(UNICODE_CI, "IXi", "İ", "IXi"); + assertStringTrimLeft(UNICODE_CI, "ix\u0307", "Ixİ", "x\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307x", "IXİ", ""); + assertStringTrimLeft(UNICODE_CI, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimLeft(UNICODE_CI, "İ", "i", "İ"); + assertStringTrimLeft(UNICODE_CI, "İ", "\u0307", "İ"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "Ixİ", "i\u0307", "xİ"); + assertStringTrimLeft(UNICODE_CI, "IXİ", "ix\u0307", "İ"); + assertStringTrimLeft(UNICODE_CI, "xi\u0307", "\u0307IX", "i\u0307"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTrimLeft("UTF8_BINARY", "ςxς", "σ", "ςxς"); - assertStringTrimLeft("UTF8_BINARY", "ςxς", "ς", "xς"); - assertStringTrimLeft("UTF8_BINARY", "ςxς", "Σ", "ςxς"); - assertStringTrimLeft("UTF8_BINARY", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UTF8_BINARY", "σxσ", "ς", "σxσ"); - assertStringTrimLeft("UTF8_BINARY", "σxσ", "Σ", "σxσ"); - assertStringTrimLeft("UTF8_BINARY", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimLeft("UTF8_BINARY", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimLeft("UTF8_BINARY", "ΣxΣ", "Σ", "xΣ"); - assertStringTrimLeft("UTF8_LCASE", "ςxς", "σ", "xς"); - assertStringTrimLeft("UTF8_LCASE", "ςxς", "ς", "xς"); - assertStringTrimLeft("UTF8_LCASE", "ςxς", "Σ", "xς"); - assertStringTrimLeft("UTF8_LCASE", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UTF8_LCASE", "σxσ", "ς", "xσ"); - assertStringTrimLeft("UTF8_LCASE", "σxσ", "Σ", "xσ"); - assertStringTrimLeft("UTF8_LCASE", "ΣxΣ", "σ", "xΣ"); - assertStringTrimLeft("UTF8_LCASE", "ΣxΣ", "ς", "xΣ"); - assertStringTrimLeft("UTF8_LCASE", "ΣxΣ", "Σ", "xΣ"); - assertStringTrimLeft("UNICODE", "ςxς", "σ", "ςxς"); - assertStringTrimLeft("UNICODE", "ςxς", "ς", "xς"); - assertStringTrimLeft("UNICODE", "ςxς", "Σ", "ςxς"); - assertStringTrimLeft("UNICODE", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UNICODE", "σxσ", "ς", "σxσ"); - assertStringTrimLeft("UNICODE", "σxσ", "Σ", "σxσ"); - assertStringTrimLeft("UNICODE", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimLeft("UNICODE", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimLeft("UNICODE", "ΣxΣ", "Σ", "xΣ"); - assertStringTrimLeft("UNICODE_CI", "ςxς", "σ", "xς"); - assertStringTrimLeft("UNICODE_CI", "ςxς", "ς", "xς"); - assertStringTrimLeft("UNICODE_CI", "ςxς", "Σ", "xς"); - assertStringTrimLeft("UNICODE_CI", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UNICODE_CI", "σxσ", "ς", "xσ"); - assertStringTrimLeft("UNICODE_CI", "σxσ", "Σ", "xσ"); - assertStringTrimLeft("UNICODE_CI", "ΣxΣ", "σ", "xΣ"); - assertStringTrimLeft("UNICODE_CI", "ΣxΣ", "ς", "xΣ"); - assertStringTrimLeft("UNICODE_CI", "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UTF8_BINARY, "ςxς", "σ", "ςxς"); + assertStringTrimLeft(UTF8_BINARY, "ςxς", "ς", "xς"); + assertStringTrimLeft(UTF8_BINARY, "ςxς", "Σ", "ςxς"); + assertStringTrimLeft(UTF8_BINARY, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UTF8_BINARY, "σxσ", "ς", "σxσ"); + assertStringTrimLeft(UTF8_BINARY, "σxσ", "Σ", "σxσ"); + assertStringTrimLeft(UTF8_BINARY, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimLeft(UTF8_BINARY, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimLeft(UTF8_BINARY, "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UTF8_LCASE, "ςxς", "σ", "xς"); + assertStringTrimLeft(UTF8_LCASE, "ςxς", "ς", "xς"); + assertStringTrimLeft(UTF8_LCASE, "ςxς", "Σ", "xς"); + assertStringTrimLeft(UTF8_LCASE, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UTF8_LCASE, "σxσ", "ς", "xσ"); + assertStringTrimLeft(UTF8_LCASE, "σxσ", "Σ", "xσ"); + assertStringTrimLeft(UTF8_LCASE, "ΣxΣ", "σ", "xΣ"); + assertStringTrimLeft(UTF8_LCASE, "ΣxΣ", "ς", "xΣ"); + assertStringTrimLeft(UTF8_LCASE, "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UNICODE, "ςxς", "σ", "ςxς"); + assertStringTrimLeft(UNICODE, "ςxς", "ς", "xς"); + assertStringTrimLeft(UNICODE, "ςxς", "Σ", "ςxς"); + assertStringTrimLeft(UNICODE, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UNICODE, "σxσ", "ς", "σxσ"); + assertStringTrimLeft(UNICODE, "σxσ", "Σ", "σxσ"); + assertStringTrimLeft(UNICODE, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimLeft(UNICODE, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimLeft(UNICODE, "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UNICODE_CI, "ςxς", "σ", "xς"); + assertStringTrimLeft(UNICODE_CI, "ςxς", "ς", "xς"); + assertStringTrimLeft(UNICODE_CI, "ςxς", "Σ", "xς"); + assertStringTrimLeft(UNICODE_CI, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UNICODE_CI, "σxσ", "ς", "xσ"); + assertStringTrimLeft(UNICODE_CI, "σxσ", "Σ", "xσ"); + assertStringTrimLeft(UNICODE_CI, "ΣxΣ", "σ", "xΣ"); + assertStringTrimLeft(UNICODE_CI, "ΣxΣ", "ς", "xΣ"); + assertStringTrimLeft(UNICODE_CI, "ΣxΣ", "Σ", "xΣ"); // Unicode normalization. - assertStringTrimLeft("UTF8_BINARY", "åβγδa\u030A", "å", "βγδa\u030A"); - assertStringTrimLeft("UTF8_LCASE", "åβγδa\u030A", "Å", "βγδa\u030A"); - assertStringTrimLeft("UNICODE", "åβγδa\u030A", "å", "βγδa\u030A"); - assertStringTrimLeft("UNICODE_CI", "åβγδa\u030A", "Å", "βγδa\u030A"); + assertStringTrimLeft(UTF8_BINARY, "åβγδa\u030A", "å", "βγδa\u030A"); + assertStringTrimLeft(UTF8_LCASE, "åβγδa\u030A", "Å", "βγδa\u030A"); + assertStringTrimLeft(UNICODE, "åβγδa\u030A", "å", "βγδa\u030A"); + assertStringTrimLeft(UNICODE_CI, "åβγδa\u030A", "Å", "βγδa\u030A"); // Surrogate pairs. - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UTF8_BINARY", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UTF8_LCASE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UNICODE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UNICODE_CI", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UTF8_BINARY", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UTF8_LCASE", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UNICODE", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UNICODE_CI", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UTF8_BINARY", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UTF8_LCASE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UNICODE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UNICODE_CI", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UTF8_BINARY", "𐐅", "𐐅", ""); - assertStringTrimLeft("UTF8_LCASE", "𐐅", "𐐅", ""); - assertStringTrimLeft("UNICODE", "𐐅", "𐐅", ""); - assertStringTrimLeft("UNICODE_CI", "𐐅", "𐐅", ""); - assertStringTrimLeft("UTF8_BINARY", "𐐅", "𐐭", "𐐅"); - assertStringTrimLeft("UTF8_LCASE", "𐐅", "𐐭", ""); - assertStringTrimLeft("UNICODE", "𐐅", "𐐭", "𐐅"); - assertStringTrimLeft("UNICODE_CI", "𐐅", "𐐭", ""); - assertStringTrimLeft("UTF8_BINARY", "𝔸", "𝔸", ""); - assertStringTrimLeft("UTF8_LCASE", "𝔸", "𝔸", ""); - assertStringTrimLeft("UNICODE", "𝔸", "𝔸", ""); - assertStringTrimLeft("UNICODE_CI", "𝔸", "𝔸", ""); - assertStringTrimLeft("UTF8_BINARY", "𝔸", "A", "𝔸"); - assertStringTrimLeft("UTF8_LCASE", "𝔸", "A", "𝔸"); - assertStringTrimLeft("UNICODE", "𝔸", "A", "𝔸"); - assertStringTrimLeft("UNICODE_CI", "𝔸", "A", ""); - assertStringTrimLeft("UTF8_BINARY", "𝔸", "a", "𝔸"); - assertStringTrimLeft("UTF8_LCASE", "𝔸", "a", "𝔸"); - assertStringTrimLeft("UNICODE", "𝔸", "a", "𝔸"); - assertStringTrimLeft("UNICODE_CI", "𝔸", "a", ""); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UTF8_BINARY, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UTF8_LCASE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UNICODE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UNICODE_CI, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UTF8_BINARY, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UTF8_LCASE, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UNICODE, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UNICODE_CI, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UTF8_BINARY, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UTF8_LCASE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UNICODE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UNICODE_CI, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UTF8_BINARY, "𐐅", "𐐅", ""); + assertStringTrimLeft(UTF8_LCASE, "𐐅", "𐐅", ""); + assertStringTrimLeft(UNICODE, "𐐅", "𐐅", ""); + assertStringTrimLeft(UNICODE_CI, "𐐅", "𐐅", ""); + assertStringTrimLeft(UTF8_BINARY, "𐐅", "𐐭", "𐐅"); + assertStringTrimLeft(UTF8_LCASE, "𐐅", "𐐭", ""); + assertStringTrimLeft(UNICODE, "𐐅", "𐐭", "𐐅"); + assertStringTrimLeft(UNICODE_CI, "𐐅", "𐐭", ""); + assertStringTrimLeft(UTF8_BINARY, "𝔸", "𝔸", ""); + assertStringTrimLeft(UTF8_LCASE, "𝔸", "𝔸", ""); + assertStringTrimLeft(UNICODE, "𝔸", "𝔸", ""); + assertStringTrimLeft(UNICODE_CI, "𝔸", "𝔸", ""); + assertStringTrimLeft(UTF8_BINARY, "𝔸", "A", "𝔸"); + assertStringTrimLeft(UTF8_LCASE, "𝔸", "A", "𝔸"); + assertStringTrimLeft(UNICODE, "𝔸", "A", "𝔸"); + assertStringTrimLeft(UNICODE_CI, "𝔸", "A", ""); + assertStringTrimLeft(UTF8_BINARY, "𝔸", "a", "𝔸"); + assertStringTrimLeft(UTF8_LCASE, "𝔸", "a", "𝔸"); + assertStringTrimLeft(UNICODE, "𝔸", "a", "𝔸"); + assertStringTrimLeft(UNICODE_CI, "𝔸", "a", ""); } /** @@ -3378,274 +3379,274 @@ private void assertStringTrimRight(String collationName, String sourceString, St @Test public void testStringTrimRight() throws SparkException { // Basic tests. - assertStringTrimRight("UTF8_BINARY", "", "", ""); - assertStringTrimRight("UTF8_BINARY", "", "xyz", ""); - assertStringTrimRight("UTF8_BINARY", "asd", "", "asd"); - assertStringTrimRight("UTF8_BINARY", "asd", null, "asd"); - assertStringTrimRight("UTF8_BINARY", " asd ", null, " asd"); - assertStringTrimRight("UTF8_BINARY", " a世a ", null, " a世a"); - assertStringTrimRight("UTF8_BINARY", "asd", "x", "asd"); - assertStringTrimRight("UTF8_BINARY", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UTF8_BINARY", "xa世ax", "x", "xa世a"); - assertStringTrimRight("UTF8_LCASE", "", "", ""); - assertStringTrimRight("UTF8_LCASE", "", "xyz", ""); - assertStringTrimRight("UTF8_LCASE", "asd", "", "asd"); - assertStringTrimRight("UTF8_LCASE", "asd", null, "asd"); - assertStringTrimRight("UTF8_LCASE", " asd ", null, " asd"); - assertStringTrimRight("UTF8_LCASE", " a世a ", null, " a世a"); - assertStringTrimRight("UTF8_LCASE", "asd", "x", "asd"); - assertStringTrimRight("UTF8_LCASE", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UTF8_LCASE", "xa世ax", "x", "xa世a"); - assertStringTrimRight("UNICODE", "", "", ""); - assertStringTrimRight("UNICODE", "", "xyz", ""); - assertStringTrimRight("UNICODE", "asd", "", "asd"); - assertStringTrimRight("UNICODE", "asd", null, "asd"); - assertStringTrimRight("UNICODE", " asd ", null, " asd"); - assertStringTrimRight("UNICODE", " a世a ", null, " a世a"); - assertStringTrimRight("UNICODE", "asd", "x", "asd"); - assertStringTrimRight("UNICODE", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UNICODE", "xa世ax", "x", "xa世a"); - assertStringTrimRight("UNICODE_CI", "", "", ""); - assertStringTrimRight("UNICODE_CI", "", "xyz", ""); - assertStringTrimRight("UNICODE_CI", "asd", "", "asd"); - assertStringTrimRight("UNICODE_CI", "asd", null, "asd"); - assertStringTrimRight("UNICODE_CI", " asd ", null, " asd"); - assertStringTrimRight("UNICODE_CI", " a世a ", null, " a世a"); - assertStringTrimRight("UNICODE_CI", "asd", "x", "asd"); - assertStringTrimRight("UNICODE_CI", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UNICODE_CI", "xa世ax", "x", "xa世a"); + assertStringTrimRight(UTF8_BINARY, "", "", ""); + assertStringTrimRight(UTF8_BINARY, "", "xyz", ""); + assertStringTrimRight(UTF8_BINARY, "asd", "", "asd"); + assertStringTrimRight(UTF8_BINARY, "asd", null, "asd"); + assertStringTrimRight(UTF8_BINARY, " asd ", null, " asd"); + assertStringTrimRight(UTF8_BINARY, " a世a ", null, " a世a"); + assertStringTrimRight(UTF8_BINARY, "asd", "x", "asd"); + assertStringTrimRight(UTF8_BINARY, "xxasdxx", "x", "xxasd"); + assertStringTrimRight(UTF8_BINARY, "xa世ax", "x", "xa世a"); + assertStringTrimRight(UTF8_LCASE, "", "", ""); + assertStringTrimRight(UTF8_LCASE, "", "xyz", ""); + assertStringTrimRight(UTF8_LCASE, "asd", "", "asd"); + assertStringTrimRight(UTF8_LCASE, "asd", null, "asd"); + assertStringTrimRight(UTF8_LCASE, " asd ", null, " asd"); + assertStringTrimRight(UTF8_LCASE, " a世a ", null, " a世a"); + assertStringTrimRight(UTF8_LCASE, "asd", "x", "asd"); + assertStringTrimRight(UTF8_LCASE, "xxasdxx", "x", "xxasd"); + assertStringTrimRight(UTF8_LCASE, "xa世ax", "x", "xa世a"); + assertStringTrimRight(UNICODE, "", "", ""); + assertStringTrimRight(UNICODE, "", "xyz", ""); + assertStringTrimRight(UNICODE, "asd", "", "asd"); + assertStringTrimRight(UNICODE, "asd", null, "asd"); + assertStringTrimRight(UNICODE, " asd ", null, " asd"); + assertStringTrimRight(UNICODE, " a世a ", null, " a世a"); + assertStringTrimRight(UNICODE, "asd", "x", "asd"); + assertStringTrimRight(UNICODE, "xxasdxx", "x", "xxasd"); + assertStringTrimRight(UNICODE, "xa世ax", "x", "xa世a"); + assertStringTrimRight(UNICODE_CI, "", "", ""); + assertStringTrimRight(UNICODE_CI, "", "xyz", ""); + assertStringTrimRight(UNICODE_CI, "asd", "", "asd"); + assertStringTrimRight(UNICODE_CI, "asd", null, "asd"); + assertStringTrimRight(UNICODE_CI, " asd ", null, " asd"); + assertStringTrimRight(UNICODE_CI, " a世a ", null, " a世a"); + assertStringTrimRight(UNICODE_CI, "asd", "x", "asd"); + assertStringTrimRight(UNICODE_CI, "xxasdxx", "x", "xxasd"); + assertStringTrimRight(UNICODE_CI, "xa世ax", "x", "xa世a"); // Case variation. - assertStringTrimRight("UTF8_BINARY", "ddsXXXaa", "asd", "ddsXXX"); - assertStringTrimRight("UTF8_LCASE", "ddsXXXaa", "AsD", "ddsXXX"); - assertStringTrimRight("UNICODE", "ddsXXXaa", "asd", "ddsXXX"); - assertStringTrimRight("UNICODE_CI", "ddsXXXaa", "AsD", "ddsXXX"); + assertStringTrimRight(UTF8_BINARY, "ddsXXXaa", "asd", "ddsXXX"); + assertStringTrimRight(UTF8_LCASE, "ddsXXXaa", "AsD", "ddsXXX"); + assertStringTrimRight(UNICODE, "ddsXXXaa", "asd", "ddsXXX"); + assertStringTrimRight(UNICODE_CI, "ddsXXXaa", "AsD", "ddsXXX"); // One-to-many case mapping (e.g. Turkish dotted I).. - assertStringTrimRight("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimRight("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimRight("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "Ëaaa"); - assertStringTrimRight("UTF8_LCASE", "ẞaaaẞ", "ß", "ẞaaa"); - assertStringTrimRight("UTF8_LCASE", "ßaaaß", "ẞ", "ßaaa"); - assertStringTrimRight("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "Ëaaa"); - assertStringTrimRight("UNICODE", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimRight("UNICODE", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimRight("UNICODE", "Ëaaaẞ", "Ëẞ", "Ëaaa"); - assertStringTrimRight("UNICODE_CI", "ẞaaaẞ", "ß", "ẞaaa"); - assertStringTrimRight("UNICODE_CI", "ßaaaß", "ẞ", "ßaaa"); - assertStringTrimRight("UNICODE_CI", "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UTF8_BINARY, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimRight(UTF8_BINARY, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimRight(UTF8_BINARY, "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UTF8_LCASE, "ẞaaaẞ", "ß", "ẞaaa"); + assertStringTrimRight(UTF8_LCASE, "ßaaaß", "ẞ", "ßaaa"); + assertStringTrimRight(UTF8_LCASE, "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UNICODE, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimRight(UNICODE, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimRight(UNICODE, "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UNICODE_CI, "ẞaaaẞ", "ß", "ẞaaa"); + assertStringTrimRight(UNICODE_CI, "ßaaaß", "ẞ", "ßaaa"); + assertStringTrimRight(UNICODE_CI, "Ëaaaẞ", "Ëẞ", "Ëaaa"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTrimRight("UTF8_BINARY", "i", "i", ""); - assertStringTrimRight("UTF8_BINARY", "iii", "I", "iii"); - assertStringTrimRight("UTF8_BINARY", "I", "iii", "I"); - assertStringTrimRight("UTF8_BINARY", "ixi", "i", "ix"); - assertStringTrimRight("UTF8_BINARY", "i", "İ", "i"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "İ", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307", "İi", "ii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iii\u0307", "İi", "iii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iiii\u0307", "iİ", "iiii\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "\u0307", "i"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307i", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UTF8_BINARY", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UTF8_BINARY", "i\u0307İ", "İ", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "İ", "İ", ""); - assertStringTrimRight("UTF8_BINARY", "IXi", "İ", "IXi"); - assertStringTrimRight("UTF8_BINARY", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UTF8_BINARY", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimRight("UTF8_BINARY", "i\u0307x", "ix\u0307İ", ""); - assertStringTrimRight("UTF8_BINARY", "İ", "i", "İ"); - assertStringTrimRight("UTF8_BINARY", "İ", "\u0307", "İ"); - assertStringTrimRight("UTF8_BINARY", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UTF8_BINARY", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UTF8_BINARY", "xi\u0307", "\u0307IX", "xi"); - assertStringTrimRight("UTF8_LCASE", "i", "i", ""); - assertStringTrimRight("UTF8_LCASE", "iii", "I", ""); - assertStringTrimRight("UTF8_LCASE", "I", "iii", ""); - assertStringTrimRight("UTF8_LCASE", "ixi", "i", "ix"); - assertStringTrimRight("UTF8_LCASE", "i", "İ", "i"); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "İ", ""); - assertStringTrimRight("UTF8_LCASE", "ii\u0307", "İi", ""); - assertStringTrimRight("UTF8_LCASE", "iii\u0307", "İi", ""); - assertStringTrimRight("UTF8_LCASE", "iiii\u0307", "iİ", ""); - assertStringTrimRight("UTF8_LCASE", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "\u0307", "i"); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307i", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UTF8_LCASE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UTF8_LCASE", "i\u0307İ", "İ", ""); - assertStringTrimRight("UTF8_LCASE", "İ", "İ", ""); - assertStringTrimRight("UTF8_LCASE", "IXi", "İ", "IXi"); - assertStringTrimRight("UTF8_LCASE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UTF8_LCASE", "i\u0307x", "IXİ", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimRight("UTF8_LCASE", "İ", "i", "İ"); - assertStringTrimRight("UTF8_LCASE", "İ", "\u0307", "İ"); - assertStringTrimRight("UTF8_LCASE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UTF8_LCASE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UTF8_LCASE", "xi\u0307", "\u0307IX", ""); - assertStringTrimRight("UNICODE", "i", "i", ""); - assertStringTrimRight("UNICODE", "iii", "I", "iii"); - assertStringTrimRight("UNICODE", "I", "iii", "I"); - assertStringTrimRight("UNICODE", "ixi", "i", "ix"); - assertStringTrimRight("UNICODE", "i", "İ", "i"); - assertStringTrimRight("UNICODE", "i\u0307", "İ", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307", "İi", "ii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iii\u0307", "İi", "iii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iiii\u0307", "iİ", "iiii\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); - assertStringTrimRight("UNICODE", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimRight("UNICODE", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UNICODE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UNICODE", "i\u0307İ", "İ", "i\u0307"); - assertStringTrimRight("UNICODE", "İ", "İ", ""); - assertStringTrimRight("UNICODE", "IXi", "İ", "IXi"); - assertStringTrimRight("UNICODE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UNICODE", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimRight("UNICODE", "i\u0307x", "ix\u0307İ", "i\u0307"); - assertStringTrimRight("UNICODE", "İ", "i", "İ"); - assertStringTrimRight("UNICODE", "İ", "\u0307", "İ"); - assertStringTrimRight("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UNICODE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UNICODE", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrimRight("UNICODE_CI", "i", "i", ""); - assertStringTrimRight("UNICODE_CI", "iii", "I", ""); - assertStringTrimRight("UNICODE_CI", "I", "iii", ""); - assertStringTrimRight("UNICODE_CI", "ixi", "i", "ix"); - assertStringTrimRight("UNICODE_CI", "i", "İ", "i"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "İ", ""); - assertStringTrimRight("UNICODE_CI", "ii\u0307", "İi", ""); - assertStringTrimRight("UNICODE_CI", "iii\u0307", "İi", ""); - assertStringTrimRight("UNICODE_CI", "iiii\u0307", "iİ", ""); - assertStringTrimRight("UNICODE_CI", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimRight("UNICODE_CI", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UNICODE_CI", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UNICODE_CI", "i\u0307İ", "İ", ""); - assertStringTrimRight("UNICODE_CI", "İ", "İ", ""); - assertStringTrimRight("UNICODE_CI", "IXi", "İ", "IXi"); - assertStringTrimRight("UNICODE_CI", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307x", "IXİ", ""); - assertStringTrimRight("UNICODE_CI", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimRight("UNICODE_CI", "İ", "i", "İ"); - assertStringTrimRight("UNICODE_CI", "İ", "\u0307", "İ"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UNICODE_CI", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UNICODE_CI", "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimRight(UTF8_BINARY, "i", "i", ""); + assertStringTrimRight(UTF8_BINARY, "iii", "I", "iii"); + assertStringTrimRight(UTF8_BINARY, "I", "iii", "I"); + assertStringTrimRight(UTF8_BINARY, "ixi", "i", "ix"); + assertStringTrimRight(UTF8_BINARY, "i", "İ", "i"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "İ", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307", "İi", "ii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iii\u0307", "İi", "iii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iiii\u0307", "iİ", "iiii\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "\u0307", "i"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307i", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UTF8_BINARY, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UTF8_BINARY, "i\u0307İ", "İ", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "İ", "İ", ""); + assertStringTrimRight(UTF8_BINARY, "IXi", "İ", "IXi"); + assertStringTrimRight(UTF8_BINARY, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UTF8_BINARY, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimRight(UTF8_BINARY, "i\u0307x", "ix\u0307İ", ""); + assertStringTrimRight(UTF8_BINARY, "İ", "i", "İ"); + assertStringTrimRight(UTF8_BINARY, "İ", "\u0307", "İ"); + assertStringTrimRight(UTF8_BINARY, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UTF8_BINARY, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UTF8_BINARY, "xi\u0307", "\u0307IX", "xi"); + assertStringTrimRight(UTF8_LCASE, "i", "i", ""); + assertStringTrimRight(UTF8_LCASE, "iii", "I", ""); + assertStringTrimRight(UTF8_LCASE, "I", "iii", ""); + assertStringTrimRight(UTF8_LCASE, "ixi", "i", "ix"); + assertStringTrimRight(UTF8_LCASE, "i", "İ", "i"); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "İ", ""); + assertStringTrimRight(UTF8_LCASE, "ii\u0307", "İi", ""); + assertStringTrimRight(UTF8_LCASE, "iii\u0307", "İi", ""); + assertStringTrimRight(UTF8_LCASE, "iiii\u0307", "iİ", ""); + assertStringTrimRight(UTF8_LCASE, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "\u0307", "i"); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307i", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UTF8_LCASE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UTF8_LCASE, "i\u0307İ", "İ", ""); + assertStringTrimRight(UTF8_LCASE, "İ", "İ", ""); + assertStringTrimRight(UTF8_LCASE, "IXi", "İ", "IXi"); + assertStringTrimRight(UTF8_LCASE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UTF8_LCASE, "i\u0307x", "IXİ", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimRight(UTF8_LCASE, "İ", "i", "İ"); + assertStringTrimRight(UTF8_LCASE, "İ", "\u0307", "İ"); + assertStringTrimRight(UTF8_LCASE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UTF8_LCASE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UTF8_LCASE, "xi\u0307", "\u0307IX", ""); + assertStringTrimRight(UNICODE, "i", "i", ""); + assertStringTrimRight(UNICODE, "iii", "I", "iii"); + assertStringTrimRight(UNICODE, "I", "iii", "I"); + assertStringTrimRight(UNICODE, "ixi", "i", "ix"); + assertStringTrimRight(UNICODE, "i", "İ", "i"); + assertStringTrimRight(UNICODE, "i\u0307", "İ", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307", "İi", "ii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iii\u0307", "İi", "iii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iiii\u0307", "iİ", "iiii\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); + assertStringTrimRight(UNICODE, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimRight(UNICODE, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UNICODE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UNICODE, "i\u0307İ", "İ", "i\u0307"); + assertStringTrimRight(UNICODE, "İ", "İ", ""); + assertStringTrimRight(UNICODE, "IXi", "İ", "IXi"); + assertStringTrimRight(UNICODE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UNICODE, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimRight(UNICODE, "i\u0307x", "ix\u0307İ", "i\u0307"); + assertStringTrimRight(UNICODE, "İ", "i", "İ"); + assertStringTrimRight(UNICODE, "İ", "\u0307", "İ"); + assertStringTrimRight(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UNICODE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UNICODE, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimRight(UNICODE_CI, "i", "i", ""); + assertStringTrimRight(UNICODE_CI, "iii", "I", ""); + assertStringTrimRight(UNICODE_CI, "I", "iii", ""); + assertStringTrimRight(UNICODE_CI, "ixi", "i", "ix"); + assertStringTrimRight(UNICODE_CI, "i", "İ", "i"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "İ", ""); + assertStringTrimRight(UNICODE_CI, "ii\u0307", "İi", ""); + assertStringTrimRight(UNICODE_CI, "iii\u0307", "İi", ""); + assertStringTrimRight(UNICODE_CI, "iiii\u0307", "iİ", ""); + assertStringTrimRight(UNICODE_CI, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimRight(UNICODE_CI, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UNICODE_CI, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UNICODE_CI, "i\u0307İ", "İ", ""); + assertStringTrimRight(UNICODE_CI, "İ", "İ", ""); + assertStringTrimRight(UNICODE_CI, "IXi", "İ", "IXi"); + assertStringTrimRight(UNICODE_CI, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307x", "IXİ", ""); + assertStringTrimRight(UNICODE_CI, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimRight(UNICODE_CI, "İ", "i", "İ"); + assertStringTrimRight(UNICODE_CI, "İ", "\u0307", "İ"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UNICODE_CI, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UNICODE_CI, "xi\u0307", "\u0307IX", "xi\u0307"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTrimRight("UTF8_BINARY", "ςxς", "σ", "ςxς"); - assertStringTrimRight("UTF8_BINARY", "ςxς", "ς", "ςx"); - assertStringTrimRight("UTF8_BINARY", "ςxς", "Σ", "ςxς"); - assertStringTrimRight("UTF8_BINARY", "σxσ", "σ", "σx"); - assertStringTrimRight("UTF8_BINARY", "σxσ", "ς", "σxσ"); - assertStringTrimRight("UTF8_BINARY", "σxσ", "Σ", "σxσ"); - assertStringTrimRight("UTF8_BINARY", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimRight("UTF8_BINARY", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimRight("UTF8_BINARY", "ΣxΣ", "Σ", "Σx"); - assertStringTrimRight("UTF8_LCASE", "ςxς", "σ", "ςx"); - assertStringTrimRight("UTF8_LCASE", "ςxς", "ς", "ςx"); - assertStringTrimRight("UTF8_LCASE", "ςxς", "Σ", "ςx"); - assertStringTrimRight("UTF8_LCASE", "σxσ", "σ", "σx"); - assertStringTrimRight("UTF8_LCASE", "σxσ", "ς", "σx"); - assertStringTrimRight("UTF8_LCASE", "σxσ", "Σ", "σx"); - assertStringTrimRight("UTF8_LCASE", "ΣxΣ", "σ", "Σx"); - assertStringTrimRight("UTF8_LCASE", "ΣxΣ", "ς", "Σx"); - assertStringTrimRight("UTF8_LCASE", "ΣxΣ", "Σ", "Σx"); - assertStringTrimRight("UNICODE", "ςxς", "σ", "ςxς"); - assertStringTrimRight("UNICODE", "ςxς", "ς", "ςx"); - assertStringTrimRight("UNICODE", "ςxς", "Σ", "ςxς"); - assertStringTrimRight("UNICODE", "σxσ", "σ", "σx"); - assertStringTrimRight("UNICODE", "σxσ", "ς", "σxσ"); - assertStringTrimRight("UNICODE", "σxσ", "Σ", "σxσ"); - assertStringTrimRight("UNICODE", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimRight("UNICODE", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimRight("UNICODE", "ΣxΣ", "Σ", "Σx"); - assertStringTrimRight("UNICODE_CI", "ςxς", "σ", "ςx"); - assertStringTrimRight("UNICODE_CI", "ςxς", "ς", "ςx"); - assertStringTrimRight("UNICODE_CI", "ςxς", "Σ", "ςx"); - assertStringTrimRight("UNICODE_CI", "σxσ", "σ", "σx"); - assertStringTrimRight("UNICODE_CI", "σxσ", "ς", "σx"); - assertStringTrimRight("UNICODE_CI", "σxσ", "Σ", "σx"); - assertStringTrimRight("UNICODE_CI", "ΣxΣ", "σ", "Σx"); - assertStringTrimRight("UNICODE_CI", "ΣxΣ", "ς", "Σx"); - assertStringTrimRight("UNICODE_CI", "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UTF8_BINARY, "ςxς", "σ", "ςxς"); + assertStringTrimRight(UTF8_BINARY, "ςxς", "ς", "ςx"); + assertStringTrimRight(UTF8_BINARY, "ςxς", "Σ", "ςxς"); + assertStringTrimRight(UTF8_BINARY, "σxσ", "σ", "σx"); + assertStringTrimRight(UTF8_BINARY, "σxσ", "ς", "σxσ"); + assertStringTrimRight(UTF8_BINARY, "σxσ", "Σ", "σxσ"); + assertStringTrimRight(UTF8_BINARY, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimRight(UTF8_BINARY, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimRight(UTF8_BINARY, "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UTF8_LCASE, "ςxς", "σ", "ςx"); + assertStringTrimRight(UTF8_LCASE, "ςxς", "ς", "ςx"); + assertStringTrimRight(UTF8_LCASE, "ςxς", "Σ", "ςx"); + assertStringTrimRight(UTF8_LCASE, "σxσ", "σ", "σx"); + assertStringTrimRight(UTF8_LCASE, "σxσ", "ς", "σx"); + assertStringTrimRight(UTF8_LCASE, "σxσ", "Σ", "σx"); + assertStringTrimRight(UTF8_LCASE, "ΣxΣ", "σ", "Σx"); + assertStringTrimRight(UTF8_LCASE, "ΣxΣ", "ς", "Σx"); + assertStringTrimRight(UTF8_LCASE, "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UNICODE, "ςxς", "σ", "ςxς"); + assertStringTrimRight(UNICODE, "ςxς", "ς", "ςx"); + assertStringTrimRight(UNICODE, "ςxς", "Σ", "ςxς"); + assertStringTrimRight(UNICODE, "σxσ", "σ", "σx"); + assertStringTrimRight(UNICODE, "σxσ", "ς", "σxσ"); + assertStringTrimRight(UNICODE, "σxσ", "Σ", "σxσ"); + assertStringTrimRight(UNICODE, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimRight(UNICODE, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimRight(UNICODE, "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UNICODE_CI, "ςxς", "σ", "ςx"); + assertStringTrimRight(UNICODE_CI, "ςxς", "ς", "ςx"); + assertStringTrimRight(UNICODE_CI, "ςxς", "Σ", "ςx"); + assertStringTrimRight(UNICODE_CI, "σxσ", "σ", "σx"); + assertStringTrimRight(UNICODE_CI, "σxσ", "ς", "σx"); + assertStringTrimRight(UNICODE_CI, "σxσ", "Σ", "σx"); + assertStringTrimRight(UNICODE_CI, "ΣxΣ", "σ", "Σx"); + assertStringTrimRight(UNICODE_CI, "ΣxΣ", "ς", "Σx"); + assertStringTrimRight(UNICODE_CI, "ΣxΣ", "Σ", "Σx"); // Unicode normalization. - assertStringTrimRight("UTF8_BINARY", "åβγδa\u030A", "å", "åβγδa\u030A"); - assertStringTrimRight("UTF8_LCASE", "åβγδa\u030A", "Å", "åβγδa\u030A"); - assertStringTrimRight("UNICODE", "åβγδa\u030A", "å", "åβγδ"); - assertStringTrimRight("UNICODE_CI", "åβγδa\u030A", "Å", "åβγδ"); + assertStringTrimRight(UTF8_BINARY, "åβγδa\u030A", "å", "åβγδa\u030A"); + assertStringTrimRight(UTF8_LCASE, "åβγδa\u030A", "Å", "åβγδa\u030A"); + assertStringTrimRight(UNICODE, "åβγδa\u030A", "å", "åβγδ"); + assertStringTrimRight(UNICODE_CI, "åβγδa\u030A", "Å", "åβγδ"); // Surrogate pairs. - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UTF8_BINARY", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UTF8_LCASE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UNICODE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UNICODE_CI", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UTF8_BINARY", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UTF8_LCASE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UNICODE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UNICODE_CI", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UTF8_BINARY", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UTF8_LCASE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UNICODE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UNICODE_CI", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UTF8_BINARY", "𐐅", "𐐅", ""); - assertStringTrimRight("UTF8_LCASE", "𐐅", "𐐅", ""); - assertStringTrimRight("UNICODE", "𐐅", "𐐅", ""); - assertStringTrimRight("UNICODE_CI", "𐐅", "𐐅", ""); - assertStringTrimRight("UTF8_BINARY", "𐐅", "𐐭", "𐐅"); - assertStringTrimRight("UTF8_LCASE", "𐐅", "𐐭", ""); - assertStringTrimRight("UNICODE", "𐐅", "𐐭", "𐐅"); - assertStringTrimRight("UNICODE_CI", "𐐅", "𐐭", ""); - assertStringTrimRight("UTF8_BINARY", "𝔸", "𝔸", ""); - assertStringTrimRight("UTF8_LCASE", "𝔸", "𝔸", ""); - assertStringTrimRight("UNICODE", "𝔸", "𝔸", ""); - assertStringTrimRight("UNICODE_CI", "𝔸", "𝔸", ""); - assertStringTrimRight("UTF8_BINARY", "𝔸", "A", "𝔸"); - assertStringTrimRight("UTF8_LCASE", "𝔸", "A", "𝔸"); - assertStringTrimRight("UNICODE", "𝔸", "A", "𝔸"); - assertStringTrimRight("UNICODE_CI", "𝔸", "A", ""); - assertStringTrimRight("UTF8_BINARY", "𝔸", "a", "𝔸"); - assertStringTrimRight("UTF8_LCASE", "𝔸", "a", "𝔸"); - assertStringTrimRight("UNICODE", "𝔸", "a", "𝔸"); - assertStringTrimRight("UNICODE_CI", "𝔸", "a", ""); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UTF8_BINARY, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UTF8_LCASE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UNICODE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UNICODE_CI, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UTF8_BINARY, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UTF8_LCASE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UNICODE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UNICODE_CI, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UTF8_BINARY, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UTF8_LCASE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UNICODE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UNICODE_CI, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UTF8_BINARY, "𐐅", "𐐅", ""); + assertStringTrimRight(UTF8_LCASE, "𐐅", "𐐅", ""); + assertStringTrimRight(UNICODE, "𐐅", "𐐅", ""); + assertStringTrimRight(UNICODE_CI, "𐐅", "𐐅", ""); + assertStringTrimRight(UTF8_BINARY, "𐐅", "𐐭", "𐐅"); + assertStringTrimRight(UTF8_LCASE, "𐐅", "𐐭", ""); + assertStringTrimRight(UNICODE, "𐐅", "𐐭", "𐐅"); + assertStringTrimRight(UNICODE_CI, "𐐅", "𐐭", ""); + assertStringTrimRight(UTF8_BINARY, "𝔸", "𝔸", ""); + assertStringTrimRight(UTF8_LCASE, "𝔸", "𝔸", ""); + assertStringTrimRight(UNICODE, "𝔸", "𝔸", ""); + assertStringTrimRight(UNICODE_CI, "𝔸", "𝔸", ""); + assertStringTrimRight(UTF8_BINARY, "𝔸", "A", "𝔸"); + assertStringTrimRight(UTF8_LCASE, "𝔸", "A", "𝔸"); + assertStringTrimRight(UNICODE, "𝔸", "A", "𝔸"); + assertStringTrimRight(UNICODE_CI, "𝔸", "A", ""); + assertStringTrimRight(UTF8_BINARY, "𝔸", "a", "𝔸"); + assertStringTrimRight(UTF8_LCASE, "𝔸", "a", "𝔸"); + assertStringTrimRight(UNICODE, "𝔸", "a", "𝔸"); + assertStringTrimRight(UNICODE_CI, "𝔸", "a", ""); } /** @@ -3664,211 +3665,211 @@ private void assertStringTranslate(String inputString, String matchingString, @Test public void testStringTranslate() throws SparkException { // Empty strings. - assertStringTranslate("", "", "", "UTF8_BINARY", ""); - assertStringTranslate("", "", "", "UTF8_LCASE", ""); - assertStringTranslate("", "", "", "UNICODE", ""); - assertStringTranslate("", "", "", "UNICODE_CI", ""); - assertStringTranslate("abc", "", "", "UTF8_BINARY", "abc"); - assertStringTranslate("abc", "", "", "UTF8_LCASE", "abc"); - assertStringTranslate("abc", "", "", "UNICODE", "abc"); - assertStringTranslate("abc", "", "", "UNICODE_CI", "abc"); - assertStringTranslate("", "b", "", "UTF8_BINARY", ""); - assertStringTranslate("", "b", "", "UTF8_LCASE", ""); - assertStringTranslate("", "b", "", "UNICODE", ""); - assertStringTranslate("", "b", "", "UNICODE_CI", ""); - assertStringTranslate("", "", "x", "UTF8_BINARY", ""); - assertStringTranslate("", "", "x", "UTF8_LCASE", ""); - assertStringTranslate("", "", "x", "UNICODE", ""); - assertStringTranslate("", "", "x", "UNICODE_CI", ""); - assertStringTranslate("abc", "b", "", "UTF8_BINARY", "ac"); - assertStringTranslate("abc", "b", "", "UTF8_LCASE", "ac"); - assertStringTranslate("abc", "b", "", "UNICODE", "ac"); - assertStringTranslate("abc", "b", "", "UNICODE_CI", "ac"); - assertStringTranslate("abc", "", "x", "UTF8_BINARY", "abc"); - assertStringTranslate("abc", "", "x", "UTF8_LCASE", "abc"); - assertStringTranslate("abc", "", "x", "UNICODE", "abc"); - assertStringTranslate("abc", "", "x", "UNICODE_CI", "abc"); - assertStringTranslate("", "b", "x", "UTF8_BINARY", ""); - assertStringTranslate("", "b", "x", "UTF8_LCASE", ""); - assertStringTranslate("", "b", "x", "UNICODE", ""); - assertStringTranslate("", "b", "x", "UNICODE_CI", ""); + assertStringTranslate("", "", "", UTF8_BINARY, ""); + assertStringTranslate("", "", "", UTF8_LCASE, ""); + assertStringTranslate("", "", "", UNICODE, ""); + assertStringTranslate("", "", "", UNICODE_CI, ""); + assertStringTranslate("abc", "", "", UTF8_BINARY, "abc"); + assertStringTranslate("abc", "", "", UTF8_LCASE, "abc"); + assertStringTranslate("abc", "", "", UNICODE, "abc"); + assertStringTranslate("abc", "", "", UNICODE_CI, "abc"); + assertStringTranslate("", "b", "", UTF8_BINARY, ""); + assertStringTranslate("", "b", "", UTF8_LCASE, ""); + assertStringTranslate("", "b", "", UNICODE, ""); + assertStringTranslate("", "b", "", UNICODE_CI, ""); + assertStringTranslate("", "", "x", UTF8_BINARY, ""); + assertStringTranslate("", "", "x", UTF8_LCASE, ""); + assertStringTranslate("", "", "x", UNICODE, ""); + assertStringTranslate("", "", "x", UNICODE_CI, ""); + assertStringTranslate("abc", "b", "", UTF8_BINARY, "ac"); + assertStringTranslate("abc", "b", "", UTF8_LCASE, "ac"); + assertStringTranslate("abc", "b", "", UNICODE, "ac"); + assertStringTranslate("abc", "b", "", UNICODE_CI, "ac"); + assertStringTranslate("abc", "", "x", UTF8_BINARY, "abc"); + assertStringTranslate("abc", "", "x", UTF8_LCASE, "abc"); + assertStringTranslate("abc", "", "x", UNICODE, "abc"); + assertStringTranslate("abc", "", "x", UNICODE_CI, "abc"); + assertStringTranslate("", "b", "x", UTF8_BINARY, ""); + assertStringTranslate("", "b", "x", UTF8_LCASE, ""); + assertStringTranslate("", "b", "x", UNICODE, ""); + assertStringTranslate("", "b", "x", UNICODE_CI, ""); // Basic tests. - assertStringTranslate("abc", "b", "x", "UTF8_BINARY", "axc"); - assertStringTranslate("abc", "b", "x", "UTF8_LCASE", "axc"); - assertStringTranslate("abc", "b", "x", "UNICODE", "axc"); - assertStringTranslate("abc", "b", "x", "UNICODE_CI", "axc"); - assertStringTranslate("Translate", "Rnlt", "12", "UTF8_BINARY", "Tra2sae"); - assertStringTranslate("Translate", "Rnlt", "12", "UTF8_LCASE", "1a2sae"); - assertStringTranslate("Translate", "Rnlt", "12", "UNICODE", "Tra2sae"); - assertStringTranslate("Translate", "Rnlt", "12", "UNICODE_CI", "1a2sae"); - assertStringTranslate("Translate", "Rn", "1234", "UTF8_BINARY", "Tra2slate"); - assertStringTranslate("Translate", "Rn", "1234", "UTF8_LCASE", "T1a2slate"); - assertStringTranslate("Translate", "Rn", "1234", "UNICODE", "Tra2slate"); - assertStringTranslate("Translate", "Rn", "1234", "UNICODE_CI", "T1a2slate"); - assertStringTranslate("Translate", "Rnlt", "1234", "UTF8_BINARY", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "1234", "UNICODE", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "1234", "UNICODE_CI", "41a2s3a4e"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UTF8_BINARY", "TRaxsXaxe"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UTF8_LCASE", "xXaxsXaxe"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UNICODE", "TRaxsXaxe"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UNICODE_CI", "xXaxsXaxe"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UTF8_BINARY", "TxaxsXaxeX"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UTF8_LCASE", "xxaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UNICODE", "TxaxsXaxeX"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UNICODE_CI", "xxaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UTF8_BINARY", "TXaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UTF8_LCASE", "xXaxsXaxeX"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UNICODE", "TXaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UNICODE_CI", "xXaxsXaxeX"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UTF8_BINARY", "test大千世AX大千世A"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UTF8_LCASE", "test大千世AB大千世A"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UNICODE", "test大千世AX大千世A"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UNICODE_CI", "test大千世AB大千世A"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UTF8_BINARY", "大千世界test大千世界"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UTF8_LCASE", "大千世界abca大千世界"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UNICODE", "大千世界test大千世界"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UNICODE_CI", "大千世界abca大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UTF8_BINARY", "Oeso大千世界大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UTF8_LCASE", "oeso大千世界大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UNICODE", "Oeso大千世界大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UNICODE_CI", "oeso大千世界大千世界"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UTF8_BINARY", "大千世界大千世界oesO"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UTF8_LCASE", "大千世界大千世界OesO"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UNICODE", "大千世界大千世界oesO"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UNICODE_CI", "大千世界大千世界OesO"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UTF8_BINARY", "世世世界世世世界tesT"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UTF8_LCASE", "世世世界世世世界tesT"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UNICODE", "世世世界世世世界tesT"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UNICODE_CI", "世世世界世世世界tesT"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UTF8_BINARY", "Tr4234e"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UTF8_LCASE", "14234e"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UNICODE", "Tr4234e"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UNICODE_CI", "14234e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UTF8_BINARY", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UTF8_LCASE", "41a2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UNICODE", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UNICODE_CI", "41a2s3a4e"); - assertStringTranslate("abcdef", "abcde", "123", "UTF8_BINARY", "123f"); - assertStringTranslate("abcdef", "abcde", "123", "UTF8_LCASE", "123f"); - assertStringTranslate("abcdef", "abcde", "123", "UNICODE", "123f"); - assertStringTranslate("abcdef", "abcde", "123", "UNICODE_CI", "123f"); + assertStringTranslate("abc", "b", "x", UTF8_BINARY, "axc"); + assertStringTranslate("abc", "b", "x", UTF8_LCASE, "axc"); + assertStringTranslate("abc", "b", "x", UNICODE, "axc"); + assertStringTranslate("abc", "b", "x", UNICODE_CI, "axc"); + assertStringTranslate("Translate", "Rnlt", "12", UTF8_BINARY, "Tra2sae"); + assertStringTranslate("Translate", "Rnlt", "12", UTF8_LCASE, "1a2sae"); + assertStringTranslate("Translate", "Rnlt", "12", UNICODE, "Tra2sae"); + assertStringTranslate("Translate", "Rnlt", "12", UNICODE_CI, "1a2sae"); + assertStringTranslate("Translate", "Rn", "1234", UTF8_BINARY, "Tra2slate"); + assertStringTranslate("Translate", "Rn", "1234", UTF8_LCASE, "T1a2slate"); + assertStringTranslate("Translate", "Rn", "1234", UNICODE, "Tra2slate"); + assertStringTranslate("Translate", "Rn", "1234", UNICODE_CI, "T1a2slate"); + assertStringTranslate("Translate", "Rnlt", "1234", UTF8_BINARY, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "1234", UTF8_LCASE, "41a2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "1234", UNICODE, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "1234", UNICODE_CI, "41a2s3a4e"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UTF8_BINARY, "TRaxsXaxe"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UTF8_LCASE, "xXaxsXaxe"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UNICODE, "TRaxsXaxe"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UNICODE_CI, "xXaxsXaxe"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UTF8_BINARY, "TxaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UTF8_LCASE, "xxaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UNICODE, "TxaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UNICODE_CI, "xxaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UTF8_BINARY, "TXaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UTF8_LCASE, "xXaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UNICODE, "TXaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UNICODE_CI, "xXaxsXaxeX"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UTF8_BINARY, "test大千世AX大千世A"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UTF8_LCASE, "test大千世AB大千世A"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UNICODE, "test大千世AX大千世A"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UNICODE_CI, "test大千世AB大千世A"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UTF8_BINARY, "大千世界test大千世界"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UTF8_LCASE, "大千世界abca大千世界"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UNICODE, "大千世界test大千世界"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UNICODE_CI, "大千世界abca大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UTF8_BINARY, "Oeso大千世界大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UTF8_LCASE, "oeso大千世界大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UNICODE, "Oeso大千世界大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UNICODE_CI, "oeso大千世界大千世界"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UTF8_BINARY, "大千世界大千世界oesO"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UTF8_LCASE, "大千世界大千世界OesO"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UNICODE, "大千世界大千世界oesO"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UNICODE_CI, "大千世界大千世界OesO"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UTF8_BINARY, "世世世界世世世界tesT"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UTF8_LCASE, "世世世界世世世界tesT"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UNICODE, "世世世界世世世界tesT"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UNICODE_CI, "世世世界世世世界tesT"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UTF8_BINARY, "Tr4234e"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UTF8_LCASE, "14234e"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UNICODE, "Tr4234e"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UNICODE_CI, "14234e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UTF8_BINARY, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UTF8_LCASE, "41a2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UNICODE, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UNICODE_CI, "41a2s3a4e"); + assertStringTranslate("abcdef", "abcde", "123", UTF8_BINARY, "123f"); + assertStringTranslate("abcdef", "abcde", "123", UTF8_LCASE, "123f"); + assertStringTranslate("abcdef", "abcde", "123", UNICODE, "123f"); + assertStringTranslate("abcdef", "abcde", "123", UNICODE_CI, "123f"); assertStringTranslate("abcdëÈêf", "ÊèË", "123", "AF_CI", "abcd321f"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTranslate("İ", "i\u0307", "xy", "UTF8_BINARY", "İ"); - assertStringTranslate("İ", "i\u0307", "xy", "UTF8_LCASE", "İ"); - assertStringTranslate("İ", "i\u0307", "xy", "UNICODE", "İ"); - assertStringTranslate("İ", "i\u0307", "xy", "UNICODE_CI", "İ"); - assertStringTranslate("i\u0307", "İ", "xy", "UTF8_BINARY", "i\u0307"); - assertStringTranslate("i\u0307", "İ", "xy", "UTF8_LCASE", "x"); - assertStringTranslate("i\u0307", "İ", "xy", "UNICODE", "i\u0307"); - assertStringTranslate("i\u0307", "İ", "xy", "UNICODE_CI", "x"); - assertStringTranslate("i\u030A", "İ", "x", "UTF8_BINARY", "i\u030A"); - assertStringTranslate("i\u030A", "İ", "x", "UTF8_LCASE", "i\u030A"); - assertStringTranslate("i\u030A", "İ", "x", "UNICODE", "i\u030A"); - assertStringTranslate("i\u030A", "İ", "x", "UNICODE_CI", "i\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UTF8_BINARY", "y\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UTF8_LCASE", "y\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UNICODE", "i\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UNICODE_CI", "i\u030A"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UTF8_BINARY", "123"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UTF8_LCASE", "11"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UNICODE", "1i\u0307"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UNICODE_CI", "11"); - assertStringTranslate("İi\u0307", "İyz", "123", "UTF8_BINARY", "1i\u0307"); - assertStringTranslate("İi\u0307", "İyz", "123", "UTF8_LCASE", "11"); - assertStringTranslate("İi\u0307", "İyz", "123", "UNICODE", "1i\u0307"); - assertStringTranslate("İi\u0307", "İyz", "123", "UNICODE_CI", "11"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UTF8_BINARY", "İ23"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UTF8_LCASE", "İ23"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UNICODE", "İi\u0307"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UNICODE_CI", "İi\u0307"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UTF8_BINARY", "12bc3"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UTF8_LCASE", "12bc3"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UNICODE", "3bc3"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UNICODE_CI", "3bc3"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UTF8_BINARY", "a2bcå"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UTF8_LCASE", "12bc3"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UNICODE", "a\u030Abcå"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UNICODE_CI", "3bc3"); - assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", "UTF8_BINARY", "3\u030Aβφδ1\u0307"); - assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", "UTF8_LCASE", "3\u030Aβφδ2"); - assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", "UNICODE", "4βφδ2"); - assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", "UNICODE_CI", "4βφδ2"); + assertStringTranslate("İ", "i\u0307", "xy", UTF8_BINARY, "İ"); + assertStringTranslate("İ", "i\u0307", "xy", UTF8_LCASE, "İ"); + assertStringTranslate("İ", "i\u0307", "xy", UNICODE, "İ"); + assertStringTranslate("İ", "i\u0307", "xy", UNICODE_CI, "İ"); + assertStringTranslate("i\u0307", "İ", "xy", UTF8_BINARY, "i\u0307"); + assertStringTranslate("i\u0307", "İ", "xy", UTF8_LCASE, "x"); + assertStringTranslate("i\u0307", "İ", "xy", UNICODE, "i\u0307"); + assertStringTranslate("i\u0307", "İ", "xy", UNICODE_CI, "x"); + assertStringTranslate("i\u030A", "İ", "x", UTF8_BINARY, "i\u030A"); + assertStringTranslate("i\u030A", "İ", "x", UTF8_LCASE, "i\u030A"); + assertStringTranslate("i\u030A", "İ", "x", UNICODE, "i\u030A"); + assertStringTranslate("i\u030A", "İ", "x", UNICODE_CI, "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UTF8_BINARY, "y\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UTF8_LCASE, "y\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UNICODE, "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UNICODE_CI, "i\u030A"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UTF8_BINARY, "123"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UTF8_LCASE, "11"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UNICODE, "1i\u0307"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UNICODE_CI, "11"); + assertStringTranslate("İi\u0307", "İyz", "123", UTF8_BINARY, "1i\u0307"); + assertStringTranslate("İi\u0307", "İyz", "123", UTF8_LCASE, "11"); + assertStringTranslate("İi\u0307", "İyz", "123", UNICODE, "1i\u0307"); + assertStringTranslate("İi\u0307", "İyz", "123", UNICODE_CI, "11"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UTF8_BINARY, "İ23"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UTF8_LCASE, "İ23"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UNICODE, "İi\u0307"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UNICODE_CI, "İi\u0307"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UTF8_BINARY, "12bc3"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UTF8_LCASE, "12bc3"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UNICODE, "3bc3"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UNICODE_CI, "3bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UTF8_BINARY, "a2bcå"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UTF8_LCASE, "12bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UNICODE, "a\u030Abcå"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UNICODE_CI, "3bc3"); + assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", UTF8_BINARY, "3\u030Aβφδ1\u0307"); + assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", UTF8_LCASE, "3\u030Aβφδ2"); + assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", UNICODE, "4βφδ2"); + assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", UNICODE_CI, "4βφδ2"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UTF8_BINARY", "σΥσΤΗΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UNICODE", "σΥσΤΗΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UTF8_BINARY", "ςΥςΤΗΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UNICODE", "ςΥςΤΗΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UTF8_BINARY", "σιστιματικος"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UNICODE", "σιστιματικος"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UNICODE_CI", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UTF8_BINARY", "σιστιματικος"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UNICODE", "σιστιματικος"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UNICODE_CI", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UTF8_BINARY", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UNICODE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UNICODE_CI", "σιστιματικοσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UTF8_BINARY, "σΥσΤΗΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UTF8_LCASE, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UNICODE, "σΥσΤΗΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UNICODE_CI, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UTF8_LCASE, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UNICODE_CI, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UTF8_LCASE, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UNICODE_CI, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UTF8_LCASE, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UNICODE_CI, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UTF8_BINARY, "ςΥςΤΗΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UTF8_LCASE, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UNICODE, "ςΥςΤΗΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UNICODE_CI, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UTF8_LCASE, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UNICODE_CI, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UTF8_BINARY, "σιστιματικος"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UTF8_LCASE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UNICODE, "σιστιματικος"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UNICODE_CI, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "συη", "σιι", UTF8_BINARY, "σιστιματικος"); + assertStringTranslate("συστηματικος", "συη", "σιι", UTF8_LCASE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "συη", "σιι", UNICODE, "σιστιματικος"); + assertStringTranslate("συστηματικος", "συη", "σιι", UNICODE_CI, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UTF8_BINARY, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UTF8_LCASE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UNICODE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UNICODE_CI, "σιστιματικοσ"); // Surrogate pairs. - assertStringTranslate("a🙃b🙃c", "a", "x", "UTF8_BINARY", "x🙃b🙃c"); - assertStringTranslate("a🙃b🙃c", "a🙃", "xy", "UTF8_BINARY", "xybyc"); - assertStringTranslate("a🙃b🙃c", "a🙃b", "xyz", "UTF8_BINARY", "xyzyc"); - assertStringTranslate("a🙃b🙃c", "a🙃bc", "xyzw", "UTF8_BINARY", "xyzyw"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UTF8_BINARY", "😀😂😃😅"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UTF8_LCASE", "😀😂😃😅"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UNICODE", "😀😂😃😅"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UNICODE_CI", "😀😂😃😅"); - assertStringTranslate("𐐅", "𐐅", "x", "UTF8_BINARY", "x"); - assertStringTranslate("𐐅", "𐐅", "x", "UTF8_LCASE", "x"); - assertStringTranslate("𐐅", "𐐅", "x", "UNICODE", "x"); - assertStringTranslate("𐐅", "𐐅", "x", "UNICODE_CI", "x"); - assertStringTranslate("𐐅", "𐐭", "x", "UTF8_BINARY", "𐐅"); - assertStringTranslate("𐐅", "𐐭", "x", "UTF8_LCASE", "x"); - assertStringTranslate("𐐅", "𐐭", "x", "UNICODE", "𐐅"); - assertStringTranslate("𐐅", "𐐭", "x", "UNICODE_CI", "x"); - assertStringTranslate("A", "A", "𐐅", "UTF8_BINARY", "𐐅"); - assertStringTranslate("A", "A", "𐐅", "UTF8_LCASE", "𐐅"); - assertStringTranslate("A", "A", "𐐅", "UNICODE", "𐐅"); - assertStringTranslate("A", "A", "𐐅", "UNICODE_CI", "𐐅"); - assertStringTranslate("A", "a", "𐐅", "UTF8_BINARY", "A"); - assertStringTranslate("A", "a", "𐐅", "UTF8_LCASE", "𐐅"); - assertStringTranslate("A", "a", "𐐅", "UNICODE", "A"); - assertStringTranslate("A", "a", "𐐅", "UNICODE_CI", "𐐅"); - assertStringTranslate("a", "A", "𐐅", "UTF8_BINARY", "a"); - assertStringTranslate("a", "A", "𐐅", "UTF8_LCASE", "𐐅"); - assertStringTranslate("a", "A", "𐐅", "UNICODE", "a"); - assertStringTranslate("a", "A", "𐐅", "UNICODE_CI", "𐐅"); - assertStringTranslate("𝔸", "𝔸", "x", "UTF8_BINARY", "x"); - assertStringTranslate("𝔸", "𝔸", "x", "UTF8_LCASE", "x"); - assertStringTranslate("𝔸", "𝔸", "x", "UNICODE", "x"); - assertStringTranslate("𝔸", "𝔸", "x", "UNICODE_CI", "x"); - assertStringTranslate("𝔸", "𝕒", "x", "UTF8_BINARY", "𝔸"); - assertStringTranslate("𝔸", "𝕒", "x", "UTF8_LCASE", "𝔸"); - assertStringTranslate("𝔸", "𝕒", "x", "UNICODE", "𝔸"); - assertStringTranslate("𝔸", "𝕒", "x", "UNICODE_CI", "x"); + assertStringTranslate("a🙃b🙃c", "a", "x", UTF8_BINARY, "x🙃b🙃c"); + assertStringTranslate("a🙃b🙃c", "a🙃", "xy", UTF8_BINARY, "xybyc"); + assertStringTranslate("a🙃b🙃c", "a🙃b", "xyz", UTF8_BINARY, "xyzyc"); + assertStringTranslate("a🙃b🙃c", "a🙃bc", "xyzw", UTF8_BINARY, "xyzyw"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UTF8_BINARY, "😀😂😃😅"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UTF8_LCASE, "😀😂😃😅"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UNICODE, "😀😂😃😅"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UNICODE_CI, "😀😂😃😅"); + assertStringTranslate("𐐅", "𐐅", "x", UTF8_BINARY, "x"); + assertStringTranslate("𐐅", "𐐅", "x", UTF8_LCASE, "x"); + assertStringTranslate("𐐅", "𐐅", "x", UNICODE, "x"); + assertStringTranslate("𐐅", "𐐅", "x", UNICODE_CI, "x"); + assertStringTranslate("𐐅", "𐐭", "x", UTF8_BINARY, "𐐅"); + assertStringTranslate("𐐅", "𐐭", "x", UTF8_LCASE, "x"); + assertStringTranslate("𐐅", "𐐭", "x", UNICODE, "𐐅"); + assertStringTranslate("𐐅", "𐐭", "x", UNICODE_CI, "x"); + assertStringTranslate("A", "A", "𐐅", UTF8_BINARY, "𐐅"); + assertStringTranslate("A", "A", "𐐅", UTF8_LCASE, "𐐅"); + assertStringTranslate("A", "A", "𐐅", UNICODE, "𐐅"); + assertStringTranslate("A", "A", "𐐅", UNICODE_CI, "𐐅"); + assertStringTranslate("A", "a", "𐐅", UTF8_BINARY, "A"); + assertStringTranslate("A", "a", "𐐅", UTF8_LCASE, "𐐅"); + assertStringTranslate("A", "a", "𐐅", UNICODE, "A"); + assertStringTranslate("A", "a", "𐐅", UNICODE_CI, "𐐅"); + assertStringTranslate("a", "A", "𐐅", UTF8_BINARY, "a"); + assertStringTranslate("a", "A", "𐐅", UTF8_LCASE, "𐐅"); + assertStringTranslate("a", "A", "𐐅", UNICODE, "a"); + assertStringTranslate("a", "A", "𐐅", UNICODE_CI, "𐐅"); + assertStringTranslate("𝔸", "𝔸", "x", UTF8_BINARY, "x"); + assertStringTranslate("𝔸", "𝔸", "x", UTF8_LCASE, "x"); + assertStringTranslate("𝔸", "𝔸", "x", UNICODE, "x"); + assertStringTranslate("𝔸", "𝔸", "x", UNICODE_CI, "x"); + assertStringTranslate("𝔸", "𝕒", "x", UTF8_BINARY, "𝔸"); + assertStringTranslate("𝔸", "𝕒", "x", UTF8_LCASE, "𝔸"); + assertStringTranslate("𝔸", "𝕒", "x", UNICODE, "𝔸"); + assertStringTranslate("𝔸", "𝕒", "x", UNICODE_CI, "x"); } private Map buildDict(String matching, String replace) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6077e55561e62..727d54b6bbd2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler -import org.apache.spark.sql.catalyst.util.{CollationFactory, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.{CollationFactory, CollationNames, DateTimeUtils} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{AtomicType, StringType, TimestampNTZType, TimestampType} @@ -5787,7 +5787,7 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def trimCollationEnabled: Boolean = getConf(TRIM_COLLATION_ENABLED) override def defaultStringType: StringType = { - if (getConf(DEFAULT_COLLATION).toUpperCase(Locale.ROOT) == "UTF8_BINARY") { + if (getConf(DEFAULT_COLLATION).toUpperCase(Locale.ROOT) == CollationNames.UTF8_BINARY) { StringType } else { StringType(getConf(DEFAULT_COLLATION)) From 94a31bd9c144da16f4fe5d42a709c1dc415e1277 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 14 Jan 2025 12:52:55 +0900 Subject: [PATCH 416/438] [SPARK-48809][PYTHON][DOCS] Reimplemented `spark version drop down` of the `PySpark doc site` and fix bug ### What changes were proposed in this pull request? The pr aims to using `pydata_sphinx_theme`'s `embedded` `version-switcher` to reimplement `spark version drop down` of the `PySpark doc site` and fix bug for `4.0.0-preview1`'s python api docs. ### Why are the changes needed? - When I was reviewing `4.0.0-preview1`'s docs, I found that `spark version drop down of the PySpark doc site` is no longer usable (when clicking, `the dropdown menu` will no longer display), as follows: https://spark.apache.org/docs/4.0.0-preview1/api/python/index.html image With the continuous updates of sphinx and its dependent component versions that generate Python documents, the `version-switcher` originally implemented by custom extensions is no longer usable, and we need to fix it - After image **It looks more in line with the current theme `CSS style` and looks more `beautiful`** - In addition, to cooperate with this fix, we need to update the content of the `spark-website/site/static/versions.json` file, as follows: image **I will complete it in another separate PR** The modified file `versions.json` will be compatible with the implementation of `spark 3.5.1` and the `new implementation`. https://github.com/apache/spark/blob/310f8ea2456dad7cec0f22bfed05a679764c3d7e/python/docs/source/_templates/version-switcher.html#L63-L73 ### 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 #47214 from panbingkun/SPARK-48809. Lead-authored-by: panbingkun Co-authored-by: panbingkun Signed-off-by: Hyukjin Kwon --- python/docs/source/_static/css/pyspark.css | 13 ---- .../source/_templates/version-switcher.html | 77 ------------------- python/docs/source/conf.py | 18 ++--- 3 files changed, 9 insertions(+), 99 deletions(-) delete mode 100644 python/docs/source/_templates/version-switcher.html diff --git a/python/docs/source/_static/css/pyspark.css b/python/docs/source/_static/css/pyspark.css index 565eaea299359..6f47dd80e9503 100644 --- a/python/docs/source/_static/css/pyspark.css +++ b/python/docs/source/_static/css/pyspark.css @@ -91,16 +91,3 @@ u.bd-sidebar .nav>li>ul>.active:hover>a,.bd-sidebar .nav>li>ul>.active>a { .spec_table tr, td, th { border-top: none!important; } - -/* Styling to the version dropdown */ -#version-button { - padding-left: 0.2rem; - padding-right: 3.2rem; -} - -#version_switcher { - height: auto; - max-height: 300px; - width: 165px; - overflow-y: auto; -} diff --git a/python/docs/source/_templates/version-switcher.html b/python/docs/source/_templates/version-switcher.html deleted file mode 100644 index 16c443229f4be..0000000000000 --- a/python/docs/source/_templates/version-switcher.html +++ /dev/null @@ -1,77 +0,0 @@ - - - - - diff --git a/python/docs/source/conf.py b/python/docs/source/conf.py index 5640ba151176d..20c13cd768deb 100644 --- a/python/docs/source/conf.py +++ b/python/docs/source/conf.py @@ -188,19 +188,19 @@ # a list of builtin themes. html_theme = 'pydata_sphinx_theme' -html_context = { - # When releasing a new Spark version, please update the file - # "site/static/versions.json" under the code repository "spark-website" - # (item should be added in order), and also set the local environment - # variable "RELEASE_VERSION". - "switcher_json_url": "https://spark.apache.org/static/versions.json", - "switcher_template_url": "https://spark.apache.org/docs/{version}/api/python/index.html", -} - # Theme options are theme-specific and customize the look and feel of a theme # further. For a list of options available for each theme, see the # documentation. html_theme_options = { + "check_switcher": False, + "switcher": { + # When releasing a new Spark version, please update the file + # "site/static/versions.json" under the code repository "spark-website" + # (item should be added in order), and also set the local environment + # variable "RELEASE_VERSION". + "json_url": "https://spark.apache.org/static/versions.json", + "version_match": release, + }, "header_links_before_dropdown": 6, "navbar_end": ["version-switcher", "theme-switcher", "navbar-icon-links"], "footer_start": ["spark_footer", "sphinx-version"], From 3c7f5e25b70ce8332c31bee50b704dc55d810bf1 Mon Sep 17 00:00:00 2001 From: Gene Pang Date: Tue, 14 Jan 2025 12:58:42 +0900 Subject: [PATCH 417/438] [SPARK-50790][PYTHON] Implement parse json in pyspark ### What changes were proposed in this pull request? Implement the parseJson functionality in PySpark, for parsing a json string to a VariantVal. ### Why are the changes needed? Currently, there is no way to create a VariantVal from python. It can only be created from Spark SQL. ### Does this PR introduce _any_ user-facing change? Added `VariantVal.parseJson`, which takes a json string, and returns a `VariantVal`. ### How was this patch tested? Added unittests. ### Was this patch authored or co-authored using generative AI tooling? no Closes #49450 from gene-db/py-parse-json. Authored-by: Gene Pang Signed-off-by: Hyukjin Kwon --- .../reference/pyspark.sql/variant_val.rst | 1 + python/pyspark/sql/tests/test_types.py | 11 + python/pyspark/sql/types.py | 9 + python/pyspark/sql/variant_utils.py | 327 +++++++++++++++++- 4 files changed, 346 insertions(+), 2 deletions(-) diff --git a/python/docs/source/reference/pyspark.sql/variant_val.rst b/python/docs/source/reference/pyspark.sql/variant_val.rst index 8630ae8aace14..883b4c8fdc3d5 100644 --- a/python/docs/source/reference/pyspark.sql/variant_val.rst +++ b/python/docs/source/reference/pyspark.sql/variant_val.rst @@ -26,3 +26,4 @@ VariantVal VariantVal.toPython VariantVal.toJson + VariantVal.parseJson diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 432ddd083c802..75c28ac0dec1d 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -2240,6 +2240,17 @@ def test_variant_type(self): PySparkValueError, lambda: str(VariantVal(bytes([32, 10, 1, 0, 0, 0]), metadata)) ) + # check parse_json + for key, json, obj in expected_values: + self.assertEqual(VariantVal.parseJson(json).toJson(), json) + self.assertEqual(VariantVal.parseJson(json).toPython(), obj) + + # compare the parse_json in Spark vs python. `json_str` contains all of `expected_values`. + parse_json_spark_output = variants[0] + parse_json_python_output = VariantVal.parseJson(json_str) + self.assertEqual(parse_json_spark_output.value, parse_json_python_output.value) + self.assertEqual(parse_json_spark_output.metadata, parse_json_python_output.metadata) + def test_to_ddl(self): schema = StructType().add("a", NullType()).add("b", BooleanType()).add("c", BinaryType()) self.assertEqual(schema.toDDL(), "a VOID,b BOOLEAN,c BINARY") diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index f40a8bf62b290..b913e05e16d2a 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1770,6 +1770,15 @@ def toJson(self, zone_id: str = "UTC") -> str: """ return VariantUtils.to_json(self.value, self.metadata, zone_id) + @classmethod + def parseJson(cls, json_str: str) -> "VariantVal": + """ + Convert the VariantVal to a nested Python object of Python data types. + :return: Python representation of the Variant nested structure + """ + (value, metadata) = VariantUtils.parse_json(json_str) + return VariantVal(value, metadata) + _atomic_types: List[Type[DataType]] = [ StringType, diff --git a/python/pyspark/sql/variant_utils.py b/python/pyspark/sql/variant_utils.py index 40cc69c1f0961..3025523064e1d 100644 --- a/python/pyspark/sql/variant_utils.py +++ b/python/pyspark/sql/variant_utils.py @@ -21,7 +21,7 @@ import json import struct from array import array -from typing import Any, Callable, Dict, List, Tuple +from typing import Any, Callable, Dict, List, NamedTuple, Tuple from pyspark.errors import PySparkValueError from zoneinfo import ZoneInfo @@ -108,8 +108,25 @@ class VariantUtils: # string size) + (size bytes of string content). LONG_STR = 16 + VERSION = 1 + # The lower 4 bits of the first metadata byte contain the version. + VERSION_MASK = 0x0F + + U8_MAX = 0xFF + U16_MAX = 0xFFFF + U24_MAX = 0xFFFFFF + U24_SIZE = 3 U32_SIZE = 4 + I8_MAX = 0x7F + I8_MIN = -0x80 + I16_MAX = 0x7FFF + I16_MIN = -0x8000 + I32_MAX = 0x7FFFFFFF + I32_MIN = -0x80000000 + I64_MAX = 0x7FFFFFFFFFFFFFFF + I64_MIN = -0x8000000000000000 + EPOCH = datetime.datetime( year=1970, month=1, day=1, hour=0, minute=0, second=0, tzinfo=datetime.timezone.utc ) @@ -140,6 +157,15 @@ def to_python(cls, value: bytes, metadata: bytes) -> str: """ return cls._to_python(value, metadata, 0) + @classmethod + def parse_json(cls, json_str: str) -> Tuple[bytes, bytes]: + """ + Parses the JSON string and creates the Variant binary (value, metadata) + :return: tuple of 2 binary values (value, metadata) + """ + builder = VariantBuilder() + return builder.build(json_str) + @classmethod def _read_long(cls, data: bytes, pos: int, num_bytes: int, signed: bool) -> int: cls._check_index(pos, len(data)) @@ -468,7 +494,10 @@ def _handle_object( value, offset_start + offset_size * i, offset_size, signed=False ) value_pos = data_start + offset - key_value_pos_list.append((cls._get_metadata_key(metadata, id), value_pos)) + if metadata is not None: + key_value_pos_list.append((cls._get_metadata_key(metadata, id), value_pos)) + else: + key_value_pos_list.append(("", value_pos)) return func(key_value_pos_list) @classmethod @@ -496,3 +525,297 @@ def _handle_array(cls, value: bytes, pos: int, func: Callable[[List[int]], Any]) element_pos = data_start + offset value_pos_list.append(element_pos) return func(value_pos_list) + + +class FieldEntry(NamedTuple): + """ + Info about an object field + """ + + key: str + id: int + offset: int + + +class VariantBuilder: + """ + A utility class for building VariantVal. + """ + + DEFAULT_SIZE_LIMIT = 16 * 1024 * 1024 + + def __init__(self, size_limit: int = DEFAULT_SIZE_LIMIT): + self.value = bytearray() + self.dictionary = dict[str, int]() + self.dictionary_keys = list[bytes]() + self.size_limit = size_limit + + def build(self, json_str: str) -> Tuple[bytes, bytes]: + parsed = json.loads(json_str, parse_float=self._handle_float) + self._process_parsed_json(parsed) + + num_keys = len(self.dictionary_keys) + dictionary_string_size = sum(len(key) for key in self.dictionary_keys) + + # Determine the number of bytes required per offset entry. + # The largest offset is the one-past-the-end value, which is total string size. It's very + # unlikely that the number of keys could be larger, but incorporate that into the + # calculation in case of pathological data. + max_size = max(dictionary_string_size, num_keys) + if max_size > self.size_limit: + raise PySparkValueError(errorClass="VARIANT_SIZE_LIMIT_EXCEEDED", messageParameters={}) + offset_size = self._get_integer_size(max_size) + + offset_start = 1 + offset_size + string_start = offset_start + (num_keys + 1) * offset_size + metadata_size = string_start + dictionary_string_size + if metadata_size > self.size_limit: + raise PySparkValueError(errorClass="VARIANT_SIZE_LIMIT_EXCEEDED", messageParameters={}) + + metadata = bytearray() + header_byte = VariantUtils.VERSION | ((offset_size - 1) << 6) + metadata.extend(header_byte.to_bytes(1, byteorder="little")) + metadata.extend(num_keys.to_bytes(offset_size, byteorder="little")) + # write offsets + current_offset = 0 + for key in self.dictionary_keys: + metadata.extend(current_offset.to_bytes(offset_size, byteorder="little")) + current_offset += len(key) + metadata.extend(current_offset.to_bytes(offset_size, byteorder="little")) + # write key data + for key in self.dictionary_keys: + metadata.extend(key) + return (bytes(self.value), bytes(metadata)) + + def _process_parsed_json(self, parsed: Any) -> None: + if type(parsed) is dict: + fields = list[FieldEntry]() + start = len(self.value) + for key, value in parsed.items(): + id = self._add_key(key) + fields.append(FieldEntry(key, id, len(self.value) - start)) + self._process_parsed_json(value) + self._finish_writing_object(start, fields) + elif type(parsed) is list: + offsets = [] + start = len(self.value) + for elem in parsed: + offsets.append(len(self.value) - start) + self._process_parsed_json(elem) + self._finish_writing_array(start, offsets) + elif type(parsed) is str: + self._append_string(parsed) + elif type(parsed) is int: + if not self._append_int(parsed): + self._process_parsed_json(self._handle_float(str(parsed))) + elif type(parsed) is float: + self._append_float(parsed) + elif type(parsed) is decimal.Decimal: + self._append_decimal(parsed) + elif type(parsed) is bool: + self._append_boolean(parsed) + elif parsed is None: + self._append_null() + else: + raise PySparkValueError(errorClass="MALFORMED_VARIANT", messageParameters={}) + + # Choose the smallest unsigned integer type that can store `value`. It must be within + # [0, U24_MAX]. + def _get_integer_size(self, value: int) -> int: + if value <= VariantUtils.U8_MAX: + return 1 + if value <= VariantUtils.U16_MAX: + return 2 + return VariantUtils.U24_SIZE + + def _check_capacity(self, additional: int) -> None: + required = len(self.value) + additional + if required > self.size_limit: + raise PySparkValueError(errorClass="VARIANT_SIZE_LIMIT_EXCEEDED", messageParameters={}) + + def _primitive_header(self, type: int) -> bytes: + return bytes([(type << 2) | VariantUtils.PRIMITIVE]) + + def _short_string_header(self, size: int) -> bytes: + return bytes([size << 2 | VariantUtils.SHORT_STR]) + + def _array_header(self, large_size: bool, offset_size: int) -> bytes: + return bytes( + [ + ( + (large_size << (VariantUtils.BASIC_TYPE_BITS + 2)) + | ((offset_size - 1) << VariantUtils.BASIC_TYPE_BITS) + | VariantUtils.ARRAY + ) + ] + ) + + def _object_header(self, large_size: bool, id_size: int, offset_size: int) -> bytes: + return bytes( + [ + ( + (large_size << (VariantUtils.BASIC_TYPE_BITS + 4)) + | ((id_size - 1) << (VariantUtils.BASIC_TYPE_BITS + 2)) + | ((offset_size - 1) << VariantUtils.BASIC_TYPE_BITS) + | VariantUtils.OBJECT + ) + ] + ) + + # Add a key to the variant dictionary. If the key already exists, the dictionary is + # not modified. In either case, return the id of the key. + def _add_key(self, key: str) -> int: + if key in self.dictionary: + return self.dictionary[key] + id = len(self.dictionary_keys) + self.dictionary[key] = id + self.dictionary_keys.append(key.encode("utf-8")) + return id + + def _handle_float(self, num_str: str) -> Any: + # a float can be a decimal if it only contains digits, '-', or '-'. + if all([ch.isdecimal() or ch == "-" or ch == "." for ch in num_str]): + dec = decimal.Decimal(num_str) + precision = len(dec.as_tuple().digits) + scale = -int(dec.as_tuple().exponent) + + if ( + scale <= VariantUtils.MAX_DECIMAL16_PRECISION + and precision <= VariantUtils.MAX_DECIMAL16_PRECISION + ): + return dec + return float(num_str) + + def _append_boolean(self, b: bool) -> None: + self._check_capacity(1) + self.value.extend(self._primitive_header(VariantUtils.TRUE if b else VariantUtils.FALSE)) + + def _append_null(self) -> None: + self._check_capacity(1) + self.value.extend(self._primitive_header(VariantUtils.NULL)) + + def _append_string(self, s: str) -> None: + text = s.encode("utf-8") + long_str = len(text) > VariantUtils.MAX_SHORT_STR_SIZE + additional = (1 + VariantUtils.U32_SIZE) if long_str else 1 + self._check_capacity(additional + len(text)) + if long_str: + self.value.extend(self._primitive_header(VariantUtils.LONG_STR)) + self.value.extend(len(text).to_bytes(VariantUtils.U32_SIZE, byteorder="little")) + else: + self.value.extend(self._short_string_header(len(text))) + self.value.extend(text) + + def _append_int(self, i: int) -> bool: + self._check_capacity(1 + 8) + if i >= VariantUtils.I8_MIN and i <= VariantUtils.I8_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT1)) + self.value.extend(i.to_bytes(1, byteorder="little", signed=True)) + elif i >= VariantUtils.I16_MIN and i <= VariantUtils.I16_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT2)) + self.value.extend(i.to_bytes(2, byteorder="little", signed=True)) + elif i >= VariantUtils.I32_MIN and i <= VariantUtils.I32_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT4)) + self.value.extend(i.to_bytes(4, byteorder="little", signed=True)) + elif i >= VariantUtils.I64_MIN and i <= VariantUtils.I64_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT8)) + self.value.extend(i.to_bytes(8, byteorder="little", signed=True)) + else: + return False + return True + + # Append a decimal value to the variant builder. The caller should guarantee that its precision + # and scale fit into `MAX_DECIMAL16_PRECISION`. + def _append_decimal(self, d: decimal.Decimal) -> None: + self._check_capacity(2 + 16) + precision = len(d.as_tuple().digits) + scale = -int(d.as_tuple().exponent) + unscaled = int("".join(map(str, d.as_tuple().digits))) + unscaled = -unscaled if d < 0 else unscaled + if ( + scale <= VariantUtils.MAX_DECIMAL4_PRECISION + and precision <= VariantUtils.MAX_DECIMAL4_PRECISION + ): + self.value.extend(self._primitive_header(VariantUtils.DECIMAL4)) + self.value.extend(scale.to_bytes(1, byteorder="little")) + self.value.extend(unscaled.to_bytes(4, byteorder="little", signed=True)) + elif ( + scale <= VariantUtils.MAX_DECIMAL8_PRECISION + and precision <= VariantUtils.MAX_DECIMAL8_PRECISION + ): + self.value.extend(self._primitive_header(VariantUtils.DECIMAL8)) + self.value.extend(scale.to_bytes(1, byteorder="little")) + self.value.extend(unscaled.to_bytes(8, byteorder="little", signed=True)) + else: + assert ( + scale <= VariantUtils.MAX_DECIMAL16_PRECISION + and precision <= VariantUtils.MAX_DECIMAL16_PRECISION + ) + self.value.extend(self._primitive_header(VariantUtils.DECIMAL16)) + self.value.extend(scale.to_bytes(1, byteorder="little")) + self.value.extend(unscaled.to_bytes(16, byteorder="little", signed=True)) + + def _append_float(self, f: float) -> None: + self._check_capacity(1 + 8) + self.value.extend(self._primitive_header(VariantUtils.DOUBLE)) + self.value.extend(struct.pack(" None: + data_size = len(self.value) - start + num_offsets = len(offsets) + large_size = num_offsets > VariantUtils.U8_MAX + size_bytes = VariantUtils.U32_SIZE if large_size else 1 + offset_size = self._get_integer_size(data_size) + # The space for header byte, object size, and offset list. + header_size = 1 + size_bytes + (num_offsets + 1) * offset_size + self._check_capacity(header_size) + self.value.extend(bytearray(header_size)) + # Shift the just-written element data to make room for the header section. + self.value[start + header_size :] = bytes(self.value[start : start + data_size]) + # Write the header byte, num offsets + offset_start = start + 1 + size_bytes + self.value[start : start + 1] = self._array_header(large_size, offset_size) + self.value[start + 1 : offset_start] = num_offsets.to_bytes(size_bytes, byteorder="little") + # write offset list + offset_list = bytearray() + for offset in offsets: + offset_list.extend(offset.to_bytes(offset_size, byteorder="little")) + offset_list.extend(data_size.to_bytes(offset_size, byteorder="little")) + self.value[offset_start : offset_start + len(offset_list)] = offset_list + + # Finish writing a variant object after all of its fields have already been written. + def _finish_writing_object(self, start: int, fields: List[FieldEntry]) -> None: + num_fields = len(fields) + # object fields are from a python dictionary, so keys are already distinct + fields.sort(key=lambda f: f.key) + max_id = 0 + for field in fields: + max_id = max(max_id, field.id) + + data_size = len(self.value) - start + large_size = num_fields > VariantUtils.U8_MAX + size_bytes = VariantUtils.U32_SIZE if large_size else 1 + id_size = self._get_integer_size(max_id) + offset_size = self._get_integer_size(data_size) + # The space for header byte, object size, id list, and offset list. + header_size = 1 + size_bytes + num_fields * id_size + (num_fields + 1) * offset_size + self._check_capacity(header_size) + self.value.extend(bytearray(header_size)) + # Shift the just-written field data to make room for the object header section. + self.value[start + header_size :] = self.value[start : start + data_size] + # Write the header byte, num fields, id list, offset list + self.value[start : start + 1] = self._object_header(large_size, id_size, offset_size) + self.value[start + 1 : start + 1 + size_bytes] = num_fields.to_bytes( + size_bytes, byteorder="little" + ) + id_start = start + 1 + size_bytes + offset_start = id_start + num_fields * id_size + id_list = bytearray() + offset_list = bytearray() + for field in fields: + id_list.extend(field.id.to_bytes(id_size, byteorder="little")) + offset_list.extend(field.offset.to_bytes(offset_size, byteorder="little")) + offset_list.extend(data_size.to_bytes(offset_size, byteorder="little")) + self.value[id_start : id_start + len(id_list)] = id_list + self.value[offset_start : offset_start + len(offset_list)] = offset_list From bba6839d87144a251464bda410540e9877cbba2b Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Tue, 14 Jan 2025 14:58:58 +0800 Subject: [PATCH 418/438] [SPARK-50762][SQL] Add Analyzer rule for resolving SQL scalar UDFs ### What changes were proposed in this pull request? This PR adds a new Analyzer rule `ResolveSQLFunctions` to resolve scalar SQL UDFs by replacing a `SQLFunctionExpression` with an actual function body. It currently supports the following operators: Project, Filter, Join and Aggregate. For example: ``` CREATE FUNCTION area(width DOUBLE, height DOUBLE) RETURNS DOUBLE RETURN width * height; ``` and this query ``` SELECT area(a, b) FROM t; ``` will be resolved as ``` Project [area(width, height) AS area] +- Project [a, b, CAST(a AS DOUBLE) AS width, CAST(b AS DOUBLE) AS height] +- Relation [a, b] ``` ### Why are the changes needed? To support SQL UDFs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New SQL query tests. More tests will be added once table function resolution is supported. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49414 from allisonwang-db/spark-50762-resolve-scalar-udf. Authored-by: Allison Wang Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 13 + .../catalyst/expressions/ExpressionInfo.java | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 272 +++++++++ .../sql/catalyst/analysis/CheckAnalysis.scala | 2 + .../analysis/SQLFunctionExpression.scala | 53 +- .../sql/catalyst/catalog/SessionCatalog.scala | 103 +++- .../catalog/UserDefinedFunction.scala | 21 + .../optimizer/EliminateSQLFunctionNode.scala | 47 ++ .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../sql/catalyst/trees/TreePatterns.scala | 1 + .../analyzer-results/sql-udf.sql.out | 575 ++++++++++++++++++ .../resources/sql-tests/inputs/sql-udf.sql | 122 ++++ .../sql-tests/results/sql-udf.sql.out | 484 +++++++++++++++ .../sql/execution/SQLFunctionSuite.scala | 61 ++ .../sql/expressions/ExpressionInfoSuite.scala | 3 +- 15 files changed, 1753 insertions(+), 7 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSQLFunctionNode.scala create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/sql-udf.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 8b266e9d6ac11..5037b52475422 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3126,6 +3126,13 @@ ], "sqlState" : "42K08" }, + "INVALID_SQL_FUNCTION_PLAN_STRUCTURE" : { + "message" : [ + "Invalid SQL function plan structure", + "" + ], + "sqlState" : "XXKD0" + }, "INVALID_SQL_SYNTAX" : { "message" : [ "Invalid SQL syntax:" @@ -5757,6 +5764,12 @@ ], "sqlState" : "0A000" }, + "UNSUPPORTED_SQL_UDF_USAGE" : { + "message" : [ + "Using SQL function in is not supported." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_STREAMING_OPERATOR_WITHOUT_WATERMARK" : { "message" : [ " output mode not supported for on streaming DataFrames/DataSets without watermark." diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java index 4200619d3c5f9..310d18ddb3486 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java @@ -51,7 +51,7 @@ public class ExpressionInfo { "window_funcs", "xml_funcs", "table_funcs", "url_funcs", "variant_funcs")); private static final Set validSources = - new HashSet<>(Arrays.asList("built-in", "hive", "python_udf", "scala_udf", + new HashSet<>(Arrays.asList("built-in", "hive", "python_udf", "scala_udf", "sql_udf", "java_udf", "python_udtf", "internal")); public String getClassName() { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9282e0554a2d4..92cfc4119dd0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -374,6 +374,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor BindProcedures :: ResolveTableSpec :: ValidateAndStripPipeExpressions :: + ResolveSQLFunctions :: ResolveAliases :: ResolveSubquery :: ResolveSubqueryColumnAliases :: @@ -2364,6 +2365,277 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } + /** + * This rule resolves SQL function expressions. It pulls out function inputs and place them + * in a separate [[Project]] node below the operator and replace the SQL function with its + * actual function body. SQL function expressions in [[Aggregate]] are handled in a special + * way. Non-aggregated SQL functions in the aggregate expressions of an Aggregate need to be + * pulled out into a Project above the Aggregate before replacing the SQL function expressions + * with actual function bodies. For example: + * + * Before: + * Aggregate [c1] [foo(c1), foo(max(c2)), sum(foo(c2)) AS sum] + * +- Relation [c1, c2] + * + * After: + * Project [foo(c1), foo(max_c2), sum] + * +- Aggregate [c1] [c1, max(c2) AS max_c2, sum(foo(c2)) AS sum] + * +- Relation [c1, c2] + */ + object ResolveSQLFunctions extends Rule[LogicalPlan] { + + private def hasSQLFunctionExpression(exprs: Seq[Expression]): Boolean = { + exprs.exists(_.find(_.isInstanceOf[SQLFunctionExpression]).nonEmpty) + } + + /** + * Check if the function input contains aggregate expressions. + */ + private def checkFunctionInput(f: SQLFunctionExpression): Unit = { + if (f.inputs.exists(AggregateExpression.containsAggregate)) { + // The input of a SQL function should not contain aggregate functions after + // `extractAndRewrite`. If there are aggregate functions, it means they are + // nested in another aggregate function, which is not allowed. + // For example: SELECT sum(foo(sum(c1))) FROM t + // We have to throw the error here because otherwise the query plan after + // resolving the SQL function will not be valid. + throw new AnalysisException( + errorClass = "NESTED_AGGREGATE_FUNCTION", + messageParameters = Map.empty) + } + } + + /** + * Resolve a SQL function expression as a logical plan check if it can be analyzed. + */ + private def resolve(f: SQLFunctionExpression): LogicalPlan = { + // Validate the SQL function input. + checkFunctionInput(f) + val plan = v1SessionCatalog.makeSQLFunctionPlan(f.name, f.function, f.inputs) + val resolved = SQLFunctionContext.withSQLFunction { + // Resolve the SQL function plan using its context. + val conf = new SQLConf() + f.function.getSQLConfigs.foreach { case (k, v) => conf.settings.put(k, v) } + SQLConf.withExistingConf(conf) { + executeSameContext(plan) + } + } + // Fail the analysis eagerly if a SQL function cannot be resolved using its input. + SimpleAnalyzer.checkAnalysis(resolved) + resolved + } + + /** + * Rewrite SQL function expressions into actual resolved function bodies and extract + * function inputs into the given project list. + */ + private def rewriteSQLFunctions[E <: Expression]( + expression: E, + projectList: ArrayBuffer[NamedExpression]): E = { + val newExpr = expression match { + case f: SQLFunctionExpression if !hasSQLFunctionExpression(f.inputs) && + // Make sure LateralColumnAliasReference in parameters is resolved and eliminated first. + // Otherwise, the projectList can contain the LateralColumnAliasReference, which will be + // pushed down to a Project without the 'referenced' alias by LCA present, leaving it + // unresolved. + !f.inputs.exists(_.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE)) => + withPosition(f) { + val plan = resolve(f) + // Extract the function input project list from the SQL function plan and + // inline the SQL function expression. + plan match { + case Project(body :: Nil, Project(aliases, _: LocalRelation)) => + projectList ++= aliases + SQLScalarFunction(f.function, aliases.map(_.toAttribute), body) + case o => + throw new AnalysisException( + errorClass = "INVALID_SQL_FUNCTION_PLAN_STRUCTURE", + messageParameters = Map("plan" -> o.toString)) + } + } + case o => o.mapChildren(rewriteSQLFunctions(_, projectList)) + } + newExpr.asInstanceOf[E] + } + + /** + * Check if the given expression contains expressions that should be extracted, + * i.e. non-aggregated SQL functions with non-foldable inputs. + */ + private def shouldExtract(e: Expression): Boolean = e match { + // Return false if the expression is already an aggregate expression. + case _: AggregateExpression => false + case _: SQLFunctionExpression => true + case _: LeafExpression => false + case o => o.children.exists(shouldExtract) + } + + /** + * Extract aggregate expressions from the given expression and replace + * them with attribute references. + * Example: + * Before: foo(c1) + foo(max(c2)) + max(foo(c2)) + * After: foo(c1) + foo(max_c2) + max_foo_c2 + * Extracted expressions: [c1, max(c2) AS max_c2, max(foo(c2)) AS max_foo_c2] + */ + private def extractAndRewrite[T <: Expression]( + expression: T, + extractedExprs: ArrayBuffer[NamedExpression]): T = { + val newExpr = expression match { + case e if !shouldExtract(e) => + val exprToAdd: NamedExpression = e match { + case o: OuterReference => Alias(o, toPrettySQL(o.e))() + case ne: NamedExpression => ne + case o => Alias(o, toPrettySQL(o))() + } + extractedExprs += exprToAdd + exprToAdd.toAttribute + case f: SQLFunctionExpression => + val newInputs = f.inputs.map(extractAndRewrite(_, extractedExprs)) + f.copy(inputs = newInputs) + case o => o.mapChildren(extractAndRewrite(_, extractedExprs)) + } + newExpr.asInstanceOf[T] + } + + /** + * Replace all [[SQLFunctionExpression]]s in an expression with attribute references + * from the aliasMap. + */ + private def replaceSQLFunctionWithAttr[T <: Expression]( + expr: T, + aliasMap: mutable.HashMap[Expression, Alias]): T = { + expr.transform { + case f: SQLFunctionExpression if aliasMap.contains(f.canonicalized) => + aliasMap(f.canonicalized).toAttribute + }.asInstanceOf[T] + } + + private def rewrite(plan: LogicalPlan): LogicalPlan = plan match { + // Return if a sub-tree does not contain SQLFunctionExpression. + case p: LogicalPlan if !p.containsPattern(SQL_FUNCTION_EXPRESSION) => p + + case f @ Filter(cond, a: Aggregate) + if !f.resolved || AggregateExpression.containsAggregate(cond) || + ResolveGroupingAnalytics.hasGroupingFunction(cond) || + cond.containsPattern(TEMP_RESOLVED_COLUMN) => + // If the filter's condition contains aggregate expressions or grouping expressions or temp + // resolved column, we cannot rewrite both the filter and the aggregate until they are + // resolved by ResolveAggregateFunctions or ResolveGroupingAnalytics, because rewriting SQL + // functions in aggregate can add an additional project on top of the aggregate + // which breaks the pattern matching in those rules. + f.copy(child = a.copy(child = rewrite(a.child))) + + case h @ UnresolvedHaving(_, a: Aggregate) => + // Similarly UnresolvedHaving should be resolved by ResolveAggregateFunctions first + // before rewriting aggregate. + h.copy(child = a.copy(child = rewrite(a.child))) + + case a: Aggregate if a.resolved && hasSQLFunctionExpression(a.expressions) => + val child = rewrite(a.child) + // Extract SQL functions in the grouping expressions and place them in a project list + // below the current aggregate. Also update their appearances in the aggregate expressions. + val bottomProjectList = ArrayBuffer.empty[NamedExpression] + val aliasMap = mutable.HashMap.empty[Expression, Alias] + val newGrouping = a.groupingExpressions.map { expr => + expr.transformDown { + case f: SQLFunctionExpression => + val alias = aliasMap.getOrElseUpdate(f.canonicalized, Alias(f, f.name)()) + bottomProjectList += alias + alias.toAttribute + } + } + val aggregateExpressions = a.aggregateExpressions.map( + replaceSQLFunctionWithAttr(_, aliasMap)) + + // Rewrite SQL functions in the aggregate expressions that are not wrapped in + // aggregate functions. They need to be extracted into a project list above the + // current aggregate. + val aggExprs = ArrayBuffer.empty[NamedExpression] + val topProjectList = aggregateExpressions.map(extractAndRewrite(_, aggExprs)) + + // Rewrite SQL functions in the new aggregate expressions that are wrapped inside + // aggregate functions. + val newAggExprs = aggExprs.map(rewriteSQLFunctions(_, bottomProjectList)) + + val bottomProject = if (bottomProjectList.nonEmpty) { + Project(child.output ++ bottomProjectList, child) + } else { + child + } + val newAgg = if (newGrouping.nonEmpty || newAggExprs.nonEmpty) { + a.copy( + groupingExpressions = newGrouping, + aggregateExpressions = newAggExprs.toSeq, + child = bottomProject) + } else { + bottomProject + } + if (topProjectList.nonEmpty) Project(topProjectList, newAgg) else newAgg + + case p: Project if p.resolved && hasSQLFunctionExpression(p.expressions) => + val newChild = rewrite(p.child) + val projectList = ArrayBuffer.empty[NamedExpression] + val newPList = p.projectList.map(rewriteSQLFunctions(_, projectList)) + if (newPList != newChild.output) { + p.copy(newPList, Project(newChild.output ++ projectList, newChild)) + } else { + assert(projectList.isEmpty) + p.copy(child = newChild) + } + + case f: Filter if f.resolved && hasSQLFunctionExpression(f.expressions) => + val newChild = rewrite(f.child) + val projectList = ArrayBuffer.empty[NamedExpression] + val newCond = rewriteSQLFunctions(f.condition, projectList) + if (newCond != f.condition) { + Project(f.output, Filter(newCond, Project(newChild.output ++ projectList, newChild))) + } else { + assert(projectList.isEmpty) + f.copy(child = newChild) + } + + case j: Join if j.resolved && hasSQLFunctionExpression(j.expressions) => + val newLeft = rewrite(j.left) + val newRight = rewrite(j.right) + val projectList = ArrayBuffer.empty[NamedExpression] + val joinCond = j.condition.map(rewriteSQLFunctions(_, projectList)) + if (joinCond != j.condition) { + // Join condition cannot have non-deterministic expressions. We can safely + // replace the aliases with the original SQL function input expressions. + val aliasMap = projectList.collect { case a: Alias => a.toAttribute -> a.child }.toMap + val newJoinCond = joinCond.map(_.transform { + case a: Attribute => aliasMap.getOrElse(a, a) + }) + j.copy(left = newLeft, right = newRight, condition = newJoinCond) + } else { + assert(projectList.isEmpty) + j.copy(left = newLeft, right = newRight) + } + + case o: LogicalPlan if o.resolved && hasSQLFunctionExpression(o.expressions) => + o.transformExpressionsWithPruning(_.containsPattern(SQL_FUNCTION_EXPRESSION)) { + case f: SQLFunctionExpression => + f.failAnalysis( + errorClass = "UNSUPPORTED_SQL_UDF_USAGE", + messageParameters = Map( + "functionName" -> toSQLId(f.function.name.nameParts), + "nodeName" -> o.nodeName.toString)) + } + + case p: LogicalPlan => p.mapChildren(rewrite) + } + + def apply(plan: LogicalPlan): LogicalPlan = { + // Only rewrite SQL functions when they are not in nested function calls. + if (SQLFunctionContext.get.nestedSQLFunctionDepth > 0) { + plan + } else { + rewrite(plan) + } + } + } + /** * Turns projections that contain aggregate expressions into aggregations. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 46ca8e793218b..0a68524c31241 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1106,6 +1106,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB @scala.annotation.tailrec def cleanQueryInScalarSubquery(p: LogicalPlan): LogicalPlan = p match { case s: SubqueryAlias => cleanQueryInScalarSubquery(s.child) + // Skip SQL function node added by the Analyzer + case s: SQLFunctionNode => cleanQueryInScalarSubquery(s.child) case p: Project => cleanQueryInScalarSubquery(p.child) case h: ResolvedHint => cleanQueryInScalarSubquery(h.child) case child => child diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala index fb6935d64d4c4..37981f47287da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.catalog.SQLFunction -import org.apache.spark.sql.catalyst.expressions.{Expression, Unevaluable} -import org.apache.spark.sql.catalyst.trees.TreePattern.{SQL_FUNCTION_EXPRESSION, TreePattern} +import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression, Unevaluable} +import org.apache.spark.sql.catalyst.trees.TreePattern.{SQL_FUNCTION_EXPRESSION, SQL_SCALAR_FUNCTION, TreePattern} import org.apache.spark.sql.types.DataType /** @@ -39,3 +39,52 @@ case class SQLFunctionExpression( newChildren: IndexedSeq[Expression]): SQLFunctionExpression = copy(inputs = newChildren) final override val nodePatterns: Seq[TreePattern] = Seq(SQL_FUNCTION_EXPRESSION) } + +/** + * A wrapper node for a SQL scalar function expression. + */ +case class SQLScalarFunction(function: SQLFunction, inputs: Seq[Expression], child: Expression) + extends UnaryExpression with Unevaluable { + override def dataType: DataType = child.dataType + override def toString: String = s"${function.name}(${inputs.mkString(", ")})" + override def sql: String = s"${function.name}(${inputs.map(_.sql).mkString(", ")})" + override protected def withNewChildInternal(newChild: Expression): SQLScalarFunction = { + copy(child = newChild) + } + final override val nodePatterns: Seq[TreePattern] = Seq(SQL_SCALAR_FUNCTION) + // The `inputs` is for display only and does not matter in execution. + override lazy val canonicalized: Expression = copy(inputs = Nil, child = child.canonicalized) + override lazy val deterministic: Boolean = { + function.deterministic.getOrElse(true) && children.forall(_.deterministic) + } +} + +/** + * Provide a way to keep state during analysis for resolving nested SQL functions. + * + * @param nestedSQLFunctionDepth The nested depth in the SQL function resolution. A SQL function + * expression should only be expanded as a [[SQLScalarFunction]] if + * the nested depth is 0. + */ +case class SQLFunctionContext(nestedSQLFunctionDepth: Int = 0) + +object SQLFunctionContext { + + private val value = new ThreadLocal[SQLFunctionContext]() { + override def initialValue: SQLFunctionContext = SQLFunctionContext() + } + + def get: SQLFunctionContext = value.get() + + def reset(): Unit = value.remove() + + private def set(context: SQLFunctionContext): Unit = value.set(context) + + def withSQLFunction[A](f: => A): A = { + val originContext = value.get() + val context = originContext.copy( + nestedSQLFunctionDepth = originContext.nestedSQLFunctionDepth + 1) + set(context) + try f finally { set(originContext) } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 3c6dfe5ac8445..b123952c5f086 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -38,9 +38,9 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry.TableFunctionBuilder import org.apache.spark.sql.catalyst.catalog.SQLFunction.parseDefault -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, ExpressionInfo, NamedArgumentExpression, NamedExpression, UpCast} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, ExpressionInfo, NamedArgumentExpression, NamedExpression, ScalarSubquery, UpCast} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter, LogicalPlan, NamedParametersSupport, Project, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter, LocalRelation, LogicalPlan, NamedParametersSupport, Project, SubqueryAlias, View} import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils} import org.apache.spark.sql.connector.catalog.CatalogManager @@ -48,7 +48,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType} import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -1561,6 +1561,103 @@ class SessionCatalog( } } + /** + * Constructs a scalar SQL function logical plan. The logical plan will be used to + * construct actual expression from the function inputs and body. + * + * The body of a scalar SQL function can either be an expression or a query returns + * one single column. + * + * Example scalar SQL function with an expression: + * + * CREATE FUNCTION area(width DOUBLE, height DOUBLE) RETURNS DOUBLE + * RETURN width * height; + * + * Query: + * + * SELECT area(a, b) FROM t; + * + * SQL function plan: + * + * Project [CAST(width * height AS DOUBLE) AS area] + * +- Project [CAST(a AS DOUBLE) AS width, CAST(b AS DOUBLE) AS height] + * +- LocalRelation [a, b] + * + * Example scalar SQL function with a subquery: + * + * CREATE FUNCTION foo(x INT) RETURNS INT + * RETURN SELECT SUM(b) FROM t WHERE x = a; + * + * SELECT foo(a) FROM t; + * + * SQL function plan: + * + * Project [scalar-subquery AS foo] + * : +- Aggregate [] [sum(b)] + * : +- Filter [outer(x) = a] + * : +- Relation [a, b] + * +- Project [CAST(a AS INT) AS x] + * +- LocalRelation [a, b] + */ + def makeSQLFunctionPlan( + name: String, + function: SQLFunction, + input: Seq[Expression]): LogicalPlan = { + def metaForFuncInputAlias = { + new MetadataBuilder() + .putString("__funcInputAlias", "true") + .build() + } + assert(!function.isTableFunc) + val funcName = function.name.funcName + + // Use captured SQL configs when parsing a SQL function. + val conf = new SQLConf() + function.getSQLConfigs.foreach { case (k, v) => conf.settings.put(k, v) } + SQLConf.withExistingConf(conf) { + val inputParam = function.inputParam + val returnType = function.getScalarFuncReturnType + val (expression, query) = function.getExpressionAndQuery(parser, isTableFunc = false) + assert(expression.isDefined || query.isDefined) + + // Check function arguments + val paramSize = inputParam.map(_.size).getOrElse(0) + if (input.size > paramSize) { + throw QueryCompilationErrors.wrongNumArgsError( + name, paramSize.toString, input.size) + } + + val inputs = inputParam.map { param => + // Attributes referencing the input parameters inside the function can use the + // function name as a qualifier. E.G.: + // `create function foo(a int) returns int return foo.a` + val qualifier = Seq(funcName) + val paddedInput = input ++ + param.takeRight(paramSize - input.size).map { p => + val defaultExpr = p.getDefault() + if (defaultExpr.isDefined) { + Cast(parseDefault(defaultExpr.get, parser), p.dataType) + } else { + throw QueryCompilationErrors.wrongNumArgsError( + name, paramSize.toString, input.size) + } + } + + paddedInput.zip(param.fields).map { + case (expr, param) => + Alias(Cast(expr, param.dataType), param.name)( + qualifier = qualifier, + // mark the alias as function input + explicitMetadata = Some(metaForFuncInputAlias)) + } + }.getOrElse(Nil) + + val body = if (query.isDefined) ScalarSubquery(query.get) else expression.get + Project(Alias(Cast(body, returnType), funcName)() :: Nil, + Project(inputs, LocalRelation(inputs.flatMap(_.references)))) + } + } + /** * Constructs a [[TableFunctionBuilder]] based on the provided class that represents a function. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala index b00cae22cf9c0..a76ca7b15c278 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala @@ -45,6 +45,14 @@ trait UserDefinedFunction { */ def properties: Map[String, String] + /** + * Get SQL configs from the function properties. + * Use this to restore the SQL configs that should be used for this function. + */ + def getSQLConfigs: Map[String, String] = { + UserDefinedFunction.propertiesToSQLConfigs(properties) + } + /** * Owner of the function */ @@ -142,4 +150,17 @@ object UserDefinedFunction { * Verify if the function is a [[UserDefinedFunction]]. */ def isUserDefinedFunction(className: String): Boolean = SQLFunction.isSQLFunction(className) + + /** + * Covert properties to SQL configs. + */ + def propertiesToSQLConfigs(properties: Map[String, String]): Map[String, String] = { + try { + for ((key, value) <- properties if key.startsWith(SQL_CONFIG_PREFIX)) + yield (key.substring(SQL_CONFIG_PREFIX.length), value) + } catch { + case e: Exception => throw SparkException.internalError( + "Corrupted user defined function SQL configs in catalog", cause = e) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSQLFunctionNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSQLFunctionNode.scala new file mode 100644 index 0000000000000..d9da38b4c2af4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSQLFunctionNode.scala @@ -0,0 +1,47 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.analysis.{SQLFunctionExpression, SQLFunctionNode, SQLScalarFunction, SQLTableFunction} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * This rule removes [[SQLScalarFunction]] and [[SQLFunctionNode]] wrapper. They are respected + * till the end of analysis stage because we want to see which part of an analyzed logical + * plan is generated from a SQL function and also perform ACL checks. + */ +object EliminateSQLFunctionNode extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + // Include subqueries when eliminating SQL function expressions otherwise we might miss + // expressions in subqueries which can be inlined by the rule `OptimizeOneRowRelationSubquery`. + plan.transformWithSubqueries { + case SQLFunctionNode(_, child) => child + case f: SQLTableFunction => + throw SparkException.internalError( + s"SQL table function plan should be rewritten during analysis: $f") + case p: LogicalPlan => p.transformExpressions { + case f: SQLScalarFunction => f.child + case f: SQLFunctionExpression => + throw SparkException.internalError( + s"SQL function expression should be rewritten during analysis: $f") + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 8ee2226947ec9..9d269f37e58b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -315,6 +315,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateSubqueryAliases, EliminatePipeOperators, EliminateView, + EliminateSQLFunctionNode, ReplaceExpressions, RewriteNonCorrelatedExists, PullOutGroupingExpressions, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index b56085ecae8d6..9856a26346f6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -93,6 +93,7 @@ object TreePattern extends Enumeration { val SESSION_WINDOW: Value = Value val SORT: Value = Value val SQL_FUNCTION_EXPRESSION: Value = Value + val SQL_SCALAR_FUNCTION: Value = Value val SQL_TABLE_FUNCTION: Value = Value val SUBQUERY_ALIAS: Value = Value val SUM: Value = Value diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out new file mode 100644 index 0000000000000..b3c10e929f297 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out @@ -0,0 +1,575 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1 +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo1a0`" + } +} + + +-- !query +SELECT foo1a0() +-- !query analysis +Project [spark_catalog.default.foo1a0() AS spark_catalog.default.foo1a0()#x] ++- Project + +- OneRowRelation + + +-- !query +SELECT foo1a0(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`spark_catalog`.`default`.`foo1a0`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "foo1a0(1)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1 +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo1a1`" + } +} + + +-- !query +SELECT foo1a1(1) +-- !query analysis +Project [spark_catalog.default.foo1a1(a#x) AS spark_catalog.default.foo1a1(1)#x] ++- Project [cast(1 as int) AS a#x] + +- OneRowRelation + + +-- !query +SELECT foo1a1(1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`foo1a1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "foo1a1(1, 2)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1 +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo1a2`" + } +} + + +-- !query +SELECT foo1a2(1, 2, 3, 4) +-- !query analysis +Project [spark_catalog.default.foo1a2(a#x, b#x, c#x, d#x) AS spark_catalog.default.foo1a2(1, 2, 3, 4)#x] ++- Project [cast(1 as int) AS a#x, cast(2 as int) AS b#x, cast(3 as int) AS c#x, cast(4 as int) AS d#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1a`" + } +} + + +-- !query +SELECT foo2_1a(5) +-- !query analysis +Project [spark_catalog.default.foo2_1a(a#x) AS spark_catalog.default.foo2_1a(5)#x] ++- Project [cast(5 as int) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1b`" + } +} + + +-- !query +SELECT foo2_1b(5, 6) +-- !query analysis +Project [spark_catalog.default.foo2_1b(a#x, b#x) AS spark_catalog.default.foo2_1b(5, 6)#x] ++- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1c`" + } +} + + +-- !query +SELECT foo2_1c(5, 6) +-- !query analysis +Project [spark_catalog.default.foo2_1c(a#x, b#x) AS spark_catalog.default.foo2_1c(5, 6)#x] ++- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1d`" + } +} + + +-- !query +SELECT foo2_1d(-5, 6) +-- !query analysis +Project [spark_catalog.default.foo2_1d(a#x, b#x) AS spark_catalog.default.foo2_1d(-5, 6)#x] ++- Project [cast(-5 as int) AS a#x, cast(6 as int) AS b#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2a`" + } +} + + +-- !query +SELECT foo2_2a(5) +-- !query analysis +Project [spark_catalog.default.foo2_2a(a#x) AS spark_catalog.default.foo2_2a(5)#x] ++- Project [cast(5 as int) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2b`" + } +} + + +-- !query +SELECT foo2_2b(5) +-- !query analysis +Project [spark_catalog.default.foo2_2b(a#x) AS spark_catalog.default.foo2_2b(5)#x] +: +- Project [outer(a#x)] +: +- OneRowRelation ++- Project [cast(5 as int) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 21, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2e`" + } +} + + +-- !query +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2f`" + } +} + + +-- !query +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2g`" + } +} + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2 + + +-- !query +DROP TABLE IF EXISTS ts +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.ts + + +-- !query +DROP TABLE IF EXISTS tm +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tm + + +-- !query +DROP TABLE IF EXISTS ta +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.ta + + +-- !query +DROP TABLE IF EXISTS V1 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V1 + + +-- !query +DROP TABLE IF EXISTS V2 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V2 + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t1`, true, true, false + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t2`, true, true, false + + +-- !query +DROP VIEW IF EXISTS ts +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`ts`, true, true, false + + +-- !query +DROP VIEW IF EXISTS tm +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tm`, true, true, false + + +-- !query +DROP VIEW IF EXISTS ta +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`ta`, true, true, false + + +-- !query +DROP VIEW IF EXISTS V1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V1`, true, true, false + + +-- !query +DROP VIEW IF EXISTS V2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V2`, true, true, false + + +-- !query +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_3`" + } +} + + +-- !query +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`V1`, [(c1,None), (c2,None)], VALUES (1, 2), (3, 4), (5, 6), false, false, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`V2`, [(c1,None), (c2,None)], VALUES (-1, -2), (-3, -4), (-5, -6), false, false, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3 +-- !query analysis +Sort [spark_catalog.default.foo2_3(c1, c2)#x ASC NULLS FIRST, spark_catalog.default.foo2_3(c2, 1)#x ASC NULLS FIRST, (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x ASC NULLS FIRST], true ++- Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c1, c2)#x, spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c2, 1)#x, (spark_catalog.default.foo2_3(a#x, b#x) - spark_catalog.default.foo2_3(a#x, b#x)) AS (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x] + +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(1 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast((c1#x - 1) as int) AS b#x] + +- SubqueryAlias spark_catalog.default.v1 + +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8 +-- !query analysis +Project [c1#x, c2#x] ++- Project [c1#x, c2#x] + +- Filter ((spark_catalog.default.foo2_3(a#x, b#x) = c1#x) AND (spark_catalog.default.foo2_3(a#x, b#x) < 8)) + +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(0 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x] + +- SubqueryAlias spark_catalog.default.v1 + +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1 +-- !query analysis +Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(sum(c1), sum(c2))#x, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL] ++- Project [sum(c1)#xL, sum(c2)#xL, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL, cast(sum(c1)#xL as int) AS a#x, cast(sum(c2)#xL as int) AS b#x] + +- Aggregate [sum(c1#x) AS sum(c1)#xL, sum(c2#x) AS sum(c2)#xL, (sum(c1#x) + sum(c2#x)) AS (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(a#x, b#x) + spark_catalog.default.foo2_3(a#x, b#x)) - spark_catalog.default.foo2_3(a#x, b#x))) AS sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL] + +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x] + +- SubqueryAlias spark_catalog.default.v1 + +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_4a`" + } +} + + +-- !query +SELECT foo2_4a(ARRAY('a', 'b')) +-- !query analysis +Project [spark_catalog.default.foo2_4a(a#x) AS spark_catalog.default.foo2_4a(array(a, b))#x] +: +- Project [array_sort(outer(a#x), lambdafunction((rank#x[lambda i#x] - rank#x[lambda j#x]), lambda i#x, lambda j#x, false), false)[0] AS array_sort(outer(foo2_4a.a), lambdafunction((rank[namedlambdavariable()] - rank[namedlambdavariable()]), namedlambdavariable(), namedlambdavariable()))[0]#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [map(a, 1, b, 2) AS rank#x] +: +- OneRowRelation ++- Project [cast(array(a, b) as array) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_4b`" + } +} + + +-- !query +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a') +-- !query analysis +Project [spark_catalog.default.foo2_4b(m#x, k#x) AS spark_catalog.default.foo2_4b(map(a, hello, b, world), a)#x] +: +- Project [concat(concat(v#x, ), v#x) AS concat(concat(v, ), v)#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [upper(outer(m#x)[outer(k#x)]) AS v#x] +: +- OneRowRelation ++- Project [cast(map(a, hello, b, world) as map) AS m#x, cast(a as string) AS k#x] + +- OneRowRelation + + +-- !query +DROP VIEW V2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V2`, false, true, false + + +-- !query +DROP VIEW V1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V1`, false, true, false diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-udf.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-udf.sql new file mode 100644 index 0000000000000..34cb41d726766 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-udf.sql @@ -0,0 +1,122 @@ +-- test cases for SQL User Defined Functions + +-- 1. CREATE FUNCTION +-- 1.1 Parameter +-- 1.1.a A scalar function with various numbers of parameter +-- Expect success +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a0(); +-- Expect failure +SELECT foo1a0(1); + +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a1(1); +-- Expect failure +SELECT foo1a1(1, 2); + +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a2(1, 2, 3, 4); + +------------------------------- +-- 2. Scalar SQL UDF +-- 2.1 deterministic simple expressions +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a; +SELECT foo2_1a(5); + +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b; +SELECT foo2_1b(5, 6); + +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b); +SELECT foo2_1c(5, 6); + +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))); +SELECT foo2_1d(-5, 6); + +-- 2.2 deterministic complex expression with subqueries +-- 2.2.1 Nested Scalar subqueries +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a; +SELECT foo2_2a(5); + +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a); +SELECT foo2_2b(5); + +-- Expect error: deep correlation is not yet supported +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)); +-- SELECT foo2_2c(5); + +-- Expect error: deep correlation is not yet supported +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))); +-- SELECT foo2_2d(5); + +-- 2.2.2 Set operations +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1); +-- SELECT foo2_2e(5); + +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a); +-- SELECT foo2_2f(5); + +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a); +-- SELECT foo2_2g(5); + +-- Prepare by dropping views or tables if they already exist. +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS ts; +DROP TABLE IF EXISTS tm; +DROP TABLE IF EXISTS ta; +DROP TABLE IF EXISTS V1; +DROP TABLE IF EXISTS V2; +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS ts; +DROP VIEW IF EXISTS tm; +DROP VIEW IF EXISTS ta; +DROP VIEW IF EXISTS V1; +DROP VIEW IF EXISTS V2; + +-- 2.3 Calling Scalar UDF from various places +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b; +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6); +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6); + +-- 2.3.1 Multiple times in the select list +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3; + +-- 2.3.2 In the WHERE clause +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8; + +-- 2.3.3 Different places around an aggregate +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1; + +-- 2.4 Scalar UDF with complex one row relation subquery +-- 2.4.1 higher order functions +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank); + +SELECT foo2_4a(ARRAY('a', 'b')); + +-- 2.4.2 built-in functions +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v); + +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a'); + +-- Clean up +DROP VIEW V2; +DROP VIEW V1; diff --git a/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out new file mode 100644 index 0000000000000..9f7af7c644871 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out @@ -0,0 +1,484 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a0() +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo1a0(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`spark_catalog`.`default`.`foo1a0`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "foo1a0(1)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a1(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo1a1(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`foo1a1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "foo1a1(1, 2)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a2(1, 2, 3, 4) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1a(5) +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1b(5, 6) +-- !query schema +struct +-- !query output +11 + + +-- !query +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1c(5, 6) +-- !query schema +struct +-- !query output +10 + + +-- !query +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1d(-5, 6) +-- !query schema +struct +-- !query output +4 + + +-- !query +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_2a(5) +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_2b(5) +-- !query schema +struct +-- !query output +6 + + +-- !query +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 21, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS ts +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tm +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS ta +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS V1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS ts +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tm +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS ta +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS V1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3 +-- !query schema +struct +-- !query output +3 3 1 +7 5 1 +11 7 1 + + +-- !query +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8 +-- !query schema +struct +-- !query output +1 2 +3 4 + + +-- !query +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1 +-- !query schema +struct +-- !query output +21 21 21 + + +-- !query +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_4a(ARRAY('a', 'b')) +-- !query schema +struct +-- !query output +a + + +-- !query +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a') +-- !query schema +struct +-- !query output +HELLO HELLO + + +-- !query +DROP VIEW V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW V1 +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala new file mode 100644 index 0000000000000..4da3b9ab1d06b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala @@ -0,0 +1,61 @@ +/* + * 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 + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession + +/** + * Test suite for SQL user-defined functions (UDFs). + */ +class SQLFunctionSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + protected override def beforeAll(): Unit = { + super.beforeAll() + Seq((0, 1), (1, 2)).toDF("a", "b").createOrReplaceTempView("t") + } + + test("SQL scalar function") { + withUserDefinedFunction("area" -> false) { + sql( + """ + |CREATE FUNCTION area(width DOUBLE, height DOUBLE) + |RETURNS DOUBLE + |RETURN width * height + |""".stripMargin) + checkAnswer(sql("SELECT area(1, 2)"), Row(2)) + checkAnswer(sql("SELECT area(a, b) FROM t"), Seq(Row(0), Row(2))) + } + } + + test("SQL scalar function with subquery in the function body") { + withUserDefinedFunction("foo" -> false) { + withTable("tbl") { + sql("CREATE TABLE tbl AS SELECT * FROM VALUES (1, 2), (1, 3), (2, 3) t(a, b)") + sql( + """ + |CREATE FUNCTION foo(x INT) RETURNS INT + |RETURN SELECT SUM(b) FROM tbl WHERE x = a; + |""".stripMargin) + checkAnswer(sql("SELECT foo(1)"), Row(5)) + checkAnswer(sql("SELECT foo(a) FROM t"), Seq(Row(null), Row(5))) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index c00f00ceaa355..a7af22a0554e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -79,7 +79,8 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { assert(info.getSource === "built-in") val validSources = Seq( - "built-in", "hive", "python_udf", "scala_udf", "java_udf", "python_udtf", "internal") + "built-in", "hive", "python_udf", "scala_udf", "java_udf", "python_udtf", "internal", + "sql_udf") validSources.foreach { source => val info = new ExpressionInfo( "testClass", null, "testName", null, "", "", "", "", "", "", source) From e4cbb91eb2723ebaa91ac308670bbbb39cca6b0b Mon Sep 17 00:00:00 2001 From: Siying Dong Date: Tue, 14 Jan 2025 17:41:34 +0800 Subject: [PATCH 419/438] [SPARK-50804][SQL] to_protobuf() should not throw MatchError ### What changes were proposed in this pull request? Right now, if `to_protobuf()` is called with an argument that is not StructType, ` scala.MatchError` is thrown. We should categorize this error. ### Why are the changes needed? scala.MatchError is confusing to users and doesn't include enough information for debugging. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Add a new unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49473 from siying/categorize_match. Authored-by: Siying Dong Signed-off-by: Wenchen Fan --- .../sql/protobuf/ProtobufSerializer.scala | 3 +++ .../sql/protobuf/ProtobufFunctionsSuite.scala | 27 +++++++++++++++++++ .../toFromProtobufSqlFunctions.scala | 20 ++++++++++---- 3 files changed, 45 insertions(+), 5 deletions(-) diff --git a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala index 1c64e70755d5c..65e8cce0d056e 100644 --- a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala +++ b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala @@ -47,6 +47,9 @@ private[sql] class ProtobufSerializer( } private val converter: Any => Any = { + assert( + rootCatalystType.isInstanceOf[StructType], + "ProtobufSerializer's root catalyst type must be a struct type") val baseConverter = try { rootCatalystType match { diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala index 44a8339ac1f02..c1f5186b8333f 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala @@ -1721,6 +1721,33 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot } } + test("non-struct SQL type") { + val dfWithInt = spark + .range(1) + .select( + lit(9999).as("int_col") + ) + + val parseError = intercept[AnalysisException] { + dfWithInt.select( + to_protobuf_wrapper($"int_col", "SimpleMessageEnum", Some(testFileDesc))).collect() + } + val descMsg = testFileDesc.map("%02X".format(_)).mkString("") + checkError( + exception = parseError, + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + parameters = Map( + "sqlExpr" -> + s"""\"to_protobuf(int_col, SimpleMessageEnum, X'$descMsg', NULL)\"""", + "msg" -> ("The first argument of the TO_PROTOBUF SQL function must be a struct type"), + "hint" -> "" + ), + queryContext = Array(ExpectedContext( + fragment = "fn", + callSitePattern = ".*")) + ) + } + test("test unsigned integer types") { // Test that we correctly handle unsigned integer parsing. // We're using Integer/Long's `MIN_VALUE` as it has a 1 in the sign bit. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala index 96bcf49dbd097..380df38ab049b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.util.ArrayBasedMapData import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{BinaryType, MapType, NullType, StringType} +import org.apache.spark.sql.types.{BinaryType, MapType, NullType, StringType, StructType} import org.apache.spark.sql.util.ProtobufUtils import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -238,6 +238,15 @@ case class ToProtobuf( } override def checkInputDataTypes(): TypeCheckResult = { + val colTypeCheck = first.dataType match { + case _: StructType => None + case _ => + Some( + TypeCheckResult.TypeCheckFailure( + "The first argument of the TO_PROTOBUF SQL function must be a struct type") + ) + } + val messageNameCheck = messageName.dataType match { case _: StringType if messageName.foldable => None case _ => @@ -262,10 +271,11 @@ case class ToProtobuf( "strings to strings containing the options to use for converting the value to " + "Protobuf format")) } - - messageNameCheck.getOrElse( - descFilePathCheck.getOrElse( - optionsCheck.getOrElse(TypeCheckResult.TypeCheckSuccess) + colTypeCheck.getOrElse( + messageNameCheck.getOrElse( + descFilePathCheck.getOrElse( + optionsCheck.getOrElse(TypeCheckResult.TypeCheckSuccess) + ) ) ) } From f355293f8bc82fc5bd56f34e1eb2518c8b47e483 Mon Sep 17 00:00:00 2001 From: mihailoale-db Date: Tue, 14 Jan 2025 17:43:17 +0800 Subject: [PATCH 420/438] [SPARK-50801][SQL] Improve `PlanLogger.logPlanResolution` so it shows just unresolved and resolved plans ### What changes were proposed in this pull request? Improve `PlanLogger.logPlanResolution` so it shows just unresolved and resolved plans. ### Why are the changes needed? Currently `PlanLogger` dumps `unresolvedPlan.withNewChildren(resolvedPlan.children` and `resolvedPlan` in `logPlanResolution` method. This leads to internal errors because in `withNewChildren` we have `children.size == newChildren.size` check which can fail sometimes (e.g when `UnresolvedRelation` is substituted with `SubqueryAlias(..., child, ...)`). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests with the `ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER` value set to `true`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49468 from mihailoale-db/mihailo-aleksic_data/master. Authored-by: mihailoale-db Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/analysis/resolver/PlanLogger.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala index 8870befede4f4..f778915008dbb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala @@ -46,7 +46,7 @@ class PlanLogger extends Logging { |${MDC( QUERY_PLAN, sideBySide( - unresolvedPlan.withNewChildren(resolvedPlan.children).treeString, + unresolvedPlan.treeString, resolvedPlan.treeString ).mkString("\n") )} From f96417fe0d37c665d331311fffae5ad27d8db4c2 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 14 Jan 2025 19:24:51 +0900 Subject: [PATCH 421/438] [SPARK-47081][CONNECT][FOLLOW-UP] Respect spark.connect.progress.reportInterval over timeout ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/49474 that properly assigns the value to `timeout`. ### Why are the changes needed? It was a mistake that did not assign the value back. ### Does this PR introduce _any_ user-facing change? Yes, same as https://github.com/apache/spark/pull/49474 ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49484 from HyukjinKwon/SPARK-47081-followup4. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../sql/connect/execution/ExecuteGrpcResponseSender.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala index 72c2b0e3f1095..21b5e057fb77e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala @@ -241,13 +241,13 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( // The state of interrupted, response and lastIndex are changed under executionObserver // monitor, and will notify upon state change. if (response.isEmpty) { - val timeout = Math.max(1, deadlineTimeMillis - System.currentTimeMillis()) + var timeout = Math.max(1, deadlineTimeMillis - System.currentTimeMillis()) // Wake up more frequently to send the progress updates. val progressTimeout = executeHolder.sessionHolder.session.sessionState.conf .getConf(CONNECT_PROGRESS_REPORT_INTERVAL) // If the progress feature is disabled, wait for the deadline. if (progressTimeout > 0L) { - Math.min(progressTimeout, timeout) + timeout = Math.min(progressTimeout, timeout) } logTrace(s"Wait for response to become available with timeout=$timeout ms.") executionObserver.responseLock.wait(timeout) From fafe43c87d8bc68c0b447de3513ee05ab581943a Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Tue, 14 Jan 2025 19:29:24 +0800 Subject: [PATCH 422/438] [SPARK-49907][ML][CONNECT] Support spark.ml on Connect ### What changes were proposed in this pull request? This PR which derived from https://github.com/apache/spark/pull/40479 authored by WeichenXu123 enables running spark.ml on Connect. Currently, this PR supports the following functionalities: - Fit operation in LogisticRegression - Transform/predict operation in LogisticRegressionModel - Retrieving attributes in LogisticRegressionModel - Retrieving summary and its attributes in LogisticRegressionModel - Read/write operations for LogisticRegressionModel - Read/write operations for LogisticRegression - Evaluate a dataset in LogisticRegressionModel and return a summary and retrieve attributes from it. ### Why are the changes needed? It's a new feature that makes spark.ml run on connect environment. ### Does this PR introduce _any_ user-facing change? Yes, new feature. ### How was this patch tested? Make sure the CI (especially the newly added tests) pass. And we can manually run below code without any exception. ``` python (pyspark) userbobby:~ $ pyspark --remote sc://localhost Python 3.11.10 (main, Oct 3 2024, 07:29:13) [GCC 11.2.0] on linux Type "help", "copyright", "credits" or "license" for more information. Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /__ / .__/\_,_/_/ /_/\_\ version 4.0.0.dev0 /_/ Using Python version 3.11.10 (main, Oct 3 2024 07:29:13) Client connected to the Spark Connect server at localhost SparkSession available as 'spark'. >>> from pyspark.ml.classification import (LogisticRegression, LogisticRegressionModel) >>> from pyspark.ml.linalg import Vectors >>> >>> df = spark.createDataFrame([ ... (Vectors.dense([1.0, 2.0]), 1), ... (Vectors.dense([2.0, -1.0]), 1), ... (Vectors.dense([-3.0, -2.0]), 0), ... (Vectors.dense([-1.0, -2.0]), 0), ... ], schema=['features', 'label']) >>> lr = LogisticRegression() >>> lr.setMaxIter(30) LogisticRegression_27d6a4e4f39d >>> lr.setThreshold(0.8) LogisticRegression_27d6a4e4f39d >>> lr.write().overwrite().save("/tmp/connect-ml-demo/estimator") >>> loaded_lr = LogisticRegression.load("/tmp/connect-ml-demo/estimator") >>> assert (loaded_lr.getThreshold() == 0.8) >>> assert loaded_lr.getMaxIter() == 30 >>> >>> model: LogisticRegressionModel = lr.fit(df) >>> assert (model.getThreshold() == 0.8) >>> assert model.getMaxIter() == 30 >>> model.predictRaw(Vectors.dense([1.0, 2.0])) DenseVector([-21.1048, 21.1048]) >>> model.summary.roc.show() +---+---+ |FPR|TPR| +---+---+ |0.0|0.0| |0.0|0.5| |0.0|1.0| |0.5|1.0| |1.0|1.0| |1.0|1.0| +---+---+ >>> model.summary.weightedRecall 1.0 >>> model.summary.recallByLabel [1.0, 1.0] >>> model.coefficients DenseVector([10.3964, 4.513]) >>> model.intercept 1.682348909633995 >>> model.transform(df).show() +-----------+-----+--------------------+--------------------+----------+ | features|label| rawPrediction| probability|prediction| +-----------+-----+--------------------+--------------------+----------+ | [1.0,2.0]| 1|[-21.104818251026...|[6.82800596289009...| 1.0| | [2.0,-1.0]| 1|[-17.962094978515...|[1.58183529116627...| 1.0| |[-3.0,-2.0]| 0|[38.5329050234205...| [1.0,0.0]| 0.0| |[-1.0,-2.0]| 0|[17.7401204317581...|[0.99999998025016...| 0.0| +-----------+-----+--------------------+--------------------+----------+ >>> model.write().overwrite().save("/tmp/connect-ml-demo/model") >>> loaded_model = LogisticRegressionModel.load("/tmp/connect-ml-demo/model") >>> assert loaded_model.getMaxIter() == 30 >>> loaded_model.transform(df).show() +-----------+-----+--------------------+--------------------+----------+ | features|label| rawPrediction| probability|prediction| +-----------+-----+--------------------+--------------------+----------+ | [1.0,2.0]| 1|[-21.104818251026...|[6.82800596289009...| 1.0| | [2.0,-1.0]| 1|[-17.962094978515...|[1.58183529116627...| 1.0| |[-3.0,-2.0]| 0|[38.5329050234205...| [1.0,0.0]| 0.0| |[-1.0,-2.0]| 0|[17.7401204317581...|[0.99999998025016...| 0.0| +-----------+-----+--------------------+--------------------+----------+ >>> >>> summary = loaded_model.evaluate(df) >>> summary.weightCol 'weightCol' >>> summary.recallByLabel [1.0, 1.0] >>> summary.accuracy 1.0 >>> summary.predictions.show() +-----------+-----+--------------------+--------------------+----------+ | features|label| rawPrediction| probability|prediction| +-----------+-----+--------------------+--------------------+----------+ | [1.0,2.0]| 1|[-21.104818251026...|[6.82800596289009...| 1.0| | [2.0,-1.0]| 1|[-17.962094978515...|[1.58183529116627...| 1.0| |[-3.0,-2.0]| 0|[38.5329050234205...| [1.0,0.0]| 0.0| |[-1.0,-2.0]| 0|[17.7401204317581...|[0.99999998025016...| 0.0| +-----------+-----+--------------------+--------------------+----------+ ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48791 from wbo4958/connect-ml. Authored-by: Bobby Wang Signed-off-by: Ruifeng Zheng --- .../resources/error/error-conditions.json | 18 + dev/sparktestsupport/modules.py | 2 + .../services/org.apache.spark.ml.Estimator | 20 + .../services/org.apache.spark.ml.Transformer | 20 + .../ClassificationSummary.scala | 3 +- .../org/apache/spark/ml/param/params.scala | 9 +- .../org/apache/spark/ml/util/Summary.scala | 28 ++ .../apache/spark/ml/param/JavaTestParams.java | 3 +- python/pyspark/ml/classification.py | 7 + python/pyspark/ml/remote/__init__.py | 16 + python/pyspark/ml/remote/proto.py | 76 ++++ python/pyspark/ml/remote/readwrite.py | 134 ++++++ python/pyspark/ml/remote/serialize.py | 132 ++++++ python/pyspark/ml/remote/util.py | 293 ++++++++++++ .../test_connect_spark_ml_classification.py | 49 ++ python/pyspark/ml/tests/test_algorithms.py | 82 +--- .../pyspark/ml/tests/test_classification.py | 304 +++++++++++++ python/pyspark/ml/tests/test_persistence.py | 23 - .../pyspark/ml/tests/test_training_summary.py | 91 ---- python/pyspark/ml/util.py | 4 + python/pyspark/ml/wrapper.py | 18 +- python/pyspark/sql/connect/client/core.py | 37 ++ python/pyspark/sql/connect/proto/__init__.py | 2 + python/pyspark/sql/connect/proto/base_pb2.py | 359 +++++++-------- python/pyspark/sql/connect/proto/base_pb2.pyi | 11 + .../pyspark/sql/connect/proto/commands_pb2.py | 203 ++++----- .../sql/connect/proto/commands_pb2.pyi | 10 + .../sql/connect/proto/ml_common_pb2.py | 80 ++++ .../sql/connect/proto/ml_common_pb2.pyi | 427 ++++++++++++++++++ python/pyspark/sql/connect/proto/ml_pb2.py | 72 +++ python/pyspark/sql/connect/proto/ml_pb2.pyi | 393 ++++++++++++++++ .../sql/connect/proto/relations_pb2.py | 325 ++++++------- .../sql/connect/proto/relations_pb2.pyi | 203 +++++++++ .../main/protobuf/spark/connect/base.proto | 4 + .../protobuf/spark/connect/commands.proto | 3 +- .../src/main/protobuf/spark/connect/ml.proto | 107 +++++ .../protobuf/spark/connect/ml_common.proto | 108 +++++ .../protobuf/spark/connect/relations.proto | 53 +++ .../apache/spark/sql/connect/ml/MLCache.scala | 70 +++ .../spark/sql/connect/ml/MLException.scala | 32 ++ .../spark/sql/connect/ml/MLHandler.scala | 268 +++++++++++ .../apache/spark/sql/connect/ml/MLUtils.scala | 353 +++++++++++++++ .../spark/sql/connect/ml/Serializer.scala | 151 +++++++ .../connect/planner/SparkConnectPlanner.scala | 21 + .../sql/connect/service/SessionHolder.scala | 6 + .../apache/spark/sql/connect/ml/MLSuite.scala | 390 ++++++++++++++++ .../planner/SparkConnectPlannerSuite.scala | 4 +- 47 files changed, 4384 insertions(+), 640 deletions(-) create mode 100644 mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Estimator create mode 100644 mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Transformer create mode 100644 mllib/src/main/scala/org/apache/spark/ml/util/Summary.scala create mode 100644 python/pyspark/ml/remote/__init__.py create mode 100644 python/pyspark/ml/remote/proto.py create mode 100644 python/pyspark/ml/remote/readwrite.py create mode 100644 python/pyspark/ml/remote/serialize.py create mode 100644 python/pyspark/ml/remote/util.py create mode 100644 python/pyspark/ml/tests/connect/test_connect_spark_ml_classification.py create mode 100644 python/pyspark/ml/tests/test_classification.py create mode 100644 python/pyspark/sql/connect/proto/ml_common_pb2.py create mode 100644 python/pyspark/sql/connect/proto/ml_common_pb2.pyi create mode 100644 python/pyspark/sql/connect/proto/ml_pb2.py create mode 100644 python/pyspark/sql/connect/proto/ml_pb2.pyi create mode 100644 sql/connect/common/src/main/protobuf/spark/connect/ml.proto create mode 100644 sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala create mode 100644 sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 5037b52475422..3c11f5b360bb2 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -752,6 +752,24 @@ }, "sqlState" : "56K00" }, + "CONNECT_ML" : { + "message" : [ + "Generic Spark Connect ML error." + ], + "subClass" : { + "ATTRIBUTE_NOT_ALLOWED" : { + "message" : [ + " is not allowed to be accessed." + ] + }, + "UNSUPPORTED_EXCEPTION" : { + "message" : [ + "" + ] + } + }, + "sqlState" : "XX000" + }, "CONVERSION_INVALID_INPUT" : { "message" : [ "The value () cannot be converted to because it is malformed. Correct the value as per the syntax, or change its format. Use to tolerate malformed input and return NULL instead." diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 71817a64b3581..d43feceb76648 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -686,6 +686,7 @@ def __hash__(self): "pyspark.ml.tests.connect.test_legacy_mode_classification", "pyspark.ml.tests.connect.test_legacy_mode_pipeline", "pyspark.ml.tests.connect.test_legacy_mode_tuning", + "pyspark.ml.tests.test_classification", ], excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy and it isn't available there @@ -1106,6 +1107,7 @@ def __hash__(self): "pyspark.ml.tests.connect.test_connect_classification", "pyspark.ml.tests.connect.test_connect_pipeline", "pyspark.ml.tests.connect.test_connect_tuning", + "pyspark.ml.tests.connect.test_connect_spark_ml_classification", ], excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy, pandas, and pyarrow and diff --git a/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Estimator b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Estimator new file mode 100644 index 0000000000000..e6902f62c4d60 --- /dev/null +++ b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Estimator @@ -0,0 +1,20 @@ +# +# 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. +# + +# Spark Connect ML uses ServiceLoader to find out the supported Spark Ml estimators. +# So register the supported estimator here if you're trying to add a new one. +org.apache.spark.ml.classification.LogisticRegression diff --git a/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Transformer b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Transformer new file mode 100644 index 0000000000000..004ec8aeff8cf --- /dev/null +++ b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Transformer @@ -0,0 +1,20 @@ +# +# 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. +# + +# Spark Connect ML uses ServiceLoader to find out the supported Spark Ml non-model transformer. +# So register the supported transformer here if you're trying to add a new one. +org.apache.spark.ml.feature.VectorAssembler diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala index 9f3428db484c2..88cfb703fca41 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.util.Summary import org.apache.spark.mllib.evaluation.{BinaryClassificationMetrics, MulticlassMetrics} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions.{col, lit} @@ -28,7 +29,7 @@ import org.apache.spark.sql.types.DoubleType /** * Abstraction for multiclass classification results for a given model. */ -private[classification] trait ClassificationSummary extends Serializable { +private[classification] trait ClassificationSummary extends Summary with Serializable { /** * Dataframe output by the model's `transform` method. diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 055c1c4d4228e..43016a32e570b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml.param import java.lang.reflect.Modifier import java.util.{List => JList} -import java.util.NoSuchElementException import scala.annotation.varargs import scala.collection.mutable import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag import org.json4s._ import org.json4s.jackson.JsonMethods._ @@ -45,9 +45,14 @@ import org.apache.spark.util.ArrayImplicits._ * See [[ParamValidators]] for factory methods for common validation functions. * @tparam T param value type */ -class Param[T](val parent: String, val name: String, val doc: String, val isValid: T => Boolean) +class Param[T: ClassTag]( + val parent: String, val name: String, val doc: String, val isValid: T => Boolean) extends Serializable { + // Spark Connect ML needs T type information which has been erased when compiling, + // Use classTag to preserve the T type. + val paramValueClassTag = implicitly[ClassTag[T]] + def this(parent: Identifiable, name: String, doc: String, isValid: T => Boolean) = this(parent.uid, name, doc, isValid) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Summary.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Summary.scala new file mode 100644 index 0000000000000..6205fea92ef83 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Summary.scala @@ -0,0 +1,28 @@ +/* + * 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.ml.util + +import org.apache.spark.annotation.Since + +/** + * Trait for the Summary + * All the summaries should extend from this Summary in order to + * support connect. + */ +@Since("4.0.0") +private[spark] trait Summary diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java index 1ad5f7a442daa..b3993c453e91f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java @@ -21,6 +21,7 @@ import java.util.List; import org.apache.spark.ml.util.Identifiable$; +import scala.reflect.ClassTag; /** * A subclass of Params for testing. @@ -110,7 +111,7 @@ private void init() { ParamValidators.inRange(0.0, 1.0)); List validStrings = Arrays.asList("a", "b"); myStringParam_ = new Param<>(this, "myStringParam", "this is a string param", - ParamValidators.inArray(validStrings)); + ParamValidators.inArray(validStrings), ClassTag.apply(String.class)); myDoubleArrayParam_ = new DoubleArrayParam(this, "myDoubleArrayParam", "this is a double param"); diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index b7ae794d3d613..e003ba43ec7c8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -62,6 +62,7 @@ HasSolver, HasParallelism, ) +from pyspark.ml.remote.util import try_remote_attribute_relation from pyspark.ml.tree import ( _DecisionTreeModel, _DecisionTreeParams, @@ -336,6 +337,7 @@ class _ClassificationSummary(JavaWrapper): @property @since("3.1.0") + @try_remote_attribute_relation def predictions(self) -> DataFrame: """ Dataframe outputted by the model's `transform` method. @@ -521,6 +523,7 @@ def scoreCol(self) -> str: return self._call_java("scoreCol") @property + @try_remote_attribute_relation def roc(self) -> DataFrame: """ Returns the receiver operating characteristic (ROC) curve, @@ -546,6 +549,7 @@ def areaUnderROC(self) -> float: @property @since("3.1.0") + @try_remote_attribute_relation def pr(self) -> DataFrame: """ Returns the precision-recall curve, which is a Dataframe @@ -556,6 +560,7 @@ def pr(self) -> DataFrame: @property @since("3.1.0") + @try_remote_attribute_relation def fMeasureByThreshold(self) -> DataFrame: """ Returns a dataframe with two fields (threshold, F-Measure) curve @@ -565,6 +570,7 @@ def fMeasureByThreshold(self) -> DataFrame: @property @since("3.1.0") + @try_remote_attribute_relation def precisionByThreshold(self) -> DataFrame: """ Returns a dataframe with two fields (threshold, precision) curve. @@ -575,6 +581,7 @@ def precisionByThreshold(self) -> DataFrame: @property @since("3.1.0") + @try_remote_attribute_relation def recallByThreshold(self) -> DataFrame: """ Returns a dataframe with two fields (threshold, recall) curve. diff --git a/python/pyspark/ml/remote/__init__.py b/python/pyspark/ml/remote/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/python/pyspark/ml/remote/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/python/pyspark/ml/remote/proto.py b/python/pyspark/ml/remote/proto.py new file mode 100644 index 0000000000000..3a81e74b6aec3 --- /dev/null +++ b/python/pyspark/ml/remote/proto.py @@ -0,0 +1,76 @@ +# +# 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. +# +from typing import Optional, TYPE_CHECKING, List + +import pyspark.sql.connect.proto as pb2 +from pyspark.sql.connect.plan import LogicalPlan + +if TYPE_CHECKING: + from pyspark.sql.connect.client import SparkConnectClient + + +class TransformerRelation(LogicalPlan): + """A logical plan for transforming of a transformer which could be a cached model + or a non-model transformer like VectorAssembler.""" + + def __init__( + self, + child: Optional["LogicalPlan"], + name: str, + ml_params: pb2.MlParams, + uid: str = "", + is_model: bool = True, + ) -> None: + super().__init__(child) + self._name = name + self._ml_params = ml_params + self._uid = uid + self._is_model = is_model + + def plan(self, session: "SparkConnectClient") -> pb2.Relation: + assert self._child is not None + plan = self._create_proto_relation() + plan.ml_relation.transform.input.CopyFrom(self._child.plan(session)) + + if self._is_model: + plan.ml_relation.transform.obj_ref.CopyFrom(pb2.ObjectRef(id=self._name)) + else: + plan.ml_relation.transform.transformer.CopyFrom( + pb2.MlOperator(name=self._name, uid=self._uid, type=pb2.MlOperator.TRANSFORMER) + ) + + if self._ml_params is not None: + plan.ml_relation.transform.params.CopyFrom(self._ml_params) + + return plan + + +class AttributeRelation(LogicalPlan): + """A logical plan used in ML to represent an attribute of an instance, which + could be a model or a summary. This attribute returns a DataFrame. + """ + + def __init__(self, ref_id: str, methods: List[pb2.Fetch.Method]) -> None: + super().__init__(None) + self._ref_id = ref_id + self._methods = methods + + def plan(self, session: "SparkConnectClient") -> pb2.Relation: + plan = self._create_proto_relation() + plan.ml_relation.fetch.obj_ref.CopyFrom(pb2.ObjectRef(id=self._ref_id)) + plan.ml_relation.fetch.methods.extend(self._methods) + return plan diff --git a/python/pyspark/ml/remote/readwrite.py b/python/pyspark/ml/remote/readwrite.py new file mode 100644 index 0000000000000..9149ab3bfd454 --- /dev/null +++ b/python/pyspark/ml/remote/readwrite.py @@ -0,0 +1,134 @@ +# +# 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. +# + +from typing import cast, Type, TYPE_CHECKING + +import pyspark.sql.connect.proto as pb2 +from pyspark.ml.remote.serialize import serialize_ml_params, deserialize, deserialize_param +from pyspark.ml.util import MLWriter, MLReader, RL +from pyspark.ml.wrapper import JavaWrapper + +if TYPE_CHECKING: + from pyspark.ml.util import JavaMLReadable, JavaMLWritable + from pyspark.core.context import SparkContext + + +class RemoteMLWriter(MLWriter): + def __init__(self, instance: "JavaMLWritable") -> None: + super().__init__() + self._instance = instance + + @property + def sc(self) -> "SparkContext": + raise RuntimeError("Accessing SparkContext is not supported on Connect") + + def save(self, path: str) -> None: + from pyspark.ml.wrapper import JavaModel, JavaEstimator + from pyspark.sql.connect.session import SparkSession + + session = SparkSession.getActiveSession() + assert session is not None + + # Spark Connect ML is built on scala Spark.ML, that means we're only + # supporting JavaModel or JavaEstimator or JavaEvaluator + if isinstance(self._instance, JavaModel): + model = cast("JavaModel", self._instance) + params = serialize_ml_params(model, session.client) + assert isinstance(model._java_obj, str) + writer = pb2.MlCommand.Write( + obj_ref=pb2.ObjectRef(id=model._java_obj), + params=params, + path=path, + should_overwrite=self.shouldOverwrite, + options=self.optionMap, + ) + elif isinstance(self._instance, JavaEstimator): + estimator = cast("JavaEstimator", self._instance) + params = serialize_ml_params(estimator, session.client) + assert isinstance(estimator._java_obj, str) + writer = pb2.MlCommand.Write( + operator=pb2.MlOperator( + name=estimator._java_obj, uid=estimator.uid, type=pb2.MlOperator.ESTIMATOR + ), + params=params, + path=path, + should_overwrite=self.shouldOverwrite, + options=self.optionMap, + ) + else: + raise NotImplementedError(f"Unsupported writing for {self._instance}") + + command = pb2.Command() + command.ml_command.write.CopyFrom(writer) + session.client.execute_command(command) + + +class RemoteMLReader(MLReader[RL]): + def __init__(self, clazz: Type["JavaMLReadable[RL]"]) -> None: + super().__init__() + self._clazz = clazz + + def load(self, path: str) -> RL: + from pyspark.sql.connect.session import SparkSession + from pyspark.ml.wrapper import JavaModel, JavaEstimator + + session = SparkSession.getActiveSession() + assert session is not None + # to get the java corresponding qualified class name + java_qualified_class_name = ( + self._clazz.__module__.replace("pyspark", "org.apache.spark") + + "." + + self._clazz.__name__ + ) + + if issubclass(self._clazz, JavaModel): + ml_type = pb2.MlOperator.MODEL + elif issubclass(self._clazz, JavaEstimator): + ml_type = pb2.MlOperator.ESTIMATOR + else: + raise ValueError(f"Unsupported reading for {java_qualified_class_name}") + + command = pb2.Command() + command.ml_command.read.CopyFrom( + pb2.MlCommand.Read( + operator=pb2.MlOperator(name=java_qualified_class_name, type=ml_type), path=path + ) + ) + (_, properties, _) = session.client.execute_command(command) + result = deserialize(properties) + + # Get the python type + def _get_class() -> Type[RL]: + parts = (self._clazz.__module__ + "." + self._clazz.__name__).split(".") + module = ".".join(parts[:-1]) + m = __import__(module, fromlist=[parts[-1]]) + return getattr(m, parts[-1]) + + py_type = _get_class() + # It must be JavaWrapper, since we're passing the string to the _java_obj + if issubclass(py_type, JavaWrapper): + if ml_type == pb2.MlOperator.MODEL: + session.client.add_ml_cache(result.obj_ref.id) + instance = py_type(result.obj_ref.id) + else: + instance = py_type() + instance._resetUid(result.uid) + params = {k: deserialize_param(v) for k, v in result.params.params.items()} + instance._set(**params) + return instance + else: + raise RuntimeError(f"Unsupported class {self._clazz}") diff --git a/python/pyspark/ml/remote/serialize.py b/python/pyspark/ml/remote/serialize.py new file mode 100644 index 0000000000000..69e3af1f4c787 --- /dev/null +++ b/python/pyspark/ml/remote/serialize.py @@ -0,0 +1,132 @@ +# +# 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. +# +from typing import Any, List, TYPE_CHECKING, Mapping, Dict + +import pyspark.sql.connect.proto as pb2 +from pyspark.ml.linalg import ( + Vectors, + Matrices, + DenseVector, + SparseVector, + DenseMatrix, + SparseMatrix, +) +from pyspark.sql.connect.expressions import LiteralExpression + +if TYPE_CHECKING: + from pyspark.sql.connect.client import SparkConnectClient + from pyspark.ml.param import Params + + +def serialize_param(value: Any, client: "SparkConnectClient") -> pb2.Param: + if isinstance(value, DenseVector): + return pb2.Param(vector=pb2.Vector(dense=pb2.Vector.Dense(value=value.values.tolist()))) + elif isinstance(value, SparseVector): + return pb2.Param( + vector=pb2.Vector( + sparse=pb2.Vector.Sparse( + size=value.size, index=value.indices.tolist(), value=value.values.tolist() + ) + ) + ) + elif isinstance(value, DenseMatrix): + return pb2.Param( + matrix=pb2.Matrix( + dense=pb2.Matrix.Dense( + num_rows=value.numRows, num_cols=value.numCols, value=value.values.tolist() + ) + ) + ) + elif isinstance(value, SparseMatrix): + return pb2.Param( + matrix=pb2.Matrix( + sparse=pb2.Matrix.Sparse( + num_rows=value.numRows, + num_cols=value.numCols, + colptr=value.colPtrs.tolist(), + row_index=value.rowIndices.tolist(), + value=value.values.tolist(), + ) + ) + ) + else: + literal = LiteralExpression._from_value(value).to_plan(client).literal + return pb2.Param(literal=literal) + + +def serialize(client: "SparkConnectClient", *args: Any) -> List[Any]: + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + + result = [] + for arg in args: + if isinstance(arg, ConnectDataFrame): + result.append(pb2.Fetch.Method.Args(input=arg._plan.plan(client))) + else: + result.append(pb2.Fetch.Method.Args(param=serialize_param(arg, client))) + return result + + +def deserialize_param(param: pb2.Param) -> Any: + if param.HasField("literal"): + return LiteralExpression._to_value(param.literal) + if param.HasField("vector"): + vector = param.vector + if vector.HasField("dense"): + return Vectors.dense(vector.dense.value) + elif vector.HasField("sparse"): + return Vectors.sparse(vector.sparse.size, vector.sparse.index, vector.sparse.value) + else: + raise ValueError("Unsupported vector type") + if param.HasField("matrix"): + matrix = param.matrix + if matrix.HasField("dense"): + return DenseMatrix( + matrix.dense.num_rows, + matrix.dense.num_cols, + matrix.dense.value, + matrix.dense.is_transposed, + ) + elif matrix.HasField("sparse"): + return Matrices.sparse( + matrix.sparse.num_rows, + matrix.sparse.num_cols, + matrix.sparse.colptr, + matrix.sparse.row_index, + matrix.sparse.value, + ) + else: + raise ValueError("Unsupported matrix type") + + raise ValueError("Unsupported param type") + + +def deserialize(ml_command_result_properties: Dict[str, Any]) -> Any: + ml_command_result = ml_command_result_properties["ml_command_result"] + if ml_command_result.HasField("operator_info"): + return ml_command_result.operator_info + + if ml_command_result.HasField("param"): + return deserialize_param(ml_command_result.param) + + raise ValueError("Unsupported result type") + + +def serialize_ml_params(instance: "Params", client: "SparkConnectClient") -> pb2.MlParams: + params: Mapping[str, pb2.Param] = { + k.name: serialize_param(v, client) for k, v in instance._paramMap.items() + } + return pb2.MlParams(params=params) diff --git a/python/pyspark/ml/remote/util.py b/python/pyspark/ml/remote/util.py new file mode 100644 index 0000000000000..cb34dae165863 --- /dev/null +++ b/python/pyspark/ml/remote/util.py @@ -0,0 +1,293 @@ +# +# 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. +# +import functools +import os +from typing import Any, cast, TypeVar, Callable, TYPE_CHECKING, Type, List, Tuple + +import pyspark.sql.connect.proto as pb2 +from pyspark.ml.remote.serialize import serialize_ml_params, serialize, deserialize +from pyspark.sql import is_remote + +if TYPE_CHECKING: + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + from pyspark.ml.wrapper import JavaWrapper, JavaEstimator + from pyspark.ml.util import JavaMLReadable, JavaMLWritable + +FuncT = TypeVar("FuncT", bound=Callable[..., Any]) + + +def _extract_id_methods(obj_identifier: str) -> Tuple[List[pb2.Fetch.Method], str]: + """Extract the obj reference id and the methods. Eg, model.summary""" + method_chain = obj_identifier.split(".") + obj_ref = method_chain[0] + methods: List[pb2.Fetch.Method] = [] + if len(method_chain) > 1: + methods = [pb2.Fetch.Method(method=m) for m in method_chain[1:]] + return methods, obj_ref + + +def try_remote_intermediate_result(f: FuncT) -> FuncT: + """Mark the function/property that returns the intermediate result of the remote call. + Eg, model.summary""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + return f"{self._java_obj}.{f.__name__}" + else: + return f(self) + + return cast(FuncT, wrapped) + + +def try_remote_attribute_relation(f: FuncT) -> FuncT: + """Mark the function/property that returns a Relation. + Eg, model.summary.roc""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper", *args: Any, **kwargs: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + # The attribute returns a dataframe, we need to wrap it + # in the AttributeRelation + from pyspark.ml.remote.proto import AttributeRelation + from pyspark.sql.connect.session import SparkSession + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + + session = SparkSession.getActiveSession() + assert session is not None + + assert isinstance(self._java_obj, str) + + methods, obj_ref = _extract_id_methods(self._java_obj) + methods.append( + pb2.Fetch.Method(method=f.__name__, args=serialize(session.client, *args)) + ) + plan = AttributeRelation(obj_ref, methods) + return ConnectDataFrame(plan, session) + else: + return f(self, *args, **kwargs) + + return cast(FuncT, wrapped) + + +def try_remote_fit(f: FuncT) -> FuncT: + """Mark the function that fits a model.""" + + @functools.wraps(f) + def wrapped(self: "JavaEstimator", dataset: "ConnectDataFrame") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + client = dataset.sparkSession.client + input = dataset._plan.plan(client) + assert isinstance(self._java_obj, str) + estimator = pb2.MlOperator( + name=self._java_obj, uid=self.uid, type=pb2.MlOperator.ESTIMATOR + ) + command = pb2.Command() + command.ml_command.fit.CopyFrom( + pb2.MlCommand.Fit( + estimator=estimator, + params=serialize_ml_params(self, client), + dataset=input, + ) + ) + (_, properties, _) = client.execute_command(command) + model_info = deserialize(properties) + client.add_ml_cache(model_info.obj_ref.id) + return model_info.obj_ref.id + else: + return f(self, dataset) + + return cast(FuncT, wrapped) + + +def try_remote_transform_relation(f: FuncT) -> FuncT: + """Mark the function/property that returns a relation for model transform.""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper", dataset: "ConnectDataFrame") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + from pyspark.ml import Model, Transformer + from pyspark.sql.connect.session import SparkSession + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + + session = SparkSession.getActiveSession() + assert session is not None + # Model is also a Transformer, so we much match Model first + if isinstance(self, Model): + params = serialize_ml_params(self, session.client) + from pyspark.ml.remote.proto import TransformerRelation + + assert isinstance(self._java_obj, str) + return ConnectDataFrame( + TransformerRelation( + child=dataset._plan, name=self._java_obj, ml_params=params, is_model=True + ), + session, + ) + elif isinstance(self, Transformer): + params = serialize_ml_params(self, session.client) + from pyspark.ml.remote.proto import TransformerRelation + + assert isinstance(self._java_obj, str) + return ConnectDataFrame( + TransformerRelation( + child=dataset._plan, + name=self._java_obj, + ml_params=params, + uid=self.uid, + is_model=False, + ), + session, + ) + else: + raise RuntimeError(f"Unsupported {self}") + else: + return f(self, dataset) + + return cast(FuncT, wrapped) + + +def try_remote_call(f: FuncT) -> FuncT: + """Mark the function/property for the remote call. + Eg, model.coefficients""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper", name: str, *args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + """Launch a remote call if possible""" + from pyspark.sql.connect.session import SparkSession + + session = SparkSession.getActiveSession() + assert session is not None + assert isinstance(self._java_obj, str) + methods, obj_ref = _extract_id_methods(self._java_obj) + methods.append(pb2.Fetch.Method(method=name, args=serialize(session.client, *args))) + command = pb2.Command() + command.ml_command.fetch.CopyFrom( + pb2.Fetch(obj_ref=pb2.ObjectRef(id=obj_ref), methods=methods) + ) + (_, properties, _) = session.client.execute_command(command) + ml_command_result = properties["ml_command_result"] + if ml_command_result.HasField("summary"): + summary = ml_command_result.summary + session.client.add_ml_cache(summary) + return summary + else: + return deserialize(properties) + else: + return f(self, name, *args) + + return cast(FuncT, wrapped) + + +def try_remote_del(f: FuncT) -> FuncT: + """Mark the function/property to delete a model on the server side.""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper") -> Any: + try: + in_remote = is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ + except Exception: + return + + if in_remote: + # Delete the model if possible + model_id = self._java_obj + if model_id is not None and "." not in model_id: + try: + from pyspark.sql.connect.session import SparkSession + + session = SparkSession.getActiveSession() + if session is not None: + session.client.remove_ml_cache(model_id) + return + except Exception: + # SparkSession's down. + return + else: + return f(self) + + return cast(FuncT, wrapped) + + +def try_remote_return_java_class(f: FuncT) -> FuncT: + """Mark the function/property that returns none.""" + + @functools.wraps(f) + def wrapped(java_class: str, *args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + return java_class + else: + return f(java_class, *args) + + return cast(FuncT, wrapped) + + +def try_remote_write(f: FuncT) -> FuncT: + """Mark the function that write an estimator/model or evaluator""" + + @functools.wraps(f) + def wrapped(self: "JavaMLWritable") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + from pyspark.ml.remote.readwrite import RemoteMLWriter + + return RemoteMLWriter(self) + else: + return f(self) + + return cast(FuncT, wrapped) + + +def try_remote_read(f: FuncT) -> FuncT: + """Mark the function to read an estimator/model or evaluator""" + + @functools.wraps(f) + def wrapped(cls: Type["JavaMLReadable"]) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + from pyspark.ml.remote.readwrite import RemoteMLReader + + return RemoteMLReader(cls) + else: + return f(cls) + + return cast(FuncT, wrapped) + + +def try_remote_intercept(f: FuncT) -> FuncT: + """Mark the function/property that returns none.""" + + @functools.wraps(f) + def wrapped(java_class: str, *args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + return None + else: + return f(java_class, *args) + + return cast(FuncT, wrapped) + + +def try_remote_not_supporting(f: FuncT) -> FuncT: + """Mark the function/property that has not been supported yet""" + + @functools.wraps(f) + def wrapped(*args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + raise NotImplementedError("") + else: + return f(*args) + + return cast(FuncT, wrapped) diff --git a/python/pyspark/ml/tests/connect/test_connect_spark_ml_classification.py b/python/pyspark/ml/tests/connect/test_connect_spark_ml_classification.py new file mode 100644 index 0000000000000..2000a38d9e616 --- /dev/null +++ b/python/pyspark/ml/tests/connect/test_connect_spark_ml_classification.py @@ -0,0 +1,49 @@ +# +# 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. +# + +import os +import unittest + +from pyspark.ml.tests.test_classification import ClassificationTestsMixin +from pyspark.sql import SparkSession + + +class ClassificationTestsOnConnect(ClassificationTestsMixin, unittest.TestCase): + def setUp(self) -> None: + self.spark = SparkSession.builder.remote( + os.environ.get("SPARK_CONNECT_TESTING_REMOTE", "local[2]") + ).getOrCreate() + + def test_assert_remote_mode(self): + from pyspark.sql import is_remote + + self.assertTrue(is_remote()) + + def tearDown(self) -> None: + self.spark.stop() + + +if __name__ == "__main__": + from pyspark.ml.tests.connect.test_connect_spark_ml_classification import * # noqa: F401 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/ml/tests/test_algorithms.py b/python/pyspark/ml/tests/test_algorithms.py index eeb342c4238dd..d0e2600a9a8b3 100644 --- a/python/pyspark/ml/tests/test_algorithms.py +++ b/python/pyspark/ml/tests/test_algorithms.py @@ -29,93 +29,13 @@ ) from pyspark.ml.clustering import DistributedLDAModel, KMeans, LocalLDAModel, LDA, LDAModel from pyspark.ml.fpm import FPGrowth -from pyspark.ml.linalg import Matrices, Vectors, DenseVector +from pyspark.ml.linalg import Vectors, DenseVector from pyspark.ml.recommendation import ALS from pyspark.ml.regression import GeneralizedLinearRegression, LinearRegression from pyspark.sql import Row from pyspark.testing.mlutils import SparkSessionTestCase -class LogisticRegressionTest(SparkSessionTestCase): - def test_binomial_logistic_regression_with_bound(self): - df = self.spark.createDataFrame( - [ - (1.0, 1.0, Vectors.dense(0.0, 5.0)), - (0.0, 2.0, Vectors.dense(1.0, 2.0)), - (1.0, 3.0, Vectors.dense(2.0, 1.0)), - (0.0, 4.0, Vectors.dense(3.0, 3.0)), - ], - ["label", "weight", "features"], - ) - - lor = LogisticRegression( - regParam=0.01, - weightCol="weight", - lowerBoundsOnCoefficients=Matrices.dense(1, 2, [-1.0, -1.0]), - upperBoundsOnIntercepts=Vectors.dense(0.0), - ) - model = lor.fit(df) - self.assertTrue(np.allclose(model.coefficients.toArray(), [-0.2944, -0.0484], atol=1e-4)) - self.assertTrue(np.isclose(model.intercept, 0.0, atol=1e-4)) - - def test_multinomial_logistic_regression_with_bound(self): - data_path = "data/mllib/sample_multiclass_classification_data.txt" - df = self.spark.read.format("libsvm").load(data_path) - - lor = LogisticRegression( - regParam=0.01, - lowerBoundsOnCoefficients=Matrices.dense(3, 4, range(12)), - upperBoundsOnIntercepts=Vectors.dense(0.0, 0.0, 0.0), - ) - model = lor.fit(df) - expected = [ - [4.593, 4.5516, 9.0099, 12.2904], - [1.0, 8.1093, 7.0, 10.0], - [3.041, 5.0, 8.0, 11.0], - ] - for i in range(0, len(expected)): - self.assertTrue( - np.allclose(model.coefficientMatrix.toArray()[i], expected[i], atol=1e-4) - ) - self.assertTrue( - np.allclose(model.interceptVector.toArray(), [-0.9057, -1.1392, -0.0033], atol=1e-4) - ) - - def test_logistic_regression_with_threshold(self): - df = self.spark.createDataFrame( - [ - (1.0, 1.0, Vectors.dense(0.0, 5.0)), - (0.0, 2.0, Vectors.dense(1.0, 2.0)), - (1.0, 3.0, Vectors.dense(2.0, 1.0)), - (0.0, 4.0, Vectors.dense(3.0, 3.0)), - ], - ["label", "weight", "features"], - ) - - lor = LogisticRegression(weightCol="weight") - model = lor.fit(df) - - # status changes 1 - for t in [0.0, 0.1, 0.2, 0.5, 1.0]: - model.setThreshold(t).transform(df) - - # status changes 2 - [model.setThreshold(t).predict(Vectors.dense(0.0, 5.0)) for t in [0.0, 0.1, 0.2, 0.5, 1.0]] - - self.assertEqual( - [row.prediction for row in model.setThreshold(0.0).transform(df).collect()], - [1.0, 1.0, 1.0, 1.0], - ) - self.assertEqual( - [row.prediction for row in model.setThreshold(0.5).transform(df).collect()], - [0.0, 1.0, 1.0, 0.0], - ) - self.assertEqual( - [row.prediction for row in model.setThreshold(1.0).transform(df).collect()], - [0.0, 0.0, 0.0, 0.0], - ) - - class MultilayerPerceptronClassifierTest(SparkSessionTestCase): def test_raw_and_probability_prediction(self): data_path = "data/mllib/sample_multiclass_classification_data.txt" diff --git a/python/pyspark/ml/tests/test_classification.py b/python/pyspark/ml/tests/test_classification.py new file mode 100644 index 0000000000000..ee72e0394e3a0 --- /dev/null +++ b/python/pyspark/ml/tests/test_classification.py @@ -0,0 +1,304 @@ +# +# 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. +# + +import os +import tempfile +import unittest +from shutil import rmtree + +import numpy as np + +from pyspark.ml.linalg import Vectors, Matrices +from pyspark.sql import SparkSession, DataFrame +from pyspark.ml.classification import ( + LogisticRegression, + LogisticRegressionModel, + LogisticRegressionSummary, + BinaryLogisticRegressionSummary, +) + + +class ClassificationTestsMixin: + def test_binomial_logistic_regression_with_bound(self): + df = self.spark.createDataFrame( + [ + (1.0, 1.0, Vectors.dense(0.0, 5.0)), + (0.0, 2.0, Vectors.dense(1.0, 2.0)), + (1.0, 3.0, Vectors.dense(2.0, 1.0)), + (0.0, 4.0, Vectors.dense(3.0, 3.0)), + ], + ["label", "weight", "features"], + ) + + lor = LogisticRegression( + regParam=0.01, + weightCol="weight", + lowerBoundsOnCoefficients=Matrices.dense(1, 2, [-1.0, -1.0]), + upperBoundsOnIntercepts=Vectors.dense(0.0), + ) + lor_model = lor.fit(df) + + def check_result(model: LogisticRegressionModel) -> None: + self.assertTrue( + np.allclose(model.coefficients.toArray(), [-0.2944, -0.0484], atol=1e-4) + ) + self.assertTrue(np.isclose(model.intercept, 0.0, atol=1e-4)) + + check_result(lor_model) + + # Model save + with tempfile.TemporaryDirectory(prefix="model_save") as tmp_dir: + local_path = os.path.join(tmp_dir, "model") + lor_model.write().save(local_path) + loaded_model = LogisticRegressionModel.load(local_path) + check_result(loaded_model) + + def test_multinomial_logistic_regression_with_bound(self): + data_path = "data/mllib/sample_multiclass_classification_data.txt" + df = self.spark.read.format("libsvm").load(data_path) + + lor = LogisticRegression( + regParam=0.01, + lowerBoundsOnCoefficients=Matrices.dense(3, 4, range(12)), + upperBoundsOnIntercepts=Vectors.dense(0.0, 0.0, 0.0), + ) + lor_model = lor.fit(df) + + def check_result(model: LogisticRegressionModel) -> None: + expected = [ + [4.593, 4.5516, 9.0099, 12.2904], + [1.0, 8.1093, 7.0, 10.0], + [3.041, 5.0, 8.0, 11.0], + ] + for i in range(0, len(expected)): + self.assertTrue( + np.allclose(model.coefficientMatrix.toArray()[i], expected[i], atol=1e-4) + ) + self.assertTrue( + np.allclose(model.interceptVector.toArray(), [-0.9057, -1.1392, -0.0033], atol=1e-4) + ) + + check_result(lor_model) + + # Model save + with tempfile.TemporaryDirectory(prefix="model_save") as tmp_dir: + local_path = os.path.join(tmp_dir, "model") + lor_model.write().save(local_path) + loaded_model = LogisticRegressionModel.load(local_path) + check_result(loaded_model) + + def test_logistic_regression_with_threshold(self): + df = self.spark.createDataFrame( + [ + (1.0, 1.0, Vectors.dense(0.0, 5.0)), + (0.0, 2.0, Vectors.dense(1.0, 2.0)), + (1.0, 3.0, Vectors.dense(2.0, 1.0)), + (0.0, 4.0, Vectors.dense(3.0, 3.0)), + ], + ["label", "weight", "features"], + ) + + lor = LogisticRegression(weightCol="weight") + model = lor.fit(df) + + # status changes 1 + for t in [0.0, 0.1, 0.2, 0.5, 1.0]: + model.setThreshold(t).transform(df) + + # status changes 2 + [model.setThreshold(t).predict(Vectors.dense(0.0, 5.0)) for t in [0.0, 0.1, 0.2, 0.5, 1.0]] + + self.assertEqual( + [row.prediction for row in model.setThreshold(0.0).transform(df).collect()], + [1.0, 1.0, 1.0, 1.0], + ) + self.assertEqual( + [row.prediction for row in model.setThreshold(0.5).transform(df).collect()], + [0.0, 1.0, 1.0, 0.0], + ) + self.assertEqual( + [row.prediction for row in model.setThreshold(1.0).transform(df).collect()], + [0.0, 0.0, 0.0, 0.0], + ) + + def test_binary_logistic_regression_summary(self): + df = self.spark.createDataFrame( + [(1.0, 2.0, Vectors.dense(1.0)), (0.0, 2.0, Vectors.sparse(1, [], []))], + ["label", "weight", "features"], + ) + lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) + model = lr.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.probabilityCol, "probability") + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.featuresCol, "features") + self.assertEqual(s.predictionCol, "prediction") + objHist = s.objectiveHistory + self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertTrue(isinstance(s.roc, DataFrame)) + self.assertAlmostEqual(s.areaUnderROC, 1.0, 2) + self.assertTrue(isinstance(s.pr, DataFrame)) + self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame)) + self.assertTrue(isinstance(s.precisionByThreshold, DataFrame)) + self.assertTrue(isinstance(s.recallByThreshold, DataFrame)) + self.assertAlmostEqual(s.accuracy, 1.0, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) + self.assertAlmostEqual(s.weightedRecall, 1.0, 2) + self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) + + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, BinaryLogisticRegressionSummary)) + self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + self.assertEqual(sorted(sameSummary.predictions.collect()), sorted(s.predictions.collect())) + + def test_multiclass_logistic_regression_summary(self): + df = self.spark.createDataFrame( + [ + (1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], [])), + (2.0, 2.0, Vectors.dense(2.0)), + (2.0, 2.0, Vectors.dense(1.9)), + ], + ["label", "weight", "features"], + ) + lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) + model = lr.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.probabilityCol, "probability") + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.featuresCol, "features") + self.assertEqual(s.predictionCol, "prediction") + objHist = s.objectiveHistory + self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertAlmostEqual(s.accuracy, 0.75, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 0.75, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.25, 2) + self.assertAlmostEqual(s.weightedRecall, 0.75, 2) + self.assertAlmostEqual(s.weightedPrecision, 0.583, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 0.65, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.65, 2) + + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, LogisticRegressionSummary)) + self.assertFalse(isinstance(sameSummary, BinaryLogisticRegressionSummary)) + self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) + + # We can't use sorted(s.predictions.collect()), since the DenseVector doesn't support "<" + self.assertEqual( + sameSummary.predictions.coalesce(1).sort("label", "weight", "prediction").collect(), + s.predictions.coalesce(1).sort("label", "weight", "prediction").collect(), + ) + + def test_logistic_regression(self): + # test sparse/dense vector and matrix + lower_intercepts = Vectors.dense([1, 2, 3, 4]) + upper_intercepts = Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) + lower_coefficients = Matrices.dense(3, 2, [0, 1, 4, 5, 9, 10]) + upper_coefficients = Matrices.sparse(1, 1, [0, 1], [0], [2.0]) + + lr = LogisticRegression( + maxIter=1, + lowerBoundsOnIntercepts=lower_intercepts, + upperBoundsOnIntercepts=upper_intercepts, + lowerBoundsOnCoefficients=lower_coefficients, + upperBoundsOnCoefficients=upper_coefficients, + ) + path = tempfile.mkdtemp() + lr_path = path + "/logreg" + lr.save(lr_path) + lr2 = LogisticRegression.load(lr_path) + self.assertEqual( + lr2.uid, + lr2.maxIter.parent, + "Loaded LogisticRegression instance uid (%s) " + "did not match Param's uid (%s)" % (lr2.uid, lr2.maxIter.parent), + ) + self.assertEqual( + lr._defaultParamMap[lr.maxIter], + lr2._defaultParamMap[lr2.maxIter], + "Loaded LogisticRegression instance default params did not match " + + "original defaults", + ) + self.assertEqual( + lr.getLowerBoundsOnIntercepts(), + lr2.getLowerBoundsOnIntercepts(), + ) + self.assertEqual( + lr.getUpperBoundsOnIntercepts(), + lr2.getUpperBoundsOnIntercepts(), + ) + self.assertEqual( + lr.getLowerBoundsOnCoefficients(), + lr2.getLowerBoundsOnCoefficients(), + ) + self.assertEqual( + lr.getUpperBoundsOnCoefficients(), + lr2.getUpperBoundsOnCoefficients(), + ) + try: + rmtree(path) + except OSError: + pass + + +class ClassificationTests(ClassificationTestsMixin, unittest.TestCase): + def setUp(self) -> None: + self.spark = SparkSession.builder.master("local[4]").getOrCreate() + + def tearDown(self) -> None: + self.spark.stop() + + +if __name__ == "__main__": + from pyspark.ml.tests.test_classification import * # noqa: F401,F403 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/ml/tests/test_persistence.py b/python/pyspark/ml/tests/test_persistence.py index 406180d9a6391..481c2f236d46f 100644 --- a/python/pyspark/ml/tests/test_persistence.py +++ b/python/pyspark/ml/tests/test_persistence.py @@ -153,29 +153,6 @@ def test_linear_regression_pmml_basic(self): self.assertIn("Apache Spark", pmml_text) self.assertIn("PMML", pmml_text) - def test_logistic_regression(self): - lr = LogisticRegression(maxIter=1) - path = tempfile.mkdtemp() - lr_path = path + "/logreg" - lr.save(lr_path) - lr2 = LogisticRegression.load(lr_path) - self.assertEqual( - lr2.uid, - lr2.maxIter.parent, - "Loaded LogisticRegression instance uid (%s) " - "did not match Param's uid (%s)" % (lr2.uid, lr2.maxIter.parent), - ) - self.assertEqual( - lr._defaultParamMap[lr.maxIter], - lr2._defaultParamMap[lr2.maxIter], - "Loaded LogisticRegression instance default params did not match " - + "original defaults", - ) - try: - rmtree(path) - except OSError: - pass - def test_kmeans(self): kmeans = KMeans(k=2, seed=1) path = tempfile.mkdtemp() diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index 5704d7186734f..e1c8f4197e3c7 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -18,14 +18,11 @@ import unittest from pyspark.ml.classification import ( - BinaryLogisticRegressionSummary, BinaryRandomForestClassificationSummary, FMClassifier, FMClassificationSummary, LinearSVC, LinearSVCSummary, - LogisticRegression, - LogisticRegressionSummary, MultilayerPerceptronClassifier, MultilayerPerceptronClassificationSummary, RandomForestClassificationSummary, @@ -122,94 +119,6 @@ def test_glr_summary(self): sameSummary = model.evaluate(df) self.assertAlmostEqual(sameSummary.deviance, s.deviance) - def test_binary_logistic_regression_summary(self): - df = self.spark.createDataFrame( - [(1.0, 2.0, Vectors.dense(1.0)), (0.0, 2.0, Vectors.sparse(1, [], []))], - ["label", "weight", "features"], - ) - lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) - model = lr.fit(df) - self.assertTrue(model.hasSummary) - s = model.summary - # test that api is callable and returns expected types - self.assertTrue(isinstance(s.predictions, DataFrame)) - self.assertEqual(s.probabilityCol, "probability") - self.assertEqual(s.labelCol, "label") - self.assertEqual(s.featuresCol, "features") - self.assertEqual(s.predictionCol, "prediction") - objHist = s.objectiveHistory - self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) - self.assertGreater(s.totalIterations, 0) - self.assertTrue(isinstance(s.labels, list)) - self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.precisionByLabel, list)) - self.assertTrue(isinstance(s.recallByLabel, list)) - self.assertTrue(isinstance(s.fMeasureByLabel(), list)) - self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) - self.assertTrue(isinstance(s.roc, DataFrame)) - self.assertAlmostEqual(s.areaUnderROC, 1.0, 2) - self.assertTrue(isinstance(s.pr, DataFrame)) - self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame)) - self.assertTrue(isinstance(s.precisionByThreshold, DataFrame)) - self.assertTrue(isinstance(s.recallByThreshold, DataFrame)) - self.assertAlmostEqual(s.accuracy, 1.0, 2) - self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) - self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) - self.assertAlmostEqual(s.weightedRecall, 1.0, 2) - self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) - self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) - self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) - # test evaluation (with training dataset) produces a summary with same values - # one check is enough to verify a summary is returned, Scala version runs full test - sameSummary = model.evaluate(df) - self.assertTrue(isinstance(sameSummary, BinaryLogisticRegressionSummary)) - self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) - - def test_multiclass_logistic_regression_summary(self): - df = self.spark.createDataFrame( - [ - (1.0, 2.0, Vectors.dense(1.0)), - (0.0, 2.0, Vectors.sparse(1, [], [])), - (2.0, 2.0, Vectors.dense(2.0)), - (2.0, 2.0, Vectors.dense(1.9)), - ], - ["label", "weight", "features"], - ) - lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) - model = lr.fit(df) - self.assertTrue(model.hasSummary) - s = model.summary - # test that api is callable and returns expected types - self.assertTrue(isinstance(s.predictions, DataFrame)) - self.assertEqual(s.probabilityCol, "probability") - self.assertEqual(s.labelCol, "label") - self.assertEqual(s.featuresCol, "features") - self.assertEqual(s.predictionCol, "prediction") - objHist = s.objectiveHistory - self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) - self.assertGreater(s.totalIterations, 0) - self.assertTrue(isinstance(s.labels, list)) - self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.precisionByLabel, list)) - self.assertTrue(isinstance(s.recallByLabel, list)) - self.assertTrue(isinstance(s.fMeasureByLabel(), list)) - self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) - self.assertAlmostEqual(s.accuracy, 0.75, 2) - self.assertAlmostEqual(s.weightedTruePositiveRate, 0.75, 2) - self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.25, 2) - self.assertAlmostEqual(s.weightedRecall, 0.75, 2) - self.assertAlmostEqual(s.weightedPrecision, 0.583, 2) - self.assertAlmostEqual(s.weightedFMeasure(), 0.65, 2) - self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.65, 2) - # test evaluation (with training dataset) produces a summary with same values - # one check is enough to verify a summary is returned, Scala version runs full test - sameSummary = model.evaluate(df) - self.assertTrue(isinstance(sameSummary, LogisticRegressionSummary)) - self.assertFalse(isinstance(sameSummary, BinaryLogisticRegressionSummary)) - self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) - def test_linear_svc_summary(self): df = self.spark.createDataFrame( [(1.0, 2.0, Vectors.dense(1.0, 1.0, 1.0)), (0.0, 2.0, Vectors.dense(1.0, 2.0, 3.0))], diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 9bbd64d2aef5a..3fe97f44619c0 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -37,6 +37,7 @@ from pyspark import since from pyspark.ml.common import inherit_doc +from pyspark.ml.remote.util import try_remote_intermediate_result, try_remote_write, try_remote_read from pyspark.sql import SparkSession from pyspark.sql.utils import is_remote from pyspark.util import VersionUtils @@ -270,6 +271,7 @@ class JavaMLWritable(MLWritable): (Private) Mixin for ML instances that provide :py:class:`JavaMLWriter`. """ + @try_remote_write def write(self) -> JavaMLWriter: """Returns an MLWriter instance for this ML instance.""" return JavaMLWriter(self) @@ -378,6 +380,7 @@ class JavaMLReadable(MLReadable[RL]): """ @classmethod + @try_remote_read def read(cls) -> JavaMLReader[RL]: """Returns an MLReader instance for this class.""" return JavaMLReader(cls) @@ -680,6 +683,7 @@ def hasSummary(self) -> bool: @property @since("2.1.0") + @try_remote_intermediate_result def summary(self) -> T: """ Gets summary of the model trained on the training set. An exception is thrown if diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index eed7781dc71e3..e2bf25386c77a 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -19,7 +19,15 @@ from typing import Any, Generic, Optional, List, Type, TypeVar, TYPE_CHECKING from pyspark import since -from pyspark.sql import DataFrame +from pyspark.ml.remote.util import ( + try_remote_transform_relation, + try_remote_call, + try_remote_fit, + try_remote_del, + try_remote_return_java_class, + try_remote_intercept, +) +from pyspark.sql import DataFrame, is_remote from pyspark.ml import Estimator, Predictor, PredictionModel, Transformer, Model from pyspark.ml.base import _PredictorParams from pyspark.ml.param import Param, Params @@ -47,6 +55,7 @@ def __init__(self, java_obj: Optional["JavaObject"] = None): super(JavaWrapper, self).__init__() self._java_obj = java_obj + @try_remote_del def __del__(self) -> None: from pyspark.core.context import SparkContext @@ -63,6 +72,7 @@ def _create_from_java_class(cls: Type[JW], java_class: str, *args: Any) -> JW: java_obj = JavaWrapper._new_java_obj(java_class, *args) return cls(java_obj) + @try_remote_call def _call_java(self, name: str, *args: Any) -> Any: from pyspark.core.context import SparkContext @@ -74,6 +84,7 @@ def _call_java(self, name: str, *args: Any) -> Any: return _java2py(sc, m(*java_args)) @staticmethod + @try_remote_return_java_class def _new_java_obj(java_class: str, *args: Any) -> "JavaObject": """ Returns a new Java object. @@ -347,6 +358,7 @@ def copy(self: "JP", extra: Optional["ParamMap"] = None) -> "JP": that._transfer_params_to_java() return that + @try_remote_intercept def clear(self, param: Param) -> None: """ Clears a param from the param map if it has been explicitly set. @@ -372,6 +384,7 @@ def _create_model(self, java_model: "JavaObject") -> JM: """ raise NotImplementedError() + @try_remote_fit def _fit_java(self, dataset: DataFrame) -> "JavaObject": """ Fits a Java model to the input dataset. @@ -405,6 +418,7 @@ class JavaTransformer(JavaParams, Transformer, metaclass=ABCMeta): available as _java_obj. """ + @try_remote_transform_relation def _transform(self, dataset: DataFrame) -> DataFrame: assert self._java_obj is not None @@ -435,7 +449,7 @@ def __init__(self, java_model: Optional["JavaObject"] = None): other ML classes). """ super(JavaModel, self).__init__(java_model) - if java_model is not None: + if java_model is not None and not is_remote(): # SPARK-10931: This is a temporary fix to allow models to own params # from estimators. Eventually, these params should be in models through # using common base classes between estimators and models. diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 162d309a6f40a..4a85ca26b64de 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -20,6 +20,8 @@ "SparkConnectClient", ] +import atexit + from pyspark.sql.connect.utils import check_dependencies check_dependencies(__name__) @@ -675,6 +677,9 @@ def __init__( self._progress_handlers: List[ProgressHandler] = [] + # cleanup ml cache if possible + atexit.register(self._cleanup_ml) + def register_progress_handler(self, handler: ProgressHandler) -> None: """ Register a progress handler to be called when a progress message is received. @@ -1479,6 +1484,8 @@ def handle_response( b.checkpoint_command_result.relation ) } + if b.HasField("ml_command_result"): + yield {"ml_command_result": b.ml_command_result} try: if self._use_reattachable_execute: @@ -1931,3 +1938,33 @@ def _create_profile(self, profile: pb2.ResourceProfile) -> int: (_, properties, _) = self.execute_command(cmd) profile_id = properties["create_resource_profile_command_result"] return profile_id + + def add_ml_cache(self, cache_id: str) -> None: + if not hasattr(self.thread_local, "ml_caches"): + self.thread_local.ml_caches = set() + self.thread_local.ml_caches.add(cache_id) + + def remove_ml_cache(self, cache_id: str) -> None: + if not hasattr(self.thread_local, "ml_caches"): + self.thread_local.ml_caches = set() + + if cache_id in self.thread_local.ml_caches: + self._delete_ml_cache(cache_id) + + def _delete_ml_cache(self, cache_id: str) -> None: + # try best to delete the cache + try: + command = pb2.Command() + command.ml_command.delete.obj_ref.CopyFrom(pb2.ObjectRef(id=cache_id)) + self.execute_command(command) + except Exception: + pass + + def _cleanup_ml(self) -> None: + if not hasattr(self.thread_local, "ml_caches"): + self.thread_local.ml_caches = set() + + self.disable_reattachable_execute() + # Todo add a pattern to delete all model in one command + for model_id in self.thread_local.ml_caches: + self._delete_ml_cache(model_id) diff --git a/python/pyspark/sql/connect/proto/__init__.py b/python/pyspark/sql/connect/proto/__init__.py index 3e8d074d963dc..0877696c2680e 100644 --- a/python/pyspark/sql/connect/proto/__init__.py +++ b/python/pyspark/sql/connect/proto/__init__.py @@ -23,3 +23,5 @@ from pyspark.sql.connect.proto.relations_pb2 import * from pyspark.sql.connect.proto.catalog_pb2 import * from pyspark.sql.connect.proto.common_pb2 import * +from pyspark.sql.connect.proto.ml_pb2 import * +from pyspark.sql.connect.proto.ml_common_pb2 import * diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 64c549ffe9565..2fbc4287db786 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -40,10 +40,11 @@ from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 from pyspark.sql.connect.proto import types_pb2 as spark_dot_connect_dot_types__pb2 +from pyspark.sql.connect.proto import ml_pb2 as spark_dot_connect_dot_ml__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto\x1a\x16spark/connect/ml.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xb4\x17\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12L\n\x11ml_command_result\x18\x14 \x01(\x0b\x32\x1e.spark.connect.MlCommandResultH\x00R\x0fmlCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -68,190 +69,190 @@ _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" ]._serialized_options = b"8\001" - _globals["_PLAN"]._serialized_start = 219 - _globals["_PLAN"]._serialized_end = 335 - _globals["_USERCONTEXT"]._serialized_start = 337 - _globals["_USERCONTEXT"]._serialized_end = 459 - _globals["_ANALYZEPLANREQUEST"]._serialized_start = 462 - _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3139 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1824 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1873 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1876 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2191 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2019 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2191 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2193 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2283 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2285 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2335 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2337 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2391 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2393 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2446 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2448 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2462 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2464 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2505 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2507 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2628 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2630 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2685 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2688 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2839 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2841 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 2951 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 2953 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3023 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3025 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3069 - _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3142 - _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5008 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4383 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4440 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4442 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4490 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4492 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4537 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4539 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4575 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4577 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4625 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4627 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4661 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4663 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4703 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4705 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4764 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4766 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4805 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4807 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4845 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2688 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2697 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2841 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2852 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4871 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 4954 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 4956 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 4998 - _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5011 - _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5686 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5445 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5610 - _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5689 - _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 8607 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7381 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 7452 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 7454 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 7572 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 7575 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8092 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 7670 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8002 + _globals["_PLAN"]._serialized_start = 243 + _globals["_PLAN"]._serialized_end = 359 + _globals["_USERCONTEXT"]._serialized_start = 361 + _globals["_USERCONTEXT"]._serialized_end = 483 + _globals["_ANALYZEPLANREQUEST"]._serialized_start = 486 + _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3163 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1848 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1897 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1900 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2215 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2043 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2215 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2217 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2307 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2309 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2359 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2361 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2415 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2417 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2470 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2472 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2486 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2488 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2529 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2531 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2652 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2654 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2709 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2712 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2863 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2865 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 2975 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 2977 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3047 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3049 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3093 + _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3166 + _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5032 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4407 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4464 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4466 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4514 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4516 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4561 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4563 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4599 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4601 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4649 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4651 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4685 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4687 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4727 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4729 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4788 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4790 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4829 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4831 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4869 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2712 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2721 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2865 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2876 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4895 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 4978 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 4980 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5022 + _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5035 + _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5710 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5469 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5634 + _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5713 + _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 8709 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7483 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 7554 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 7556 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 7674 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 7677 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8194 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 7772 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8104 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 7879 + ]._serialized_start = 7981 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_end = 8002 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8004 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8092 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8095 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8236 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8238 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8254 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8257 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 8590 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 8413 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 8590 - _globals["_KEYVALUE"]._serialized_start = 8609 - _globals["_KEYVALUE"]._serialized_end = 8674 - _globals["_CONFIGREQUEST"]._serialized_start = 8677 - _globals["_CONFIGREQUEST"]._serialized_end = 9876 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 8985 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 9483 - _globals["_CONFIGREQUEST_SET"]._serialized_start = 9485 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 9577 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 9579 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 9604 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9606 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9669 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9671 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9702 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9704 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9752 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9754 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9781 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9783 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9817 - _globals["_CONFIGRESPONSE"]._serialized_start = 9879 - _globals["_CONFIGRESPONSE"]._serialized_end = 10054 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10057 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11059 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10532 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10585 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10587 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10698 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10700 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10793 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10796 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 10989 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11062 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11334 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11253 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11334 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11337 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11663 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11666 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12018 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11861 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 11976 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 11978 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12018 - _globals["_INTERRUPTREQUEST"]._serialized_start = 12021 - _globals["_INTERRUPTREQUEST"]._serialized_end = 12624 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12424 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12552 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 12627 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 12771 - _globals["_REATTACHOPTIONS"]._serialized_start = 12773 - _globals["_REATTACHOPTIONS"]._serialized_end = 12826 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12829 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 13235 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 13238 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13823 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13692 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13704 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13706 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13753 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13826 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 13991 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 13994 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14206 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14208 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14316 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14319 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14651 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14654 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16209 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14883 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15057 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15060 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15428 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15391 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15428 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15431 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15840 + ]._serialized_end = 8104 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8106 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8194 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8197 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8338 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8340 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8356 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8359 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 8692 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 8515 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 8692 + _globals["_KEYVALUE"]._serialized_start = 8711 + _globals["_KEYVALUE"]._serialized_end = 8776 + _globals["_CONFIGREQUEST"]._serialized_start = 8779 + _globals["_CONFIGREQUEST"]._serialized_end = 9978 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 9087 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 9585 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 9587 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 9679 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 9681 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 9706 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9708 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9771 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9773 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9804 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9806 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9854 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9856 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9883 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9885 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9919 + _globals["_CONFIGRESPONSE"]._serialized_start = 9981 + _globals["_CONFIGRESPONSE"]._serialized_end = 10156 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10159 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11161 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10634 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10687 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10689 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10800 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10802 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10895 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10898 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 11091 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11164 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11436 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11355 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11436 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11439 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11765 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11768 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12120 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11963 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 12078 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 12080 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12120 + _globals["_INTERRUPTREQUEST"]._serialized_start = 12123 + _globals["_INTERRUPTREQUEST"]._serialized_end = 12726 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12526 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12654 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 12729 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 12873 + _globals["_REATTACHOPTIONS"]._serialized_start = 12875 + _globals["_REATTACHOPTIONS"]._serialized_end = 12928 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12931 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 13337 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 13340 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13925 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13794 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13806 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13808 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13855 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13928 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 14093 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 14096 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14308 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14310 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14418 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14421 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14753 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14756 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16311 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14985 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15159 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15162 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15530 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15493 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15530 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15533 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15942 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 15742 + ]._serialized_start = 15844 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 15810 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15843 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16190 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16211 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16301 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16304 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17250 + ]._serialized_end = 15912 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15945 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16292 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16313 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16403 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16406 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17352 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index adea62ed6b805..738339fa968ec 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -43,6 +43,7 @@ import google.protobuf.message import pyspark.sql.connect.proto.commands_pb2 import pyspark.sql.connect.proto.common_pb2 import pyspark.sql.connect.proto.expressions_pb2 +import pyspark.sql.connect.proto.ml_pb2 import pyspark.sql.connect.proto.relations_pb2 import pyspark.sql.connect.proto.types_pb2 import sys @@ -1581,6 +1582,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): CREATE_RESOURCE_PROFILE_COMMAND_RESULT_FIELD_NUMBER: builtins.int EXECUTION_PROGRESS_FIELD_NUMBER: builtins.int CHECKPOINT_COMMAND_RESULT_FIELD_NUMBER: builtins.int + ML_COMMAND_RESULT_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int METRICS_FIELD_NUMBER: builtins.int OBSERVED_METRICS_FIELD_NUMBER: builtins.int @@ -1645,6 +1647,9 @@ class ExecutePlanResponse(google.protobuf.message.Message): def checkpoint_command_result(self) -> global___CheckpointCommandResult: """Response for command that checkpoints a DataFrame.""" @property + def ml_command_result(self) -> pyspark.sql.connect.proto.ml_pb2.MlCommandResult: + """ML command response""" + @property def extension(self) -> google.protobuf.any_pb2.Any: """Support arbitrary result objects.""" @property @@ -1686,6 +1691,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): | None = ..., execution_progress: global___ExecutePlanResponse.ExecutionProgress | None = ..., checkpoint_command_result: global___CheckpointCommandResult | None = ..., + ml_command_result: pyspark.sql.connect.proto.ml_pb2.MlCommandResult | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., metrics: global___ExecutePlanResponse.Metrics | None = ..., observed_metrics: collections.abc.Iterable[global___ExecutePlanResponse.ObservedMetrics] @@ -1709,6 +1715,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"get_resources_command_result", "metrics", b"metrics", + "ml_command_result", + b"ml_command_result", "response_type", b"response_type", "result_complete", @@ -1744,6 +1752,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"get_resources_command_result", "metrics", b"metrics", + "ml_command_result", + b"ml_command_result", "observed_metrics", b"observed_metrics", "operation_id", @@ -1787,6 +1797,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): "create_resource_profile_command_result", "execution_progress", "checkpoint_command_result", + "ml_command_result", "extension", ] | None diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index a7fcc1d7e0908..57a770f0226d9 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -38,10 +38,11 @@ from pyspark.sql.connect.proto import common_pb2 as spark_dot_connect_dot_common__pb2 from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 +from pyspark.sql.connect.proto import ml_pb2 as spark_dot_connect_dot_ml__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\x90\r\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12_\n\x18merge_into_table_command\x18\x10 \x01(\x0b\x32$.spark.connect.MergeIntoTableCommandH\x00R\x15mergeIntoTableCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xd8\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x12\x36\n\x17\x63lustering_column_names\x18\x0f \x03(\tR\x15\x63lusteringColumnNames\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xcd\x01\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger\x12\x45\n\rstorage_level\x18\x04 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level"\xe8\x03\n\x15MergeIntoTableCommand\x12*\n\x11target_table_name\x18\x01 \x01(\tR\x0ftargetTableName\x12\x43\n\x11source_table_plan\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x0fsourceTablePlan\x12\x42\n\x0fmerge_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0emergeCondition\x12>\n\rmatch_actions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cmatchActions\x12I\n\x13not_matched_actions\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x11notMatchedActions\x12[\n\x1dnot_matched_by_source_actions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x19notMatchedBySourceActions\x12\x32\n\x15with_schema_evolution\x18\x07 \x01(\x08R\x13withSchemaEvolution*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x16spark/connect/ml.proto"\xcb\r\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12_\n\x18merge_into_table_command\x18\x10 \x01(\x0b\x32$.spark.connect.MergeIntoTableCommandH\x00R\x15mergeIntoTableCommand\x12\x39\n\nml_command\x18\x11 \x01(\x0b\x32\x18.spark.connect.MlCommandH\x00R\tmlCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xd8\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x12\x36\n\x17\x63lustering_column_names\x18\x0f \x03(\tR\x15\x63lusteringColumnNames\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xcd\x01\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger\x12\x45\n\rstorage_level\x18\x04 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level"\xe8\x03\n\x15MergeIntoTableCommand\x12*\n\x11target_table_name\x18\x01 \x01(\tR\x0ftargetTableName\x12\x43\n\x11source_table_plan\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x0fsourceTablePlan\x12\x42\n\x0fmerge_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0emergeCondition\x12>\n\rmatch_actions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cmatchActions\x12I\n\x13not_matched_actions\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x11notMatchedActions\x12[\n\x1dnot_matched_by_source_actions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x19notMatchedBySourceActions\x12\x32\n\x15with_schema_evolution\x18\x07 \x01(\x08R\x13withSchemaEvolution*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -78,114 +79,114 @@ _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_options = b"8\001" _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._loaded_options = None _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_options = b"8\001" - _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_start = 11252 - _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_end = 11385 - _globals["_COMMAND"]._serialized_start = 167 - _globals["_COMMAND"]._serialized_end = 1847 - _globals["_SQLCOMMAND"]._serialized_start = 1850 - _globals["_SQLCOMMAND"]._serialized_end = 2404 - _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_start = 2220 - _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_end = 2310 - _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_start = 2312 - _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_end = 2404 - _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_start = 2407 - _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_end = 2557 - _globals["_WRITEOPERATION"]._serialized_start = 2560 - _globals["_WRITEOPERATION"]._serialized_end = 3658 - _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_start = 3082 - _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_end = 3140 - _globals["_WRITEOPERATION_SAVETABLE"]._serialized_start = 3143 - _globals["_WRITEOPERATION_SAVETABLE"]._serialized_end = 3401 - _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_start = 3277 - _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_end = 3401 - _globals["_WRITEOPERATION_BUCKETBY"]._serialized_start = 3403 - _globals["_WRITEOPERATION_BUCKETBY"]._serialized_end = 3494 - _globals["_WRITEOPERATION_SAVEMODE"]._serialized_start = 3497 - _globals["_WRITEOPERATION_SAVEMODE"]._serialized_end = 3634 - _globals["_WRITEOPERATIONV2"]._serialized_start = 3661 - _globals["_WRITEOPERATIONV2"]._serialized_end = 4521 - _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_start = 3082 - _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_end = 3140 - _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_start = 4280 - _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_end = 4346 - _globals["_WRITEOPERATIONV2_MODE"]._serialized_start = 4349 - _globals["_WRITEOPERATIONV2_MODE"]._serialized_end = 4508 - _globals["_WRITESTREAMOPERATIONSTART"]._serialized_start = 4524 - _globals["_WRITESTREAMOPERATIONSTART"]._serialized_end = 5380 - _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_start = 3082 - _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_end = 3140 - _globals["_STREAMINGFOREACHFUNCTION"]._serialized_start = 5383 - _globals["_STREAMINGFOREACHFUNCTION"]._serialized_end = 5562 - _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_start = 5565 - _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_end = 5777 - _globals["_STREAMINGQUERYINSTANCEID"]._serialized_start = 5779 - _globals["_STREAMINGQUERYINSTANCEID"]._serialized_end = 5844 - _globals["_STREAMINGQUERYCOMMAND"]._serialized_start = 5847 - _globals["_STREAMINGQUERYCOMMAND"]._serialized_end = 6479 - _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_start = 6346 - _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_end = 6390 - _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_start = 6392 - _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_end = 6468 - _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_start = 6482 - _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_end = 7623 - _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_start = 7065 - _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_end = 7235 - _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_start = 7237 - _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_end = 7309 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_start = 7311 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_end = 7350 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_start = 7353 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_end = 7550 - _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_start = 7552 - _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_end = 7608 - _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_start = 7626 - _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_end = 8455 - _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_start = 8157 - _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_end = 8236 - _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_start = 8239 - _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_end = 8444 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_start = 8458 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_end = 9534 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_start = 9066 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_end = 9193 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_start = 9195 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_end = 9310 + _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_start = 11335 + _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_end = 11468 + _globals["_COMMAND"]._serialized_start = 191 + _globals["_COMMAND"]._serialized_end = 1930 + _globals["_SQLCOMMAND"]._serialized_start = 1933 + _globals["_SQLCOMMAND"]._serialized_end = 2487 + _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_start = 2303 + _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_end = 2393 + _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_start = 2395 + _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_end = 2487 + _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_start = 2490 + _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_end = 2640 + _globals["_WRITEOPERATION"]._serialized_start = 2643 + _globals["_WRITEOPERATION"]._serialized_end = 3741 + _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_start = 3165 + _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_end = 3223 + _globals["_WRITEOPERATION_SAVETABLE"]._serialized_start = 3226 + _globals["_WRITEOPERATION_SAVETABLE"]._serialized_end = 3484 + _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_start = 3360 + _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_end = 3484 + _globals["_WRITEOPERATION_BUCKETBY"]._serialized_start = 3486 + _globals["_WRITEOPERATION_BUCKETBY"]._serialized_end = 3577 + _globals["_WRITEOPERATION_SAVEMODE"]._serialized_start = 3580 + _globals["_WRITEOPERATION_SAVEMODE"]._serialized_end = 3717 + _globals["_WRITEOPERATIONV2"]._serialized_start = 3744 + _globals["_WRITEOPERATIONV2"]._serialized_end = 4604 + _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_start = 3165 + _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_end = 3223 + _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_start = 4363 + _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_end = 4429 + _globals["_WRITEOPERATIONV2_MODE"]._serialized_start = 4432 + _globals["_WRITEOPERATIONV2_MODE"]._serialized_end = 4591 + _globals["_WRITESTREAMOPERATIONSTART"]._serialized_start = 4607 + _globals["_WRITESTREAMOPERATIONSTART"]._serialized_end = 5463 + _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_start = 3165 + _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_end = 3223 + _globals["_STREAMINGFOREACHFUNCTION"]._serialized_start = 5466 + _globals["_STREAMINGFOREACHFUNCTION"]._serialized_end = 5645 + _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_start = 5648 + _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_end = 5860 + _globals["_STREAMINGQUERYINSTANCEID"]._serialized_start = 5862 + _globals["_STREAMINGQUERYINSTANCEID"]._serialized_end = 5927 + _globals["_STREAMINGQUERYCOMMAND"]._serialized_start = 5930 + _globals["_STREAMINGQUERYCOMMAND"]._serialized_end = 6562 + _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_start = 6429 + _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_end = 6473 + _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_start = 6475 + _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_end = 6551 + _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_start = 6565 + _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_end = 7706 + _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_start = 7148 + _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_end = 7318 + _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_start = 7320 + _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_end = 7392 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_start = 7394 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_end = 7433 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_start = 7436 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_end = 7633 + _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_start = 7635 + _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_end = 7691 + _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_start = 7709 + _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_end = 8538 + _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_start = 8240 + _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_end = 8319 + _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_start = 8322 + _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_end = 8527 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_start = 8541 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_end = 9617 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_start = 9149 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_end = 9276 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_start = 9278 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_end = 9393 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT" - ]._serialized_start = 9312 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT"]._serialized_end = 9371 + ]._serialized_start = 9395 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT"]._serialized_end = 9454 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE" - ]._serialized_start = 9373 + ]._serialized_start = 9456 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE" - ]._serialized_end = 9448 + ]._serialized_end = 9531 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT" - ]._serialized_start = 9450 + ]._serialized_start = 9533 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT" - ]._serialized_end = 9519 - _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_start = 9537 - _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_end = 9710 - _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_start = 9713 - _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_end = 9844 - _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_start = 9847 - _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_end = 10051 - _globals["_GETRESOURCESCOMMAND"]._serialized_start = 10053 - _globals["_GETRESOURCESCOMMAND"]._serialized_end = 10074 - _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_start = 10077 - _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_end = 10289 - _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_start = 10193 - _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_end = 10289 - _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_start = 10291 - _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_end = 10379 - _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_start = 10381 - _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_end = 10448 - _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_start = 10450 - _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_end = 10550 - _globals["_CHECKPOINTCOMMAND"]._serialized_start = 10553 - _globals["_CHECKPOINTCOMMAND"]._serialized_end = 10758 - _globals["_MERGEINTOTABLECOMMAND"]._serialized_start = 10761 - _globals["_MERGEINTOTABLECOMMAND"]._serialized_end = 11249 + ]._serialized_end = 9602 + _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_start = 9620 + _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_end = 9793 + _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_start = 9796 + _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_end = 9927 + _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_start = 9930 + _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_end = 10134 + _globals["_GETRESOURCESCOMMAND"]._serialized_start = 10136 + _globals["_GETRESOURCESCOMMAND"]._serialized_end = 10157 + _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_start = 10160 + _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_end = 10372 + _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_start = 10276 + _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_end = 10372 + _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_start = 10374 + _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_end = 10462 + _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_start = 10464 + _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_end = 10531 + _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_start = 10533 + _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_end = 10633 + _globals["_CHECKPOINTCOMMAND"]._serialized_start = 10636 + _globals["_CHECKPOINTCOMMAND"]._serialized_end = 10841 + _globals["_MERGEINTOTABLECOMMAND"]._serialized_start = 10844 + _globals["_MERGEINTOTABLECOMMAND"]._serialized_end = 11332 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index 6192a29607cbf..906f1aad10574 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -42,6 +42,7 @@ import google.protobuf.internal.enum_type_wrapper import google.protobuf.message import pyspark.sql.connect.proto.common_pb2 import pyspark.sql.connect.proto.expressions_pb2 +import pyspark.sql.connect.proto.ml_pb2 import pyspark.sql.connect.proto.relations_pb2 import sys import typing @@ -104,6 +105,7 @@ class Command(google.protobuf.message.Message): CHECKPOINT_COMMAND_FIELD_NUMBER: builtins.int REMOVE_CACHED_REMOTE_RELATION_COMMAND_FIELD_NUMBER: builtins.int MERGE_INTO_TABLE_COMMAND_FIELD_NUMBER: builtins.int + ML_COMMAND_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def register_function( @@ -146,6 +148,8 @@ class Command(google.protobuf.message.Message): @property def merge_into_table_command(self) -> global___MergeIntoTableCommand: ... @property + def ml_command(self) -> pyspark.sql.connect.proto.ml_pb2.MlCommand: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary Commands they can add them here. During the planning the correct resolution is done. @@ -174,6 +178,7 @@ class Command(google.protobuf.message.Message): remove_cached_remote_relation_command: global___RemoveCachedRemoteRelationCommand | None = ..., merge_into_table_command: global___MergeIntoTableCommand | None = ..., + ml_command: pyspark.sql.connect.proto.ml_pb2.MlCommand | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -193,6 +198,8 @@ class Command(google.protobuf.message.Message): b"get_resources_command", "merge_into_table_command", b"merge_into_table_command", + "ml_command", + b"ml_command", "register_data_source", b"register_data_source", "register_function", @@ -234,6 +241,8 @@ class Command(google.protobuf.message.Message): b"get_resources_command", "merge_into_table_command", b"merge_into_table_command", + "ml_command", + b"ml_command", "register_data_source", b"register_data_source", "register_function", @@ -278,6 +287,7 @@ class Command(google.protobuf.message.Message): "checkpoint_command", "remove_cached_remote_relation_command", "merge_into_table_command", + "ml_command", "extension", ] | None diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.py b/python/pyspark/sql/connect/proto/ml_common_pb2.py new file mode 100644 index 0000000000000..70e0e91652892 --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.py @@ -0,0 +1,80 @@ +# +# 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. +# +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# NO CHECKED-IN PROTOBUF GENCODE +# source: spark/connect/ml_common.proto +# Protobuf Python Version: 5.28.3 +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import runtime_version as _runtime_version +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import builder as _builder + +_runtime_version.ValidateProtobufRuntimeVersion( + _runtime_version.Domain.PUBLIC, 5, 28, 3, "", "spark/connect/ml_common.proto" +) +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x1dspark/connect/ml_common.proto\x12\rspark.connect\x1a\x1fspark/connect/expressions.proto"\x98\x01\n\x08MlParams\x12;\n\x06params\x18\x01 \x03(\x0b\x32#.spark.connect.MlParams.ParamsEntryR\x06params\x1aO\n\x0bParamsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12*\n\x05value\x18\x02 \x01(\x0b\x32\x14.spark.connect.ParamR\x05value:\x02\x38\x01"\xb6\x01\n\x05Param\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12/\n\x06vector\x18\x02 \x01(\x0b\x32\x15.spark.connect.VectorH\x00R\x06vector\x12/\n\x06matrix\x18\x03 \x01(\x0b\x32\x15.spark.connect.MatrixH\x00R\x06matrixB\x0c\n\nparam_type"\xc9\x01\n\nMlOperator\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x10\n\x03uid\x18\x02 \x01(\tR\x03uid\x12:\n\x04type\x18\x03 \x01(\x0e\x32&.spark.connect.MlOperator.OperatorTypeR\x04type"Y\n\x0cOperatorType\x12\x0f\n\x0bUNSPECIFIED\x10\x00\x12\r\n\tESTIMATOR\x10\x01\x12\x0f\n\x0bTRANSFORMER\x10\x02\x12\r\n\tEVALUATOR\x10\x03\x12\t\n\x05MODEL\x10\x04"\x1b\n\tObjectRef\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id"\xed\x01\n\x06Vector\x12\x33\n\x05\x64\x65nse\x18\x01 \x01(\x0b\x32\x1b.spark.connect.Vector.DenseH\x00R\x05\x64\x65nse\x12\x36\n\x06sparse\x18\x02 \x01(\x0b\x32\x1c.spark.connect.Vector.SparseH\x00R\x06sparse\x1a\x1d\n\x05\x44\x65nse\x12\x14\n\x05value\x18\x01 \x03(\x01R\x05value\x1aH\n\x06Sparse\x12\x12\n\x04size\x18\x01 \x01(\x05R\x04size\x12\x14\n\x05index\x18\x02 \x03(\x05R\x05index\x12\x14\n\x05value\x18\x03 \x03(\x01R\x05valueB\r\n\x0bvector_type"\xaf\x03\n\x06Matrix\x12\x33\n\x05\x64\x65nse\x18\x01 \x01(\x0b\x32\x1b.spark.connect.Matrix.DenseH\x00R\x05\x64\x65nse\x12\x36\n\x06sparse\x18\x02 \x01(\x0b\x32\x1c.spark.connect.Matrix.SparseH\x00R\x06sparse\x1ax\n\x05\x44\x65nse\x12\x19\n\x08num_rows\x18\x01 \x01(\x05R\x07numRows\x12\x19\n\x08num_cols\x18\x02 \x01(\x05R\x07numCols\x12\x14\n\x05value\x18\x03 \x03(\x01R\x05value\x12#\n\ris_transposed\x18\x04 \x01(\x08R\x0cisTransposed\x1a\xae\x01\n\x06Sparse\x12\x19\n\x08num_rows\x18\x01 \x01(\x05R\x07numRows\x12\x19\n\x08num_cols\x18\x02 \x01(\x05R\x07numCols\x12\x16\n\x06\x63olptr\x18\x03 \x03(\x05R\x06\x63olptr\x12\x1b\n\trow_index\x18\x04 \x03(\x05R\x08rowIndex\x12\x14\n\x05value\x18\x05 \x03(\x01R\x05value\x12#\n\ris_transposed\x18\x06 \x01(\x08R\x0cisTransposedB\r\n\x0bmatrix_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' +) + +_globals = globals() +_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) +_builder.BuildTopDescriptorsAndMessages( + DESCRIPTOR, "pyspark.sql.connect.proto.ml_common_pb2", _globals +) +if not _descriptor._USE_C_DESCRIPTORS: + _globals["DESCRIPTOR"]._loaded_options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" + _globals["_MLPARAMS_PARAMSENTRY"]._loaded_options = None + _globals["_MLPARAMS_PARAMSENTRY"]._serialized_options = b"8\001" + _globals["_MLPARAMS"]._serialized_start = 82 + _globals["_MLPARAMS"]._serialized_end = 234 + _globals["_MLPARAMS_PARAMSENTRY"]._serialized_start = 155 + _globals["_MLPARAMS_PARAMSENTRY"]._serialized_end = 234 + _globals["_PARAM"]._serialized_start = 237 + _globals["_PARAM"]._serialized_end = 419 + _globals["_MLOPERATOR"]._serialized_start = 422 + _globals["_MLOPERATOR"]._serialized_end = 623 + _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_start = 534 + _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_end = 623 + _globals["_OBJECTREF"]._serialized_start = 625 + _globals["_OBJECTREF"]._serialized_end = 652 + _globals["_VECTOR"]._serialized_start = 655 + _globals["_VECTOR"]._serialized_end = 892 + _globals["_VECTOR_DENSE"]._serialized_start = 774 + _globals["_VECTOR_DENSE"]._serialized_end = 803 + _globals["_VECTOR_SPARSE"]._serialized_start = 805 + _globals["_VECTOR_SPARSE"]._serialized_end = 877 + _globals["_MATRIX"]._serialized_start = 895 + _globals["_MATRIX"]._serialized_end = 1326 + _globals["_MATRIX_DENSE"]._serialized_start = 1014 + _globals["_MATRIX_DENSE"]._serialized_end = 1134 + _globals["_MATRIX_SPARSE"]._serialized_start = 1137 + _globals["_MATRIX_SPARSE"]._serialized_end = 1311 +# @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.pyi b/python/pyspark/sql/connect/proto/ml_common_pb2.pyi new file mode 100644 index 0000000000000..64029b6679f19 --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.pyi @@ -0,0 +1,427 @@ +# +# 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. +# +""" +@generated by mypy-protobuf. Do not edit manually! +isort:skip_file + +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. +""" +import builtins +import collections.abc +import google.protobuf.descriptor +import google.protobuf.internal.containers +import google.protobuf.internal.enum_type_wrapper +import google.protobuf.message +import pyspark.sql.connect.proto.expressions_pb2 +import sys +import typing + +if sys.version_info >= (3, 10): + import typing as typing_extensions +else: + import typing_extensions + +DESCRIPTOR: google.protobuf.descriptor.FileDescriptor + +class MlParams(google.protobuf.message.Message): + """MlParams stores param settings for ML Estimator / Transformer / Evaluator""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class ParamsEntry(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + @property + def value(self) -> global___Param: ... + def __init__( + self, + *, + key: builtins.str = ..., + value: global___Param | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["value", b"value"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + + PARAMS_FIELD_NUMBER: builtins.int + @property + def params( + self, + ) -> google.protobuf.internal.containers.MessageMap[builtins.str, global___Param]: + """User-supplied params""" + def __init__( + self, + *, + params: collections.abc.Mapping[builtins.str, global___Param] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["params", b"params"]) -> None: ... + +global___MlParams = MlParams + +class Param(google.protobuf.message.Message): + """Represents the parameter type of the ML instance, or the returned value + of the attribute + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + LITERAL_FIELD_NUMBER: builtins.int + VECTOR_FIELD_NUMBER: builtins.int + MATRIX_FIELD_NUMBER: builtins.int + @property + def literal(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression.Literal: ... + @property + def vector(self) -> global___Vector: ... + @property + def matrix(self) -> global___Matrix: ... + def __init__( + self, + *, + literal: pyspark.sql.connect.proto.expressions_pb2.Expression.Literal | None = ..., + vector: global___Vector | None = ..., + matrix: global___Matrix | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "literal", + b"literal", + "matrix", + b"matrix", + "param_type", + b"param_type", + "vector", + b"vector", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "literal", + b"literal", + "matrix", + b"matrix", + "param_type", + b"param_type", + "vector", + b"vector", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["param_type", b"param_type"] + ) -> typing_extensions.Literal["literal", "vector", "matrix"] | None: ... + +global___Param = Param + +class MlOperator(google.protobuf.message.Message): + """MLOperator represents the ML operators like (Estimator, Transformer or Evaluator)""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _OperatorType: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _OperatorTypeEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + MlOperator._OperatorType.ValueType + ], + builtins.type, + ): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + UNSPECIFIED: MlOperator._OperatorType.ValueType # 0 + ESTIMATOR: MlOperator._OperatorType.ValueType # 1 + TRANSFORMER: MlOperator._OperatorType.ValueType # 2 + EVALUATOR: MlOperator._OperatorType.ValueType # 3 + MODEL: MlOperator._OperatorType.ValueType # 4 + + class OperatorType(_OperatorType, metaclass=_OperatorTypeEnumTypeWrapper): ... + UNSPECIFIED: MlOperator.OperatorType.ValueType # 0 + ESTIMATOR: MlOperator.OperatorType.ValueType # 1 + TRANSFORMER: MlOperator.OperatorType.ValueType # 2 + EVALUATOR: MlOperator.OperatorType.ValueType # 3 + MODEL: MlOperator.OperatorType.ValueType # 4 + + NAME_FIELD_NUMBER: builtins.int + UID_FIELD_NUMBER: builtins.int + TYPE_FIELD_NUMBER: builtins.int + name: builtins.str + """The qualified name of the ML operator.""" + uid: builtins.str + """Unique id of the ML operator""" + type: global___MlOperator.OperatorType.ValueType + """Represents what the ML operator is""" + def __init__( + self, + *, + name: builtins.str = ..., + uid: builtins.str = ..., + type: global___MlOperator.OperatorType.ValueType = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["name", b"name", "type", b"type", "uid", b"uid"] + ) -> None: ... + +global___MlOperator = MlOperator + +class ObjectRef(google.protobuf.message.Message): + """Represents a reference to the cached object which could be a model + or summary evaluated by a model + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + ID_FIELD_NUMBER: builtins.int + id: builtins.str + """The ID is used to lookup the object on the server side.""" + def __init__( + self, + *, + id: builtins.str = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["id", b"id"]) -> None: ... + +global___ObjectRef = ObjectRef + +class Vector(google.protobuf.message.Message): + """See pyspark.ml.linalg.Vector""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Dense(google.protobuf.message.Message): + """See pyspark.ml.linalg.DenseVector""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUE_FIELD_NUMBER: builtins.int + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + def __init__( + self, + *, + value: collections.abc.Iterable[builtins.float] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["value", b"value"]) -> None: ... + + class Sparse(google.protobuf.message.Message): + """See pyspark.ml.linalg.SparseVector""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SIZE_FIELD_NUMBER: builtins.int + INDEX_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + size: builtins.int + @property + def index( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + def __init__( + self, + *, + size: builtins.int = ..., + index: collections.abc.Iterable[builtins.int] | None = ..., + value: collections.abc.Iterable[builtins.float] | None = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "index", b"index", "size", b"size", "value", b"value" + ], + ) -> None: ... + + DENSE_FIELD_NUMBER: builtins.int + SPARSE_FIELD_NUMBER: builtins.int + @property + def dense(self) -> global___Vector.Dense: ... + @property + def sparse(self) -> global___Vector.Sparse: ... + def __init__( + self, + *, + dense: global___Vector.Dense | None = ..., + sparse: global___Vector.Sparse | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "sparse", b"sparse", "vector_type", b"vector_type" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "sparse", b"sparse", "vector_type", b"vector_type" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["vector_type", b"vector_type"] + ) -> typing_extensions.Literal["dense", "sparse"] | None: ... + +global___Vector = Vector + +class Matrix(google.protobuf.message.Message): + """See pyspark.ml.linalg.Matrix""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Dense(google.protobuf.message.Message): + """See pyspark.ml.linalg.DenseMatrix""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NUM_ROWS_FIELD_NUMBER: builtins.int + NUM_COLS_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + IS_TRANSPOSED_FIELD_NUMBER: builtins.int + num_rows: builtins.int + num_cols: builtins.int + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + is_transposed: builtins.bool + def __init__( + self, + *, + num_rows: builtins.int = ..., + num_cols: builtins.int = ..., + value: collections.abc.Iterable[builtins.float] | None = ..., + is_transposed: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "is_transposed", + b"is_transposed", + "num_cols", + b"num_cols", + "num_rows", + b"num_rows", + "value", + b"value", + ], + ) -> None: ... + + class Sparse(google.protobuf.message.Message): + """See pyspark.ml.linalg.SparseMatrix""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NUM_ROWS_FIELD_NUMBER: builtins.int + NUM_COLS_FIELD_NUMBER: builtins.int + COLPTR_FIELD_NUMBER: builtins.int + ROW_INDEX_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + IS_TRANSPOSED_FIELD_NUMBER: builtins.int + num_rows: builtins.int + num_cols: builtins.int + @property + def colptr( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + @property + def row_index( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + is_transposed: builtins.bool + def __init__( + self, + *, + num_rows: builtins.int = ..., + num_cols: builtins.int = ..., + colptr: collections.abc.Iterable[builtins.int] | None = ..., + row_index: collections.abc.Iterable[builtins.int] | None = ..., + value: collections.abc.Iterable[builtins.float] | None = ..., + is_transposed: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "colptr", + b"colptr", + "is_transposed", + b"is_transposed", + "num_cols", + b"num_cols", + "num_rows", + b"num_rows", + "row_index", + b"row_index", + "value", + b"value", + ], + ) -> None: ... + + DENSE_FIELD_NUMBER: builtins.int + SPARSE_FIELD_NUMBER: builtins.int + @property + def dense(self) -> global___Matrix.Dense: ... + @property + def sparse(self) -> global___Matrix.Sparse: ... + def __init__( + self, + *, + dense: global___Matrix.Dense | None = ..., + sparse: global___Matrix.Sparse | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "matrix_type", b"matrix_type", "sparse", b"sparse" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "matrix_type", b"matrix_type", "sparse", b"sparse" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["matrix_type", b"matrix_type"] + ) -> typing_extensions.Literal["dense", "sparse"] | None: ... + +global___Matrix = Matrix diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py new file mode 100644 index 0000000000000..46692010e3ffc --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_pb2.py @@ -0,0 +1,72 @@ +# +# 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. +# +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# NO CHECKED-IN PROTOBUF GENCODE +# source: spark/connect/ml.proto +# Protobuf Python Version: 5.28.3 +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import runtime_version as _runtime_version +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import builder as _builder + +_runtime_version.ValidateProtobufRuntimeVersion( + _runtime_version.Domain.PUBLIC, 5, 28, 3, "", "spark/connect/ml.proto" +) +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 +from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 +from pyspark.sql.connect.proto import ml_common_pb2 as spark_dot_connect_dot_ml__common__pb2 + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x1dspark/connect/ml_common.proto"\xc6\x07\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x1a\xa2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12/\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61taset\x1a;\n\x06\x44\x65lete\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x1a\xf0\x02\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12/\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12)\n\x10should_overwrite\x18\x05 \x01(\x08R\x0fshouldOverwrite\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x06\n\x04type\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04pathB\t\n\x07\x63ommand"\xe9\x02\n\x0fMlCommandResult\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\xa6\x01\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x10\n\x03uid\x18\x03 \x01(\tR\x03uid\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\x06\n\x04typeB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' +) + +_globals = globals() +_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "pyspark.sql.connect.proto.ml_pb2", _globals) +if not _descriptor._USE_C_DESCRIPTORS: + _globals["DESCRIPTOR"]._loaded_options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._loaded_options = None + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_options = b"8\001" + _globals["_MLCOMMAND"]._serialized_start = 137 + _globals["_MLCOMMAND"]._serialized_end = 1103 + _globals["_MLCOMMAND_FIT"]._serialized_start = 415 + _globals["_MLCOMMAND_FIT"]._serialized_end = 577 + _globals["_MLCOMMAND_DELETE"]._serialized_start = 579 + _globals["_MLCOMMAND_DELETE"]._serialized_end = 638 + _globals["_MLCOMMAND_WRITE"]._serialized_start = 641 + _globals["_MLCOMMAND_WRITE"]._serialized_end = 1009 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 943 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 1001 + _globals["_MLCOMMAND_READ"]._serialized_start = 1011 + _globals["_MLCOMMAND_READ"]._serialized_end = 1092 + _globals["_MLCOMMANDRESULT"]._serialized_start = 1106 + _globals["_MLCOMMANDRESULT"]._serialized_end = 1467 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 1286 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 1452 +# @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/ml_pb2.pyi b/python/pyspark/sql/connect/proto/ml_pb2.pyi new file mode 100644 index 0000000000000..95bfefb524e2a --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_pb2.pyi @@ -0,0 +1,393 @@ +# +# 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. +# +""" +@generated by mypy-protobuf. Do not edit manually! +isort:skip_file + +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. +""" +import builtins +import collections.abc +import google.protobuf.descriptor +import google.protobuf.internal.containers +import google.protobuf.message +import pyspark.sql.connect.proto.ml_common_pb2 +import pyspark.sql.connect.proto.relations_pb2 +import sys + +if sys.version_info >= (3, 8): + import typing as typing_extensions +else: + import typing_extensions + +DESCRIPTOR: google.protobuf.descriptor.FileDescriptor + +class MlCommand(google.protobuf.message.Message): + """Command for ML""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Fit(google.protobuf.message.Message): + """Command for estimator.fit(dataset)""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + ESTIMATOR_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + DATASET_FIELD_NUMBER: builtins.int + @property + def estimator(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """Estimator information""" + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: + """parameters of the Estimator""" + @property + def dataset(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: + """the training dataset""" + def __init__( + self, + *, + estimator: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + dataset: pyspark.sql.connect.proto.relations_pb2.Relation | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "dataset", b"dataset", "estimator", b"estimator", "params", b"params" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "dataset", b"dataset", "estimator", b"estimator", "params", b"params" + ], + ) -> None: ... + + class Delete(google.protobuf.message.Message): + """Command to delete the cached object which could be a model + or summary evaluated by a model + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OBJ_REF_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: ... + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["obj_ref", b"obj_ref"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["obj_ref", b"obj_ref"] + ) -> None: ... + + class Write(google.protobuf.message.Message): + """Command to write ML operator""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class OptionsEntry(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + value: builtins.str + def __init__( + self, + *, + key: builtins.str = ..., + value: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + + OPERATOR_FIELD_NUMBER: builtins.int + OBJ_REF_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + PATH_FIELD_NUMBER: builtins.int + SHOULD_OVERWRITE_FIELD_NUMBER: builtins.int + OPTIONS_FIELD_NUMBER: builtins.int + @property + def operator(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """Estimator or evaluator""" + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """The cached model""" + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: + """The parameters of operator which could be estimator/evaluator or a cached model""" + path: builtins.str + """Save the ML instance to the path""" + should_overwrite: builtins.bool + """Overwrites if the output path already exists.""" + @property + def options( + self, + ) -> google.protobuf.internal.containers.ScalarMap[builtins.str, builtins.str]: + """The options of the writer""" + def __init__( + self, + *, + operator: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + path: builtins.str = ..., + should_overwrite: builtins.bool = ..., + options: collections.abc.Mapping[builtins.str, builtins.str] | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "obj_ref", b"obj_ref", "operator", b"operator", "params", b"params", "type", b"type" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "obj_ref", + b"obj_ref", + "operator", + b"operator", + "options", + b"options", + "params", + b"params", + "path", + b"path", + "should_overwrite", + b"should_overwrite", + "type", + b"type", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["type", b"type"] + ) -> typing_extensions.Literal["operator", "obj_ref"] | None: ... + + class Read(google.protobuf.message.Message): + """Command to load ML operator.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OPERATOR_FIELD_NUMBER: builtins.int + PATH_FIELD_NUMBER: builtins.int + @property + def operator(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """ML operator information""" + path: builtins.str + """Load the ML instance from the input path""" + def __init__( + self, + *, + operator: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + path: builtins.str = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["operator", b"operator"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["operator", b"operator", "path", b"path"] + ) -> None: ... + + FIT_FIELD_NUMBER: builtins.int + FETCH_FIELD_NUMBER: builtins.int + DELETE_FIELD_NUMBER: builtins.int + WRITE_FIELD_NUMBER: builtins.int + READ_FIELD_NUMBER: builtins.int + @property + def fit(self) -> global___MlCommand.Fit: ... + @property + def fetch(self) -> pyspark.sql.connect.proto.relations_pb2.Fetch: ... + @property + def delete(self) -> global___MlCommand.Delete: ... + @property + def write(self) -> global___MlCommand.Write: ... + @property + def read(self) -> global___MlCommand.Read: ... + def __init__( + self, + *, + fit: global___MlCommand.Fit | None = ..., + fetch: pyspark.sql.connect.proto.relations_pb2.Fetch | None = ..., + delete: global___MlCommand.Delete | None = ..., + write: global___MlCommand.Write | None = ..., + read: global___MlCommand.Read | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "command", + b"command", + "delete", + b"delete", + "fetch", + b"fetch", + "fit", + b"fit", + "read", + b"read", + "write", + b"write", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "command", + b"command", + "delete", + b"delete", + "fetch", + b"fetch", + "fit", + b"fit", + "read", + b"read", + "write", + b"write", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["command", b"command"] + ) -> typing_extensions.Literal["fit", "fetch", "delete", "write", "read"] | None: ... + +global___MlCommand = MlCommand + +class MlCommandResult(google.protobuf.message.Message): + """The result of MlCommand""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class MlOperatorInfo(google.protobuf.message.Message): + """Represents an operator info""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OBJ_REF_FIELD_NUMBER: builtins.int + NAME_FIELD_NUMBER: builtins.int + UID_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """The cached object which could be a model or summary evaluated by a model""" + name: builtins.str + """Operator name""" + uid: builtins.str + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: ... + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + name: builtins.str = ..., + uid: builtins.str = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "name", b"name", "obj_ref", b"obj_ref", "params", b"params", "type", b"type" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "name", + b"name", + "obj_ref", + b"obj_ref", + "params", + b"params", + "type", + b"type", + "uid", + b"uid", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["type", b"type"] + ) -> typing_extensions.Literal["obj_ref", "name"] | None: ... + + PARAM_FIELD_NUMBER: builtins.int + SUMMARY_FIELD_NUMBER: builtins.int + OPERATOR_INFO_FIELD_NUMBER: builtins.int + @property + def param(self) -> pyspark.sql.connect.proto.ml_common_pb2.Param: + """The result of the attribute""" + summary: builtins.str + """Evaluate a Dataset in a model and return the cached ID of summary""" + @property + def operator_info(self) -> global___MlCommandResult.MlOperatorInfo: + """Operator information""" + def __init__( + self, + *, + param: pyspark.sql.connect.proto.ml_common_pb2.Param | None = ..., + summary: builtins.str = ..., + operator_info: global___MlCommandResult.MlOperatorInfo | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "operator_info", + b"operator_info", + "param", + b"param", + "result_type", + b"result_type", + "summary", + b"summary", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "operator_info", + b"operator_info", + "param", + b"param", + "result_type", + b"result_type", + "summary", + b"summary", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["result_type", b"result_type"] + ) -> typing_extensions.Literal["param", "summary", "operator_info"] | None: ... + +global___MlCommandResult = MlCommandResult diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index b7248d4b17080..4327d0240b355 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -39,10 +39,11 @@ from pyspark.sql.connect.proto import types_pb2 as spark_dot_connect_dot_types__pb2 from pyspark.sql.connect.proto import catalog_pb2 as spark_dot_connect_dot_catalog__pb2 from pyspark.sql.connect.proto import common_pb2 as spark_dot_connect_dot_common__pb2 +from pyspark.sql.connect.proto import ml_common_pb2 as spark_dot_connect_dot_ml__common__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto"\xdd\x1c\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xbe\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xca\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1ar\n\x04\x41rgs\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -77,160 +78,170 @@ ]._serialized_options = b"\030\001" _globals["_PARSE_OPTIONSENTRY"]._loaded_options = None _globals["_PARSE_OPTIONSENTRY"]._serialized_options = b"8\001" - _globals["_RELATION"]._serialized_start = 193 - _globals["_RELATION"]._serialized_end = 3870 - _globals["_UNKNOWN"]._serialized_start = 3872 - _globals["_UNKNOWN"]._serialized_end = 3881 - _globals["_RELATIONCOMMON"]._serialized_start = 3884 - _globals["_RELATIONCOMMON"]._serialized_end = 4026 - _globals["_SQL"]._serialized_start = 4029 - _globals["_SQL"]._serialized_end = 4507 - _globals["_SQL_ARGSENTRY"]._serialized_start = 4323 - _globals["_SQL_ARGSENTRY"]._serialized_end = 4413 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 4415 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 4507 - _globals["_WITHRELATIONS"]._serialized_start = 4509 - _globals["_WITHRELATIONS"]._serialized_end = 4626 - _globals["_READ"]._serialized_start = 4629 - _globals["_READ"]._serialized_end = 5292 - _globals["_READ_NAMEDTABLE"]._serialized_start = 4807 - _globals["_READ_NAMEDTABLE"]._serialized_end = 4999 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 4941 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 4999 - _globals["_READ_DATASOURCE"]._serialized_start = 5002 - _globals["_READ_DATASOURCE"]._serialized_end = 5279 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 4941 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 4999 - _globals["_PROJECT"]._serialized_start = 5294 - _globals["_PROJECT"]._serialized_end = 5411 - _globals["_FILTER"]._serialized_start = 5413 - _globals["_FILTER"]._serialized_end = 5525 - _globals["_JOIN"]._serialized_start = 5528 - _globals["_JOIN"]._serialized_end = 6189 - _globals["_JOIN_JOINDATATYPE"]._serialized_start = 5867 - _globals["_JOIN_JOINDATATYPE"]._serialized_end = 5959 - _globals["_JOIN_JOINTYPE"]._serialized_start = 5962 - _globals["_JOIN_JOINTYPE"]._serialized_end = 6170 - _globals["_SETOPERATION"]._serialized_start = 6192 - _globals["_SETOPERATION"]._serialized_end = 6671 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 6508 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 6622 - _globals["_LIMIT"]._serialized_start = 6673 - _globals["_LIMIT"]._serialized_end = 6749 - _globals["_OFFSET"]._serialized_start = 6751 - _globals["_OFFSET"]._serialized_end = 6830 - _globals["_TAIL"]._serialized_start = 6832 - _globals["_TAIL"]._serialized_end = 6907 - _globals["_AGGREGATE"]._serialized_start = 6910 - _globals["_AGGREGATE"]._serialized_end = 7676 - _globals["_AGGREGATE_PIVOT"]._serialized_start = 7325 - _globals["_AGGREGATE_PIVOT"]._serialized_end = 7436 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 7438 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 7514 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 7517 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 7676 - _globals["_SORT"]._serialized_start = 7679 - _globals["_SORT"]._serialized_end = 7839 - _globals["_DROP"]._serialized_start = 7842 - _globals["_DROP"]._serialized_end = 7983 - _globals["_DEDUPLICATE"]._serialized_start = 7986 - _globals["_DEDUPLICATE"]._serialized_end = 8226 - _globals["_LOCALRELATION"]._serialized_start = 8228 - _globals["_LOCALRELATION"]._serialized_end = 8317 - _globals["_CACHEDLOCALRELATION"]._serialized_start = 8319 - _globals["_CACHEDLOCALRELATION"]._serialized_end = 8391 - _globals["_CACHEDREMOTERELATION"]._serialized_start = 8393 - _globals["_CACHEDREMOTERELATION"]._serialized_end = 8448 - _globals["_SAMPLE"]._serialized_start = 8451 - _globals["_SAMPLE"]._serialized_end = 8724 - _globals["_RANGE"]._serialized_start = 8727 - _globals["_RANGE"]._serialized_end = 8872 - _globals["_SUBQUERYALIAS"]._serialized_start = 8874 - _globals["_SUBQUERYALIAS"]._serialized_end = 8988 - _globals["_REPARTITION"]._serialized_start = 8991 - _globals["_REPARTITION"]._serialized_end = 9133 - _globals["_SHOWSTRING"]._serialized_start = 9136 - _globals["_SHOWSTRING"]._serialized_end = 9278 - _globals["_HTMLSTRING"]._serialized_start = 9280 - _globals["_HTMLSTRING"]._serialized_end = 9394 - _globals["_STATSUMMARY"]._serialized_start = 9396 - _globals["_STATSUMMARY"]._serialized_end = 9488 - _globals["_STATDESCRIBE"]._serialized_start = 9490 - _globals["_STATDESCRIBE"]._serialized_end = 9571 - _globals["_STATCROSSTAB"]._serialized_start = 9573 - _globals["_STATCROSSTAB"]._serialized_end = 9674 - _globals["_STATCOV"]._serialized_start = 9676 - _globals["_STATCOV"]._serialized_end = 9772 - _globals["_STATCORR"]._serialized_start = 9775 - _globals["_STATCORR"]._serialized_end = 9912 - _globals["_STATAPPROXQUANTILE"]._serialized_start = 9915 - _globals["_STATAPPROXQUANTILE"]._serialized_end = 10079 - _globals["_STATFREQITEMS"]._serialized_start = 10081 - _globals["_STATFREQITEMS"]._serialized_end = 10206 - _globals["_STATSAMPLEBY"]._serialized_start = 10209 - _globals["_STATSAMPLEBY"]._serialized_end = 10518 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 10410 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 10509 - _globals["_NAFILL"]._serialized_start = 10521 - _globals["_NAFILL"]._serialized_end = 10655 - _globals["_NADROP"]._serialized_start = 10658 - _globals["_NADROP"]._serialized_end = 10792 - _globals["_NAREPLACE"]._serialized_start = 10795 - _globals["_NAREPLACE"]._serialized_end = 11091 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 10950 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11091 - _globals["_TODF"]._serialized_start = 11093 - _globals["_TODF"]._serialized_end = 11181 - _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11184 - _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 11566 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 11428 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 11495 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 11497 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 11566 - _globals["_WITHCOLUMNS"]._serialized_start = 11568 - _globals["_WITHCOLUMNS"]._serialized_end = 11687 - _globals["_WITHWATERMARK"]._serialized_start = 11690 - _globals["_WITHWATERMARK"]._serialized_end = 11824 - _globals["_HINT"]._serialized_start = 11827 - _globals["_HINT"]._serialized_end = 11959 - _globals["_UNPIVOT"]._serialized_start = 11962 - _globals["_UNPIVOT"]._serialized_end = 12289 - _globals["_UNPIVOT_VALUES"]._serialized_start = 12219 - _globals["_UNPIVOT_VALUES"]._serialized_end = 12278 - _globals["_TRANSPOSE"]._serialized_start = 12291 - _globals["_TRANSPOSE"]._serialized_end = 12413 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 12415 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 12540 - _globals["_TOSCHEMA"]._serialized_start = 12542 - _globals["_TOSCHEMA"]._serialized_end = 12648 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 12651 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 12854 - _globals["_MAPPARTITIONS"]._serialized_start = 12857 - _globals["_MAPPARTITIONS"]._serialized_end = 13089 - _globals["_GROUPMAP"]._serialized_start = 13092 - _globals["_GROUPMAP"]._serialized_end = 13809 - _globals["_COGROUPMAP"]._serialized_start = 13812 - _globals["_COGROUPMAP"]._serialized_end = 14338 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 14341 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 14698 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 14701 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 14945 - _globals["_PYTHONUDTF"]._serialized_start = 14948 - _globals["_PYTHONUDTF"]._serialized_end = 15125 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15128 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 15279 - _globals["_PYTHONDATASOURCE"]._serialized_start = 15281 - _globals["_PYTHONDATASOURCE"]._serialized_end = 15356 - _globals["_COLLECTMETRICS"]._serialized_start = 15359 - _globals["_COLLECTMETRICS"]._serialized_end = 15495 - _globals["_PARSE"]._serialized_start = 15498 - _globals["_PARSE"]._serialized_end = 15886 - _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 4941 - _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 4999 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 15787 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 15875 - _globals["_ASOFJOIN"]._serialized_start = 15889 - _globals["_ASOFJOIN"]._serialized_end = 16364 - _globals["_LATERALJOIN"]._serialized_start = 16367 - _globals["_LATERALJOIN"]._serialized_end = 16597 + _globals["_RELATION"]._serialized_start = 224 + _globals["_RELATION"]._serialized_end = 3964 + _globals["_MLRELATION"]._serialized_start = 3967 + _globals["_MLRELATION"]._serialized_end = 4343 + _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4097 + _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4332 + _globals["_FETCH"]._serialized_start = 4346 + _globals["_FETCH"]._serialized_end = 4664 + _globals["_FETCH_METHOD"]._serialized_start = 4462 + _globals["_FETCH_METHOD"]._serialized_end = 4664 + _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4550 + _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4664 + _globals["_UNKNOWN"]._serialized_start = 4666 + _globals["_UNKNOWN"]._serialized_end = 4675 + _globals["_RELATIONCOMMON"]._serialized_start = 4678 + _globals["_RELATIONCOMMON"]._serialized_end = 4820 + _globals["_SQL"]._serialized_start = 4823 + _globals["_SQL"]._serialized_end = 5301 + _globals["_SQL_ARGSENTRY"]._serialized_start = 5117 + _globals["_SQL_ARGSENTRY"]._serialized_end = 5207 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5209 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5301 + _globals["_WITHRELATIONS"]._serialized_start = 5303 + _globals["_WITHRELATIONS"]._serialized_end = 5420 + _globals["_READ"]._serialized_start = 5423 + _globals["_READ"]._serialized_end = 6086 + _globals["_READ_NAMEDTABLE"]._serialized_start = 5601 + _globals["_READ_NAMEDTABLE"]._serialized_end = 5793 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5735 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5793 + _globals["_READ_DATASOURCE"]._serialized_start = 5796 + _globals["_READ_DATASOURCE"]._serialized_end = 6073 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5735 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5793 + _globals["_PROJECT"]._serialized_start = 6088 + _globals["_PROJECT"]._serialized_end = 6205 + _globals["_FILTER"]._serialized_start = 6207 + _globals["_FILTER"]._serialized_end = 6319 + _globals["_JOIN"]._serialized_start = 6322 + _globals["_JOIN"]._serialized_end = 6983 + _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6661 + _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6753 + _globals["_JOIN_JOINTYPE"]._serialized_start = 6756 + _globals["_JOIN_JOINTYPE"]._serialized_end = 6964 + _globals["_SETOPERATION"]._serialized_start = 6986 + _globals["_SETOPERATION"]._serialized_end = 7465 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7302 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7416 + _globals["_LIMIT"]._serialized_start = 7467 + _globals["_LIMIT"]._serialized_end = 7543 + _globals["_OFFSET"]._serialized_start = 7545 + _globals["_OFFSET"]._serialized_end = 7624 + _globals["_TAIL"]._serialized_start = 7626 + _globals["_TAIL"]._serialized_end = 7701 + _globals["_AGGREGATE"]._serialized_start = 7704 + _globals["_AGGREGATE"]._serialized_end = 8470 + _globals["_AGGREGATE_PIVOT"]._serialized_start = 8119 + _globals["_AGGREGATE_PIVOT"]._serialized_end = 8230 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8232 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8308 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8311 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8470 + _globals["_SORT"]._serialized_start = 8473 + _globals["_SORT"]._serialized_end = 8633 + _globals["_DROP"]._serialized_start = 8636 + _globals["_DROP"]._serialized_end = 8777 + _globals["_DEDUPLICATE"]._serialized_start = 8780 + _globals["_DEDUPLICATE"]._serialized_end = 9020 + _globals["_LOCALRELATION"]._serialized_start = 9022 + _globals["_LOCALRELATION"]._serialized_end = 9111 + _globals["_CACHEDLOCALRELATION"]._serialized_start = 9113 + _globals["_CACHEDLOCALRELATION"]._serialized_end = 9185 + _globals["_CACHEDREMOTERELATION"]._serialized_start = 9187 + _globals["_CACHEDREMOTERELATION"]._serialized_end = 9242 + _globals["_SAMPLE"]._serialized_start = 9245 + _globals["_SAMPLE"]._serialized_end = 9518 + _globals["_RANGE"]._serialized_start = 9521 + _globals["_RANGE"]._serialized_end = 9666 + _globals["_SUBQUERYALIAS"]._serialized_start = 9668 + _globals["_SUBQUERYALIAS"]._serialized_end = 9782 + _globals["_REPARTITION"]._serialized_start = 9785 + _globals["_REPARTITION"]._serialized_end = 9927 + _globals["_SHOWSTRING"]._serialized_start = 9930 + _globals["_SHOWSTRING"]._serialized_end = 10072 + _globals["_HTMLSTRING"]._serialized_start = 10074 + _globals["_HTMLSTRING"]._serialized_end = 10188 + _globals["_STATSUMMARY"]._serialized_start = 10190 + _globals["_STATSUMMARY"]._serialized_end = 10282 + _globals["_STATDESCRIBE"]._serialized_start = 10284 + _globals["_STATDESCRIBE"]._serialized_end = 10365 + _globals["_STATCROSSTAB"]._serialized_start = 10367 + _globals["_STATCROSSTAB"]._serialized_end = 10468 + _globals["_STATCOV"]._serialized_start = 10470 + _globals["_STATCOV"]._serialized_end = 10566 + _globals["_STATCORR"]._serialized_start = 10569 + _globals["_STATCORR"]._serialized_end = 10706 + _globals["_STATAPPROXQUANTILE"]._serialized_start = 10709 + _globals["_STATAPPROXQUANTILE"]._serialized_end = 10873 + _globals["_STATFREQITEMS"]._serialized_start = 10875 + _globals["_STATFREQITEMS"]._serialized_end = 11000 + _globals["_STATSAMPLEBY"]._serialized_start = 11003 + _globals["_STATSAMPLEBY"]._serialized_end = 11312 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11204 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11303 + _globals["_NAFILL"]._serialized_start = 11315 + _globals["_NAFILL"]._serialized_end = 11449 + _globals["_NADROP"]._serialized_start = 11452 + _globals["_NADROP"]._serialized_end = 11586 + _globals["_NAREPLACE"]._serialized_start = 11589 + _globals["_NAREPLACE"]._serialized_end = 11885 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11744 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11885 + _globals["_TODF"]._serialized_start = 11887 + _globals["_TODF"]._serialized_end = 11975 + _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11978 + _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12360 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12222 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12289 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12291 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12360 + _globals["_WITHCOLUMNS"]._serialized_start = 12362 + _globals["_WITHCOLUMNS"]._serialized_end = 12481 + _globals["_WITHWATERMARK"]._serialized_start = 12484 + _globals["_WITHWATERMARK"]._serialized_end = 12618 + _globals["_HINT"]._serialized_start = 12621 + _globals["_HINT"]._serialized_end = 12753 + _globals["_UNPIVOT"]._serialized_start = 12756 + _globals["_UNPIVOT"]._serialized_end = 13083 + _globals["_UNPIVOT_VALUES"]._serialized_start = 13013 + _globals["_UNPIVOT_VALUES"]._serialized_end = 13072 + _globals["_TRANSPOSE"]._serialized_start = 13085 + _globals["_TRANSPOSE"]._serialized_end = 13207 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13209 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13334 + _globals["_TOSCHEMA"]._serialized_start = 13336 + _globals["_TOSCHEMA"]._serialized_end = 13442 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13445 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13648 + _globals["_MAPPARTITIONS"]._serialized_start = 13651 + _globals["_MAPPARTITIONS"]._serialized_end = 13883 + _globals["_GROUPMAP"]._serialized_start = 13886 + _globals["_GROUPMAP"]._serialized_end = 14603 + _globals["_COGROUPMAP"]._serialized_start = 14606 + _globals["_COGROUPMAP"]._serialized_end = 15132 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15135 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15492 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15495 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 15739 + _globals["_PYTHONUDTF"]._serialized_start = 15742 + _globals["_PYTHONUDTF"]._serialized_end = 15919 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15922 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16073 + _globals["_PYTHONDATASOURCE"]._serialized_start = 16075 + _globals["_PYTHONDATASOURCE"]._serialized_end = 16150 + _globals["_COLLECTMETRICS"]._serialized_start = 16153 + _globals["_COLLECTMETRICS"]._serialized_end = 16289 + _globals["_PARSE"]._serialized_start = 16292 + _globals["_PARSE"]._serialized_end = 16680 + _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5735 + _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5793 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 16581 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 16669 + _globals["_ASOFJOIN"]._serialized_start = 16683 + _globals["_ASOFJOIN"]._serialized_end = 17158 + _globals["_LATERALJOIN"]._serialized_start = 17161 + _globals["_LATERALJOIN"]._serialized_end = 17391 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index 371d735b9e871..0c8cf8dd3eda8 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -43,6 +43,7 @@ import google.protobuf.message import pyspark.sql.connect.proto.catalog_pb2 import pyspark.sql.connect.proto.common_pb2 import pyspark.sql.connect.proto.expressions_pb2 +import pyspark.sql.connect.proto.ml_common_pb2 import pyspark.sql.connect.proto.types_pb2 import sys import typing @@ -119,6 +120,7 @@ class Relation(google.protobuf.message.Message): FREQ_ITEMS_FIELD_NUMBER: builtins.int SAMPLE_BY_FIELD_NUMBER: builtins.int CATALOG_FIELD_NUMBER: builtins.int + ML_RELATION_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int UNKNOWN_FIELD_NUMBER: builtins.int @property @@ -241,6 +243,9 @@ class Relation(google.protobuf.message.Message): def catalog(self) -> pyspark.sql.connect.proto.catalog_pb2.Catalog: """Catalog API (experimental / unstable)""" @property + def ml_relation(self) -> global___MlRelation: + """ML relation""" + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary relations they can add them here. During the planning the correct resolution is done. @@ -308,6 +313,7 @@ class Relation(google.protobuf.message.Message): freq_items: global___StatFreqItems | None = ..., sample_by: global___StatSampleBy | None = ..., catalog: pyspark.sql.connect.proto.catalog_pb2.Catalog | None = ..., + ml_relation: global___MlRelation | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., unknown: global___Unknown | None = ..., ) -> None: ... @@ -376,6 +382,8 @@ class Relation(google.protobuf.message.Message): b"local_relation", "map_partitions", b"map_partitions", + "ml_relation", + b"ml_relation", "offset", b"offset", "parse", @@ -499,6 +507,8 @@ class Relation(google.protobuf.message.Message): b"local_relation", "map_partitions", b"map_partitions", + "ml_relation", + b"ml_relation", "offset", b"offset", "parse", @@ -616,6 +626,7 @@ class Relation(google.protobuf.message.Message): "freq_items", "sample_by", "catalog", + "ml_relation", "extension", "unknown", ] @@ -624,6 +635,198 @@ class Relation(google.protobuf.message.Message): global___Relation = Relation +class MlRelation(google.protobuf.message.Message): + """Relation to represent ML world""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Transform(google.protobuf.message.Message): + """Relation to represent transform(input) of the operator + which could be a cached model or a new transformer + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OBJ_REF_FIELD_NUMBER: builtins.int + TRANSFORMER_FIELD_NUMBER: builtins.int + INPUT_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """Object reference""" + @property + def transformer(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """Could be an ML transformer like VectorAssembler""" + @property + def input(self) -> global___Relation: + """the input dataframe""" + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: + """the operator specific parameters""" + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + transformer: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + input: global___Relation | None = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "input", + b"input", + "obj_ref", + b"obj_ref", + "operator", + b"operator", + "params", + b"params", + "transformer", + b"transformer", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "input", + b"input", + "obj_ref", + b"obj_ref", + "operator", + b"operator", + "params", + b"params", + "transformer", + b"transformer", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["operator", b"operator"] + ) -> typing_extensions.Literal["obj_ref", "transformer"] | None: ... + + TRANSFORM_FIELD_NUMBER: builtins.int + FETCH_FIELD_NUMBER: builtins.int + @property + def transform(self) -> global___MlRelation.Transform: ... + @property + def fetch(self) -> global___Fetch: ... + def __init__( + self, + *, + transform: global___MlRelation.Transform | None = ..., + fetch: global___Fetch | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "fetch", b"fetch", "ml_type", b"ml_type", "transform", b"transform" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "fetch", b"fetch", "ml_type", b"ml_type", "transform", b"transform" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["ml_type", b"ml_type"] + ) -> typing_extensions.Literal["transform", "fetch"] | None: ... + +global___MlRelation = MlRelation + +class Fetch(google.protobuf.message.Message): + """Message for fetching attribute from object on the server side. + Fetch can be represented as a Relation or a ML command + Command: model.coefficients, model.summary.weightedPrecision which + returns the final literal result + Relation: model.summary.roc which returns a DataFrame (Relation) + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Method(google.protobuf.message.Message): + """Represents a method with inclusion of method name and its arguments""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Args(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PARAM_FIELD_NUMBER: builtins.int + INPUT_FIELD_NUMBER: builtins.int + @property + def param(self) -> pyspark.sql.connect.proto.ml_common_pb2.Param: ... + @property + def input(self) -> global___Relation: ... + def __init__( + self, + *, + param: pyspark.sql.connect.proto.ml_common_pb2.Param | None = ..., + input: global___Relation | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "args_type", b"args_type", "input", b"input", "param", b"param" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "args_type", b"args_type", "input", b"input", "param", b"param" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["args_type", b"args_type"] + ) -> typing_extensions.Literal["param", "input"] | None: ... + + METHOD_FIELD_NUMBER: builtins.int + ARGS_FIELD_NUMBER: builtins.int + method: builtins.str + """(Required) the method name""" + @property + def args( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___Fetch.Method.Args + ]: + """(Optional) the arguments of the method""" + def __init__( + self, + *, + method: builtins.str = ..., + args: collections.abc.Iterable[global___Fetch.Method.Args] | None = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["args", b"args", "method", b"method"] + ) -> None: ... + + OBJ_REF_FIELD_NUMBER: builtins.int + METHODS_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """(Required) reference to the object on the server side""" + @property + def methods( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[global___Fetch.Method]: + """(Required) the calling method chains""" + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + methods: collections.abc.Iterable[global___Fetch.Method] | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["obj_ref", b"obj_ref"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["methods", b"methods", "obj_ref", b"obj_ref"] + ) -> None: ... + +global___Fetch = Fetch + class Unknown(google.protobuf.message.Message): """Used for testing purposes only.""" diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index 74413509ebc14..1d34011b20e78 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -25,6 +25,7 @@ import "spark/connect/common.proto"; import "spark/connect/expressions.proto"; import "spark/connect/relations.proto"; import "spark/connect/types.proto"; +import "spark/connect/ml.proto"; option java_multiple_files = true; option java_package = "org.apache.spark.connect.proto"; @@ -395,6 +396,9 @@ message ExecutePlanResponse { // Response for command that checkpoints a DataFrame. CheckpointCommandResult checkpoint_command_result = 19; + // ML command response + MlCommandResult ml_command_result = 20; + // Support arbitrary result objects. google.protobuf.Any extension = 999; } diff --git a/sql/connect/common/src/main/protobuf/spark/connect/commands.proto b/sql/connect/common/src/main/protobuf/spark/connect/commands.proto index a01d4369a7aed..10033b6400b53 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -21,6 +21,7 @@ import "google/protobuf/any.proto"; import "spark/connect/common.proto"; import "spark/connect/expressions.proto"; import "spark/connect/relations.proto"; +import "spark/connect/ml.proto"; package spark.connect; @@ -48,7 +49,7 @@ message Command { CheckpointCommand checkpoint_command = 14; RemoveCachedRemoteRelationCommand remove_cached_remote_relation_command = 15; MergeIntoTableCommand merge_into_table_command = 16; - + MlCommand ml_command = 17; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // Commands they can add them here. During the planning the correct resolution is done. google.protobuf.Any extension = 999; diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto new file mode 100644 index 0000000000000..591004b000daa --- /dev/null +++ b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto @@ -0,0 +1,107 @@ +/* + * 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. + */ + +syntax = 'proto3'; + +package spark.connect; + +import "spark/connect/expressions.proto"; +import "spark/connect/relations.proto"; +import "spark/connect/ml_common.proto"; + +option java_multiple_files = true; +option java_package = "org.apache.spark.connect.proto"; +option go_package = "internal/generated"; + +// Command for ML +message MlCommand { + oneof command { + Fit fit = 1; + Fetch fetch = 2; + Delete delete = 3; + Write write = 4; + Read read = 5; + } + + // Command for estimator.fit(dataset) + message Fit { + // Estimator information + MlOperator estimator = 1; + // parameters of the Estimator + MlParams params = 2; + // the training dataset + Relation dataset = 3; + } + + // Command to delete the cached object which could be a model + // or summary evaluated by a model + message Delete { + ObjectRef obj_ref = 1; + } + + // Command to write ML operator + message Write { + // It could be an estimator/evaluator or the cached model + oneof type { + // Estimator or evaluator + MlOperator operator = 1; + // The cached model + ObjectRef obj_ref = 2; + } + // The parameters of operator which could be estimator/evaluator or a cached model + MlParams params = 3; + // Save the ML instance to the path + string path = 4; + // Overwrites if the output path already exists. + bool should_overwrite = 5; + // The options of the writer + map options = 6; + } + + // Command to load ML operator. + message Read { + // ML operator information + MlOperator operator = 1; + // Load the ML instance from the input path + string path = 2; + } +} + +// The result of MlCommand +message MlCommandResult { + oneof result_type { + // The result of the attribute + Param param = 1; + // Evaluate a Dataset in a model and return the cached ID of summary + string summary = 2; + // Operator information + MlOperatorInfo operator_info = 3; + } + + // Represents an operator info + message MlOperatorInfo { + oneof type { + // The cached object which could be a model or summary evaluated by a model + ObjectRef obj_ref = 1; + // Operator name + string name = 2; + } + string uid = 3; + MlParams params = 4; + } + +} diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto new file mode 100644 index 0000000000000..f91c2489ed947 --- /dev/null +++ b/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto @@ -0,0 +1,108 @@ +/* + * 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. + */ + +syntax = 'proto3'; + +package spark.connect; + +import "spark/connect/expressions.proto"; + +option java_multiple_files = true; +option java_package = "org.apache.spark.connect.proto"; +option go_package = "internal/generated"; + +// MlParams stores param settings for ML Estimator / Transformer / Evaluator +message MlParams { + // User-supplied params + map params = 1; +} + +// Represents the parameter type of the ML instance, or the returned value +// of the attribute +message Param { + oneof param_type { + Expression.Literal literal = 1; + Vector vector = 2; + Matrix matrix = 3; + } +} + +// MLOperator represents the ML operators like (Estimator, Transformer or Evaluator) +message MlOperator { + // The qualified name of the ML operator. + string name = 1; + // Unique id of the ML operator + string uid = 2; + // Represents what the ML operator is + OperatorType type = 3; + enum OperatorType { + UNSPECIFIED = 0; + ESTIMATOR = 1; + TRANSFORMER = 2; + EVALUATOR = 3; + MODEL = 4; + } +} + +// Represents a reference to the cached object which could be a model +// or summary evaluated by a model +message ObjectRef { + // The ID is used to lookup the object on the server side. + string id = 1; +} + +// See pyspark.ml.linalg.Vector +message Vector { + oneof vector_type { + Dense dense = 1; + Sparse sparse = 2; + } + // See pyspark.ml.linalg.DenseVector + message Dense { + repeated double value = 1; + } + // See pyspark.ml.linalg.SparseVector + message Sparse { + int32 size = 1; + repeated int32 index = 2; + repeated double value = 3; + } +} + +// See pyspark.ml.linalg.Matrix +message Matrix { + oneof matrix_type { + Dense dense = 1; + Sparse sparse = 2; + } + // See pyspark.ml.linalg.DenseMatrix + message Dense { + int32 num_rows = 1; + int32 num_cols = 2; + repeated double value = 3; + bool is_transposed = 4; + } + // See pyspark.ml.linalg.SparseMatrix + message Sparse { + int32 num_rows = 1; + int32 num_cols = 2; + repeated int32 colptr = 3; + repeated int32 row_index = 4; + repeated double value = 5; + bool is_transposed = 6; + } +} diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index 5ab9f64149f51..c2cbed0dd22ba 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -24,6 +24,7 @@ import "spark/connect/expressions.proto"; import "spark/connect/types.proto"; import "spark/connect/catalog.proto"; import "spark/connect/common.proto"; +import "spark/connect/ml_common.proto"; option java_multiple_files = true; option java_package = "org.apache.spark.connect.proto"; @@ -98,6 +99,9 @@ message Relation { // Catalog API (experimental / unstable) Catalog catalog = 200; + // ML relation + MlRelation ml_relation = 300; + // This field is used to mark extensions to the protocol. When plugins generate arbitrary // relations they can add them here. During the planning the correct resolution is done. google.protobuf.Any extension = 998; @@ -105,6 +109,55 @@ message Relation { } } +// Relation to represent ML world +message MlRelation { + oneof ml_type { + Transform transform = 1; + Fetch fetch = 2; + } + // Relation to represent transform(input) of the operator + // which could be a cached model or a new transformer + message Transform { + oneof operator { + // Object reference + ObjectRef obj_ref = 1; + // Could be an ML transformer like VectorAssembler + MlOperator transformer = 2; + } + // the input dataframe + Relation input = 3; + // the operator specific parameters + MlParams params = 4; + } +} + +// Message for fetching attribute from object on the server side. +// Fetch can be represented as a Relation or a ML command +// Command: model.coefficients, model.summary.weightedPrecision which +// returns the final literal result +// Relation: model.summary.roc which returns a DataFrame (Relation) +message Fetch { + // (Required) reference to the object on the server side + ObjectRef obj_ref = 1; + // (Required) the calling method chains + repeated Method methods = 2; + + // Represents a method with inclusion of method name and its arguments + message Method { + // (Required) the method name + string method = 1; + // (Optional) the arguments of the method + repeated Args args = 2; + + message Args { + oneof args_type { + Param param = 1; + Relation input = 2; + } + } + } +} + // Used for testing purposes only. message Unknown {} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala new file mode 100644 index 0000000000000..a036f8b67350d --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala @@ -0,0 +1,70 @@ +/* + * 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.connect.ml + +import java.util.UUID +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.internal.Logging + +/** + * MLCache is for caching ML objects, typically for models and summaries evaluated by a model. + */ +private[connect] class MLCache extends Logging { + private val cachedModel: ConcurrentHashMap[String, Object] = + new ConcurrentHashMap[String, Object]() + + /** + * Cache an object into a map of MLCache, and return its key + * @param obj + * the object to be cached + * @return + * the key + */ + def register(obj: Object): String = { + val objectId = UUID.randomUUID().toString + cachedModel.put(objectId, obj) + objectId + } + + /** + * Get the object by the key + * @param refId + * the key used to look up the corresponding object + * @return + * the cached object + */ + def get(refId: String): Object = { + cachedModel.get(refId) + } + + /** + * Remove the object from MLCache + * @param refId + * the key used to look up the corresponding object + */ + def remove(refId: String): Unit = { + cachedModel.remove(refId) + } + + /** + * Clear all the caches + */ + def clear(): Unit = { + cachedModel.clear() + } +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala new file mode 100644 index 0000000000000..eb88bf9169d3d --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala @@ -0,0 +1,32 @@ +/* + * 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.connect.ml + +import org.apache.spark.SparkException + +private[spark] case class MlUnsupportedException(message: String) + extends SparkException( + errorClass = "CONNECT_ML.UNSUPPORTED_EXCEPTION", + messageParameters = Map("message" -> message), + cause = null) + +private[spark] case class MLAttributeNotAllowedException(attribute: String) + extends SparkException( + errorClass = "CONNECT_ML.ATTRIBUTE_NOT_ALLOWED", + messageParameters = Map("attribute" -> attribute), + cause = null) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala new file mode 100644 index 0000000000000..b4bc6bfdc66b4 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala @@ -0,0 +1,268 @@ +/* + * 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.connect.ml + +import scala.jdk.CollectionConverters.CollectionHasAsScala + +import org.apache.spark.connect.proto +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util.{MLWritable, Summary} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.ml.Serializer.deserializeMethodArguments +import org.apache.spark.sql.connect.service.SessionHolder + +private case class Method( + name: String, + argValues: Array[Object] = Array.empty, + argClasses: Array[Class[_]] = Array.empty) + +/** + * Helper function to get the attribute from an object by reflection + */ +private class AttributeHelper( + val sessionHolder: SessionHolder, + val objRef: String, + val methods: Array[Method]) { + protected lazy val instance = sessionHolder.mlCache.get(objRef) + // Get the attribute by reflection + def getAttribute: Any = { + assert(methods.length >= 1) + methods.foldLeft(instance) { (obj, m) => + if (m.argValues.isEmpty) { + MLUtils.invokeMethodAllowed(obj, m.name) + } else { + MLUtils.invokeMethodAllowed(obj, m.name, m.argValues, m.argClasses) + } + } + } +} + +// Model specific attribute helper with transform supported +private class ModelAttributeHelper( + sessionHolder: SessionHolder, + objRef: String, + methods: Array[Method]) + extends AttributeHelper(sessionHolder, objRef, methods) { + + def transform(relation: proto.MlRelation.Transform): DataFrame = { + // Create a copied model to avoid concurrently modify model params. + val model = instance.asInstanceOf[Model[_]] + val copiedModel = model.copy(ParamMap.empty).asInstanceOf[Model[_]] + MLUtils.setInstanceParams(copiedModel, relation.getParams) + val inputDF = MLUtils.parseRelationProto(relation.getInput, sessionHolder) + copiedModel.transform(inputDF) + } +} + +private object AttributeHelper { + def parseMethods( + sessionHolder: SessionHolder, + methodsProto: Array[proto.Fetch.Method] = Array.empty): Array[Method] = { + methodsProto.map { m => + val (argValues, argClasses) = + deserializeMethodArguments(m.getArgsList.asScala.toArray, sessionHolder).unzip + Method(m.getMethod, argValues, argClasses) + } + } + def apply( + sessionHolder: SessionHolder, + objId: String, + methodsProto: Array[proto.Fetch.Method] = Array.empty): AttributeHelper = { + new AttributeHelper(sessionHolder, objId, parseMethods(sessionHolder, methodsProto)) + } +} + +private object ModelAttributeHelper { + def apply( + sessionHolder: SessionHolder, + objId: String, + methodsProto: Array[proto.Fetch.Method] = Array.empty): ModelAttributeHelper = { + new ModelAttributeHelper( + sessionHolder, + objId, + AttributeHelper.parseMethods(sessionHolder, methodsProto)) + } +} + +// MLHandler is a utility to group all ML operations +private[connect] object MLHandler extends Logging { + def handleMlCommand( + sessionHolder: SessionHolder, + mlCommand: proto.MlCommand): proto.MlCommandResult = { + + val mlCache = sessionHolder.mlCache + + mlCommand.getCommandCase match { + case proto.MlCommand.CommandCase.FIT => + val fitCmd = mlCommand.getFit + val estimatorProto = fitCmd.getEstimator + assert(estimatorProto.getType == proto.MlOperator.OperatorType.ESTIMATOR) + + val dataset = MLUtils.parseRelationProto(fitCmd.getDataset, sessionHolder) + val estimator = MLUtils.getEstimator(estimatorProto, Some(fitCmd.getParams)) + val model = estimator.fit(dataset).asInstanceOf[Model[_]] + val id = mlCache.register(model) + proto.MlCommandResult + .newBuilder() + .setOperatorInfo( + proto.MlCommandResult.MlOperatorInfo + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(id))) + .build() + + case proto.MlCommand.CommandCase.FETCH => + val helper = AttributeHelper( + sessionHolder, + mlCommand.getFetch.getObjRef.getId, + mlCommand.getFetch.getMethodsList.asScala.toArray) + val attrResult = helper.getAttribute + attrResult match { + case s: Summary => + val id = mlCache.register(s) + proto.MlCommandResult.newBuilder().setSummary(id).build() + case _ => + val param = Serializer.serializeParam(attrResult) + proto.MlCommandResult.newBuilder().setParam(param).build() + } + + case proto.MlCommand.CommandCase.DELETE => + val objId = mlCommand.getDelete.getObjRef.getId + var result = false + if (!objId.contains(".")) { + mlCache.remove(objId) + result = true + } + proto.MlCommandResult + .newBuilder() + .setParam( + proto.Param + .newBuilder() + .setLiteral(LiteralValueProtoConverter.toLiteralProto(result)) + .build()) + .build() + + case proto.MlCommand.CommandCase.WRITE => + mlCommand.getWrite.getTypeCase match { + case proto.MlCommand.Write.TypeCase.OBJ_REF => // save a model + val objId = mlCommand.getWrite.getObjRef.getId + val model = mlCache.get(objId).asInstanceOf[Model[_]] + val copiedModel = model.copy(ParamMap.empty).asInstanceOf[Model[_]] + MLUtils.setInstanceParams(copiedModel, mlCommand.getWrite.getParams) + + copiedModel match { + case m: MLWritable => MLUtils.write(m, mlCommand.getWrite) + case other => throw MlUnsupportedException(s"$other is not writable") + } + + // save an estimator/evaluator/transformer + case proto.MlCommand.Write.TypeCase.OPERATOR => + val writer = mlCommand.getWrite + if (writer.getOperator.getType == proto.MlOperator.OperatorType.ESTIMATOR) { + val estimator = MLUtils.getEstimator(writer.getOperator, Some(writer.getParams)) + estimator match { + case m: MLWritable => MLUtils.write(m, mlCommand.getWrite) + case other => throw MlUnsupportedException(s"Estimator $other is not writable") + } + } else { + throw MlUnsupportedException(s"${writer.getOperator.getName} not supported") + } + + case other => throw MlUnsupportedException(s"$other not supported") + } + proto.MlCommandResult.newBuilder().build() + + case proto.MlCommand.CommandCase.READ => + val operator = mlCommand.getRead.getOperator + val name = operator.getName + val path = mlCommand.getRead.getPath + + if (operator.getType == proto.MlOperator.OperatorType.MODEL) { + val model = MLUtils.load(name, path).asInstanceOf[Model[_]] + val id = mlCache.register(model) + proto.MlCommandResult + .newBuilder() + .setOperatorInfo( + proto.MlCommandResult.MlOperatorInfo + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(id)) + .setUid(model.uid) + .setParams(Serializer.serializeParams(model))) + .build() + + } else if (operator.getType == proto.MlOperator.OperatorType.ESTIMATOR) { + val estimator = MLUtils.load(name, path).asInstanceOf[Estimator[_]] + proto.MlCommandResult + .newBuilder() + .setOperatorInfo( + proto.MlCommandResult.MlOperatorInfo + .newBuilder() + .setName(name) + .setUid(estimator.uid) + .setParams(Serializer.serializeParams(estimator))) + .build() + } else { + throw MlUnsupportedException(s"${operator.getType} not supported") + } + + case other => throw MlUnsupportedException(s"$other not supported") + } + } + + def transformMLRelation(relation: proto.MlRelation, sessionHolder: SessionHolder): DataFrame = { + relation.getMlTypeCase match { + // Ml transform + case proto.MlRelation.MlTypeCase.TRANSFORM => + relation.getTransform.getOperatorCase match { + // transform for a new ML transformer + case proto.MlRelation.Transform.OperatorCase.TRANSFORMER => + val transformProto = relation.getTransform + assert( + transformProto.getTransformer.getType == + proto.MlOperator.OperatorType.TRANSFORMER) + val dataset = MLUtils.parseRelationProto(transformProto.getInput, sessionHolder) + val transformer = MLUtils.getTransformer(transformProto) + transformer.transform(dataset) + + // transform on a cached model + case proto.MlRelation.Transform.OperatorCase.OBJ_REF => + val helper = + ModelAttributeHelper( + sessionHolder, + relation.getTransform.getObjRef.getId, + Array.empty) + helper.transform(relation.getTransform) + + case other => throw new IllegalArgumentException(s"$other not supported") + } + + // Get the attribute from a cached object which could be a model or summary + case proto.MlRelation.MlTypeCase.FETCH => + val helper = AttributeHelper( + sessionHolder, + relation.getFetch.getObjRef.getId, + relation.getFetch.getMethodsList.asScala.toArray) + helper.getAttribute.asInstanceOf[DataFrame] + + case other => throw MlUnsupportedException(s"$other not supported") + } + } + +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala new file mode 100644 index 0000000000000..72c86401eb631 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala @@ -0,0 +1,353 @@ +/* + * 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.connect.ml + +import java.util.ServiceLoader + +import scala.collection.immutable.HashSet +import scala.jdk.CollectionConverters._ + +import org.apache.commons.lang3.reflect.MethodUtils.invokeMethod + +import org.apache.spark.connect.proto +import org.apache.spark.ml.{Estimator, Transformer} +import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.ml.param.Params +import org.apache.spark.ml.util.MLWritable +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.sql.connect.service.SessionHolder +import org.apache.spark.util.{SparkClassUtils, Utils} + +private[ml] object MLUtils { + + /** + * Load the registered ML operators via ServiceLoader + * + * @param mlCls + * the operator class + * @return + * a Map with name and class + */ + private def loadOperators(mlCls: Class[_]): Map[String, Class[_]] = { + val loader = Utils.getContextOrSparkClassLoader + val serviceLoader = ServiceLoader.load(mlCls, loader) + val providers = serviceLoader.asScala.toList + providers.map(est => est.getClass.getName -> est.getClass).toMap + } + + private lazy val estimators = loadOperators(classOf[Estimator[_]]) + + private lazy val transformers = loadOperators(classOf[Transformer]) + + def deserializeVector(vector: proto.Vector): Vector = { + if (vector.hasDense) { + val values = vector.getDense.getValueList.asScala.map(_.toDouble).toArray + Vectors.dense(values) + } else { + val size = vector.getSparse.getSize + val indices = vector.getSparse.getIndexList.asScala.map(_.toInt).toArray + val values = vector.getSparse.getValueList.asScala.map(_.toDouble).toArray + Vectors.sparse(size, indices, values) + } + } + + def deserializeMatrix(matrix: proto.Matrix): Matrix = { + if (matrix.hasDense) { + val values = matrix.getDense.getValueList.asScala.map(_.toDouble).toArray + Matrices.dense(matrix.getDense.getNumRows, matrix.getDense.getNumCols, values) + } else { + val sparse = matrix.getSparse + val colPtrs = sparse.getColptrList.asScala.map(_.toInt).toArray + val rowIndices = sparse.getRowIndexList.asScala.map(_.toInt).toArray + val values = sparse.getValueList.asScala.map(_.toDouble).toArray + Matrices.sparse(sparse.getNumRows, sparse.getNumCols, colPtrs, rowIndices, values) + } + } + + /** + * Set the parameters to the ML instance + * + * @param instance + * an ML operator + * @param params + * the parameters of the ML operator + */ + def setInstanceParams(instance: Params, params: proto.MlParams): Unit = { + params.getParamsMap.asScala.foreach { case (name, paramProto) => + val p = instance.getParam(name) + val value = if (paramProto.hasLiteral) { + reconcileParam( + p.paramValueClassTag.runtimeClass, + LiteralValueProtoConverter.toCatalystValue(paramProto.getLiteral)) + } else if (paramProto.hasVector) { + deserializeVector(paramProto.getVector) + } else if (paramProto.hasMatrix) { + deserializeMatrix(paramProto.getMatrix) + } else { + throw MlUnsupportedException(s"Unsupported parameter type for ${name}") + } + instance.set(p, value) + } + } + + /** + * Convert the array from Object[] to Array[_] + * @param elementType + * the element type of the array + * @param array + * to be reconciled + * @return + * the reconciled array + */ + private def reconcileArray(elementType: Class[_], array: Array[_]): Array[_] = { + if (elementType == classOf[Byte]) { + array.map(_.asInstanceOf[Byte]) + } else if (elementType == classOf[Short]) { + array.map(_.asInstanceOf[Short]) + } else if (elementType == classOf[Int]) { + array.map(_.asInstanceOf[Int]) + } else if (elementType == classOf[Long]) { + array.map(_.asInstanceOf[Long]) + } else if (elementType == classOf[Float]) { + array.map(_.asInstanceOf[Float]) + } else if (elementType == classOf[Double]) { + array.map(_.asInstanceOf[Double]) + } else if (elementType == classOf[String]) { + array.map(_.asInstanceOf[String]) + } else { + throw MlUnsupportedException( + s"array element type unsupported, " + + s"found ${elementType.getName}") + } + } + + /** + * Reconcile the parameter value given the provided parameter type. Currently, support + * byte/short/int/long/float/double/string and array. Note that, array of array is not supported + * yet. + */ + private def reconcileParam(paramType: Class[_], value: Any): Any = { + // Some cases the param type might be mismatched with the value type. + // Because in python side we only have int / float type for numeric params. + // e.g.: + // param type is Int but client sends a Long type. + // param type is Long but client sends a Int type. + // param type is Float but client sends a Double type. + // param type is Array[Int] but client sends a Array[Long] type. + // param type is Array[Float] but client sends a Array[Double] type. + // param type is Array[Array[Int]] but client sends a Array[Array[Long]] type. + // param type is Array[Array[Float]] but client sends a Array[Array[Double]] type. + if (paramType == classOf[Byte]) { + value.asInstanceOf[java.lang.Number].byteValue() + } else if (paramType == classOf[Short]) { + value.asInstanceOf[java.lang.Number].shortValue() + } else if (paramType == classOf[Int]) { + value.asInstanceOf[java.lang.Number].intValue() + } else if (paramType == classOf[Long]) { + value.asInstanceOf[java.lang.Number].longValue() + } else if (paramType == classOf[Float]) { + value.asInstanceOf[java.lang.Number].floatValue() + } else if (paramType == classOf[Double]) { + value.asInstanceOf[java.lang.Number].doubleValue() + } else if (paramType == classOf[Boolean]) { + value.asInstanceOf[Boolean] + } else if (paramType == classOf[String]) { + value.asInstanceOf[String] + } else if (paramType.isArray) { + val compType = paramType.getComponentType + if (compType.isArray) { + throw MlUnsupportedException(s"Array of array unsupported") + } else { + val array = value.asInstanceOf[Array[_]].map { e => + reconcileParam(compType, e) + } + reconcileArray(compType, array) + } + } else { + throw MlUnsupportedException(s"Unsupported parameter type, found ${paramType.getName}") + } + } + + def parseRelationProto(relation: proto.Relation, sessionHolder: SessionHolder): DataFrame = { + val planner = new SparkConnectPlanner(sessionHolder) + val plan = planner.transformRelation(relation) + Dataset.ofRows(sessionHolder.session, plan) + } + + /** + * Get the instance according to the provided proto information. + * + * @param name + * The name of the instance (either estimator or transformer). + * @param uid + * The unique identifier for the instance. + * @param instanceMap + * A map of instance names to constructors. + * @param params + * Optional parameters for the instance. + * @tparam T + * The type of the instance (Estimator or Transformer). + * @return + * The instance of the requested type. + * @throws MlUnsupportedException + * If the instance is not supported. + */ + private def getInstance[T]( + name: String, + uid: String, + instanceMap: Map[String, Class[_]], + params: Option[proto.MlParams]): T = { + if (instanceMap.isEmpty || !instanceMap.contains(name)) { + throw MlUnsupportedException(s"Unsupported ML operator, found $name") + } + + val instance = instanceMap(name) + .getConstructor(classOf[String]) + .newInstance(uid) + .asInstanceOf[T] + + // Set parameters for the instance if they are provided + params.foreach(p => MLUtils.setInstanceParams(instance.asInstanceOf[Params], p)) + instance + } + + /** + * Get the Estimator instance according to the proto information + * + * @param operator + * MlOperator information + * @param params + * The optional parameters of the estimator + * @return + * the estimator + */ + def getEstimator(operator: proto.MlOperator, params: Option[proto.MlParams]): Estimator[_] = { + val name = operator.getName + val uid = operator.getUid + getInstance[Estimator[_]](name, uid, estimators, params) + } + + /** + * Get the transformer instance according to the transform proto + * + * @param transformProto + * transform proto + * @return + * a transformer + */ + def getTransformer(transformProto: proto.MlRelation.Transform): Transformer = { + val name = transformProto.getTransformer.getName + val uid = transformProto.getTransformer.getUid + val params = transformProto.getParams + getInstance[Transformer](name, uid, transformers, Some(params)) + } + + /** + * Call "load: function on the ML operator given the operator name + * + * @param className + * the ML operator name + * @param path + * the path to be loaded + * @return + * the ML instance + */ + def load(className: String, path: String): Object = { + val loadedMethod = SparkClassUtils.classForName(className).getMethod("load", classOf[String]) + loadedMethod.invoke(null, path) + } + + // Since we're using reflection way to get the attribute, in order not to + // leave a security hole, we define an allowed attribute list that can be accessed. + // The attributes could be retrieved from the corresponding python class + private lazy val ALLOWED_ATTRIBUTES = HashSet( + "toString", + "numFeatures", + "predict", // PredictionModel + "numClasses", + "predictRaw", // ClassificationModel + "predictProbability", // ProbabilisticClassificationModel + "coefficients", + "intercept", + "coefficientMatrix", + "interceptVector", // LogisticRegressionModel + "summary", + "hasSummary", + "evaluate", // LogisticRegressionModel + "predictions", + "predictionCol", + "labelCol", + "weightCol", + "labels", // _ClassificationSummary + "truePositiveRateByLabel", + "falsePositiveRateByLabel", // _ClassificationSummary + "precisionByLabel", + "recallByLabel", + "fMeasureByLabel", + "accuracy", // _ClassificationSummary + "weightedTruePositiveRate", + "weightedFalsePositiveRate", // _ClassificationSummary + "weightedRecall", + "weightedPrecision", + "weightedFMeasure", // _ClassificationSummary + "scoreCol", + "roc", + "areaUnderROC", + "pr", + "fMeasureByThreshold", // _BinaryClassificationSummary + "precisionByThreshold", + "recallByThreshold", // _BinaryClassificationSummary + "probabilityCol", + "featuresCol", // LogisticRegressionSummary + "objectiveHistory", + "totalIterations" // _TrainingSummary + ) + + def invokeMethodAllowed(obj: Object, methodName: String): Object = { + if (!ALLOWED_ATTRIBUTES.contains(methodName)) { + throw MLAttributeNotAllowedException(methodName) + } + invokeMethod(obj, methodName) + } + + def invokeMethodAllowed( + obj: Object, + methodName: String, + args: Array[Object], + parameterTypes: Array[Class[_]]): Object = { + if (!ALLOWED_ATTRIBUTES.contains(methodName)) { + throw MLAttributeNotAllowedException(methodName) + } + invokeMethod(obj, methodName, args, parameterTypes) + } + + def write(instance: MLWritable, writeProto: proto.MlCommand.Write): Unit = { + val writer = if (writeProto.getShouldOverwrite) { + instance.write.overwrite() + } else { + instance.write + } + val path = writeProto.getPath + val options = writeProto.getOptionsMap + options.forEach((k, v) => writer.option(k, v)) + writer.save(path) + } + +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala new file mode 100644 index 0000000000000..ad6735997f834 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala @@ -0,0 +1,151 @@ +/* + * 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.connect.ml + +import org.apache.spark.connect.proto +import org.apache.spark.ml.linalg.{DenseMatrix, DenseVector, SparseMatrix, SparseVector} +import org.apache.spark.ml.param.Params +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.service.SessionHolder + +private[ml] object Serializer { + + /** + * Serialize the ML parameters, currently support Vector/Matrix and literals + * @param data + * the value of parameter + * @return + * proto.Param + */ + def serializeParam(data: Any): proto.Param = { + data match { + case v: DenseVector => + val denseBuilder = proto.Vector.Dense.newBuilder() + v.values.foreach(denseBuilder.addValue) + proto.Param + .newBuilder() + .setVector(proto.Vector.newBuilder().setDense(denseBuilder)) + .build() + case v: SparseVector => + val sparseBuilder = proto.Vector.Sparse.newBuilder().setSize(v.size) + v.indices.foreach(sparseBuilder.addIndex) + v.values.foreach(sparseBuilder.addValue) + proto.Param + .newBuilder() + .setVector(proto.Vector.newBuilder().setSparse(sparseBuilder)) + .build() + case v: DenseMatrix => + val denseBuilder = proto.Matrix.Dense.newBuilder() + v.values.foreach(denseBuilder.addValue) + denseBuilder.setNumCols(v.numCols) + denseBuilder.setNumRows(v.numRows) + denseBuilder.setIsTransposed(v.isTransposed) + proto.Param + .newBuilder() + .setMatrix(proto.Matrix.newBuilder().setDense(denseBuilder)) + .build() + case v: SparseMatrix => + val sparseBuilder = proto.Matrix.Sparse + .newBuilder() + .setNumCols(v.numCols) + .setNumRows(v.numRows) + v.values.foreach(sparseBuilder.addValue) + v.colPtrs.foreach(sparseBuilder.addColptr) + v.rowIndices.foreach(sparseBuilder.addRowIndex) + proto.Param + .newBuilder() + .setMatrix(proto.Matrix.newBuilder().setSparse(sparseBuilder)) + .build() + case _: Byte | _: Short | _: Int | _: Long | _: Float | _: Double | _: Boolean | _: String | + _: Array[_] => + proto.Param + .newBuilder() + .setLiteral(LiteralValueProtoConverter.toLiteralProto(data)) + .build() + + case other => throw MlUnsupportedException(s"$other not supported") + } + } + + def deserializeMethodArguments( + args: Array[proto.Fetch.Method.Args], + sessionHolder: SessionHolder): Array[(Object, Class[_])] = { + args.map { arg => + if (arg.hasParam) { + val param = arg.getParam + if (param.hasLiteral) { + param.getLiteral.getLiteralTypeCase match { + case proto.Expression.Literal.LiteralTypeCase.INTEGER => + (param.getLiteral.getInteger.asInstanceOf[Object], classOf[Int]) + case proto.Expression.Literal.LiteralTypeCase.FLOAT => + (param.getLiteral.getFloat.toDouble.asInstanceOf[Object], classOf[Double]) + case proto.Expression.Literal.LiteralTypeCase.STRING => + (param.getLiteral.getString, classOf[String]) + case proto.Expression.Literal.LiteralTypeCase.DOUBLE => + (param.getLiteral.getDouble.asInstanceOf[Object], classOf[Double]) + case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => + (param.getLiteral.getBoolean.asInstanceOf[Object], classOf[Boolean]) + case other => + throw MlUnsupportedException(s"$other not supported") + } + } else if (param.hasVector) { + val vector = MLUtils.deserializeVector(param.getVector) + val vectorType = if (param.getVector.hasDense) { + classOf[DenseVector] + } else { + classOf[SparseVector] + } + (vector, vectorType) + } else if (param.hasMatrix) { + val matrix = MLUtils.deserializeMatrix(param.getMatrix) + val matrixType = if (param.getMatrix.hasDense) { + classOf[DenseMatrix] + } else { + classOf[SparseMatrix] + } + (matrix, matrixType) + } else { + throw MlUnsupportedException(s"$param not supported") + } + } else if (arg.hasInput) { + (MLUtils.parseRelationProto(arg.getInput, sessionHolder), classOf[Dataset[_]]) + } else { + throw MlUnsupportedException(s"$arg not supported") + } + } + } + + /** + * Serialize an instance of "Params" which could be estimator/model/evaluator ... + * @param instance + * of Params + * @return + * proto.MlParams + */ + def serializeParams(instance: Params): proto.MlParams = { + val builder = proto.MlParams.newBuilder() + instance.params.foreach { param => + if (instance.isSet(param)) { + val v = serializeParam(instance.get(param).get) + builder.putParams(param.name, v) + } + } + builder.build() + } +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index acbbeb49b267b..94a1ab1618086 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -61,6 +61,7 @@ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidCommandInput, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE +import org.apache.spark.sql.connect.ml.MLHandler import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry import org.apache.spark.sql.connect.service.{ExecuteHolder, SessionHolder, SparkConnectService} import org.apache.spark.sql.connect.utils.MetricGenerator @@ -223,6 +224,10 @@ class SparkConnectPlanner( // Catalog API (internal-only) case proto.Relation.RelTypeCase.CATALOG => transformCatalog(rel.getCatalog) + // ML Relation + case proto.Relation.RelTypeCase.ML_RELATION => + MLHandler.transformMLRelation(rel.getMlRelation, sessionHolder).logicalPlan + // Handle plugins for Spark Connect Relation types. case proto.Relation.RelTypeCase.EXTENSION => transformRelationPlugin(rel.getExtension) @@ -2460,11 +2465,27 @@ class SparkConnectPlanner( handleRemoveCachedRemoteRelationCommand(command.getRemoveCachedRemoteRelationCommand) case proto.Command.CommandTypeCase.MERGE_INTO_TABLE_COMMAND => handleMergeIntoTableCommand(command.getMergeIntoTableCommand) + case proto.Command.CommandTypeCase.ML_COMMAND => + handleMlCommand(command.getMlCommand, responseObserver) case _ => throw new UnsupportedOperationException(s"$command not supported.") } } + private def handleMlCommand( + command: proto.MlCommand, + responseObserver: StreamObserver[proto.ExecutePlanResponse]): Unit = { + val result = MLHandler.handleMlCommand(sessionHolder, command) + executeHolder.eventsManager.postFinished() + responseObserver.onNext( + proto.ExecutePlanResponse + .newBuilder() + .setSessionId(sessionId) + .setServerSideSessionId(sessionHolder.serverSessionId) + .setMlCommandResult(result) + .build()) + } + private def handleSqlCommand( command: SqlCommand, responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 36f3bcd1e6cd7..5b56b7079a897 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connect.common.InvalidPlanInput import org.apache.spark.sql.connect.config.Connect +import org.apache.spark.sql.connect.ml.MLCache import org.apache.spark.sql.connect.planner.PythonStreamingQueryListener import org.apache.spark.sql.connect.planner.StreamingForeachBatchHelper import org.apache.spark.sql.connect.service.ExecuteKey @@ -111,6 +112,9 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio private[spark] lazy val dataFrameCache: ConcurrentMap[String, DataFrame] = new ConcurrentHashMap() + // ML model cache + private[connect] lazy val mlCache = new MLCache() + // Mapping from id to StreamingQueryListener. Used for methods like removeListener() in // StreamingQueryManager. private lazy val listenerCache: ConcurrentMap[String, StreamingQueryListener] = @@ -322,6 +326,8 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio // remove all executions and no new executions will be added in the meanwhile. SparkConnectService.executionManager.removeAllExecutionsForSession(this.key) + mlCache.clear() + eventManager.postClosed() } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala new file mode 100644 index 0000000000000..bea7072b2034e --- /dev/null +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala @@ -0,0 +1,390 @@ +/* + * 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.connect.ml + +import java.io.File + +import org.apache.spark.SparkFunSuite +import org.apache.spark.connect.proto +import org.apache.spark.ml.classification.LogisticRegressionModel +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.util.Identifiable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.connect.SparkConnectTestUtils +import org.apache.spark.sql.connect.planner.SparkConnectPlanTest +import org.apache.spark.sql.types.{FloatType, Metadata, StructField, StructType} +import org.apache.spark.util.Utils + +trait FakeArrayParams extends Params { + final val arrayString: StringArrayParam = + new StringArrayParam(this, "arrayString", "array string") + + final def getArrayString: Array[String] = $(arrayString) + + final val arrayDouble: DoubleArrayParam = + new DoubleArrayParam(this, "arrayDouble", "array double") + + final def getArrayDouble: Array[Double] = $(arrayDouble) + + final val arrayInt: IntArrayParam = new IntArrayParam(this, "arrayInt", "array int") + + final def getArrayInt: Array[Int] = $(arrayInt) + + final val int: IntParam = new IntParam(this, "int", "int") + + final def getInt: Int = $(int) + + final val float: FloatParam = new FloatParam(this, "float", "float") + + final def getFloat: Float = $(float) + + final val boolean: BooleanParam = new BooleanParam(this, "boolean", "boolean") + + final def getBoolean: Boolean = $(boolean) + + final val double: DoubleParam = new DoubleParam(this, "double", "double") + + final def getDouble: Double = $(double) +} + +class FakedML(override val uid: String) extends FakeArrayParams { + def this() = this(Identifiable.randomUID("FakedML")) + + override def copy(extra: ParamMap): Params = this +} + +class MLSuite extends SparkFunSuite with SparkConnectPlanTest { + + def createLocalRelationProto: proto.Relation = { + val udt = new VectorUDT() + val rows = Seq( + InternalRow(1.0f, udt.serialize(Vectors.dense(Array(1.0, 2.0)))), + InternalRow(1.0f, udt.serialize(Vectors.dense(Array(2.0, -1.0)))), + InternalRow(0.0f, udt.serialize(Vectors.dense(Array(-3.0, -2.0)))), + InternalRow(0.0f, udt.serialize(Vectors.dense(Array(-1.0, -2.0))))) + + val schema = StructType( + Seq( + StructField("label", FloatType), + StructField("features", new VectorUDT(), false, Metadata.empty))) + + val inputRows = rows.map { row => + val proj = UnsafeProjection.create(schema) + proj(row).copy() + } + createLocalRelationProto(DataTypeUtils.toAttributes(schema), inputRows, "UTC", Some(schema)) + } + + test("reconcileParam") { + val fakedML = new FakedML + val params = proto.MlParams + .newBuilder() + .putParams( + "boolean", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setBoolean(true)) + .build()) + .putParams( + "double", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setDouble(1.0)) + .build()) + .putParams( + "int", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setInteger(10)) + .build()) + .putParams( + "float", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setFloat(10.0f)) + .build()) + .putParams( + "arrayString", + proto.Param + .newBuilder() + .setLiteral( + proto.Expression.Literal + .newBuilder() + .setArray( + proto.Expression.Literal.Array + .newBuilder() + .setElementType(proto.DataType + .newBuilder() + .setString(proto.DataType.String.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setString("hello")) + .addElements(proto.Expression.Literal.newBuilder().setString("world")) + .build()) + .build()) + .build()) + .putParams( + "arrayInt", + proto.Param + .newBuilder() + .setLiteral( + proto.Expression.Literal + .newBuilder() + .setArray( + proto.Expression.Literal.Array + .newBuilder() + .setElementType(proto.DataType + .newBuilder() + .setInteger(proto.DataType.Integer.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setInteger(1)) + .addElements(proto.Expression.Literal.newBuilder().setInteger(2)) + .build()) + .build()) + .build()) + .putParams( + "arrayDouble", + proto.Param + .newBuilder() + .setLiteral( + proto.Expression.Literal + .newBuilder() + .setArray( + proto.Expression.Literal.Array + .newBuilder() + .setElementType(proto.DataType + .newBuilder() + .setDouble(proto.DataType.Double.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setDouble(11.0)) + .addElements(proto.Expression.Literal.newBuilder().setDouble(12.0)) + .build()) + .build()) + .build()) + .build() + MLUtils.setInstanceParams(fakedML, params) + assert(fakedML.getInt === 10) + assert(fakedML.getFloat === 10.0) + assert(fakedML.getArrayInt === Array(1, 2)) + assert(fakedML.getArrayDouble === Array(11.0, 12.0)) + assert(fakedML.getArrayString === Array("hello", "world")) + assert(fakedML.getBoolean === true) + assert(fakedML.getDouble === 1.0) + } + + test("LogisticRegression works") { + val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark) + + def verifyModel(modelId: String, hasSummary: Boolean = false): Unit = { + val model = sessionHolder.mlCache.get(modelId) + // Model is cached + assert(model != null) + assert(model.isInstanceOf[LogisticRegressionModel]) + val lrModel = model.asInstanceOf[LogisticRegressionModel] + assert(lrModel.getMaxIter === 2) + + // Fetch double attribute + val interceptCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("intercept"))) + .build() + val interceptResult = MLHandler.handleMlCommand(sessionHolder, interceptCommand) + assert(interceptResult.getParam.getLiteral.getDouble === lrModel.intercept) + + // Fetch Vector attribute + val coefficientsCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("coefficients"))) + .build() + val coefficientsResult = MLHandler.handleMlCommand(sessionHolder, coefficientsCommand) + val deserializedCoefficients = + MLUtils.deserializeVector(coefficientsResult.getParam.getVector) + assert(deserializedCoefficients === lrModel.coefficients) + + // Fetch Matrix attribute + val coefficientsMatrixCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("coefficientMatrix"))) + .build() + val coefficientsMatrixResult = + MLHandler.handleMlCommand(sessionHolder, coefficientsMatrixCommand) + val deserializedCoefficientsMatrix = + MLUtils.deserializeMatrix(coefficientsMatrixResult.getParam.getMatrix) + assert(lrModel.coefficientMatrix === deserializedCoefficientsMatrix) + + // Predict with sparse vector + val sparseVector = Vectors.dense(Array(0.0, 2.0)).toSparse + val predictCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods( + proto.Fetch.Method + .newBuilder() + .setMethod("predict") + .addArgs(proto.Fetch.Method.Args + .newBuilder() + .setParam(Serializer.serializeParam(sparseVector))))) + .build() + val predictResult = MLHandler.handleMlCommand(sessionHolder, predictCommand) + val predictValue = predictResult.getParam.getLiteral.getDouble + assert(lrModel.predict(sparseVector) === predictValue) + + // The loaded model doesn't have summary + if (hasSummary) { + // Fetch summary attribute + val accuracyCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("summary")) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("accuracy"))) + .build() + val accuracyResult = MLHandler.handleMlCommand(sessionHolder, accuracyCommand) + assert(lrModel.summary.accuracy === accuracyResult.getParam.getLiteral.getDouble) + + val weightedFMeasureCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("summary")) + .addMethods( + proto.Fetch.Method + .newBuilder() + .setMethod("weightedFMeasure") + .addArgs(proto.Fetch.Method.Args + .newBuilder() + .setParam(Serializer.serializeParam(2.5))))) + .build() + val weightedFMeasureResult = + MLHandler.handleMlCommand(sessionHolder, weightedFMeasureCommand) + assert( + lrModel.summary.weightedFMeasure(2.5) === + weightedFMeasureResult.getParam.getLiteral.getDouble) + } + } + + try { + val fitCommand = proto.MlCommand + .newBuilder() + .setFit( + proto.MlCommand.Fit + .newBuilder() + .setDataset(createLocalRelationProto) + .setEstimator( + proto.MlOperator + .newBuilder() + .setName("org.apache.spark.ml.classification.LogisticRegression") + .setUid("LogisticRegression") + .setType(proto.MlOperator.OperatorType.ESTIMATOR)) + .setParams( + proto.MlParams + .newBuilder() + .putParams( + "maxIter", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal + .newBuilder() + .setInteger(2)) + .build()))) + .build() + val fitResult = MLHandler.handleMlCommand(sessionHolder, fitCommand) + val modelId = fitResult.getOperatorInfo.getObjRef.getId + + verifyModel(modelId, true) + + // read/write + val tempDir = Utils.createTempDir(namePrefix = this.getClass.getName) + try { + val path = new File(tempDir, Identifiable.randomUID("LogisticRegression")).getPath + val writeCmd = proto.MlCommand + .newBuilder() + .setWrite( + proto.MlCommand.Write + .newBuilder() + .setPath(path) + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId))) + .build() + MLHandler.handleMlCommand(sessionHolder, writeCmd) + + val readCmd = proto.MlCommand + .newBuilder() + .setRead( + proto.MlCommand.Read + .newBuilder() + .setOperator( + proto.MlOperator + .newBuilder() + .setName("org.apache.spark.ml.classification.LogisticRegressionModel") + .setType(proto.MlOperator.OperatorType.MODEL)) + .setPath(path)) + .build() + + val readResult = MLHandler.handleMlCommand(sessionHolder, readCmd) + verifyModel(readResult.getOperatorInfo.getObjRef.getId) + + } finally { + Utils.deleteRecursively(tempDir) + } + + } finally { + sessionHolder.mlCache.clear() + } + } + + test("Exception: Unsupported ML operator") { + intercept[MlUnsupportedException] { + val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark) + val command = proto.MlCommand + .newBuilder() + .setFit( + proto.MlCommand.Fit + .newBuilder() + .setDataset(createLocalRelationProto) + .setEstimator( + proto.MlOperator + .newBuilder() + .setName("org.apache.spark.ml.NotExistingML") + .setUid("FakedUid") + .setType(proto.MlOperator.OperatorType.ESTIMATOR))) + .build() + MLHandler.handleMlCommand(sessionHolder, command) + } + } +} diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala index 054a32179935d..55c492f511049 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala @@ -89,7 +89,8 @@ trait SparkConnectPlanTest extends SharedSparkSession { def createLocalRelationProto( attrs: Seq[AttributeReference], data: Seq[InternalRow], - timeZoneId: String = "UTC"): proto.Relation = { + timeZoneId: String = "UTC", + schema: Option[StructType] = None): proto.Relation = { val localRelationBuilder = proto.LocalRelation.newBuilder() val bytes = ArrowConverters @@ -103,6 +104,7 @@ trait SparkConnectPlanTest extends SharedSparkSession { .next() localRelationBuilder.setData(ByteString.copyFrom(bytes)) + schema.foreach(s => localRelationBuilder.setSchema(s.json)) proto.Relation.newBuilder().setLocalRelation(localRelationBuilder.build()).build() } } From 216b533046139405c673646379cf4d3b0710836e Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 14 Jan 2025 21:19:20 +0800 Subject: [PATCH 423/438] [SPARK-50795][SQL] Display all DESCRIBE AS JSON dates in ISO-8601 format and types as `dataType.simpleString` ### What changes were proposed in this pull request? The PR does two updates for consistency: 1. Display all `DESCRIBE AS JSON` dates in ISO-8601 format and add regex tests in `DescribeTableSuite.scala` to ensure dates adhere to the format. 2. Display data type names as the `dataType.simpleString` ### Why are the changes needed? Ensure uniform date and dataType format in `DESCRIBE AS JSON` ### Does this PR introduce _any_ user-facing change? Yes, affects the date and dataType format produced by `DESCRIBE AS JSON`. ### How was this patch tested? Added tests in `DescribeTableSuite.scala` ### Was this patch authored or co-authored using generative AI tooling? No Closes #49455 from asl3/asl3/describeasjson-date. Authored-by: Amanda Liu Signed-off-by: Wenchen Fan --- docs/sql-ref-syntax-aux-describe-table.md | 4 +- .../sql/catalyst/catalog/interface.scala | 21 +++++++--- .../spark/sql/execution/command/tables.scala | 2 +- .../sql-tests/results/describe.sql.out | 4 +- .../command/v1/DescribeTableSuite.scala | 39 +++++++++++++------ 5 files changed, 47 insertions(+), 23 deletions(-) diff --git a/docs/sql-ref-syntax-aux-describe-table.md b/docs/sql-ref-syntax-aux-describe-table.md index 6a14da1e43801..5f5fd27c865e6 100644 --- a/docs/sql-ref-syntax-aux-describe-table.md +++ b/docs/sql-ref-syntax-aux-describe-table.md @@ -118,9 +118,9 @@ to return the metadata pertaining to a partition or column respectively. "num_buckets": , "bucket_columns": [""], "sort_columns": [""], - "created_time": "", + "created_time": "", "created_by": "", - "last_access": "", + "last_access": "", "partition_provider": "" } ``` diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index de828dfd3e280..7836e533c8b5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI import java.time.{ZoneId, ZoneOffset} -import java.util.Date import scala.collection.mutable import scala.util.control.NonFatal @@ -87,6 +86,14 @@ trait MetadataMapSupport { } map } + + val timestampFormatter = new Iso8601TimestampFormatter( + pattern = "yyyy-MM-dd'T'HH:mm:ss'Z'", + zoneId = ZoneId.of("UTC"), + locale = DateFormatter.defaultLocale, + legacyFormat = LegacyDateFormats.LENIENT_SIMPLE_DATE_FORMAT, + isParsing = true + ) } @@ -184,10 +191,12 @@ case class CatalogTablePartition( map += ("Partition Parameters" -> paramsJson) } - map += ("Created Time" -> JString(new Date(createTime).toString)) + map += ("Created Time" -> JString( + timestampFormatter.format(DateTimeUtils.millisToMicros(createTime)))) val lastAccess = if (lastAccessTime <= 0) JString("UNKNOWN") - else JString(new Date(lastAccessTime).toString) + else JString( + timestampFormatter.format(DateTimeUtils.millisToMicros(createTime))) map += ("Last Access" -> lastAccess) stats.foreach(s => map += ("Partition Statistics" -> JString(s.simpleString))) @@ -595,8 +604,8 @@ case class CatalogTable( else JNull val lastAccess: JValue = - if (lastAccessTime <= 0) JString("UNKNOWN") else JString( - DateTimeUtils.microsToInstant(DateTimeUtils.millisToMicros(lastAccessTime)).toString) + if (lastAccessTime <= 0) JString("UNKNOWN") + else JString(timestampFormatter.format(DateTimeUtils.millisToMicros(createTime))) val viewQueryOutputColumns: JValue = if (viewQueryColumnNames.nonEmpty) JArray(viewQueryColumnNames.map(JString).toList) @@ -609,7 +618,7 @@ case class CatalogTable( map += "Table" -> JString(identifier.table) if (Option(owner).exists(_.nonEmpty)) map += "Owner" -> JString(owner) map += "Created Time" -> - JString(DateTimeUtils.microsToInstant(DateTimeUtils.millisToMicros(createTime)).toString) + JString(timestampFormatter.format(DateTimeUtils.millisToMicros(createTime))) if (lastAccess != JNull) map += "Last Access" -> lastAccess map += "Created By" -> JString(s"Spark $createVersion") map += "Type" -> JString(tableType.name) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index e69e05ba7decd..73aaed0627946 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -907,7 +907,7 @@ case class DescribeTableJsonCommand( ) case _ => - JObject("name" -> JString(dataType.typeName)) + JObject("name" -> JString(dataType.simpleString)) } } diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 015b0ceff335e..870ad02e71414 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -76,7 +76,7 @@ DESCRIBE EXTENDED t AS JSON -- !query schema struct -- !query output -{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string"},"nullable":true},{"name":"b","type":{"name":"integer"},"nullable":true},{"name":"c","type":{"name":"string"},"nullable":true},{"name":"d","type":{"name":"string"},"nullable":true}],"num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"location":"file:[not included in comparison]/{warehouse_dir}/t","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","comment":"table_comment","table_properties":{"e":"3","password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"]} +{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string"},"nullable":true},{"name":"b","type":{"name":"int"},"nullable":true},{"name":"c","type":{"name":"string"},"nullable":true},{"name":"d","type":{"name":"string"},"nullable":true}],"num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"location":"file:[not included in comparison]/{warehouse_dir}/t","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","comment":"table_comment","table_properties":{"e":"3","password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"]} -- !query @@ -303,7 +303,7 @@ DESC EXTENDED t PARTITION (c='Us', d=1) AS JSON -- !query schema struct -- !query output -{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string"},"nullable":true},{"name":"b","type":{"name":"integer"},"nullable":true},{"name":"c","type":{"name":"string"},"nullable":true},{"name":"d","type":{"name":"string"},"nullable":true}],"partition_values":{"c":"Us","d":"1"},"location":"file:[not included in comparison]/{warehouse_dir}/t/c=Us/d=1","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"table_properties":{"password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"]} +{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string"},"nullable":true},{"name":"b","type":{"name":"int"},"nullable":true},{"name":"c","type":{"name":"string"},"nullable":true},{"name":"d","type":{"name":"string"},"nullable":true}],"partition_values":{"c":"Us","d":"1"},"location":"file:[not included in comparison]/{warehouse_dir}/t/c=Us/d=1","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"table_properties":{"password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"]} -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index d5dd96f55c11a..cae56754ba465 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -44,6 +44,8 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase def getProvider(): String = defaultUsing.stripPrefix("USING").trim.toLowerCase(Locale.ROOT) + val iso8601Regex = raw"^\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?Z$$".r + test("Describing of a non-existent partition") { withNamespaceAndTable("ns", "table") { tbl => spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing " + @@ -339,13 +341,11 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { namespace = Some(List("ns")), schema_name = Some("ns"), columns = Some(List( - TableColumn("employee_id", Type("integer"), true), + TableColumn("employee_id", Type("int"), true), TableColumn("employee_name", Type("string"), true), TableColumn("department", Type("string"), true), TableColumn("hire_date", Type("date"), true) )), - owner = Some(""), - created_time = Some(""), last_access = Some("UNKNOWN"), created_by = Some(s"Spark $SPARK_VERSION"), `type` = Some("MANAGED"), @@ -369,12 +369,15 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { ) if (getProvider() == "hive") { - assert(expectedOutput == parsedOutput.copy(owner = Some(""), - created_time = Some(""), + assert(expectedOutput == parsedOutput.copy(owner = None, + created_time = None, location = Some(""))) } else { assert(expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) - == parsedOutput.copy(owner = Some(""), created_time = Some(""), location = Some(""))) + == parsedOutput.copy(owner = None, created_time = None, location = Some(""))) + } + parsedOutput.created_time.foreach { createdTime => + assert(iso8601Regex.matches(createdTime)) } } } @@ -408,7 +411,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { namespace = Some(List("ns")), schema_name = Some("ns"), columns = Some(List( - TableColumn("id", Type("integer"), true), + TableColumn("id", Type("int"), true), TableColumn("name", Type("string"), true), TableColumn("region", Type("string"), true), TableColumn("category", Type("string"), true) @@ -447,6 +450,9 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { == parsedOutput.copy(location = None, created_time = None, owner = None, storage_properties = filteredParsedStorageProperties)) } + parsedOutput.created_time.foreach { createdTime => + assert(iso8601Regex.matches(createdTime)) + } } } @@ -475,7 +481,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { namespace = Some(List("ns")), schema_name = Some("ns"), columns = Some(List( - TableColumn("id", Type("integer"), default = Some("1")), + TableColumn("id", Type("int"), default = Some("1")), TableColumn("name", Type("string"), default = Some("'unknown'")), TableColumn("created_at", Type("timestamp_ltz"), default = Some("CURRENT_TIMESTAMP")), TableColumn("is_active", Type("boolean"), default = Some("true")) @@ -504,6 +510,9 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { parsedOutput.copy(location = None, created_time = None, owner = None) ) } + parsedOutput.created_time.foreach { createdTime => + assert(iso8601Regex.matches(createdTime)) + } } } @@ -528,7 +537,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { val expectedOutput = DescribeTableJson( columns = Some(List( - TableColumn("id", Type("integer")), + TableColumn("id", Type("int")), TableColumn("name", Type("string")), TableColumn("created_at", Type("timestamp_ltz")) )) @@ -564,7 +573,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { namespace = Some(List("default")), schema_name = Some("default"), columns = Some(List( - TableColumn("id", Type("integer")), + TableColumn("id", Type("int")), TableColumn("name", Type("string")), TableColumn("created_at", Type("timestamp_ltz")) )), @@ -590,6 +599,9 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { outputformat = None, serde_library = None, storage_properties = None) == parsedOutput.copy(table_properties = None, created_time = None, owner = None)) } + parsedOutput.created_time.foreach { createdTime => + assert(iso8601Regex.matches(createdTime)) + } } } } @@ -673,7 +685,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { ), Field( name = "age", - `type` = Type("integer") + `type` = Type("int") ), Field( name = "contact", @@ -709,7 +721,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { ), Field( name = "zip", - `type` = Type("integer") + `type` = Type("int") ) )) )), @@ -770,6 +782,9 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { assert(expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) == parsedOutput.copy(location = None, created_time = None, owner = None)) } + parsedOutput.created_time.foreach { createdTime => + assert(iso8601Regex.matches(createdTime)) + } } } } From bfaa814f30265f0276389f73e4ec5125bdc2c421 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 14 Jan 2025 22:06:21 +0800 Subject: [PATCH 424/438] Revert "[SPARK-46934][SQL][FOLLOWUP] Read/write roundtrip for struct type with special characters with HMS - a backward compatible approach" This reverts commit d268e0c4ac64b14c2ef9ec3c199faafa45f91c13. --- .../org/apache/spark/sql/hive/HiveUtils.scala | 9 --- .../sql/hive/client/HiveClientImpl.scala | 53 +++---------- .../sql/hive/HiveMetastoreCatalogSuite.scala | 41 +--------- .../sql/hive/execution/HiveDDLSuite.scala | 77 ++++++++++--------- 4 files changed, 52 insertions(+), 128 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 776f7045f5965..478f486eeb213 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -221,15 +221,6 @@ private[spark] object HiveUtils extends Logging { .booleanConf .createWithDefault(true) - val QUOTE_HIVE_STRUCT_FIELD_NAME = - buildConf("spark.sql.hive.quoteHiveStructFieldName") - .doc("When true, for a column defined in struct type, when it contains special characters " + - "in the field name, Spark will quote it for verification. E.g. struct" + - " is read as struct<`x`:int,`y.z`:int> for verification.") - .version("4.0.0") - .booleanConf - .createWithDefault(false) - /** * The version of the hive client that will be used to communicate with the metastore. Note that * this does not necessarily need to be the same version of Hive that is used internally by diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index acc588fb719c2..00407f0ecc178 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -56,13 +56,12 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} -import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, QuotingUtils, StringConcat} +import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_SCHEMA -import org.apache.spark.sql.hive.HiveUtils.QUOTE_HIVE_STRUCT_FIELD_NAME import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.{CircularBuffer, Utils} @@ -585,6 +584,7 @@ private[hive] class HiveClientImpl( } override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState { + verifyColumnDataType(table.dataSchema) shim.createTable(client, toHiveTable(table, Some(userName)), ignoreIfExists) } @@ -604,6 +604,7 @@ private[hive] class HiveClientImpl( // these properties are still available to the others that share the same Hive metastore. // If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect // these user-specified values. + verifyColumnDataType(table.dataSchema) val hiveTable = toHiveTable( table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName)) // Do not use `table.qualifiedName` here because this may be a rename @@ -627,6 +628,7 @@ private[hive] class HiveClientImpl( newDataSchema: StructType, schemaProps: Map[String, String]): Unit = withHiveState { val oldTable = shim.getTable(client, dbName, tableName) + verifyColumnDataType(newDataSchema) val hiveCols = newDataSchema.map(toHiveColumn) oldTable.setFields(hiveCols.asJava) @@ -1094,45 +1096,13 @@ private[hive] object HiveClientImpl extends Logging { // When reading data in parquet, orc, or avro file format with string type for char, // the tailing spaces may lost if we are not going to pad it. val typeString = if (SQLConf.get.charVarcharAsString) { - catalogString(c.dataType) + c.dataType.catalogString } else { - CharVarcharUtils.getRawTypeString(c.metadata).getOrElse(catalogString(c.dataType)) + CharVarcharUtils.getRawTypeString(c.metadata).getOrElse(c.dataType.catalogString) } new FieldSchema(c.name, typeString, c.getComment().orNull) } - /** - * This a a variant of `DataType.catalogString` that does the same thing in general but - * it will not quote the field names in the struct type. HMS API uses unquoted field names - * to store the schema of a struct type. This is fine if we in the write path, we might encounter - * issues in the read path to parse the unquoted schema strings in the Spark SQL parser. You can - * see the tricks we play in the `getSparkSQLDataType` method to handle this. To avoid the - * flakiness of those tricks, we quote the field names, make them unrecognized by HMS API, and - * then store them in custom spark properties in a fallback way. - * - * And the reason we don't add quoting in `DataType.catalogString` directly is that we don't - * want to break the compatibility of the existing query output schema. - */ - def catalogString(dataType: DataType): String = dataType match { - case ArrayType(et, _) => s"array<${catalogString(et)}>" - case MapType(k, v, _) => s"map<${catalogString(k)},${catalogString(v)}>" - case StructType(fields) => - val stringConcat = new StringConcat() - val len = fields.length - stringConcat.append("struct<") - var i = 0 - while (i < len) { - val name = QuotingUtils.quoteIfNeeded(fields(i).name) - stringConcat.append(s"$name:${catalogString(fields(i).dataType)}") - i += 1 - if (i < len) stringConcat.append(",") - } - stringConcat.append(">") - stringConcat.toString - case udt: UserDefinedType[_] => catalogString(udt.sqlType) - case _ => dataType.catalogString - } - /** Get the Spark SQL native DataType from Hive's FieldSchema. */ private def getSparkSQLDataType(hc: FieldSchema): DataType = { // For struct types, Hive metastore API uses unquoted element names, so does the spark catalyst @@ -1145,12 +1115,7 @@ private[hive] object HiveClientImpl extends Logging { // struct -> struct<`x`:int,`y.z`:int> // array> -> array> // map> -> map> - val typeStr = if (SQLConf.get.getConf(QUOTE_HIVE_STRUCT_FIELD_NAME) && - hc.getType.indexOf('`') < 0) { // This a defensive code for possible changes in HMS - hc.getType.replaceAll("(?<=struct<|,)([^,<:]+)(?=:)", "`$1`") - } else { - hc.getType - } + val typeStr = hc.getType.replaceAll("(?<=struct<|,)([^,<:]+)(?=:)", "`$1`") try { CatalystSqlParser.parseDataType(typeStr) } catch { @@ -1169,6 +1134,10 @@ private[hive] object HiveClientImpl extends Logging { Option(hc.getComment).map(field.withComment).getOrElse(field) } + private def verifyColumnDataType(schema: StructType): Unit = { + schema.foreach(col => getSparkSQLDataType(toHiveColumn(col))) + } + private def toInputFormat(name: String) = Utils.classForName[org.apache.hadoop.mapred.InputFormat[_, _]](name) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index bb274ce0578b1..72c570d1f9097 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -17,13 +17,11 @@ package org.apache.spark.sql.hive -import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias -import org.apache.spark.sql.hive.HiveUtils.QUOTE_HIVE_STRUCT_FIELD_NAME import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} @@ -131,26 +129,6 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton with SQLTestUtils { assert(schema == expectedSchema) } } - - test("SPARK-46934: HMS columns cannot handle quoted columns") { - withTable("t") { - val schema = - "a struct<" + - "`a.a`:int," + - "`a.b`:struct<" + - " `a b b`:array," + - " `a b c`:map" + - " >" + - ">" - val e = intercept[AnalysisException](sql("CREATE TABLE t(" + schema + ") USING hive")) - checkError( - exception = e, - condition = "_LEGACY_ERROR_TEMP_3065", - parameters = Map( - "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", - "msg" -> e.getCause.getMessage)) - } - } } class DataSourceWithHiveMetastoreCatalogSuite @@ -463,7 +441,7 @@ class DataSourceWithHiveMetastoreCatalogSuite } test("SPARK-46934: Handle special characters in struct types with hive DDL") { - try { + withTable("t") { val schema = "a struct<" + "`a.a`:int," + @@ -473,22 +451,7 @@ class DataSourceWithHiveMetastoreCatalogSuite " >" + ">" sparkSession.metadataHive.runSqlHive(s"CREATE TABLE t($schema)") - withSQLConf(QUOTE_HIVE_STRUCT_FIELD_NAME.key -> "true") { - assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema(schema)) - } - - withSQLConf(QUOTE_HIVE_STRUCT_FIELD_NAME.key -> "false") { - checkError(exception = - intercept[SparkException](spark.table("t")).getCause.asInstanceOf[SparkException], - condition = "CANNOT_RECOGNIZE_HIVE_TYPE", - parameters = Map( - "fieldType" -> - "\"STRUCT,A B C:MAP>>\"", - "fieldName" -> "`a`" - )) - } - } finally { - sparkSession.metadataHive.runSqlHive("DROP TABLE IF EXISTS t") + assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema(schema)) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 880d7bdc82243..a58adbce7ec52 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER +import org.apache.spark.sql.errors.DataTypeErrors.toSQLType import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, ParquetFooterReader} @@ -166,23 +167,17 @@ class HiveDDLSuite } test("SPARK-46934: quote element name before parsing struct") { - val e = intercept[AnalysisException]( - sql("CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q")) - checkError( - exception = e, - condition = "_LEGACY_ERROR_TEMP_3065", - parameters = Map( - "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", - "msg" -> e.getCause.getMessage)) + withTable("t") { + sql("CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q") + assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema( + "q STRUCT<`$a`: STRING, b: INT>")) + } - val e1 = intercept[AnalysisException]( - sql("CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT) USING hive")) - checkError( - exception = e1, - condition = "_LEGACY_ERROR_TEMP_3065", - parameters = Map( - "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", - "msg" -> e1.getCause.getMessage)) + withTable("t") { + sql("CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT) USING hive") + assert(spark.table("t").schema === CatalystSqlParser.parseTableSchema( + "q STRUCT<`$a`:INT, col2:STRING>, i1 INT")) + } withView("v") { spark.sql("CREATE VIEW v AS SELECT STRUCT('a' AS `a`, 1 AS b) q") @@ -238,29 +233,15 @@ class HiveDDLSuite } } - test("SPARK-46934: alter datasource table tests with nested types") { + test("SPARK-46934: alter table tests with nested types") { withTable("t1") { - sql("CREATE TABLE t1 (q STRUCT, i1 INT) USING parquet") + sql("CREATE TABLE t1 (q STRUCT, i1 INT) USING hive") sql("ALTER TABLE t1 ADD COLUMNS (newcol1 STRUCT<`$col1`:STRING, col2:Int>)") assert(spark.table("t1").schema == CatalystSqlParser.parseTableSchema( "q STRUCT, i1 INT,newcol1 STRUCT<`$col1`:STRING, col2:Int>")) } } - test("SPARK-46934: alter hive table tests with nested types") { - withTable("t1") { - sql("CREATE TABLE t1 (q STRUCT, i1 INT) USING hive") - val e = intercept[AnalysisException]( - sql("ALTER TABLE t1 ADD COLUMNS (newcol1 STRUCT<`$col1`:STRING, col2:Int>)")) - checkError( - exception = e, - condition = "_LEGACY_ERROR_TEMP_3065", - parameters = Map( - "clazz" -> "java.lang.IllegalArgumentException", - "msg" -> e.getCause.getMessage)) - } - } - test("SPARK-26630: table with old input format and without partitioned will use HadoopRDD") { withTable("table_old", "table_ctas_old") { sql( @@ -2868,18 +2849,38 @@ class HiveDDLSuite } test("SPARK-47101 checks if nested column names do not include invalid characters") { - Seq(",", ":", ";", "^", "\\", "/", "%").foreach { c => + // delimiter characters + Seq(",", ":").foreach { c => val typ = s"array>" + // The regex is from HiveClientImpl.getSparkSQLDataType, please keep them in sync. + val replaced = typ.replaceAll("`", "").replaceAll("(?<=struct<|,)([^,<:]+)(?=:)", "`$1`") withTable("t") { - val e = intercept[AnalysisException] { + checkError( + exception = intercept[SparkException] { sql(s"CREATE TABLE t (a $typ) USING hive") - } + }, + condition = "CANNOT_RECOGNIZE_HIVE_TYPE", + parameters = Map( + "fieldType" -> toSQLType(replaced), + "fieldName" -> "`a`") + ) + } + } + // other special characters + Seq(";", "^", "\\", "/", "%").foreach { c => + val typ = s"array>" + val replaced = typ.replaceAll("`", "") + val msg = s"java.lang.IllegalArgumentException: Error: : expected at the position " + + s"16 of '$replaced' but '$c' is found." + withTable("t") { checkError( - exception = e, + exception = intercept[AnalysisException] { + sql(s"CREATE TABLE t (a $typ) USING hive") + }, condition = "_LEGACY_ERROR_TEMP_3065", parameters = Map( - "clazz" -> e.getCause.getClass.getName, - "msg" -> e.getCause.getMessage) + "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", + "msg" -> msg) ) } } From 2d498d51ab8333238e77e5e7de952ff3b0276b3b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 14 Jan 2025 06:49:53 -0800 Subject: [PATCH 425/438] Revert "[SPARK-50804][SQL] to_protobuf() should not throw MatchError" This reverts commit e4cbb91eb2723ebaa91ac308670bbbb39cca6b0b. --- .../sql/protobuf/ProtobufSerializer.scala | 3 --- .../sql/protobuf/ProtobufFunctionsSuite.scala | 27 ------------------- .../toFromProtobufSqlFunctions.scala | 20 ++++---------- 3 files changed, 5 insertions(+), 45 deletions(-) diff --git a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala index 65e8cce0d056e..1c64e70755d5c 100644 --- a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala +++ b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala @@ -47,9 +47,6 @@ private[sql] class ProtobufSerializer( } private val converter: Any => Any = { - assert( - rootCatalystType.isInstanceOf[StructType], - "ProtobufSerializer's root catalyst type must be a struct type") val baseConverter = try { rootCatalystType match { diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala index c1f5186b8333f..44a8339ac1f02 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala @@ -1721,33 +1721,6 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot } } - test("non-struct SQL type") { - val dfWithInt = spark - .range(1) - .select( - lit(9999).as("int_col") - ) - - val parseError = intercept[AnalysisException] { - dfWithInt.select( - to_protobuf_wrapper($"int_col", "SimpleMessageEnum", Some(testFileDesc))).collect() - } - val descMsg = testFileDesc.map("%02X".format(_)).mkString("") - checkError( - exception = parseError, - condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", - parameters = Map( - "sqlExpr" -> - s"""\"to_protobuf(int_col, SimpleMessageEnum, X'$descMsg', NULL)\"""", - "msg" -> ("The first argument of the TO_PROTOBUF SQL function must be a struct type"), - "hint" -> "" - ), - queryContext = Array(ExpectedContext( - fragment = "fn", - callSitePattern = ".*")) - ) - } - test("test unsigned integer types") { // Test that we correctly handle unsigned integer parsing. // We're using Integer/Long's `MIN_VALUE` as it has a 1 in the sign bit. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala index 380df38ab049b..96bcf49dbd097 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.util.ArrayBasedMapData import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{BinaryType, MapType, NullType, StringType, StructType} +import org.apache.spark.sql.types.{BinaryType, MapType, NullType, StringType} import org.apache.spark.sql.util.ProtobufUtils import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -238,15 +238,6 @@ case class ToProtobuf( } override def checkInputDataTypes(): TypeCheckResult = { - val colTypeCheck = first.dataType match { - case _: StructType => None - case _ => - Some( - TypeCheckResult.TypeCheckFailure( - "The first argument of the TO_PROTOBUF SQL function must be a struct type") - ) - } - val messageNameCheck = messageName.dataType match { case _: StringType if messageName.foldable => None case _ => @@ -271,11 +262,10 @@ case class ToProtobuf( "strings to strings containing the options to use for converting the value to " + "Protobuf format")) } - colTypeCheck.getOrElse( - messageNameCheck.getOrElse( - descFilePathCheck.getOrElse( - optionsCheck.getOrElse(TypeCheckResult.TypeCheckSuccess) - ) + + messageNameCheck.getOrElse( + descFilePathCheck.getOrElse( + optionsCheck.getOrElse(TypeCheckResult.TypeCheckSuccess) ) ) } From 1fd836271b6f284625ba0f50551bea87071d5d65 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 14 Jan 2025 20:14:00 +0300 Subject: [PATCH 426/438] [SPARK-50403][SQL] Fix parameterized `EXECUTE IMMEDIATE` ### What changes were proposed in this pull request? 1. Remove the assert of single parameterized query because it restricts parameterization of `EXECUTE IMMEDIATE`. The assert checks that only single node of the type `ParameterizedQuery` presents in a query, but `EXECUTE IMMEDIATE` adds one `ParameterizedQuery` + `sql()` adds another `ParameterizedQuery`. So, this case is prohibited by the assert though it is a valid use case from user's perspective. 2. Modify parameters binding: stop the bind procedure when face to another parameterized query. For example, the sql text passed to `spark.sql()` contains `EXECUTE IMMEDIATE`, and `sql()` parameters don't affect on the sql query string in `EXECUTE IMMEDIATE`. 3. Allow parameters in `EXECUTE IMMEDIATE` variables. ### Why are the changes needed? Before the changes, the following query fails with the internal error: ```scala scala> spark.sql("execute immediate 'select ?' using 1", Map("param1" -> "1")) org.apache.spark.SparkException: [INTERNAL_ERROR] The Spark SQL phase analysis failed with an internal error. You hit a bug in Spark or the Spark plugins you use. Please, report this bug to the corresponding communities or vendors, and provide the full stack trace. SQLSTATE: XX000 ``` ### Does this PR introduce _any_ user-facing change? Yes, the query above returns proper results instead of the internal error: ```scala scala> spark.sql("execute immediate 'select ?' using 1", Map("param1" -> "1")) val res2: org.apache.spark.sql.DataFrame = [1: int] ``` ### How was this patch tested? By running the new test: ``` $ build/sbt "sql/test:testOnly org.apache.spark.sql.ParametersSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49442 from MaxGekk/fix-params-execute-immediate. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../catalyst/analysis/executeImmediate.scala | 9 +++-- .../sql/catalyst/analysis/parameters.scala | 33 +++++++------------ .../apache/spark/sql/ParametersSuite.scala | 32 ++++++++++++++++++ 3 files changed, 49 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index c92171ec5c750..b452ca15bed58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -54,15 +54,18 @@ class SubstituteExecuteImmediate(val catalogManager: CatalogManager) def resolveVariable(e: Expression): Expression = { /** - * We know that the expression is either UnresolvedAttribute or Alias, as passed from the - * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it - * is an Alias, we resolve the child and return an Alias with the same name. + * We know that the expression is either UnresolvedAttribute, Alias or Parameter, as passed from + * the parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If + * it is an Alias, we resolve the child and return an Alias with the same name. If it is + * a Parameter, we leave it as is because the parameter belongs to another parameterized + * query and should be resolved later. */ e match { case u: UnresolvedAttribute => getVariableReference(u, u.nameParts) case a: Alias => Alias(resolveVariable(a.child), a.name)() + case p: Parameter => p case other => throw QueryCompilationErrors.unsupportedParameterExpression(other) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index de73747769469..2cfc2a8c90dc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -104,18 +104,6 @@ case class PosParameterizedQuery(child: LogicalPlan, args: Seq[Expression]) copy(child = newChild) } -/** - * Base class for rules that process parameterized queries. - */ -abstract class ParameterizedQueryProcessor extends Rule[LogicalPlan] { - def assertUnresolvedPlanHasSingleParameterizedQuery(plan: LogicalPlan): Unit = { - if (plan.containsPattern(PARAMETERIZED_QUERY)) { - val parameterizedQueries = plan.collect { case p: ParameterizedQuery => p } - assert(parameterizedQueries.length == 1) - } - } -} - /** * Moves `ParameterizedQuery` inside `SupervisingCommand` for their supervised plans to be * resolved later by the analyzer. @@ -127,10 +115,8 @@ abstract class ParameterizedQueryProcessor extends Rule[LogicalPlan] { * `PosParameterizedQuery(ExplainCommand(ExplainCommand(SomeQuery(...))))` => * `ExplainCommand(ExplainCommand(PosParameterizedQuery(SomeQuery(...))))` */ -object MoveParameterizedQueriesDown extends ParameterizedQueryProcessor { +object MoveParameterizedQueriesDown extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { - assertUnresolvedPlanHasSingleParameterizedQuery(plan) - plan.resolveOperatorsWithPruning(_.containsPattern(PARAMETERIZED_QUERY)) { case pq: ParameterizedQuery if pq.exists(isSupervisingCommand) => moveParameterizedQueryIntoSupervisingCommand(pq) @@ -161,7 +147,7 @@ object MoveParameterizedQueriesDown extends ParameterizedQueryProcessor { * by collection constructor functions such as `map()`, `array()`, `struct()` * from the user-specified arguments. */ -object BindParameters extends ParameterizedQueryProcessor with QueryErrorsBase { +object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { private def checkArgs(args: Iterable[(String, Expression)]): Unit = { def isNotAllowed(expr: Expression): Boolean = expr.exists { case _: Literal | _: CreateArray | _: CreateNamedStruct | @@ -176,15 +162,18 @@ object BindParameters extends ParameterizedQueryProcessor with QueryErrorsBase { } } - private def bind(p: LogicalPlan)(f: PartialFunction[Expression, Expression]): LogicalPlan = { - p.resolveExpressionsWithPruning(_.containsPattern(PARAMETER)) (f orElse { - case sub: SubqueryExpression => sub.withNewPlan(bind(sub.plan)(f)) - }) + private def bind(p0: LogicalPlan)(f: PartialFunction[Expression, Expression]): LogicalPlan = { + var stop = false + p0.resolveOperatorsDownWithPruning(_.containsPattern(PARAMETER) && !stop) { + case p1 => + stop = p1.isInstanceOf[ParameterizedQuery] + p1.transformExpressionsWithPruning(_.containsPattern(PARAMETER)) (f orElse { + case sub: SubqueryExpression => sub.withNewPlan(bind(sub.plan)(f)) + }) + } } override def apply(plan: LogicalPlan): LogicalPlan = { - assertUnresolvedPlanHasSingleParameterizedQuery(plan) - plan.resolveOperatorsWithPruning(_.containsPattern(PARAMETERIZED_QUERY)) { // We should wait for `CTESubstitution` to resolve CTE before binding parameters, as CTE // relations are not children of `UnresolvedWith`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 2ac8ed26868a0..bb1363f1c58c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -769,4 +769,36 @@ class ParametersSuite extends QueryTest with SharedSparkSession with PlanTest { checkAnswer(spark.sql(query(":cte"), args = Map("cte" -> "t1")), Row(1)) checkAnswer(spark.sql(query("?"), args = Array("t1")), Row(1)) } + + test("SPARK-50403: parameterized execute immediate") { + checkAnswer(spark.sql("execute immediate 'select ?' using ?", Array(1)), Row(1)) + checkAnswer(spark.sql("execute immediate 'select ?, ?' using ?, 2", Array(1)), Row(1, 2)) + checkError( + exception = intercept[AnalysisException] { + spark.sql("execute immediate 'select ?, ?' using 1", Array(2)) + }, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "_10"), + context = ExpectedContext("?", 10, 10)) + + checkAnswer(spark.sql("execute immediate 'select ?' using 1", Map("param1" -> "1")), Row(1)) + checkAnswer(spark.sql("execute immediate 'select :param1' using :param2 as param1", + Map("param2" -> 42)), Row(42)) + checkAnswer(spark.sql( + "execute immediate 'select :param1, :param2' using :param2 as param1, 43 as param2", + Map("param2" -> 42)), Row(42, 43)) + checkAnswer(spark.sql("execute immediate 'select :param' using 0 as param", + Map("param" -> 42)), Row(0)) + checkError( + exception = intercept[AnalysisException] { + spark.sql("execute immediate 'select :param1, :param2' using 1 as param1", + Map("param2" -> 2)) + }, + condition = "UNBOUND_SQL_PARAMETER", + parameters = Map("name" -> "param2"), + context = ExpectedContext(":param2", 16, 22)) + + checkAnswer(spark.sql("execute immediate 'select ?' using :param", Map("param" -> 2)), Row(2)) + checkAnswer(spark.sql("execute immediate 'select :param' using ? as param", Array(3)), Row(3)) + } } From 488c362471687eba9a0f6ed4280b007dba8a0050 Mon Sep 17 00:00:00 2001 From: Yuchuan Huang Date: Tue, 14 Jan 2025 17:11:16 -0800 Subject: [PATCH 427/438] [SPARK-50788][TESTS] Add Benchmark for Large-Row Dataframe ### What changes were proposed in this pull request? This PR introduces LargeRowBenchmark, a micro benchmark to the suite of spark.sql.execution.benchmark. A corresponding function is also added to create large-row dataframes during the benchmark running time. ### Why are the changes needed? Large-row dataframes, especially dataframes with large string cells are becoming common with business like online customer chatting. However, it is unknown how well/bad Spark would be able to support them. This benchmark aims to provide a baseline to indicate Spark's performance and limitation on large-row dataframes. It will also be included in future performance regression check. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? It is tested in Github Action and manual reviewed. https://github.com/yhuang-db/spark/actions/runs/12716337093 (Java 17) https://github.com/yhuang-db/spark/actions/runs/12716339158 (Java 21) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49447 from yhuang-db/large-row-benchmark. Authored-by: Yuchuan Huang Signed-off-by: Gengliang Wang --- .../LargeRowBenchmark-jdk21-results.txt | 26 ++++++ .../benchmarks/LargeRowBenchmark-results.txt | 26 ++++++ .../benchmark/LargeRowBenchmark.scala | 85 +++++++++++++++++++ 3 files changed, 137 insertions(+) create mode 100644 sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt create mode 100644 sql/core/benchmarks/LargeRowBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/LargeRowBenchmark.scala diff --git a/sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt b/sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt new file mode 100644 index 0000000000000..dbcf544b492d9 --- /dev/null +++ b/sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt @@ -0,0 +1,26 @@ +================================================================================================ +Large Row Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor +#rows: 100, #cols: 10, cell: 1.3 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +built-in UPPER 5909 6154 347 0.0 59088236.5 1.0X +udf UPPER 4106 4364 364 0.0 41062501.9 1.4X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor +#rows: 1, #cols: 1, cell: 300.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +built-in UPPER 1317 1319 3 0.0 1317449498.0 1.0X +udf UPPER 954 975 25 0.0 953744994.0 1.4X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor +#rows: 1, #cols: 200, cell: 1.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +built-in UPPER 1118 1138 28 0.0 1117901962.0 1.0X +udf UPPER 1145 1210 91 0.0 1145234313.0 1.0X + + diff --git a/sql/core/benchmarks/LargeRowBenchmark-results.txt b/sql/core/benchmarks/LargeRowBenchmark-results.txt new file mode 100644 index 0000000000000..9fafe282238b6 --- /dev/null +++ b/sql/core/benchmarks/LargeRowBenchmark-results.txt @@ -0,0 +1,26 @@ +================================================================================================ +Large Row Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor +#rows: 100, #cols: 10, cell: 1.3 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +built-in UPPER 6610 6651 58 0.0 66101681.9 1.0X +udf UPPER 4289 4291 3 0.0 42892607.0 1.5X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor +#rows: 1, #cols: 1, cell: 300.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +built-in UPPER 1492 1510 26 0.0 1492292577.0 1.0X +udf UPPER 1033 1034 1 0.0 1032584220.0 1.4X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1017-azure +AMD EPYC 7763 64-Core Processor +#rows: 1, #cols: 200, cell: 1.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +built-in UPPER 1271 1290 28 0.0 1270654457.0 1.0X +udf UPPER 1397 1558 228 0.0 1396607518.0 0.9X + + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/LargeRowBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/LargeRowBenchmark.scala new file mode 100644 index 0000000000000..8b4f78e79913a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/LargeRowBenchmark.scala @@ -0,0 +1,85 @@ +/* + * 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.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.functions.lit + +/** + * Benchmark to measure performance for large row table. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * --jars , + * 2. build/sbt "sql/Test/runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/Test/runMain " + * Results will be written to "benchmarks/LargeRowBenchmark-results.txt". + * }}} + */ +object LargeRowBenchmark extends SqlBasedBenchmark { + + /** + * Prepares a table with large row for benchmarking. The table will be written into + * the given path. + */ + private def writeLargeRow(path: String, rowsNum: Int, numCols: Int, cellSizeMb: Double): Unit = { + val stringLength = (cellSizeMb * 1024 * 1024).toInt + spark.range(rowsNum) + .select(Seq.tabulate(numCols)(i => lit("a" * stringLength).as(s"col$i")): _*) + .write.parquet(path) + } + + private def runLargeRowBenchmark(rowsNum: Int, numCols: Int, cellSizeMb: Double): Unit = { + withTempPath { path => + val benchmark = new Benchmark( + s"#rows: $rowsNum, #cols: $numCols, cell: $cellSizeMb MB", rowsNum, output = output) + writeLargeRow(path.getAbsolutePath, rowsNum, numCols, cellSizeMb) + val df = spark.read.parquet(path.getAbsolutePath) + df.createOrReplaceTempView("T") + benchmark.addCase("built-in UPPER") { _ => + val sqlSelect = df.columns.map(c => s"UPPER($c) as $c").mkString(", ") + spark.sql(s"SELECT $sqlSelect FROM T").noop() + } + benchmark.addCase("udf UPPER") { _ => + val sqlSelect = df.columns.map(c => s"udfUpper($c) as $c").mkString(", ") + spark.sql(s"SELECT $sqlSelect FROM T").noop() + } + benchmark.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("Large Row Benchmark") { + val udfUpper = (s: String) => s.toUpperCase() + spark.udf.register("udfUpper", udfUpper(_: String): String) + + val benchmarks = Array( + Map("rows" -> 100, "cols" -> 10, "cellSizeMb" -> 1.3), // OutOfMemory @ 100, 10, 1.4 + Map("rows" -> 1, "cols" -> 1, "cellSizeMb" -> 300.0), // OutOfMemory @ 1, 1, 400 + Map("rows" -> 1, "cols" -> 200, "cellSizeMb" -> 1.0) // OutOfMemory @ 1, 300, 1 + ) + + benchmarks.foreach { b => + val rows = b("rows").asInstanceOf[Int] + val cols = b("cols").asInstanceOf[Int] + val cellSizeMb = b("cellSizeMb").asInstanceOf[Double] + runLargeRowBenchmark(rows, cols, cellSizeMb) + } + } + } +} From f0842d049ded394f7dbf397bc23548ae6c1adc86 Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Wed, 15 Jan 2025 10:00:13 +0800 Subject: [PATCH 428/438] [SPARK-50813][SQL] Allow only unqualified label names inside SQL Scripts ### What changes were proposed in this pull request? Allow only unqualified label names inside SQL Scripts. Valid examples: - `label` - `label_1` Invalid examples: - `part1.part2` - `lbl.1` ### Why are the changes needed? This change fixed the introduced bug in label behavior. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New and existing tests in `SqlScriptingParserSuite`. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49485 from miland-db/milan-dankovic_data/fix-labels-to-be-unqualified. Authored-by: Milan Dankovic Signed-off-by: Wenchen Fan --- .../main/resources/error/error-conditions.json | 5 +++++ .../spark/sql/catalyst/parser/ParserUtils.scala | 8 ++++++++ .../spark/sql/errors/SqlScriptingErrors.scala | 10 ++++++++++ .../catalyst/parser/SqlScriptingParserSuite.scala | 15 +++++++++++++++ 4 files changed, 38 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 3c11f5b360bb2..71eae4de89c8a 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2782,6 +2782,11 @@ "message" : [ "ITERATE statement cannot be used with a label that belongs to a compound (BEGIN...END) body." ] + }, + "QUALIFIED_LABEL_NAME" : { + "message" : [ + "Label cannot be qualified." + ] } }, "sqlState" : "42K0L" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 8471c9f9dff13..1bc4f95f95daf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -163,6 +163,14 @@ class SqlScriptingLabelContext { bl.multipartIdentifier().getText, el.multipartIdentifier().getText) } + case (Some(bl: BeginLabelContext), _) + if bl.multipartIdentifier().parts.size() > 1 => + withOrigin(bl) { + throw SqlScriptingErrors.labelCannotBeQualified( + CurrentOrigin.get, + bl.multipartIdentifier().getText.toLowerCase(Locale.ROOT) + ) + } case (None, Some(el: EndLabelContext)) => withOrigin(el) { throw SqlScriptingErrors.endLabelWithoutBeginLabel( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala index 2a4b8fde6989c..da492cce22f2c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala @@ -133,4 +133,14 @@ private[sql] object SqlScriptingErrors { cause = null, messageParameters = Map("labelName" -> toSQLStmt(labelName))) } + + def labelCannotBeQualified( + origin: Origin, + labelName: String): Throwable = { + new SqlScriptingException( + origin = origin, + errorClass = "INVALID_LABEL_USAGE.QUALIFIED_LABEL_NAME", + cause = null, + messageParameters = Map("labelName" -> toSQLStmt(labelName))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index c9e2f42e164f9..e129c6dbba052 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -2000,6 +2000,21 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(tree.collection(4).asInstanceOf[ForStatement].label.get == "lbl") } + test("qualified label name: label cannot be qualified") { + val sqlScriptText = + """ + |BEGIN + | part1.part2: BEGIN + | END; + |END""".stripMargin + checkError( + exception = intercept[SqlScriptingException] { + parsePlan(sqlScriptText) + }, + condition = "INVALID_LABEL_USAGE.QUALIFIED_LABEL_NAME", + parameters = Map("labelName" -> "PART1.PART2")) + } + test("unique label names: nested labeled scope statements") { val sqlScriptText = """BEGIN From f223b8da9e23e4e028e145e0d4dd74eeae5d2d52 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 15 Jan 2025 10:04:46 +0800 Subject: [PATCH 429/438] [SPARK-50818][PYTHON] Replace `has_numpy` with `have_numpy` ### What changes were proposed in this pull request? Replace `has_numpy` with `have_numpy` ### Why are the changes needed? code clean up ### 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 #49490 from zhengruifeng/py_has_numpy. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/builtin.py | 9 ++++---- python/pyspark/sql/plot/plotly.py | 4 ++-- python/pyspark/sql/streaming/state.py | 5 +++-- .../stateful_processor_api_client.py | 6 +++-- python/pyspark/sql/types.py | 22 ++++++++++++++----- python/pyspark/sql/utils.py | 8 ------- 6 files changed, 30 insertions(+), 24 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 6b74c9dc2dd74..93ac6da1e14c5 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -65,7 +65,6 @@ from pyspark.sql.utils import ( to_str as _to_str, - has_numpy as _has_numpy, try_remote_functions as _try_remote_functions, get_active_spark_context as _get_active_spark_context, enum_to_value as _enum_to_value, @@ -79,8 +78,6 @@ UserDefinedFunctionLike, ) -if _has_numpy: - import numpy as np # Note to developers: all of PySpark functions here take string as column names whenever possible. # Namely, if columns are referred as arguments, they can always be both Column or string, @@ -254,6 +251,8 @@ def lit(col: Any) -> Column: | [true, false]| []| [1.5, 0.1]| [a, b, c]| +------------------+-------+-----------------+--------------------+ """ + from pyspark.testing.utils import have_numpy + if isinstance(col, Column): return col elif isinstance(col, list): @@ -262,7 +261,9 @@ def lit(col: Any) -> Column: errorClass="COLUMN_IN_LIST", messageParameters={"func_name": "lit"} ) return array(*[lit(item) for item in col]) - elif _has_numpy: + elif have_numpy: + import numpy as np + if isinstance(col, np.generic): dt = _from_numpy_type(col.dtype) if dt is None: diff --git a/python/pyspark/sql/plot/plotly.py b/python/pyspark/sql/plot/plotly.py index c7691f144ffaf..526a36033e2fc 100644 --- a/python/pyspark/sql/plot/plotly.py +++ b/python/pyspark/sql/plot/plotly.py @@ -151,7 +151,7 @@ def plot_box(data: "DataFrame", **kwargs: Any) -> "Figure": def plot_kde(data: "DataFrame", **kwargs: Any) -> "Figure": - from pyspark.sql.utils import has_numpy + from pyspark.testing.utils import have_numpy from pyspark.sql.pandas.utils import require_minimum_pandas_version require_minimum_pandas_version() @@ -166,7 +166,7 @@ def plot_kde(data: "DataFrame", **kwargs: Any) -> "Figure": colnames = process_column_param(kwargs.pop("column", None), data) ind = PySparkKdePlotBase.get_ind(data.select(*colnames), kwargs.pop("ind", None)) - if has_numpy: + if have_numpy: import numpy as np if isinstance(ind, np.ndarray): diff --git a/python/pyspark/sql/streaming/state.py b/python/pyspark/sql/streaming/state.py index 0ea5590ef2e65..cd067a8413e1c 100644 --- a/python/pyspark/sql/streaming/state.py +++ b/python/pyspark/sql/streaming/state.py @@ -19,7 +19,6 @@ from typing import Tuple, Optional from pyspark.sql.types import Row, StructType, TimestampType -from pyspark.sql.utils import has_numpy from pyspark.errors import PySparkTypeError, PySparkValueError, PySparkRuntimeError __all__ = ["GroupState", "GroupStateTimeout"] @@ -132,6 +131,8 @@ def update(self, newValue: Tuple) -> None: """ Update the value of the state. The value of the state cannot be null. """ + from pyspark.testing.utils import have_numpy + if newValue is None: raise PySparkTypeError( errorClass="CANNOT_BE_NONE", @@ -139,7 +140,7 @@ def update(self, newValue: Tuple) -> None: ) converted = [] - if has_numpy: + if have_numpy: import numpy as np # In order to convert NumPy types to Python primitive types. diff --git a/python/pyspark/sql/streaming/stateful_processor_api_client.py b/python/pyspark/sql/streaming/stateful_processor_api_client.py index c25b2a2392a69..6fd56481bc612 100644 --- a/python/pyspark/sql/streaming/stateful_processor_api_client.py +++ b/python/pyspark/sql/streaming/stateful_processor_api_client.py @@ -28,7 +28,6 @@ Row, ) from pyspark.sql.pandas.types import convert_pandas_using_numpy_type -from pyspark.sql.utils import has_numpy from pyspark.serializers import CPickleSerializer from pyspark.errors import PySparkRuntimeError import uuid @@ -414,8 +413,11 @@ def _receive_str(self) -> str: return self.utf8_deserializer.loads(self.sockfile) def _serialize_to_bytes(self, schema: StructType, data: Tuple) -> bytes: + from pyspark.testing.utils import have_numpy + converted = [] - if has_numpy: + + if have_numpy: import numpy as np # In order to convert NumPy types to Python primitive types. diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index b913e05e16d2a..cc0285233dcc4 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -49,7 +49,6 @@ from pyspark.util import is_remote_only, JVM_INT_MAX from pyspark.serializers import CloudPickleSerializer from pyspark.sql.utils import ( - has_numpy, get_active_spark_context, escape_meta_characters, StringConcat, @@ -65,9 +64,6 @@ PySparkKeyError, ) -if has_numpy: - import numpy as np - if TYPE_CHECKING: import numpy as np from py4j.java_gateway import GatewayClient, JavaGateway, JavaClass @@ -3237,7 +3233,13 @@ def convert(self, obj: datetime.timedelta, gateway_client: "GatewayClient") -> " class NumpyScalarConverter: def can_convert(self, obj: Any) -> bool: - return has_numpy and isinstance(obj, np.generic) + from pyspark.testing.utils import have_numpy + + if have_numpy: + import numpy as np + + return isinstance(obj, np.generic) + return False def convert(self, obj: "np.generic", gateway_client: "GatewayClient") -> Any: return obj.item() @@ -3248,6 +3250,8 @@ def _from_numpy_type_to_java_type( self, nt: "np.dtype", gateway: "JavaGateway" ) -> Optional["JavaClass"]: """Convert NumPy type to Py4J Java type.""" + import numpy as np + if nt in [np.dtype("int8"), np.dtype("int16")]: # Mapping int8 to gateway.jvm.byte causes # TypeError: 'bytes' object does not support item assignment @@ -3268,7 +3272,13 @@ def _from_numpy_type_to_java_type( return None def can_convert(self, obj: Any) -> bool: - return has_numpy and isinstance(obj, np.ndarray) and obj.ndim == 1 + from pyspark.testing.utils import have_numpy + + if have_numpy: + import numpy as np + + return isinstance(obj, np.ndarray) and obj.ndim == 1 + return False def convert(self, obj: "np.ndarray", gateway_client: "GatewayClient") -> "JavaGateway": from pyspark import SparkContext diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index fbe0b274b496c..855496ff3b7ca 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -62,14 +62,6 @@ from pyspark.sql.dataframe import DataFrame from pyspark.pandas._typing import IndexOpsLike, SeriesOrIndex -has_numpy: bool = False -try: - import numpy as np # noqa: F401 - - has_numpy = True -except ImportError: - pass - FuncT = TypeVar("FuncT", bound=Callable[..., Any]) From 6f3b778e1a12901726c2a35072904f36f46f7888 Mon Sep 17 00:00:00 2001 From: Yi Wu Date: Wed, 15 Jan 2025 11:33:42 +0800 Subject: [PATCH 430/438] [SPARK-50768][CORE] Introduce TaskContext.createResourceUninterruptibly to avoid stream leak by task interruption ### What changes were proposed in this pull request? This PR fixes the potential stream leak issue by introduing `TaskContext.createResourceUninterruptibly`. When a task is using `TaskContext.createResourceUninterruptibly` to create the resource, the task would be marked as uninterruptible. Thus, any interruption request during the call to `TaskContext.createResourceUninterruptibly` would be delayed until the creation finishes. This PR introduces an new lock contention between `Task.kill` and `TaskContext.createResourceUninterruptibly`. But I think it is acceptable given that both are not on the hot-path. (I will submmit a followup to apply `TaskContext.createResourceUninterruptibly` in the codebase if this PR is approved by the community.) ### Why are the changes needed? We had https://github.com/apache/spark/pull/48483 tried to fix the potential stream leak issue by task interruption. It mitigates the issue by using ```scala def tryInitializeResource[R <: Closeable, T](createResource: => R)(initialize: R => T): T = { val resource = createResource try { initialize(resource) } catch { case e: Throwable => resource.close() throw e } } ``` But this utility function has an issue that `resource.close()` would leak open resouces if `initialize(resource)` also created some resources internally, especially when `initialize(resource)` is interrupted (See the example of `InterruptionSensitiveInputStream` in the test). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added a unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49413 from Ngone51/dev-interrupt. Authored-by: Yi Wu Signed-off-by: yangjie01 --- .../org/apache/spark/BarrierTaskContext.scala | 13 ++ .../scala/org/apache/spark/TaskContext.scala | 22 ++- .../org/apache/spark/TaskContextImpl.scala | 43 ++++++ .../org/apache/spark/scheduler/Task.scala | 20 ++- .../apache/spark/JobCancellationSuite.scala | 139 +++++++++++++++++- project/MimaExcludes.scala | 5 + 6 files changed, 235 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 5b18ab95b07eb..b5c6033bd9da4 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -17,6 +17,7 @@ package org.apache.spark +import java.io.Closeable import java.util.{Properties, TimerTask} import java.util.concurrent.{ScheduledThreadPoolExecutor, TimeUnit} @@ -273,6 +274,18 @@ class BarrierTaskContext private[spark] ( } override private[spark] def getLocalProperties: Properties = taskContext.getLocalProperties + + override private[spark] def interruptible(): Boolean = taskContext.interruptible() + + override private[spark] def pendingInterrupt(threadToInterrupt: Option[Thread], reason: String) + : Unit = { + taskContext.pendingInterrupt(threadToInterrupt, reason) + } + + override private[spark] def createResourceUninterruptibly[T <: Closeable](resourceBuilder: => T) + : T = { + taskContext.createResourceUninterruptibly(resourceBuilder) + } } @Experimental diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 15ddd08fb4aef..5384fd86a8f19 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -17,7 +17,7 @@ package org.apache.spark -import java.io.Serializable +import java.io.Closeable import java.util.Properties import org.apache.spark.annotation.{DeveloperApi, Evolving, Since} @@ -305,4 +305,24 @@ abstract class TaskContext extends Serializable { /** Gets local properties set upstream in the driver. */ private[spark] def getLocalProperties: Properties + + /** Whether the current task is allowed to interrupt. */ + private[spark] def interruptible(): Boolean + + /** + * Pending the interruption request until the task is able to + * interrupt after creating the resource uninterruptibly. + */ + private[spark] def pendingInterrupt(threadToInterrupt: Option[Thread], reason: String): Unit + + /** + * Creating a closeable resource uninterruptibly. A task is not allowed to interrupt in this + * state until the resource creation finishes. E.g., + * {{{ + * val linesReader = TaskContext.get().createResourceUninterruptibly { + * new HadoopFileLinesReader(file, parser.options.lineSeparatorInRead, conf) + * } + * }}} + */ + private[spark] def createResourceUninterruptibly[T <: Closeable](resourceBuilder: => T): T } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 8167952d6b87f..f0e844289b9db 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -17,6 +17,7 @@ package org.apache.spark +import java.io.Closeable import java.util.{Properties, Stack} import javax.annotation.concurrent.GuardedBy @@ -82,6 +83,13 @@ private[spark] class TaskContextImpl( // If defined, the corresponding task has been killed and this option contains the reason. @volatile private var reasonIfKilled: Option[String] = None + // The pending interruption request, which is blocked by uninterruptible resource creation. + // Should be protected by `TaskContext.synchronized`. + private var pendingInterruptRequest: Option[(Option[Thread], String)] = None + + // Whether this task is able to be interrupted. Should be protected by `TaskContext.synchronized`. + private var _interruptible = true + // Whether the task has completed. private var completed: Boolean = false @@ -296,4 +304,39 @@ private[spark] class TaskContextImpl( private[spark] override def fetchFailed: Option[FetchFailedException] = _fetchFailedException private[spark] override def getLocalProperties: Properties = localProperties + + + override def interruptible(): Boolean = TaskContext.synchronized(_interruptible) + + override def pendingInterrupt(threadToInterrupt: Option[Thread], reason: String): Unit = { + TaskContext.synchronized { + pendingInterruptRequest = Some((threadToInterrupt, reason)) + } + } + + def createResourceUninterruptibly[T <: Closeable](resourceBuilder: => T): T = { + + @inline def interruptIfRequired(): Unit = { + pendingInterruptRequest.foreach { case (threadToInterrupt, reason) => + markInterrupted(reason) + threadToInterrupt.foreach(_.interrupt()) + } + killTaskIfInterrupted() + } + + TaskContext.synchronized { + interruptIfRequired() + _interruptible = false + } + try { + val resource = resourceBuilder + addTaskCompletionListener[Unit](_ => resource.close()) + resource + } finally { + TaskContext.synchronized { + _interruptible = true + interruptIfRequired() + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index f511aed6d2166..e21ec77ce69ec 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -22,6 +22,7 @@ import java.util.Properties import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config.APP_CALLER_CONTEXT import org.apache.spark.internal.plugin.PluginContainer import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} @@ -70,7 +71,7 @@ private[spark] abstract class Task[T]( val jobId: Option[Int] = None, val appId: Option[String] = None, val appAttemptId: Option[String] = None, - val isBarrier: Boolean = false) extends Serializable { + val isBarrier: Boolean = false) extends Serializable with Logging { @transient lazy val metrics: TaskMetrics = SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics)) @@ -231,10 +232,19 @@ private[spark] abstract class Task[T]( require(reason != null) _reasonIfKilled = reason if (context != null) { - context.markInterrupted(reason) - } - if (interruptThread && taskThread != null) { - taskThread.interrupt() + TaskContext.synchronized { + if (context.interruptible()) { + context.markInterrupted(reason) + if (interruptThread && taskThread != null) { + taskThread.interrupt() + } + } else { + logInfo(log"Task ${MDC(LogKeys.TASK_ID, context.taskAttemptId())} " + + log"is currently not interruptible. ") + val threadToInterrupt = if (interruptThread) Option(taskThread) else None + context.pendingInterrupt(threadToInterrupt, reason) + } + } } } } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index ca51e61f5ed44..65ed2684a5b00 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import java.io.{File, FileOutputStream, InputStream, ObjectOutputStream} import java.util.concurrent.{Semaphore, TimeUnit} import java.util.concurrent.atomic.AtomicInteger @@ -35,7 +36,7 @@ import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Deploy._ import org.apache.spark.scheduler.{JobFailed, SparkListener, SparkListenerExecutorRemoved, SparkListenerJobEnd, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{ThreadUtils, Utils} /** * Test suite for cancelling running jobs. We run the cancellation tasks for single job action @@ -712,6 +713,142 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft assert(executionOfInterruptibleCounter.get() < numElements) } + Seq(true, false).foreach { interruptible => + + val (hint1, hint2) = if (interruptible) { + (" not", "") + } else { + ("", " not") + } + + val testName = s"SPARK-50768:$hint1 use TaskContext.createResourceUninterruptibly " + + s"would$hint2 cause stream leak on task interruption" + + test(testName) { + import org.apache.spark.JobCancellationSuite._ + withTempDir { dir => + + // `InterruptionSensitiveInputStream` is designed to easily leak the underlying + // stream when task thread interruption happens during its initialization, as + // the reference to the underlying stream is intentionally not available to + // `InterruptionSensitiveInputStream` at that point. + class InterruptionSensitiveInputStream(fileHint: String) extends InputStream { + private var underlying: InputStream = _ + + def initialize(): InputStream = { + val in: InputStream = new InputStream { + + open() + + private def dumpFile(typeName: String): Unit = { + var fileOut: FileOutputStream = null + var objOut: ObjectOutputStream = null + try { + val file = new File(dir, s"$typeName.$fileHint") + fileOut = new FileOutputStream(file) + objOut = new ObjectOutputStream(fileOut) + objOut.writeBoolean(true) + objOut.flush() + } finally { + if (fileOut != null) { + fileOut.close() + } + if (objOut != null) { + objOut.close() + } + } + + } + + private def open(): Unit = { + dumpFile("open") + } + + override def close(): Unit = { + dumpFile("close") + } + + override def read(): Int = -1 + } + + // Leave some time for the task to be interrupted during the + // creation of `InterruptionSensitiveInputStream`. + Thread.sleep(10000) + + underlying = in + underlying + } + + override def read(): Int = -1 + + override def close(): Unit = { + if (underlying != null) { + underlying.close() + } + } + } + + def createStream(fileHint: String): Unit = { + if (interruptible) { + Utils.tryInitializeResource { + new InterruptionSensitiveInputStream(fileHint) + } { + _.initialize() + } + } else { + TaskContext.get().createResourceUninterruptibly[java.io.InputStream] { + Utils.tryInitializeResource { + new InterruptionSensitiveInputStream(fileHint) + } { + _.initialize() + } + } + } + } + + sc = new SparkContext("local[2]", "test interrupt streams") + + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + // Sleep some time to ensure task has started + Thread.sleep(2000) + taskStartedSemaphore.release() + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + if (taskEnd.reason.isInstanceOf[TaskKilled]) { + taskCancelledSemaphore.release() + } + } + }) + + sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "true") + + val fileHint = if (interruptible) "interruptible" else "uninterruptible" + val future = sc.parallelize(1 to 100, 1).mapPartitions { _ => + createStream(fileHint) + Iterator.single(1) + }.collectAsync() + + taskStartedSemaphore.acquire() + future.cancel() + taskCancelledSemaphore.acquire() + + val fileOpen = new File(dir, s"open.$fileHint") + val fileClose = new File(dir, s"close.$fileHint") + assert(fileOpen.exists()) + + if (interruptible) { + // The underlying stream leaks when the stream creation is interruptible. + assert(!fileClose.exists()) + } else { + // The underlying stream won't leak when the stream creation is uninterruptible. + assert(fileClose.exists()) + } + } + } + } + def testCount(): Unit = { // Cancel before launching any tasks { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1c3e2f16cb0f8..a3a56a6f02dad 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -201,6 +201,11 @@ object MimaExcludes { // SPARK-50112: Moving avro files from connector to sql/core ProblemFilters.exclude[Problem]("org.apache.spark.sql.avro.*"), + + // SPARK-50768: Introduce TaskContext.createResourceUninterruptibly to avoid stream leak by task interruption + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.interruptible"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.pendingInterrupt"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.createResourceUninterruptibly"), ) ++ loggingExcludes("org.apache.spark.sql.DataFrameReader") ++ loggingExcludes("org.apache.spark.sql.streaming.DataStreamReader") ++ loggingExcludes("org.apache.spark.sql.SparkSession#Builder") From d2ecfd73a3d71bd5dba0797110e32b4170f2ba4b Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 15 Jan 2025 13:10:29 +0900 Subject: [PATCH 431/438] [SPARK-50823][PYTHON] Upgrade cloudpickle from 3.1.0 to 3.1.1 ### What changes were proposed in this pull request? This PR proposes to upgrade Cloudpickle to 3.1.1 ### Why are the changes needed? To leverage bug fixes. ### Does this PR introduce _any_ user-facing change? No for now. It contains several changes for Python 3.14 support preparation. ### How was this patch tested? Existing unittests should cover. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49496 from HyukjinKwon/upgrade-cloudpickle2. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/cloudpickle/__init__.py | 2 +- python/pyspark/cloudpickle/cloudpickle.py | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/python/pyspark/cloudpickle/__init__.py b/python/pyspark/cloudpickle/__init__.py index 4d317916d4e46..bdb1738611b3b 100644 --- a/python/pyspark/cloudpickle/__init__.py +++ b/python/pyspark/cloudpickle/__init__.py @@ -3,7 +3,7 @@ __doc__ = cloudpickle.__doc__ -__version__ = "3.1.0" +__version__ = "3.1.1" __all__ = [ # noqa "__version__", diff --git a/python/pyspark/cloudpickle/cloudpickle.py b/python/pyspark/cloudpickle/cloudpickle.py index 8c50ba175e3e7..4d532e5de9f2c 100644 --- a/python/pyspark/cloudpickle/cloudpickle.py +++ b/python/pyspark/cloudpickle/cloudpickle.py @@ -63,7 +63,7 @@ import logging import opcode import pickle -from pickle import _getattribute +from pickle import _getattribute as _pickle_getattribute import platform import struct import sys @@ -192,6 +192,14 @@ def _is_registered_pickle_by_value(module): return False +if sys.version_info >= (3, 14): + def _getattribute(obj, name): + return _pickle_getattribute(obj, name.split('.')) +else: + def _getattribute(obj, name): + return _pickle_getattribute(obj, name)[0] + + def _whichmodule(obj, name): """Find the module an object belongs to. @@ -219,7 +227,7 @@ def _whichmodule(obj, name): ): continue try: - if _getattribute(module, name)[0] is obj: + if _getattribute(module, name) is obj: return module_name except Exception: pass @@ -293,7 +301,7 @@ def _lookup_module_and_qualname(obj, name=None): return None try: - obj2, parent = _getattribute(module, name) + obj2 = _getattribute(module, name) except AttributeError: # obj was not found inside the module it points to return None From d7928654605750874371c7b375cf87c4a578076f Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 15 Jan 2025 13:11:04 +0900 Subject: [PATCH 432/438] [SPARK-50821][PYTHON] Upgrade Py4J from 0.10.9.8 to 0.10.9.9 ### What changes were proposed in this pull request? This PR aim to upgrade Py4J 0.10.9.9, with relevant changes. ### Why are the changes needed? Py4J 0.10.9.9 has several bug fixes especially https://github.com/py4j/py4j/pull/551 Release notes: https://www.py4j.org/changelog.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing test cases Closes #49494 from HyukjinKwon/upgrade-py4j-10.9.9. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_python_connect.yml | 4 ++-- .github/workflows/build_python_connect35.yml | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- dev/requirements.txt | 2 +- python/docs/Makefile | 2 +- python/docs/make2.bat | 2 +- python/docs/source/getting_started/install.rst | 2 +- python/lib/py4j-0.10.9.8-src.zip | Bin 42908 -> 0 bytes python/lib/py4j-0.10.9.9-src.zip | Bin 0 -> 43108 bytes python/packaging/classic/setup.py | 2 +- sbin/spark-config.sh | 2 +- 15 files changed, 14 insertions(+), 14 deletions(-) delete mode 100644 python/lib/py4j-0.10.9.8-src.zip create mode 100644 python/lib/py4j-0.10.9.9-src.zip diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index c2151a0093f7f..311907558f6e2 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -83,7 +83,7 @@ jobs: sed -i 's/rootLogger.level = info/rootLogger.level = warn/g' conf/log4j2.properties # Start a Spark Connect server for local - PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ + PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.9-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ --jars "`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" @@ -102,7 +102,7 @@ jobs: mv pyspark.back python/pyspark # Start a Spark Connect server for local-cluster - PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ + PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.9-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --master "local-cluster[2, 4, 1024]" \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ --jars "`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml index 95ec5af719975..ba77f2dff75a9 100644 --- a/.github/workflows/build_python_connect35.yml +++ b/.github/workflows/build_python_connect35.yml @@ -86,7 +86,7 @@ jobs: sed -i 's/rootLogger.level = info/rootLogger.level = warn/g' conf/log4j2.properties # Start a Spark Connect server for local - PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ + PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.9-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ --jars "`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" diff --git a/bin/pyspark b/bin/pyspark index d719875b3659e..650d913eea028 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -77,7 +77,7 @@ fi # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.8-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.9-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 97a4406977668..9f55d772a25cf 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.9.8-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.9.9-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 5bc007fa068a7..79563c246ec4b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -426,7 +426,7 @@ net.sf.py4j py4j - 0.10.9.8 + 0.10.9.9 org.apache.spark diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 9a944a44f655b..816ceea327aae 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.ArrayImplicits.SparkArrayOps import org.apache.spark.util.Utils private[spark] object PythonUtils extends Logging { - val PY4J_ZIP_NAME = "py4j-0.10.9.8-src.zip" + val PY4J_ZIP_NAME = "py4j-0.10.9.9-src.zip" /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ def sparkPythonPath: String = { diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index ec3ef63dbe874..59aabdf5fff19 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -248,7 +248,7 @@ parquet-format-structures/1.15.0//parquet-format-structures-1.15.0.jar parquet-hadoop/1.15.0//parquet-hadoop-1.15.0.jar parquet-jackson/1.15.0//parquet-jackson-1.15.0.jar pickle/1.5//pickle-1.5.jar -py4j/0.10.9.8//py4j-0.10.9.8.jar +py4j/0.10.9.9//py4j-0.10.9.9.jar remotetea-oncrpc/1.1.2//remotetea-oncrpc-1.1.2.jar rocksdbjni/9.7.3//rocksdbjni-9.7.3.jar scala-collection-compat_2.13/2.7.0//scala-collection-compat_2.13-2.7.0.jar diff --git a/dev/requirements.txt b/dev/requirements.txt index 845778f0e9b99..36548c2eae408 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -1,5 +1,5 @@ # PySpark dependencies (required) -py4j>=0.10.9.8 +py4j>=0.10.9.9 # PySpark dependencies (optional) numpy>=1.21 diff --git a/python/docs/Makefile b/python/docs/Makefile index f49adb0df80af..045b03a1afd1b 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -21,7 +21,7 @@ SPHINXBUILD ?= sphinx-build SOURCEDIR ?= source BUILDDIR ?= build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.9.8-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.9.9-src.zip) # Put it first so that "make" without argument is like "make help". help: diff --git a/python/docs/make2.bat b/python/docs/make2.bat index 4127a045bf2fc..ff0c8f991b958 100644 --- a/python/docs/make2.bat +++ b/python/docs/make2.bat @@ -25,7 +25,7 @@ if "%SPHINXBUILD%" == "" ( set SOURCEDIR=source set BUILDDIR=build -set PYTHONPATH=..;..\lib\py4j-0.10.9.8-src.zip +set PYTHONPATH=..;..\lib\py4j-0.10.9.9-src.zip if "%1" == "" goto help diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index 515224b52e09a..c60839025eef6 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -177,7 +177,7 @@ PySpark requires the following dependencies. ========================== ========================= ============================= Package Supported version Note ========================== ========================= ============================= -`py4j` >=0.10.9.8 Required to interact with JVM +`py4j` >=0.10.9.9 Required to interact with JVM ========================== ========================= ============================= Additional libraries that enhance functionality but are not included in the installation packages: diff --git a/python/lib/py4j-0.10.9.8-src.zip b/python/lib/py4j-0.10.9.8-src.zip deleted file mode 100644 index b587e6d620520ba562340e6b95f2e67a815e1961..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 42908 zcmagFV{mR^u%`XSw(acL&W>$6*|BZg$&PK?wr$(C?b&B)&U|OiOx0Jbx@!G+>gn~T zuevJw2oW3;=*a0ss&IxBxqM77Kbub5k3AD@QszcNJwQ z00@buzA5RyaCUt`9ZDF_AwU+xQ+7u&=YkGIjjv02v2p zV0u~wwg%*~cxATl7w-Xjh2{jSxpRZGL$5A(W?UXqdYeDBn(T^rY?lKcYV`&q_Wn>aSI7eo7B_L_LDj{Owmv>(Ki;Xy_t zaE{pTph1-=lBrrW))`k9!1db^EJiZBC>dV#@mL{eG_TsEnb?FuigK&_<(ZH5Fve0xo zHfYtXH1i2)Xfc9?>j4$E+ud*8t}pS{4Oi@%Q0ryfHng563Kfsv%==NzKm)d$<6MH{ z{SrBU)2T5z5ktV~n_~HA>#6cAQvOziu+9Vfzn?CJ(+hIq+_e}WgX}l{R2X|;Hlo30 zowadg8LLBgQ2n(hNu7_@D1kF-HI5138+_H$4=R`FwIHW-wvui}3JaqCiy-vN^14v9 zk+~cf3I1uWf-lx+CF?gD>Dt|!_Z!yQA`&akE*BU3>eSVhK>I;!$>k2)xBB~XV)1DFi|oLr0eK{FMuuad$5GmyYW$95_u9G>m?q#z2 zP?LY|>uYLifF$|q>#P3uK(?D-R5Oz*#emBo;0eA?=lA0yiPzTQyTc|AzCz8 zk6zWcA^WVyXqg-qRNhLIp09@5%;G65Kf={z5W{1QI&pr}6HeoSqKaavV}X#(s>b=H zr;c+h`I}$B?sQB!MgBPn^LnJF9Hl%x!KSUVun6T#zBOaU=W(6a`}1s}33yDAk0M&s zY31n;9}hwf@@rN+cMg`+Sy>R|zWQ&VE3}TbJJQAnF85F!tw$RJR`pTxnCVaskL&gN zQdkHzd6NhzLl)^ct{+J_6j=OM|Aca$5lj$p6CTC9#7S|rsz_H5nGM7~J=41goL*O* zNv+w!y5x$uYeqK@`;j$UT`AM2*6^2(9};a`x6yDGQS`c;$o4b3$R(UAuNn^B0@vBQ ze7b&X(;+5p(-oTG^@6bviz_lTdswN7Bc51WfNtuVIGU8b>_H*I`TQ6aXbUZ6QfafiZ0%IXx9C-2+~|Ykl5m2^BsqC*uJ#dqL03I+ZzZ5u(8X*Si|M`@qTenCq3H_w30BpPK)E| zy^4wsCK2Yt-b8pYqKu1UrJBlMYr*C|f&4%AL;qtx;D53|acGnd>)-o-#Q$c$i?M^F zxvkBAWk1>f&3;|o|8-7Rmyn0>7Xuw5105$F$1mvr$ngKE3!DO+#|iMi3{U_7{C}#8 zp{=!@zSDnA@E-#6-^8U!Rmx_A0deaB6~veXq8fdynBiPRzEB61LNnLXKVFnkDLhev zDvoX$=FtEz{#~;=0ksqZ{q1-vbl_J^4PgQ&1_HO4^xd^LVu=2&f+{QX?;|pHJk+RB z*c3q_A~co&3|vR?yWVULAj))8VW1r@$5>#See~4PJvN&I!qvJE`KG`QWUbdqpC4oh z4OKd+JR>j(MV4ZPRMOrGO)5had686k8|BJHExod2V?&0!zl5i0+Nape%vR1N-qMcp zqnBwRv3$6QpmL=wc%*yzv9&GmcewezV3Yo>1~E?k2zDJXe!QIM{3yjm9`(Q3l}u*6kbWJsJ12U@Fw3o&jeeP#h4Cq zl^ZIaKvtoa$t{ZSlptr$U22FDWh?nHuT3DNO>-cy{}d{Tc@-$?M%?lRXLKtzz;6RX z^NBoPkjp4`FK+gLp0fJjvhtd}aV*?xe~hF(Z{584o12u5`l!#c^5)iw3vMy~hGAdO znsu}#PUD?l%{xkqD543~yJK2BY<}utdhC|2SwqZ>xJ*&jQ)Q~^o6+JcBK|o=S;*9e zH?##i!T7-6N-FYNKKRs;7hmtKhqVOZ9EiB&%Sp@c69-1x92AN_YmXnqjwhX0N7rg4 zXVaN^YI?}n(Ic=$&@JjkVlB_0fJsL{T&a&1M3^T-ps~`4(yi@Ki}`HAbazhl+cSV;`o4^7?tB`D#nB!DuLV(bmsEapgUrUO_(y>CP zt|Q_g=<_D)E0%2SeyeRIR51USCreJx+Z60Oc=8S{ks4S^2JCldm&W%Y>DbVa$h|09 z6@M*SgS_4t8MdC?02#_BiNz=wGdN4SsKmfZp(d5g5BsM2;vr%xjW|qNA)mw=7Wo)G zcG?hdVi3FZKn%?5-p~tuJgH$-RpRt+y17&7+q);|9++x+#?{`L;XOwhEbe z(!6;s+b;-e`QyrR?2+t9@ytK`o1w?Rj%7TT%)>9$lb$U6y3*xv7;5cFjtn0Q{6;Mrwi zo>nxKuCfE3%$3*5hI8z~e4oANqPzWX5#idwz1_Q(Gh-M2hZX|R#fnuF{8UoBs?R2 z4@LGswbHGFjG9+2dHx2!L1Zdn1^??+mibY2{7`QLm=T5b1LJTl5Qq~pd zh!f;NNoiD=nq)j+*Mvqu8o#pJDlh=15s*yqbQj{rkkb<$bG0V50`H57UHh2WRgqXI z((2T5g9tR&!+u#BxCu(!b_k4ANRo`$!W7mdvl&9ktFyVhji4=4;(Is}>@d`n;^Zb&&r!of4kuB#-)$2ZQ=H6cVFY(HYO2haItEduHQnj)01r zv${@BI~Zw8CE8> z*LXHMaLueKPeosrV4VJ&%4e?R=EB?avU>=R^WOE@J7e(s5zX^308h8Z*R?I%r3t@H z{nshOP*6#2Z&8l>xbZ>`Oe3JHLAJ~nP5&EV^Z=>>;=q0Ij8AFHkym$i^830(~1OO2KlMZ$cwobN&wpRZ?aA@MTc3c;q-+7`wUPTC<7OqQJ z+(viZruusnlnLhpFlA4Y2^WCP6{b%ly{s+!e7VSGpAzqPU-Q&1^JC%J`uBXX36(6^ zA8U!axT~7zqbHDRT^Wk1THzH{5|R-_#+UxR-is@y{*FK++7dYnxrAvZo(OdGbi30e zxx?+nLIeF)P^_0bN+1*AkSW}pM;2T2zzar_lB%C}sMn{QF#j}anQ_(V%M41|uo4l( z9^o$+Oa6%8Z208&W9GRdqG};6ehZ0n%AwKt6MG#WjEd>#w*ZpI$?Rt5-x5hty53Zvs~-6@i42>!T!Dzj$)Bn=q~q4RUTSaw!+ z7MEWE@w zRLJ6U#&JU*@nFFHNX!Z-d_D3F5y*r54=S~vZ(yS+h>w-7)FC&A%5Gu&kQUcPZi#^% z1Ca=SQpva%{Q#$2px^s{HxOdc>Mtl&_m1+6SZ2=lG-3kxnE_AySWPv!&;%|Jv9j;l za2Mvr;30Dfx^Kd-2-48mIwSUf5Vm0hGWu@%%zpnmIb2(d^zD8PxApYf@5aI9 zJI}BS55dOE3*kHB`XGKLlfb3Yj&6JUOM}7Jb7Djvn`YP-YO_>LVWchlukQKDMYJPh zn)XLvolg^}K|)8QrEZS~(c9jsiCvYh&OUZA35e31y&miLA^}20n)?WrbV;q<6L4zQ zo8wn2n2Tt`Ip;M8d!W{rIQR7Hb$fogOXf!3Eni>?S}-2Pi3dXkI$V0|-AN63%DW7( zI(c7|uj8KMmMPxJl#z1OPGIWsw-ZB-vco!{sKqb%;^QUahZWM>uBSzDm;;C10F z$na;;z8c&SG0ErQSLK@x$D=8;7guiPvFie(9ZN=^;_cDStgtUJvmL*2D?JP=QS1a5 zX7GMcD)FqJHYB<_89HTN#SadIo|{rPQrwdsM_U_K?@5(gcm=0_R~$C55(BygPlPl= z11Zpm3daCYsbroq8#ZxsMVznq8m8s};w243VFwX(4?H4K@8#0gr^EdQhrDMgyb2*| zOa8m~UlP5<6^D!E5l7t6VCiC+N~q|r`x;vu5-nULRULwS4p8pLsnMS*##bPSubt3u zYiee0+-_XfqS}4MRJpG{8AGcO!`wiic5}tclsv@{y<4)&rnM~*!|h_tflEkneLw9O zJ-HU3A+*^hJ{vIokXAjo2N&Ty?a{<4L-b^%AX7)yQsM5@jyvCBGgxbmXUQ#G#dKR7 zJp-V7J$oL28hP`W{j*=sMJWaCbnD&}xf0#rK{4l3-_6V?Tdp>UIcH0RM*6E8hgp#b z>m=N8IC_|{(BEi(Veq^Td+Je5i*qW2yK&^;)uZ`CO&4QF~ z=Z(Rnx+sW;H@(;MQZK+4HTn(Bp;y9$*%lLh1+G6+jNvd0_fG%cI?eUpFOXuik)iJy zTIM2GK8X4z-+EN4E8RLJgrJiV(FNmV6+)wBDJ-Aj5qL%ku@xJ8)soUKOpRbsxhkIa zg2KIlr@2DSu;f&t-gqKt2Csu-Tc6K1__t;{LwRB*D)u@CcHpiLYH*aWT*T_(Lu|?c zZJUq4LYH<9QGExHyCISHOyjgw1qdf0+ZNQ8N5>-Z^Uw(tm}%tr8#Ivl`7$t*JQ#k7 zL>%z9{Epc`-p6Yls!BOdhUqkFiC*V%LOFjM65EIe*c}I;FSCa|hXvl)=|iv?ZrTrV zQ98Y5xEz|NSvHCd+~<#CszKQuC!4YAuj_uLLD_7Oj$U3tsYGmo)16oIr|`1ES9G>o z-*ZiyMWx4)woI*-be8o7B=YcsrPW(JV?e~ev>$JGov6UrG^TB))S8+y!A;HQ>YbG< zMM;0)Mcr;9d>@hpOOmKEuKsP zDU29Y@ax6qxEpy7`tj};EOY9QN~|A1sLAfT3Xac?$s_v#%9N-kOkxw*Sk zn$yv``<8;6jqXn{#hxseMytPlk7pE^4bJ&?6v)*cw z-0QGV30EunkeI;8A2H0u+*6i%EoCe+$VSzgMY`;qZJW5B-B+~Ql*=@}de%3=F$g@xdcEs0As8s!j)=|Os*x-tO&k-Wca=^+<9 zudMdxvHPtL@VYD#M8~Udu0bQ-e;d~8(wLs=R5XDAi*tkJW{mFhNTbUgnh0MdlnH@r zJb&L~>{FT;bI%Vc&p$aYiHlD+P@(QTR&%I~Ia3mQCAt3)Lm{PQuOq-jPE?OM&BvrWo$sAzPOZx4@X5H zrlAE(cNloAJD~Wbp9X-gQk1ohuV&B$3CsISHoC(TF>E29vkKq2uBlr3;mfDG@Gp+z%t`tN-J~OaD z^3DKHBzpe0T9|IOHfM7b#Jz zmaqkdmx2+BIz{Y3)d_{V8jKFLl0x#rlD%ZT9-SgFY>?aDGca_Vv`-LNRAjsG)`HrD zlH9{6UQi<=05}4eUSU*QoGRp(Ag5LXE*pmu26Ygw2eumSo=8N{wRVDzHFSVdBPeP< z(n4D^L9;?Dt2yvZEa!eIIpNv2&?NT<#C1)u*KIRRbVGHAoj&dFy!<-Td;g#eny|sU z4~^Z$?l_tTT8=h@=sw%`v^o5H9VQzOK*P*oPlTq8DAW zt{fL#!td7_Ki=LA>u6lJxp#(~CLL%Q1lQDNpd$3-erBq?x>_J%F!}&8ZukTm*cZ<2 z9w7G-X&n&a9JwAZ-jDmYH4KCelqwG4m7)q-=yS6glFY~r6y#bpL(fU_nI9aVJP!cI z)(^?o=jTMsZMAxho9+FVN_J1y4mvmiij`KG=OXf)^=;k5d1ihy;`I@}R#_M?pWcYd z-~9kcXYd<6_1{6z|PwbVa^zDRY zuQt|59)>Va`>YI|Zo|r3l}j`=&}F8O?K$I_Cf%|2^c8J*e&*?C1Y^m z9plY+O5fE6i0W+Bv=YFa8~p3Q!^u}t43+$3PVQ>m{0CQOzvdv?3)YiL8S3CVb>WBd%37oF_Ump++pP4?}(|9f#0cC4p*7%>r}|cA&=Iu?uj-) zh^!1OiFaRl_5ADU3sw(Yt#9$e%$%?^*sDiKn1@FV`f7~#Nf9+^Y8p`^!D{?T%ko{8 zRl*zP!1HYfbzmwZ&IgF&(8i^uwvsC5r5n`tH7T=`@rZSAl+1DXZy`G!Zzmo$W{dUZBm^NPDzBp>wCrPb~gkKHI-HR1S`gxy8IiX+YeCbimzqh zhyrBPBCBzn64#$Ad-jsdnhnp+Gi_T*(5Fg#mvSxTLwMMaH;!pSidb=y-@|w;Gk(Y| z*-3@ z+{{tkf2zYI7an0_bus)<6(K2ZVJTJjM~e7|`xsT8E2@u5G&BlmF7GiF7~d}hDTp8* zmU+_06R+@WtObPLM*AXwjv{@iaQJb3X>+BaXYlPyFp-50seg8bQ0?S=xmfbqu2q z!jxjD9#$a0m%5i~gwy=e_|%DqjEu1wgq1<>4?&UK@s=(`LyQzO34DWkrNikffVR?vA(j)anZKc~@@;v?Km%GA^@Uuc>~?(lV6 z#*Sa;SF!hDI8GjEB$T}aZ83x-o!)DV2z@6>aP(G#*{bc8XU8p}VuE^(C|lg0W7!M< zJw0`xTF`Z5!>YI`cBkko{gxKMFcQvv9VfdgAdJ%~O||8q9nJDRy}-}?F_@!9mw{^v ze|zrD9ad@cL5JPUw*8gpOgU26N!qKav9L^wv@em8%VA(<#F4F=$>KgJ10#t0ulxM1 z{5#a|K?crHG=mfhuPI8w$-ko`r5ei1ji$862-r*{1fWQ%Do{^T$H<3@#jM`aRU=e` z*&tPytU;QH{D}a->aRQabtOPSU-==TD_Q#W{RF6uJ1%_YY=>7uj2R&qr`wU8%i$RM z7JRx4N1((h227*|VxpZev)b7hxD#$7Sk8Y2_V=8*od-9qBXWJ3h~#zrYeM-yW z?rwhBw0JAl+hs_S9J*RT+rNk=`J88irSI(QJ>2a6dX)jGeO>G&r)^@W`qBmUU3Hy_ z0g~$8x~gCYqc@tr>|;jMH)y4u(3$gncLLeg>lq{{=Fyq&)SjDC+WaF|kiXinfA+AD z!x=(4QAY+aA;9<}quCj8qs7{(U&$j?T>YT?e|pZOdLXMZ{{}#||KfkUO1iq{Hs(&c zy8qQx`hN^*lvE*0S?&}e#igfZrR9z!smEtj!eZp8X(?$Xqhi!XCuv9NsT9PesitQr zX(ggUZ;8eq_3DkwM26Dd+_kmu?&QhkOK7Fl5hM~}{+rpQw7Jd4uldgg4*~+9{-@bx zq3@!vYpU;L?5gknUq6lY|DGRRYHPV}h{pSz)STwMWCEc;C*zsAgr1xV#F>eO+Z04* zRgOSd0m~q>K@tKe(i0BeuC!9RfeKN{rpZ5Qv%`gfx0aTcRF-sA)GoqTKMdpsMdFVi z;##}h+g@~E3s-JlW+IG|DCba#r*KrpT9GaqHNp(u{|w2%$pxeCjB!)`DN$*NPvc0q zBc4i1RT)wo-99STKy`nyUcKl_R-r0MD;H^qjZ!au5Fy!@UWUQX)1bK9x?omdub&Sb z*RH%5R&mx=F2|UyJ#m22jw~SVQF2e{mzN;A&qqp%h&)m}f@1Qek89s$^pB!+k-ue< z;w!ac2H?j~k>Mmk=6X~b+%3uf-r4!_tYMF*PE}z=-T|xyk1slqL?3A%F;yv8@T7a8 zPN`0nD=LuA8*2bSDt?1eR!~SOBx~@XI%s07l`9~%oZ8Q{jxr(APZqQ3pgb)|K0T&X zrq1XuA6qIowm2l9)}6@%^Cg)pP#2o^pxr?nb+ax(ZzBbJu+ml#t2W`i)KjSNqNe;x zG2G~5tCwSYHU#0yFJ}7g1A{iOZL!)9T8krz$7%7RGz(&aN=gQ6_i|xAuix4;sgj(ssBx z(q=`nsGCeE1y#YBPmuIiJ(|m!RE^kBvB+M|RaB_67)N^svMk;0lV6lN#UO)zvY}AM zn@|*K(+q?^ZgZz4#$Kn}>t zfg6KU4)+QDkt75;bRa~fZ7tz2o5OpAHBLRH^a?WdHmc==G>%U-;RU-EOuRnB>4uOA zJKmLDc32>I^w`<#_Tuc0sFwTogzfrs`SWGlb#ON^u(6PFdi(ln;pGh%^D&eCW7>-F zLJ)FWmH#N5Qx{uD4iH{%E z)5XDque-<5o`iFBdi(g8F;H+*aB;U}x!bUX*Y35|@p(7oI`{PQD`jqO={II5~~nzh+TnzZ)BnDD4Y zcK{D4EJAY>OQOlxlCn*7l(fSpaH00vqP+@3Ii2Uf@KCbywfW@uBZoW#Fl7EJ5WgWdSP{Wtw?o4 zkH6Z4vg*CH$Fy@nZNya>A`!82`pQ84v!g|ShF}H*R({pUPTO(}`)~U-k#XY9g5tFV zg^+}(GXu`_>4>^tVXR`^7hL^ z8$0|F5(}LfD5$vGo8sUi=n*ip1I;D&aCWx$d@C!lCn1)Qk(%fr4{9q{0ab?()_@SL zqOh>b?^Y$mw4ij7Qb_E)o1_2^l)GSo_$q6XroRj%3O{=%ZpYsF6N7Y<+AT%qh=N>Q zPfp9ch&2yy0#KDd7Pct|O(lSE6R#G%!&VJw?BfT*(l|}>?~NtqA7q%z7)Z$IJHRL- z1JTC#1f_v!25Z`1S-~iodm5U1ImpQKJS^ryodM6T-D?R-6Rm3w79Vas5teCJ-qi_s zUWxv6HtmZFvI!bC9W`IxP?A9{)%@#%_(;$P(%)dZ=i|GYPCIXjb6Zr&@6k#P(qAM_ zL|we=hz~(kHd1hBnju6hXZxo$S_nu$$gm$W%#^=kZiIi=bv9JOSZ{mHU*K}&z}80B zTBa@_HG4ljFl84gF}X+8J|=e|$(TnB_4DET*CTs0*)2Pi8R4tK?4~4bB)5wY`Q_W& zTFEz`)<7u0Ah~{xF{KYa4AupOCQ_ad$!CkWFEstrEn08d{@~ZeUYJ5ux3Na|7c@qB* zzS{?pscNn6uQg4oXE~n$Q-xD;UjG zz2-^#HK|Df(Prx!0xnxFXRG*ky&JBT^$f$X`$UhBs~DYvkTdVYc3p?i51oks=X{%% zF5OG9gAinX9`OaI6E!(KsPrq*+OhqYTkp8zXQ_BqGi@6_R(imu5ou)@#Xv2{YE=I% z};ive{dBM96dsv9`}_2SODuR-hL_Vavyx!m+XEiM!n+T0G&jm zs1wadML96sc7<}WQPnz+um6$<$PN{&6dy1MLd>^tqWRbb{Fh5wf>8ktsIjx)YX25lh6{V z*BPEGc1x4xnepJGVI0A@rY6y|`DgMfC9kFEv+JwA5cRKdjh_$m*N@3IS)3=a$v5`W z$C}!Eh^GulgZ5Bua;x^iNFkKJ7v`8xNXohJw+$)C&)RI=cRHT= zyw~+;RT%ja>rGeeiALGv*hYm0n-|EvjS!7hm**Gk z$|^%}BE&!&NB^YUXq#YDxnc`d93oZiGi-AQ#nAHD1xbMue^u*J+E@U6NW?rf^ZWBB1%1tpgj{xh5;DJBBVR5)u57`x{zz7Wl``* z;E+W(;Bt#X9py5MIvM$bgcq2y#@(1-LDH^>SspGcVWaX zGr{jep0L5aPiA3#=#356j6x(CMulz>SS48oT4b@XdrgPxFRfN9zks?CML z{_`o+u`)%z{~5JxEuv?YJ}nv-v8~L!2YYt$m*l=1|2m#Tof2-Ik+I62!6Yb!2AMN( z7(YYy+@5VdN@Iv4brXeZt_b3MJ1dEVoyWXODu*jWVXC)5YHAQj4*LvIjC?POQ+7gL zM(pz!n~r!0Xct9>#an)qCVjMdIE$rbi>8;j!Ypl>)93eS?{|M2iyE9i?dam{R0O}N z4Y(|wRf_r69ak+F`8hlSt&qLH`3&pus@0s{S~K6(Hx#J8iYA*IB+ZcAvWnyd4JF>+ zCA3j6PQ-b2=@AYE)O8$-63?5$K2NXb!a1Cu@B}va7?s^_30vE_LW@S31`U6(%HFNh z{_yoW0$LvXd%?ncOT`4y)PcA^jk-W4*V3y2KeK80aSM>nvbm5;} zx)CAiGK<{aaKbcWEvJ07r%+(2=m%U_1 zdID(3vuG9i<_Bhsy%rzyp0((A2Z=z=J6T^Ep9d$rosKZcW%mgDmvzAq`hFmv!{>D( z&~wn$P`ziYR$TD4yKy0IP|LZzVqO-9||m-5wxy4E;>2?bIwp;V_lZ&}E1Cqju{^60XPVNX#Qxn#CIMVDH@@JgoL9bdIdgwaa;C430<`>rr6SmU~h?Tj{EFbZuSTyuQ5gR9T^TWTk zA)}Hd5j>P>T+G;eE2i^2=4!j91w?gA~Cw!OPQg)(U|*%skklL#?>zd_xIi=Y@wq zAmd8E!N)SWSY}H(5f-a4+5?$YX9@&wkc6ClHO*=ga$J`xA+$xRh$yQMqinkplHcqR zD86`VE1R^b59&JWRHI8tdjT$r&2AS_)%>c=-4xb=ukxEgTr-0zSPur)eu>g8>)C;K zF*x@~4IL$U${n`qyoTLl{Iq{bVEKz&@jkvlP zmNk?s&gL%*q1!*Chiuq0eIXT0{>Oa<7J`q2K0htq>+A!p7nNb zfU2RqB^+O{Q0z=t8Na}O+VGAe=|AtP90&|K+6%2r-UVx@r3Nq1^hE z>%)UfW6HlJ(P`92A_X0eRWr6E&@;5V&=Q!m8LUSg`So-07x-(XxxASa ziF|x32#dg2bTZ7}yVkJDj>xSs;dr8_nEE5P9HMj@$S$CTjHtHU30^N09?KLwKkWCo z6A1rS#st6xIQWMUEp*D-)m7@|e?9h#Wa|d&`kpx#M~m1P@JF+w@EVsWQl~?dm?=iG zZ#god#%wG4$fK^M?!x~d$sxhz3aPPIWMP*(W|bvB*M`hiuLjAttYW8+4Bh7qj2Tm`#(elFgV_#TvMS5^vM9fj4OlzPRl6hXT7X zMIC2YlkE$}i3tSKaw$dGAV&}vdga0U*^C?Sa(eAWDS}0v=?Q-9KtrUL8!j2?xFC34 zs$fjw+tm8nJw_DuGT1B+w9EV+t`A)X#8msMi+g?*IFAUe2sLJ(1wfh_nplw(JVc&a zc|A|Pj@O32v2X{dwHuAJz7R`nxIRA!(qklXujQgymckZZ@>}dlIJw00p3J4KE8l2tuhi+zHV~m zwE+0WfJdE*=vOC-rIxE#KavB06u4$mif18Fg6k=%Y=4CtS3iV-`NdbpLg|>l9()4T zUccnIJ37*#1(5%SaqTn>$rCi0@t)xSdGyKX zX#duAQmY#CbqxV%G4D%@0~>)4RFg88hwAp0hy+|XrRYaq6u*=SOq%_1Dyw# z=^eJ+k5*yW!bJjNFuQ|>fawSDsv!n7bdYX3P8DF}x%yfPLuZR?`IHE^*R2@`8lB6F6a|RwY*ry+ zu^c!wm4}i;!8{y;ja*XbebI}#q2GvM9XuAOun28!4nx$@g8=lH_opoG!k)wo%84jp zlm5*{`!tIX5A}7kVn;`(GdW7JR|IY^t*83Z(a1GewkFee?g?!COUKQ}*`q-Sdh+{p z4*p?ZMDoER$YNiag&z2+Qz7ol-b#;7`8$XklOCN9A_?b@^?f9l*(b{7fHFHNa-2Uw z>P4>5H->3FctYXBi~Jrs0{xNThn50qAELH8ijy$CwP6DTs#;2CxfN`)9PBOxCELZw zV;w>3bHK~YYNLw|bSCFs;ZiS5=fRGriH00^f1Q9DJo~@Si)UYaX_}ffUn^|r{$7WE ze0siJQjKq)MeDC{XbSW+D?2dKb zx!Ui{W7GH9aPS~5`1HVRRh~eP-e%L&8Ai0I@;^7t}tQEy=`WHn@sqbnlG11iY0I;Zk6L4 zAiV>$dfYTny{Yyb-a;GTei!yQ9DaT0Ja>1sMUjj>_Ei~(MIthCrA!hEE_w;ZPctYP`U&?`?cKJHdLvY!$ zF1TkX%5eK}Q@pRVhqC68rss(R!}T69-rNZwvg~rbrM6hx1I9u7G^D-we&||Qs8U^W zdY5a7w@Rr<&ce@r*|o$>{bRP^w{KCW3*<#y!%?;;NHVK~@0@`HjYqeHmis%q@R-iV z5AzPLqk$}!*W1z*`IPe8`us+8V(EOV{95eCCdo4Cr>h4pw1hA4Nl!>9&vvxGgBYfR ze-&EVtko_C@&*mX#-)*ZU?rJqHpL2pb|$>}FTc)Y8}+05sislxi zy;88Zf_4~ip)4KUUKO%F#b{kd>tI$ZIPpEoQyzcmbn5FzYNr2mbK&@SdYQf51LSum zSGg~ih6a8VFJUflB*n6=l>gGPFCX)gKsrr+$;n2`h5Ujrt&xomp@)~1d&$OYba|1m zclms5^^2mA?eh`18k5R>A{*iRarG&D^}gX1Zjf`s6X*q6^ z(^oN~s5hn8tMQzx)802kSC5;-G60aX8j+}e(3+KY%bssBHKCj7tUc*6QmXT3M^#Tu zc>E?6Da7HSSXZKtK6iV_&Y1-8RE2U%r6_|K>Ado7?`u9b@69 zK{o>z1&ei*0yZlC<5bhm%}9Um<1Yt1?ZU6lCj{e-$k#) zOo<2!=D)uL6*z4dl^-QTrp##KiV`}SdMAv8il+@1D`vmfcr)c9FwD@5g4DfkXc>D` za0yOCOB~pCf;~&ZAVoCR>Rs-=EvcHCSr7Xh8+4Qk+ysBb?+iGd(K+mto2l}Dg9>EQ zg-Go?4gx#(FlJ*wF8gq2?8Ilm9yEeoeu(&|)4x8(x?oh(YuUV6_Y!w3t*k5+(#@%o z5Ch)^*Ttcjhkr)m46}EE4UD^k^&8EppsV%YWGouua^(}M+@LRbXxFH_x2^s_u(Dd55y^o`q@Tn>b&`KakH=n*~9h#XxY(-;Dk9@!b=ueiM zdc0px*HIl-_j+PSOnJJVc!}fzhwahz367s7cj@5U1 zi;iQVSi(8xhGEp=L>%W2`{vf5df5r^v%=B#Z_QyA*u}8fcbizA$YaB;M%Y3`kS5rd zbO_;lLpe`qNvelBQF6;?=P__CsoT(5Dd_IR&3AI2!r)4&f0P@d~-U!HIB=b7nZ)gt~gFFic>- zV+jdeJmzp#O$N@ZnzeR}iOuF|_h_ZbReb|Z8-1Z|(>z+;siizl4ZGIDrwLh)ng}$0lB>YcE*a>8 zu}QUgkz;IC6onWoy$(DB?d9mAgRqPmgV0H6uXPop1LxY_ZfN$T;KSa7zZk}AcSehw zu087U;Ok5g^f9gK-%uqv+CeXcOBpfk@%urnepjO65&1w5l<9M2%vdtKd*Z7{BL@2Z zj`YJ1>}kzG-H{;O#x#K%u_S}{fCG|dLH8Sdth8}w*@CG0f}9q6I{KRfS_5&X>QMmX zs{G{{$&^|7`7T3kzpjD>IBZvZ=g}WRhrT1KFJdqgL#cNRNh1SKM=l56!I$cxS>Q}v zzt--E#)|lWj}ixlr^0};#xz*rG^2cv&wSvQ?U#x>9Dlpa?*_$~t;eKX=~aNy5!;x< zpg7nfq!lJO_;PXP3iHXs)j-kHdZnd%U;mCy?OAMBUS8aU5$02LCyxYu`WT6T8IHjPTaaWygTDnNNuGWCq0ReG3Y>m3X3gwWV;s6| z5xy>Dd5;v5TRIJc^nGmV63=Z}6>1^0X!>QgN^GR21+JC zwZ)@wUdyg%wnh>&hsRH+vvGbL>kw*PF5RSPK@~|?xtlZzfT~3WzVG`}_Nr<}r*t(! z>kaQui~p{T^S2qrAmL}T-0>C00y2&Lg3)w8 zgVg^)*g3T4qI6p{<{R6#o$MGpwr$(?j&0kvZQHhO($~f{jPA`8};nDPyMb2BnX66_f(M-~(E@Vc%9k$^7 zLzmhqxKg+2iLEohgQKbXF)YhnM?Jpu0{f_t!=$cL4F+orrQEnDI0wu-mcL-Lgok(q z7qw|G2_fE43tZ}xxdQy63s*S^;U8S&s*=O1dqdM3#OMf>{`vYocDTJ8#G=U6L`S?E z&wZkmn9v^2j&yuPsNCSaiajSHm9-l1HK%Mp)XGfJ(DktTIbK2~^^k)=G8r3dF{8o!IVo z@a=o{;Vp&$+nX84vyz+EylQ}p@ML-+E z&L0=!14bOpJsCnwDB_Pl@?gW8$c_%Q>!ODB3~pXr)UF(*hMSb^QVr=5hB#TP?xI(} z-NskDvVXpLPJCUt-J0ISFdfm}!4%_#K+LLn#L^?2-70DB;l*BoCT90eRr5=Lqp9K| z_s+@W51FX3N#?<3z%H7n1QUxma_tc-vgE)|)&y1a*^Hq6kV;+8nCkDI0$vK=I%97(zywU%;xlT)EEa}yDts_i-yqnPii@COWBmk{K) zWet{P&=UBf{}%qa@{!v3U)@GNKnA9z?Dy;oCyJKYI3u~7A#Ql^z~~-y)oD#IcPht( zWIfY|T%hWS&L3!oj=W{jI$|+`I4$nzWTiIhU*9EcEfXBSEkPYwFPkTyUeci?nMwZ; z{NtNQdqZ5HbPf}fS`{MfKh@ZBhf|(^F`ESBbA_1V18W2_d9A6z?<+r9R0RxtI^Jm= zXM81B@U1PQrTq>zjaTT{zsKQgz^&(5+AkW|F(>+rP&EIXus(OzkGjC!g@C$t7r2he zG5{F^@R~ylhOd0@>$uQC(O}T)T?1M)%o_0Gat!CUeDWBc9X&lggWg9yFDTvIe>IAV zmAOO(AbqohfG4tYw<*?Pvg}H`m-qi0uH%xK&@fQT+!6l(#=hA1W1boBYu;ea*A%l$ zN3Qicpl_k9#Vss=UXV6leIMiaZpU2Jwz!^ob~7hLTJJ?L?eXyQg#Gq*sN4dZ^XSKQ z6Ku1m7JW%k%j@^kuzb4x{X8r)rtS*IGGZf*rC5!=y$yZkya*{D_m=V%qvEmLSYS|y zd|Tk$1<-4}X_KLdb3@Z9B6+LfxqOa376JP)c)Fg8Rvu(ny(ga+XE`G0N^{8otssp1 z*oZGpALmeVF`1eVDtemNqNo)w zkmWAte&`k8IH{*6EVLOut=ZpSjJ+(7ap!7AnvO=bM@xI?=(PBfjUzY;g*f$ezupgn zg^F8>YSCkx@zqeSgG}oJwQa2rE~qRh;oo4PPG)p>$)gj34*Q$WEYc>4sd11hp*6ly z(o*L)l zs`1s`9%NVwR12R#mdNruX)Ilq8DYE~TR&M>qWIjH0qXdA1a-)NQgq*TlqVw~hZ<}0 z(h*Ms>D23$6I2h(#}*hR!^Z>B!Twt=<1F`-)I6k=P^I(GbYAhvcW5`$>bDnwKyYU# z*(DCG+LwUm8+nd9k40R?{G3;wQ>Az-T_GlO3h!CFZG-CW6Gp99WXZqD&HZM3>lOdb z!HO6kiDZ6jn*~x<(Le6M?@wg>SC*!-%X{g5YAT8AE ze}DM?a@LI2*Sq(GJU{|HoNh&h)hOy){)@tVI_tD!DM`C?!!3BM*A?@L*}-j0YO=s>2_cF({?Y))|d zwshNzbrx5+(xAAZX6B2N7&-38ow4||OJ1Jw3afNKDz{ojr+ zN{`Alp8B>S>cJE;?2x`Xq>CBw^G%Gu7$wtWVX{m5#1ZH7%sMqxo!SF?f(kynF-|*nm4DUnM5_x4;3K@!ujwF6DnL}m=`ro z>ycj3*1<#jf#ky6B;M}hibCR%m`Kci?yXuustQ92apa0L;N+=yt_sO2Q3)1a{hp!A4x8;Sgzj>W=%#O+!z`}*^%>ii;^hCP6#d2(YnB$E2BQ)Fqlb3 zF?OH~&EB;02Ltpc@*s_36C|g9bu)%a^VE{*JX);n2WOV*v7`5}gg8|J-nCE=n^=G`5eVBk*66|$j1LdX3;nnb46yxyDINR12 zDnkF$U;hW;bkvZL;9+dfj*eY(|38V zP@>)Gf=U-8Z)`GOi8X{oMn309|BYngRpi~=sl_}i+tzwJoClE)9M|~i}#P#@YexoDzlI9rJS?C@Q#s}L~Bn!d@29T zp@5&fSimD>DHpCtM}3n5bw9bLIBIKFBT*&J&@8#--z8Kp9wQE8Pxg?hbE&q$;D@<_ zUT~Y@De6rmW$Qo}KN!|vb|-QgSUOcfTiU~GBo*as50DxG(xRcQJDcQA*4;VbE~I&d zruFCgmYCR(Qb;j5omhI*wbnky0=O;H8@${)OXiuDBum6c@%mH{5WB=W6J9i2&rvRS z5>*V0zL2O1tCfodEL~i0FJo$DotyqMkX6XFtOcmQhH%IVNM3(0yH25R0I$(te)+|BazXCek-LNxBw^ z^ObBw>0Uc9G+urN@KWiJW+tc3m3-QCYgbRRJr@K`jV$NEl@LB_#yy$8fp(Iwndc^H zfpBhaO<6JS=Ra?{1V^r%#N9!9-_SFJ=zoC>v=m@Ag{H>cblhfKcl&;ge9hj*hEKH& zrh>HA*1h&GKU^K@-v*zu#bcJ9_T|(Ch%sK^Y^*Ki#>9`gwBG-x42a8eBq^bjPN4DM zG1m6)*1ZM_D||*eXmt^91LsZ1&nL{38$g>_%|YU=Xw!KSPOH;t#{LOx&Fe$U$?xg5 z2E!@Qlep7hzP}R<(+EI2+G0`;bP-Wvx?LVSn26qUj!F>Y|5hGnow9Ljgrt84GpTgh6s5(cYL^mnx(3@st(s#->4?@pr1A;5|`_ zImgw#0|hs7ewp`Pa4(NH_hz?}OXywMk{vU)@g*N7-~5{HPvg-lSMPkh0HGbsT9mtfv1?66ybQ`gQuAI?_R#J zchq~W@3WH@-E$S@1PZelp*}_^V;N|_fF{h-D%}@$8-=zbp zqMEQO<)fOvu_=C}`W{AO9+KD7tv(wGff5I3&Y?N=g!bd*I@en~@=g7*G1<2CML5Nh zD8%8KEz`!>f&MsroBFz#K_4%?PMqxF@67Itqt*borr4+YE_#E1yd=UA-b6py9+ZNb z{%j_lB~h2O?`2C}#|ttA`7)Tp^!Q;#^{Bk*uIYYc&-L?goWeh+J2IICy5_2=->&|4 z;UnoDF?24lFmKOgPHAD1qQJHUk{f1!)WA2)SbMaXD}_cUAbF4fsoB4?fbmrg9+Jw% z`}2MGF+P*&PuVjv8W<9g>s?A0P&d+OUW8(pf+)aVi0db)wS_aueBq)jXwA2NdSkQ#K=nlB6k^SnuF6?l&X}(d~rZA3@!8`L~wr-6kS^++?(Fo$XGm)_*l?i z?X1C?OGcZ!2BOX%f&qG}?3H9LD_KaDdDN_Wspza9)D6Q^rE**{zq@D~b@mJneb@MW z&gBrAJ+-FvV+yE4SW&l)+TVL=uO_b&XsI0K)IL+Ox^dt|29}o+xc-1$$@wfD&nPsxI3(#qeR&nFNIo zyv(fBvN3(_88B)1kJ-0|v-HLCm6IVB0L^a%r%JGTr-K^ejt0 zu7Qjvr!1|CLG^#-Hw?4wY_9cl6O1lOZLpE+c9Lq_rstz-4;m=ro>KdPyx}O@LwxLt zEkMzTd`iLqd|@KV0yc zhKD91G}W~xB6TFdAZmkT-vvk=i?EpRu(9q)$`vmG3Gu|EGI4@^LVSQ~Se4{48A-R z^r0@2L|EkG|Cp_@d(4d+aM_~ruUJz(N3#eX~ail^(C9M8*YrWCON_0ACR4u;vT-K^Hvq=D~kXMLxY+ z>L$VuOujT(qn-xMcKg1wh?@{=b_XUmB3a~~OD~MN+F}FX4C5uBClVXuq>nJH!Td?q z(w8w5sIOwSCF`>o;MmXn(}uy1j%)@Te=QE0jVQQ5t(8Yk0UPTXcJ(2u=6Ba(p z&{0Km z&8wZzo2_oQ>%*(Gu$%VjdzgPWlGgE#F%(q`n|irF>|3F&m-W32v2MJhFcFXa%>yLc zvl0emMQX+*)u@$|yxg3dB9)u-<(}ssjGN=%;;Ve(oqZLBMout#-X8j~blULwtEZQv zc0zTq%+*dEC;+i~aKp&wS|4r0bqNS@`uH5mQ{dMh<6_MxxVvFZM+|Y8(=Fe-CrE6O=W$2(9CX3tdu_i4>n2Jd~bGBBWTv$bChIG{E}UdXYJ$>L#1_FT0i{gu}Y zBMx8{rEZ(0wBb`|)=*AYIiO&F(`~9MIBE!I35)|UD{~k8Bc6IAqArEpT zi$u|rkxB-I+ZD=Qpkgn(cRUh#$I=`?I4=-y>qO@I8Y6^{aUY8>CH=uL>-%)^K*tde zjQ{1XG-ELru1Q(W0dHrBmV{PtHL>LO$-?(|`qufmx_dVT^8kVsaDP5`TD=BOy~Jii zK4)KL6_Pi<8qVT3UFDWvNoajm`Z)d_uN_1lK);{IFPg?0;)1(dYR!Dz@c?JU-B-~rIEX#?$s@? zAK}!J?5X?fuM7wcVgE9p{!^dWxJo#i_kW%gYrGHosdM2|_6$eOoiv1WB;om95z1~s zn8w#NR9eI>%rQglJo|yl9$BMiLZm*3Zf0oonOTMGU$)$i6GQOxw_{}P*$%3;^QU^9 z+`YPtSr_Lid;LC)ZA~CdkpJnV6ch~UG>PsnT7BkT`>1rp5_KfnY}z73A8Nh7!u@IN zaD{JuxQ@~ui;0z;mDRn@#cQo3lK}rxb0X`^-MIhhF-96J>6{DK@YBXSQ6M`EjN9{aUI2 zI7+%sRCqikpl>oJjBmL_zDW7&4`ac;3)-^zQw{Xa&$fhoXsN+%ASQD`g6_~La^%** zgDhJkH|X%H&{MFE7?A?0k_OiolQPF6OJ-#E_6@7vM(Og_5NSt*s3Pf=s8*QfL6;+h z9@zGbWZW|00h#BS_y`4**i$4+tpnJMcbDF$`(OLSe#wpb^{7363l7(dJjfhXEq*o{ zr9eN)DrtjvofsyBs#VBz(-`}8X8NqTzl1Z>b|+h`V^*i1>vDqj%m_y3ti_`<>>8~d zp{)`Q0VW~g0EAq=v7}-Gc1?q#h|9^ zJN1w|tn(cP7p?-fwT3`(irbj26}DKz{p0=fKfu^+Q|qqP0Fc)+-99|IQ`Fs%alLA1 ztK8bo$Mhmj#OyXRry&R#q@_-M2)Kb&7MfkqliKrx3e6Q%8|0=#X^y7>SM@E!WQ*?> zNAo;^n!M3jV9+B&&EkawJ$>}d$>$|;2PjQqL!%Iehk@IQXIMwG1D!1(>}ri1qO!85 zrdBQI@DbhKbDOS?+fU_F$$o5neDBt5C54npu?e5eC0%}Bl)rLc5=mDt9K2w$3O**W znMgx|&zlR;H?ikB*%X^W8rUUPE!_dX`g3WeDHZUEFtFzbmB`g$kL%jSfO3CwBFLljGwHP>NOShu|r;h2ENETWvP zhN1|ML28a#5_ z>3P;F5a01ld?*7cGRkMg5FZNEeAPq|Te^ubBsLmp*0VAh5wA)k*J-$4N?gIOH~{@3zK zl(IRAT_99!L9_=^2`&j;t9YLuD#ZjdjbGWZgNt7rN~ltxgkdlh^&>CVtMd=IQSUujh(Ynke+yxkCZ9BmawZ^q@i7%gB?#= z^D;W9OCH>`c-29{G3p7A7QJmkApBMzesZ}6lStu8-ld8wbz+*v5 z%b0WovDS>iLDT`%@fX{1e9j6=>Pbk4$=lpjZ~)jNJLaVw-(hM9bGdE7NYX~V|7>{} zcI3chR6pQdO!5y0As7pRtmk?)Sz;MGkMQbuNuKOKP?1vK><*=%(@0oT^ocY0hwKGZ2{81%23wNDk4IQ()(c22IF1AVc%Rok z9ciylxqxDI608qRpSOrhNem6pQR&&@=B503d=xn8B|GwEiG8lJ962t)4m`$~S7p&E z1o~B?TH9Qq9;HWaTeY_mc^xAUnPu%S+-zQaByN7KrEXNT_b0!@mo%$yqhq3JZnbFN zGbmS+>A-X8NGg~=$H=DmH+m;0GScsqSTgJ7Jjix4AFtJM-QjS2J1_A7G5dmH)#wQ{ z-XdNB4NVdNPERs3uRVdpFV|(sR**zPjf8agZQb1Ci`O|aFJjDBUYyEcL?iNa)yF5F z6h8agg2UW(#Ri4#c+=Tef_EKXdHIvlZm5l2cvN=$S>skCSGnwolt=r1EleD)e?k`q zzogrzS=}qhxEUuYEK-nyOW*D!0oQ5Q*I+EW zb?*!dOSZ^P=ypjAsy2R^nhq37v4hf8#)*C)%MMMxdv9Dn zxBkaLdq!B0GAxv&s)#YdqhD(OG+H;J|NT6>0(0&fENtArPxa$pR~{X zy>E$+5+A?Y+l76LpYP*k??{^LZr%Cy&BlYM9H+os*m?wTyZtnPzv1+3Q&I;&sil#? z-ho2hZ!5IR+FHvtcH`pZWU}(U#kcFDQt4tvR+E@XH{ft3jb!m(P86vhEPV|>dSHY| z@{v+go=aUT>G%OL&6*j8#P=j*J?T3LY!IX#G*-fyR(anF&DV4%XpB|d;%w{3))Q)sPP2lSm_<`*MtGN>J9W7( zO@o0vApjq(He@|U(Uy{ACWkQL(}v0h45BOoK4t!WW2p?!@qzxMA>w7}D%f4`&>5i! zPS2B@V5>I}e`T*MO0)BNEKQWqJp|$1Xd*#aJoKT1s+N3%U}AWZ2`AH%BS)Zeopfj- z3zB$|Vz&C@Q7rq0l%e>-q>zhrWK+Y}uCY%VsWZdZM81W~9F?bX^Xkf)PR1($n5a4_ zd93~RD~Nx~u*)>BFzK0E{7U)k*laoOM>h5#QqS9S1j(1wA?wV(@6ed8-9NZ^leQyB zMacOptrK+hHvsTYz;Y0d$OiNMnnQHY5xhd9k^La6@Na#Fqx7H)SAQEc=ev4;Pt!jg zFU?pr`tcSRAAba%u4$+)ND_Kna2{a+fb(Jb#C>wtneHrw)s#y}0bv6^)*p$~vmdB@ z*kX`H0nj@SlZe$v69&`!I(zQn`#?}0O3x6?(y{7)2jx?ircOOxpWQjj{J=p-$Bj&( zaQaND&r{L_*n|u+4tpfMM6*sv{{;q?a^}0{`w40xs#H2&E+iQGUtftCgh;G~Ii6;8 zJ%#ush1hlOT-!hoKwH<1;E!^?6G2HT;D)VgnpKZ5LWyVW&c8wm%ASRyymvRYwh!!U zioCJ9kZch-=7Gm5YrV~v^}YDS1f)q!A=P1Cd-S~7nUHq?y6l;_N`1nC&+#urdDcf6 znETwBRwhZlsrf@PZwobvoZx%X^kX5n=ynBbYy`bPg%Tf*hd>RG+Hz%kKNo7N4AcJ! zT~_5jet%4dbZbJo$S;|Mgrn<-+y0BlB#Oz*+9F_F+tHszIwa5AjiSF99zD}p#V7wG z%u%_d{7pjyWc& zColZ~U)V)i(;Wg+t@^K8wsWCSS(I(rap58ETdg3v<}ASKu^LS$YJJq{rkwDonA?#P z6PiZMvV(gri7^QkPu@DE*WJ`9-oEeE{=a}M9Ix!Ej1NkQ*@o#e#J&K>%AII*u|am? zN-LO=)g&5P&jb@Fw9V9b@zcldMgU|dW)b$3SLYkPKzd`!!}j`twjtvD30~Ny>~8%9 zZiTUkkB!~^>~!PmFn;| zJqLKHIJJ6mEjjXTUx@gGtLurU0Kw1Xy+4h)tk}qe>2PRQ&i2&qguoaVuwLcmCHAMr z0!QV0R~y6_SVnngEZd0~`56e`2CcA6oDMmYL!0X}h#NzS$}&x0*-jowvl$hQW|%7& zs{hzP(=SJ{LI}n&%vAQ3z3s~Qokx=|5@uH)4CK9J$zbUdoVF-bVwXTv=Ht7k2~lic zqCUD3bZx}B?t9}Q zMxKeBh<-YcNR`tD&F{NGGrj3kJYih7iLJYXnXh^T{&K%wT4)yXgZw=E4voUSxaka? zrtsPB*|^|&c~Z*yS~iwo)e9>gUb|utLm|QF+V~&;tWguu2uCwzImJOV2b0irrGOp~ zED$JkYuitdO(nl1WgsfJBfv-9%A+wOsVlBWcHL?1RP#c@JfVjJQ$$zlRABW5I2mqe zh~x+P28oE%?w``(v*HY$@0!T82<&}YuIo*E?)crcRp=@8Q5PL~ar86_r3vL&eH?v! z0jpyRPY%tS=TGNbo4fvK(-@Veug^HYAlXaH_IxZtzBVMrq{ZY`hb5YU}4cn3%F!7L`pXYy?^gaL7$&7NZ>8@&!F8ZE&04P zRfcDRvG!-icCVB#eW4eD?M>{t>}_T#Lok9YIw0>JLg5z_NCgB?c?0dzOox^vXg&Ho zh-=}fv2lw`6{5rc5=LN^?M7B!VB<3s^EiHpPnawm3ntwO{4s+*eC!=;E!i~M^O5~9 zGlPH3_5)3Z(>bQx6-dj$y1F9Tx=wWDVw#}*ycHfugN@g>*QO7N0Vjz+`iq z{?AT_@!W~@?y7&Vs_~bhk5jtVDwo|11<%;vuNPk-tIcO+H-M!xPM!D94mW!ur2Xl| zwx!0#UOAdb7%RF-#aRf0OpRB{d|&n4P3fcoUjw+<&BT2yu{k&i){^;LWG%>~3vo6q zB*hU!DRM1_5^Qk406dG?iHXw7%l@)YR}Fdtft6{|@!`KFtqpe=K7~wnLL{fx6CxtzNVU zAgLqilUBidyeinPenK)K0KCFi1^mv!IV)R21*a+tXwJ_(#!%Dysb<{UA7Mq81XvN3 z+FAuwa6d*G(D&EMCg%Q3@W{APEth`=T?fdxarQxCV61S8;pQ60QBMQ<8W{yrk=v&> zmfh`zlRawMtEp;~L{6y31)8$A;@~-83G|&*!s~4LT!uU^TiUCuq~@zEFr5&lV5aJ8 ze$xKvwtqPz`TnpjR*4?DrZ}gx1FD;qv^cHH-Bf!j#&r zokbHQN*ZYAYDQNqdXXf>B0o9CVj@R$+y7dcxt2FZ)ax+8?)&K4zj75xg{#0KOxoL9)oB-Aud80GlDso-3JOz-T$%V@@e}$d%9cSpT`Os@$Ddf_iop% z(&i4WihMh=R(D%Ocyk3p@`trv8AhC68vFE7WfiRDuEjOTZ(5xCw@BOfw@%>k`B52vQTV?>bixZUud5;K@{&Q*B*EE;rx5Q2V?%Oj7Jh#;BhVbRF6l zP9pxU3(~+4WbO23W>_V< zZ50sc@8?u6hk(!XRS_fRTNDFx;>Sal> zRv6Z^7jZ{?V3ik1a?Fa0bzuwAttd-6rA|iC`}TY>@!#U2m#FTL=bJb6f9Jvm#{Rw7 z0;46!zburWR0OOqiU`6||63V_2?qctNB3VH`g1kA463f< z>I%0_=UU0o!P&fi3-UItE!E>q6BnkpnJ+KR^U!*^h*Kxw_5>UhM4DIbxl%dXsnXZ-8LLv>9aCXR0n2+E8qf-_@2eIQl!ugaCo@_+b~B{mkz8% zyWi|`Roky>xMc2xQy;XI=0doLbuZ<(grm7!!&zn)ur0W?>a;4+$ui@C_$LYb=C^i# zz*^(NK0?w8R<@L%fM;h#Wk)|{ z&cHP^$@{z7=8-neuNg2Qxj~hp-}5hzW&1Hw&X>D1Oty$v9#c2h>4EKcX&c>~1lA#C zM#^g1ycC*H$RN;z-AKTE8P6g?qQM-?5jV``ug*(^*dz~w8>CbHNznfsbx zEl!hc6~s_XU*QU`0R;sE&b9rKj*0<6No{Q3IOVIbkTQQQw3xui;XD>j3&yCt@X^rw z`r_V*=yZ8Nehmw?AX&rgDHP9nCZX4KKEMSU7v4N6mLElMmttP!4 z*4m0UmE^IfRtB4BPA=eWQ;!2zZ3pKD;o@9sDO3=B52M{$ z)VcfDs@GiJTLMoDY|T7&B-B*g1j2ard>Buo+@Vw-?DrEGV4nkI$bWkjr8T$z1=d zl`1O#!4~0!I`07~U4grzV_WIDaCXadI9Y6)I4yOtX&D%~IFkUskm+G>r)yQ-79v=S zQ1iebjM$xOvy<}fBom#StZ%QHa?}p5zY)Kc10D zM~xfUg_E6>6T!96kjY@;kQPONT0$plp7wgI{MXs$3KpgLV}u3~rh`cB;XG1Fb23Ka zV6Cc=$9rLv6$29267+HSJ1M|$w(5iRjuK9fdT5uZl+#}UuHg*vErW#u+BeiU*?hNO zUN(iZx74)9vJ`z%l2SpH#OtDEV_P2FU#{=Cij01`FuG>8Y#?p`4g(m%)&&fsEvy$J zC$J{@LWfN^(Qa}F6`21{WGkZHmdy+!k@^nGQHGXA0Fowp28M-fH*u3%DGeKaD}l-I zb3j7Gk8i7+4ObRA-5=eE7lcm`GsoL;MB1Chj%G`hz71*<;Ny!n8pxh3mqA8iP5fp( zi=80!>GKZiy~K+eOyq$gfGAmKOa~f%S|CcA?t0A5&gWJ9rfyX5pnFkiKaBop+Wg!O zM_|89CKkfMfF`YNIaJ%CQpk7}Km@VReT~rOD=o!5D(kE@5lJmM5l-{^9hYA)s+?V7 z)?Jpe?GE?mh`&}3v6s55JaNPRKBcb~_}Vky%f z4#nG<4faoR0J(Sd)hDf~dJD&bkIOyiZ@1wHOd)TrJF*NS!6R^NB#yw#AvEOit)0t? zclk1Xde7o(^BZtjz?n>>n6jeVd8?X8cr<4011k!wZ0Ghc*RZ|2N8~m_?SJU4Vau7n zy`d2nK5`Qm;gw(QtLN!{B8LKswAKz_;YugVy(L%s#8K>b7l7_xuBDyy+?0P-&^h2# zyqar2P2t2heAbn?8sahi5mEfahH#D%P6}1SOekQ{`{#XX)~|@Q^0pf#ljKe=jGn@E zmq;qhg*kw=fuCOjOL>1#RE66M+m0cpWF0&QHj(_z7PJZF+XX0I4VEPwtSixb3^UDq zZK0A7JK!Jc=dC$yiF4!C`3dqxdzd#!?kUklwTxZihHclm46IJ?vmo0b+*Oqt2EuP; zqB;c(H$26Y_3hDQw;7}<`z+Ds;fj+KvShH;bw;b}Wx82xL~iK)(caba z`21b)`}&l9CisL#%ut&krhcudomm+UcR>q6ZJ=s$PLu)Jaw+18M9tipqqPFpkk_v_ z%%IOkm;eT`=juFwIWFUGLO&Llfpi2yrX$Pd*W2wx|v zd*YZi&OOjrU%=d3l@yD0t^au3-?$v(db(Oi#X!XR#4oH!I}!v2BUTV-sal51J%A-NiBx*`eANKZ?&xaUXIR0 zWne%O8{9!NmJ{pkqCu1Jp~0{nk|QX>7r)1U?YMGF5N%;O-Y>*-EzQ`8&|A-3%7d~0*z-})jk1Lz&L9l=_d8-`79$*OYu_4$4@<>)OaYix6 zF+toFVpPHcnjHRyYRY%DQqt(L=CEDTni+U#s{?rtBP>V5Ig zR=CkH%ka34;+dM?`a~JCOqpw^JBA~sbwd|bn?}T|tEwkaj))AhuOA#6AFMVc({i#2 z&SQj3L=zi|n^@eCk*F-2Z`P}S1x6QzoUBPs3TL$cHFgb$WK+j__k{J;<=(R3|2aF_ zs~q&hJVC|rcZX!2F zHFLsBS365#B1peWITRXiL@lh+qC5)#^r`mze%%f(+QI8Zz+K+Gohcjtavha*mZw5Z z2PTAmPF@;MbC{~w0;|ep^eLKA)UF|#w&9tsF}8X#Lr?ZFS0!O{v#UkBelDpU3QxTLgq9(!?M4_3hlz!1o@i2YDos(?+!? z2{rJToK|{=(3LbQIlru%qf4->4eV=m^x5J!-ZPsyOi&mOjL@(nDfL7>`j#$oO@Phj zqaAhcLwo&njpp3mT)Y1H^oA}p{=DMIIc7nMe2@5^VfZ^hBB7LKdY%Q|@0`xSf&@mJ z!LUGUHc1*t6-gN57=N$+f=MvRs(V5#kD5IL&keR*;)&1j!Qcr95g6b(G?*FDhGE&K z%%5>=F%4w#B@qPE^Z##h)obxwCuKTzsX?^}3TlmBv~ zN|1kYj+RzQoalVS||RnMstW^AA&?OZVRs52V3o=x<4Hiy~bk4oetD9;aFp6h=1nTr|0H z{zNQ61r&WTg{b71>T+r6d4PbC(Uuu&dx-!Z#@@tZ>-*6vIIn>~#&`k|KaD4U!TeE2 z&zp_)kwJQ1!=alHI&+}WMnr|6%oB@%B*%-#B24GS z^aZK><^AatkdFm{_HpEZiQGfTqxF#@T@VI1k;8NNd?hoVI8NB7#Mcn!g#QK2f^+NZ ze9g0gaa$)fIDoOl6qZA&5$PlLsmG*QWn!I%|7dJrVqv7W1>VzIq~;r9EDuwf#?ItD zwU(4=MoXB0dOG%PUbSx&-#OnSi*8lBq!W}s{Geb}Hfm;a?Ha98V(`Y4Z#P!0*e z&=xlm$Sh)<6s9HrYG@W|9~E2*2mg{(c|hal46f}F@miiW$7$e~VsCuSTrt2&ZPiq$ z!BDn9{1Zj9Fy~rr>XRZ3NWkc8x{v(F0z{SrS9X&!55d(d)^yxwYPk;=hx+4;IU1LU zstGhvqIk^LXCWdj0Ct!zP^KcQSoMhj0g(s7o$MvizZ;xVV@CrC*S49C)Dx!S^pzlu z&}rkUVRp;3AxKL#1m78C_Pilv7+)0}(-ZYW!4qmie86UjqD0O(U18x(BmW!#>IgA= zYvu0{8d6PvszA)am`?s3tG-N^x?3O$g!2Gmf!k70Js<`ROWPI+vR&KhcimX)<>&Gl zS3B!*L)7{@Ji3k7qy63^94qmSp-w15mIzUv|5XvOU8^Vfp;vrH@2>XigiVHD|87g{ z4`Bl(KQ*lxh|*D27NqAECV4mUk_QOvKYu^-VZ@EHUj@^=9Te-8g6dB25jP?g4{6IR z`R4isuzQT_G9rP}Bh~NH4f*@)gEh#)i-D3rM>Ui)+Vm4dqj@u148fS+s#*|5?rD&) zJujW9AhI);(c?tx_Z2iE3Cb(fM&Kukhwx}kssRZmH=5g;20a&&dnMX#loK{sU^H@Y zs&AFpbp3^E&|=G>GxFLLI==RAC7MyOSqgG)KIdhflfg>M+t5rkA2r3xx0Hwj8NtaV zyc1<;(+lku>JmhRJc6%hcGj0wD`ATxktgv4cNAud%|XvIZ+l6XzY9KvVa|Y(4&k2Q zM+HYCHwrKV%(EgRUbwpkQn9F?Gv}TsZI_6}!kghzV^wBgKi{vAjh+OILabZn;m}yh5_0Tg^JF}}QUsbGcBa}8hVbVx zv*_@bZDkC@(8+<*0r{p!${;fgVdTsD5eFV1b!Y?Pt1KWRW~D?aX$tbF%D z;0cf5tTc*&MP=X9&fti`ji_Ga1Ww=mco!yfNmUU^KW_VZ=f)Gz$ z$kTolOZo-ZA>wQTVkDxWS6Ixkb0!(pkH53q>;l4XFI)Jm%LrjFB>SM_odH&=#dQA{QJuQ!NFPthg0fptKX?~a$)=fqu+%&P34VPa362;Ciai;x1uYxpq%1so*& z<=p=G7r{&gEx{G-675)DR*Hm5ZJ#ZmXTX{-N&M~ImCzu4*%2b7IsvijSvaK1cMH^* zXKPEMmM+wn;KqA?*`!V&9NpA`7G>46VxEI5X+vL;8UwJZ32b)Nr6M;ump^YL?vMGT zE#f*F6#;k5{k`I04ETBA(miE)%$B6w(KSC;PXut|L?Aq%Xb6P2Co0gezD55kBPp#= zV&+Y_M#*bgtA=V0j1-Jh-Vuui7u3xTBE>!__&RpPLal5%VnLO1!r=_wK!P1iMc0%{ zwTDOXG6RKm7T<;yR~bZLsYf534n{?FHH_>@en%u^mnAS zqtD8^-}S8dGFlY++`eG{A>t=m-IatR70^=W)1C@eOuw)~aI=Jr)CCMiRF4)D$CL=e zunBy3F|9^i`S&9Q(BZa8w3>ntV?VB5X;W6TMe1-pPO-SawM@J(N%y zWox8Ib6%OQOb1HyiywR4|Dw(ox!p@v)XN z&rcF^DSQ@TRYUwNbe|pWsrBiY_F7Lx6{HFWJlAQik=)RSBAcOxPF&YQqhjmz`kABm z>eOMxJ*^*`=~A^LSG>yzsA0INj7=|%PnqbiEk0j21!T_)c~#zIJVf93WzaN6kL$kL zDBOH9u^!trnq5lidR(n|9h`oXG1&F>XTcGigf2>Zr3=F!-)Sda%^^UMz)uUT3UzHP-e!}PrkkUj^nIwbb18yE z?}v`X67+FX%M;6{8R$HF-i>md0cb0tOvC=NOs2c*^2sGdLy?2-IyrI9lnlBv_;!PbL)#;wYQf|l4>OSI*dZ=iAPqE7v{Q50C_SkG?ztC zJj&imJ*+fZBv4W7d7Dm-##J0aJi%~S>$c3;>S1AZqMV}blKT9a74F5v2V|Tj`rB2@ z&w^c-!UN&mHRH6s9wyejaC-~J$uxa&Gq!DSirw3*wi1^Z`Ep|5DyvTRyVpeJq9Fsc zA0k7jwrbgyRh4x%!=zVnVa2kWkHW}Em9(S=4{WH z$JdqC^E&@7_q2N4lY4GEQGf^k`Kp9?oksjMpay8PD&$yU_mRPcK_(3k-i{yqt zA1%p_>))%gqp%y*+(hX_eG|O z8W39L+3Ca6&M*z9Ms}@cL`s&{dLnA$Gx`UL8}FPctVb73`t0+0l4Jp!rA{#+ca7Ys z3YL330SCnedAXHoni#mwyPbbrA2tjnSkcNP<}$WOlk0)FRSbFJ`m7LFX0gebepDkP89^X~&NqqZK@VfbLk!T7b| zvO=^aj=-V7olc3_6+%MC(B)C4gJxfDzs3%F3uIqCgxMwW-<^)0PaiyHAjv*W=}Hv& zuXBC&4#k#-Fa z01yKL{>QS&*v8V-4(M#^XkyQV(@}MBD*XL{!&e*M zAIiL2rOCYSMzWQ!;#iSf&Dty26b;&FI1Lvd|b@ zd7m#gP>A|6Cv*liMUy9LQ?>c&=j+CeMBvdg?%-bjgmCp#@JFij#SryE2(gD3Sqo8mYTWM?N<@@%4%z@Nb1ZvgYqX+pf$A3Sv|5Q zFYbc8(m;z&KM(5GUax##iJ0OxO)5L4Iq)3nD7SB?T^QDZYAQ%1k=<;~r>=US6f&!s zo<#5iveS6?K~&^O0{6%#A^zQ_Fd^{F(U1zkk!LwIh*~h^=wnMkCwx`_zmRHjRJZzR zMRx4K?A?CvdgA!q_R*2Ir{gTYyR)nBsP1H94x=9*uiRE<&fD+l*S1q1cP=02G(Dp3 z?Cj2vWJ;K@dE#mLn4&|^Qm#axnfSMiCauaZvIcr!F%FBCO?LJY;BlJz=A1051@h`H z_l~S*3RSg;)9dhf5fwY#^ku1T$YPr|zK`n-RkS^ve>~vPwHMt^r0OAmN**Spy+BK; z)2EOiAH(Ziv0PD0UbHdHqiEJ9BOU?{kZz~VajT+wkbXZr#9^7zKd^s``S}};Uj~{@ zN{=Jo%%7jZ*W1ej_;A7IG5T~BW~36JSrlmX|o08$Z%<+m#vLET73~- z{AxX*Yp>`f54(^Q_^Jcpc@fB2XJOLeBfKl9tb|~N#r-7*#W0yG$bz_heC6=|=^~DSH^yB@mKtaAIJH6frb#)wSh|pv_h_E$afmQI1R9S=yYpoOt4(p zyXGK5qE0K~Te&i(d-*tC0-QulYc4Igz_f~`v0Ld5UV`W1A!0WZkhdflR3{Ddn99{$ zP|+ndip742ePxpiAk=!Q1qoJ4+vuAuhsdTlX)b190uk>dVET!6%{~Q%%w>E=DsP15 zmrdORskzk^TqG~VyDaHbh%!bG!)ZF*LxqPJ@wOL+O$^(-HHO7=lezNv z(ZygJDSs%D!8wt%jB;-W2S-Vb58-e8FCABf&mz@AiIw`@N$DbWj zr2hlr&WVHTIIrwtiS}y`etgdv=TNC}95+*C$(RvYl`iH_zwiaoMbWn~(G%LdCCMPR z9s?wyG6FAK3k`0!o=SD961Gk8je|2|83ze4$wsEGDj}nU?zV}GHjds2JVlbo5h&F3 zBs+rgFTbKx+3i2)lB$xKk@y^~>eAG}Y%Ea2UcFUWJRHp~WrgEeekC}MuJ9Iu2`ZQ9DMl?(%-mkZ08XtSb_yEY~!zyAurc;kiOE=66)rg(pbcEeYWC)|^pK1BN>o>0zIjc48z)J^*0~lsk>LTY(g6hKWGbtabDYaoL z!0CMFE5a6_j;}M_m<9^SI5~irQHwYrmvaLHS6{MbsEMja63OF8|8l;dk|wY$8i!Ey zZ2hCen^Fm1%j+x2*<+@-9aScfEQTZ<=%eZSXQjNE@G0JORyP*(u;|r!=L_TWD|~#U zPY*j>5m*Mq2o5-5WX!!VcR;Lv*-9UYO6&PP`HU{APcO;}wssgb6Bv~UF-9T{V?=mB zr;Qsi*VjUYIM*aRzMH_Jxxa7}#Jh+ag4FLhZzY%VZcFB%O1+K_wz?Piiq&TD2bibe9Kv zCJgny0Pi63)|AE(DUx;nm{_ycs-w_8`Y;ukMjf{vU@i2rfQzbyW-vrvS4Vl~I@H@6 zj(~c6z;PhC^5>}*Thv3QuLmSvLK|V_EF329%0S>%Ms>zK#VUN9xnqwypjB(rMa=NZqH3e5)Zc50L@>SdoO85a%X-b6#OR}-sc{I< zrqUQCAocx+!no&9JE>Ve3xS&iL^ZDxibl|y&_MvYQ%45} z?%wItNP0`HF!+z-NW*!(`oi>#Pre}6IgwR4yg-vGHobli0&)gH#%nH}579uzPt6FE zdi9J6Fi^sE1wa1dPvO^tbf{e%_&ZL%0N@&`b6?K^49 zJow$(Em6#*mBc+gUss|0{04JZxQV_T)qtD4;0n9Kua@C8*G#z*jI4rpaV?umtlE-vWoov(Or}U!!eT z)4wOs8K4Z&Lupp^56#m9@bABi7MmSeb0HBoWAF#JBs!TobdwcZ=O?6V*-TI^C zid;>c_XV0!bCi_;lr!CxuThaEJ1DcF9&Fm{Voa8H(h~HXDkVXhmg2w>Fo4uQ$H%qaM)2bt3Fdnua&OVi3uEum4cSA&%!fyY)~Wu!Fm$DFM1KQT zidURFTbBTztmD%=dXItGYBv@-Y!M!>n#k(y@I@l&I#P^Dceh|6MxPNg3F@9%3Q&@P zb#1=W@8R$4bsA1**M7Xuk`9%tEIJ~_#z`=gKd_6wKi_4VV_NdM^8i&3BQyc4))c( zVs9F9CA&P|cnGY*0bM$~0A*(1q9DfMcCMd7A(*XhJjUq?Wcvci+w^Xevt|;mB0B)$#0uHXw6Z>z} zvFX#-piZ$(a&S3&A*^Ib=gT)=l;yPSLzZDPT)H*f@Y9of{ZBAG%YJu-M>E3|s#!){ zWi-4ZcJ)b{NXdHk&LRsNyE?yVHrFASP*$v6Q`)FrYH}BLf_c7G?HcPvXlqVz;}*Hf zkH9Qb8*Nw>jy$L8pl{?O&j(NhJS~5Eu zcN}-vXwitzc`7|cOv|Z`N){8J5|r2A|FA09%neemc@|5Dx3^Ams{=@!VyQVD)-xP% zesQ%fu}#ir=kA5DD^C9od>h5*e;c@{)NOUPrDfoxro^vxCuv?^Xdqokg1K9;tMDo=-Qc70QOuH9eK zvGdxbeY0v2p3AD7_~Pe;()1X?!wcf<$#3(_1ts_UI?ayunT#n)V!j;s>mxhqa>HU2_dO)g{G96}ovBJ*mE1`V-PI>%#{kHSDVL zf<{G)Uq3f~Z8s^-32wAK-1x$-P=QSd#Rl_`8zG}>dS6zgs}fYN(eAqg1(MLwdoB=f zsd1%!vJJz4`%AxOk-4Ratag*NoIw6lakH+d^wNtBp@t8a$CH#roWz@%3g#nCW|-=# zs*n+V%mUH3F!_#K2kh8zR;y#RjaO~N2G8Y(WN$9QDDSfzhu=zJVqC>>5XWqu-Wkqt z+6!0`rXg1^qh-T+-EGg1c;C?N7uUh#Di25;j9jkcLijgjnqKg+<2_49`MRg@fJn2t zCl4;6ZAyJq+O+bD~vh()LI;#Go%fsAUPk*A^q%E`fra+yEZmJ{8#k zYVmhr-tz@&JvVaV(kY!C_Y_a`7YdjX*xj}CVk&j)fUDfo$+FjnB6^8KH%fy&7g@1i zKRXRg$GKqkyrKqQ!2*Oa7FweWQ7|U)I>Yq_M-oN*qOZ5UIqiP>z_hydo5XV; zxbqd80!+Vv=xLmwRGYQ6@WSJJ`|1$8kTUhBGrDeiiBC|AP_{lIt9;63e3`~0jpx)` z#+`L0D&|{x6f6&6+OtM%6ftu#LD0*3xDNm=*U`d}r;2sOsnD2imq6gMr3yF-H73n8 zPZnxMc#Tvx!-t@3dZcCr3vcMYB_IMN^TN)l0JjS5ZxOzpNj~8S$5^MSJdEtdTsVmb|b?%UdAxurg-GYZKs{d-9c$soKM(I5;oNQC2GM9Dr=Tqm>Wg!$aO3I*Jm6sKZLiTCt}=(XYh0r>4F+{a2DP8+j^H z#@0njFi&M#rmt5(sS!nv3i6p$F3nF&EKpOWwB-RtWaA3v`gV^OihI>F~qe+;YYhD1}GGqTxmZ~OcgXq*uAvJcz9lXm#hdES9K>O zQ05pNqouWbX_?~DLGpeXNOZe2M3bC$tW6`Oi^NxiH;^)-l+}ntpIoG)-i|3;pJQ1> zHagVqHl=L#T9#tU4b`ZJ2#adgU?Pd>=X7z3{q(o_<@w0SAoY2}X;Js~P{Y?7qZ{j) zviBiA!rHU)i&^o-`xmyH%OUCs8ZSjI6^oxk^Q0#`y`vFriaoAXd-dmP84SH6yKqR# z{;)SwZpsvVp!lPE^RsWVwk;wyoLPk5jmy1U9RBDFacQiLj^16}C?Z7fH8*>_=1mR! zvXS^66)Pg<=RhJGclU6G`pD|y;i6t~&rIRl5MtiyslEBbHLepR@R(!1oAo-><^?N< zy`1acJ8Rx@tRTR58XJ*4%WTD?k%wUM7;ND-Xh~LOEqfAtBIcJJZn5;-p7*3IyS`jw z9s`QPYrZ|+xqB=w-OmO(!aqNzHvO{ZY&V6!kM$Svb@vl#=8B2mfO&QZCq~9ac4$_h)BR zs#WR*4?mJw@;Hbw{xOWROn(qk2Z{(9D9uBsK8T`-)U}*@Lk&u>%7e-&z;E(TN+A1wSKEC|qn)1=r^p#!S9s@}TDH1|34N+EDGs0E=lulIFraMm{IEqxv3zUF8HDE{!Br|C3t;`%{;+ss#ay#!ja5bwT-Eh9(4$D9^d1Fk1tn?WKhu_p*(-S zz6j3Glf9dG$i)}F<1>yLumT_r?Av{1QdnR}@LX(#gts=JzomM)$zll!x~E>GaWOne z3v^9QZ=MU6|0bQTXj+1scyygwlsxiUZ7*m1@EXnuDc)^?mLP^J#P1s8CW14a4h!`1 zk#1nsnfkE2dKIia8EqeXkGyB+to$V3`$Xp3W}V0Id$!wuQ5V|O*Q!U@pGIK^31X~) z6GRvfl%fE;^@EYo=epd$+^NTe6-f~6~!Om7J*O}+Q zPwWs3pRM|`o1V(Rr9xLtqbX&iS!G0Xx+k=G%OVKCf{pKq$u z3A4Do+TFH-#^z&NykM)I@yo_jGv_p>o%XVR!>fn+_Mk!@Z%w}|41+s1mzZenBn8gb zOBg{@v){KFURY?pglA0%OZ-A@Y{G(S2P;H;I(1&|7XeV^zZ6jF+Tua=FiYnV0pESr zEQQ{cdbxZTpU?AAdT;ob=1mDZ5iN6kf%v$jszgF93k!lJk3dM#Z^gmpm>}GXH%0{+ zFmNIWOaK4?3*gI**PMDz53Pp+0N~I70F)0wt_puYZpKvmw@6cp0>=zcKRN2>H+CqU}|p;{Tb8tR)H0!`fvJ^lmo-<7)mN~ZAo wXS{!r|8L#y|89)`p5pyix=h4B>Hn7S{eO|6J|@t|Bk*xzd`t%Cf`4577xAUoH~;_u diff --git a/python/lib/py4j-0.10.9.9-src.zip b/python/lib/py4j-0.10.9.9-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..035bbd38ba852e1ecf6528a27db5689cda73a905 GIT binary patch literal 43108 zcmZ^JQ;;r9l;qd8ZQZu5+qP}nwr<`&+&(2KjK4e5aR8-c>$vlzg z$V-8Mq5=Q_NWkyBc#Y}zjF3h!000UJ06+lX0_;6lEa{yr%xn#;o#^a6RFt6rAS9Xw zX21S{i#rSe2=o~U0QlbzRIACvZm=PCf2bjjiRf}{<9kxTzRnWM)bnQmWE`P^>1h?% z8j;Hrl-a&tdZFTE3*%k5FE(br<=>Oy=4>y|4d@oL| zLLJ9!*7u*%Hz1`rn40kEuZ;)pXPabi;@HYw4DWwAXyUaw4N#ELeh^QF2N{pRIbpwp z234U*rfGGz4(tje`(dl=Hi^DSQ*no_Gp;Ux^R-}Hmu1$S#faKFa@q7QKMfLJ!jo+O-u#s#oMUr(JdGS;UI~lfh~7tqv>>R(5hK$ z77)#%vPSiaz!)g$pin=aofd1mU^sA@wnjhD)pMXU4U)_Ort!gEGQka z*2WTEM_h^V7s_jOxi+AGm%W~IBEUlHSYEyWL|a9G{nm7c_Rl?5s!wcXAX+aEN-0w` zslh;alPGh=ljNfrwDymn$Krfrhm?ML!!YYU(H4ZO3BwyzmDb~q`_j{RJj*B#jlRtB zJf{2wR_=J##YvY#pCS~B-KqGI60SH-2L%OW^E~e#I?zGVs@&zem_zGdCTk8g`RBgA zrltl-Qm($f8g36{`%9v?CKnh0NYQn=_%Yk;_plKk^-c-zS&2M|cfY2`pMf0+f$FI} zeK<$O7!Z-bVduaEd3f^JV@wfKOX=lW@yZMnAjb`PO`6I|xIqa0nG|PrH$tKo|-`^BbG z%Q4-O0S(!)ySJHY1LlXY^IsNs^W2tzhk$IUS&|LKFq_t~{)xhzY%-Vq$FS*$iAJYJ zD;DD*MHPjYMckB$a-TeT55C4r?7$uH`gHy&o%0#`Asi4jR%Tqilxp4LN@DKmzUmp&asql zegXT_G38YG=VZ+5k=hEBii||t_O7C0lrQ%2anXA8~1V~TtfF`~{ZPr7_O z2)W3w*$LdaSW;)@L6H0Ezk#mMI@|AlH9c^7gz9KL+8VN|kCMeshjMscuQ!yzLa51` zMnD;{NXK*iNW!7O;=lSQR`85qf`FUyDCQ?kimO#ex`D`SAolB<-9_N^x#>)5%@)yG*XKrdoY6%t;Z%Fqa_AMh&E6Hz4cM3s zGijTx(EMF59Q&}mB1Ln6m6|x>iL(RfrLBphNjbo2R&DlPOP zKi+3eYgD8~&FJK4r#ZdFtP11C9Hf+n6GSG<$#Zjcj2H;I>4W=7q3$51K{qxCVCJoa z%8ueUd-Os28GzZKPN{RQ<%E^O^}yGfR7uO)vpi-z8n|u)W{yvrJ`1PSRJi%m;b}{nK(~IvU| zU+w2+2sP{e2meF+_4NMFJv}`_9zt>kIz|RMPC8C<=>K5&zwQF30OxrE{0{>Z00955 zyBOKo*c&+iOYk4&@*nsg1kI{awi^tHTNkJxCL|Cw=wl@e=OXe&I;a$yd0zerqKr!6 zNfK1?bjvW0hIk3@nl*{2WfqF$513QtmUaS86 zAVX-V(n;kTgGnf|lqjVA>Z{bGGD49TNt3r#u3FU6FHbQsVz`SUJWba=#b#!EC7;>pXy9-wET#%ZbjfqQW)Kj$cm@ z>^`l0-AML76a=!2OrGcefD2v{o z{=GpX?(-4OSe3%a6(~))ytRrsXzIvQww;JhEjDhpVVe{h^&6z4=WHAC%!Q<_#EV_p z7&D$NjK!|4CL7w0{?(UQX3v7oEh4osdQn3*5^F3te7?C;nX|vtSZfW-k?SsOuhYqY zCGx{75RPlNr{c_I%j=gbcWCEVnAab_#bC9vj>C1~!ls>~D~T1=_M{!&lzW$%V9WAY zvmveuBgGTQYSePMMe&_dqrZ zZD43Vk>?9C8O5H(&0f$`)<3wcyykBl3-{U|Bk9juH*fwHre&jl)Mr_F^XkO~w-|rJ zurFxMI@uAY^Ukm4AEieW(}e2ZF)bdpJoPd?_DI*RA!bEfrYh^JGFA7_Xz>*j|D2*M zWNE`2*@2y4eBf^-7ke)seCo)HulF^;T7hs5MqKjcrWf>!10!t?2_>9$Bn)9E{F+xs z*J>kU)0uf{e#qR>C$K}%EAB≪nQ+$v{9{X^0U-m?uS`vDS&!tLs#Y_@<{3Ty%~@ znw5#B%5aMc0VH>XkBrM%!Hb&B2g-Fz>wg{8%lATl1O9utD7_a3Er0+3^N0Wd(!b?m zYVl7mX5ne#_+KmL|Iv{TA8y*(9 z7e%Y)uS08;*B>Lr*0&!dMfoJL90g+rXUPzi7+fjRq>}kz-&9{bL`7cg9|leeVwWC_g?Zf@exXnJWmH|AG`*W)p_Sj~3DMgCC;2y8u6(f~WU0d1rqZDq zSeIQ=%bc_n*`AaG%yfXpuR0pJdRS#vJfOImqK>+m+51ED=;h9&(d&(FN@YvF{f@Q0 zQYPV7{=Amm7X-CKY`=8%j)Coo2JoYNh?+fnzAMD0Kr52vl<-hBE2y7}SC@X|<#y+s zdVy#31q>D$h|9HwqL=h*7;ne+yM}|8=E1C$be_O0d3vdgXA!y@*)RhvWFS3=(Ni=bjimmnTx}1(9x=;cA9^N9Tg1nPT~@_&pRjteYprGp0^cv zPI;J@HBFV9?4TEO)wQzG9J?^zXWzN#?!a3_xOQ+~&#u+X*oFV0r2uq^Vl@Rnm6Smp z5c)a3kj(fdHmV7%Tf>6Csgw}CH%GwE6qIBolsn=Or-;LJE`K0&6@{u7MC0ZZ2t9gF7FsgO8aCDkn_^!S2ckLY z7#;dWPL)B%(ys~c9gjWiZT70riZCB8eg>Y$MHAFdyuqzGW<0X4A3R7h>2tp`E63Qx zXTf8B-{3chOr@;gQQqZQAJxYX4Yq(8!5P{ao8gCXn9yp(08?XS zJ%P@6K^~OUCWWa<#shXuXauD3EBmcNLtq*K$wV&?A#Mygec>@Tn_t%8{jqUt9}~MO z5(~vzU0UuCffoAMFH3_rK}p+=fsqQyk`Y^&!kVPE!zlUnwwKp&mLR1hVh(39&2TIc zQWrI08<5C+ZTsdmLIhKvmo>MJ@*k&D!ZTfD(LeHFP~S#EViYSn<5}~tBi3xsYFLED{+x=(d|X5!%osghl%|Y7@{}UO z%B1y~%*F(+nK$RF7|0Tg(|=R_nJc}y@bS9r8OGzhcYF599QuAl^EwQ`)2sD$YtM0Q z#&1_AKV=vWDy{1)&h;2KS;&QH0(3XZmiwX^d?Sn=Ks7=fcnqEKDQ!8iS#S+V>&NGW z@Yo+&Wc>UWbwD$u*G$PEt0zGQ0A4r&0K|XQ!QRo%+0Mw$`d@L#pa?a~{0INjPPuKI z*2U*{o~Vyk5kjYh>k}8Z(cRXl0*->R;C}lxW{#H(5rWGXU`Qgnt}1_jxXfgl{@Uxh zhOBE36U*5pmB}5e5jfnpdpiJ;T?*qS|gLx5Rl}>AXUKH>KDtXc|)WY z?1-JGentp61EWA47?rE>dK@|F_+7+0mP8+RQMjEd>x zw?LB5$?R_L--@9i2^4u;GSOANf|9Ei8ZC8gAuvEgTfjQue?@_2m?tupZeP%3>>+8? zU}(`MW&VvMqHx*Z)so^x&6$vfc1%{d=pQU0&;UnSz?Y6_mc$TC;ZcpyXY=>W##|lR zI%!DbY_|Z^SRjH4!V_!<>jU-LyaTL5hq5M#tV(Pk4cOxD1#gO^BO*PJAQIoeD*yK( z4WxL|Wnsy^B#CrK0Xi+;D8*~ z7E=zg)PYdQpuA<>h*(EQU+?FBm`3ARY(#)gXrEk`!v|1n-FLKUVlH|8 zn4u%in?RzUZ}tpNfqRwxuwp;2AmF~le>C^=`*=9M-5I@bq{;i_7(DLQ3FrdmNC#5z z%s8-O4fI*CAQsk)&u7r>Y|`ou08-y}fvx?Ux!reW&rLil4#zONzL0I7m2)?cw`REy z5ARdaPG@>>wzU{QpR{Tx1YFCf?)C=<9Iw=#VNyMg4x5bw9(S&LGl9b!l)0fMbyz>7 z_Jtt@F-|;Ky^FcE+|!X`hwB7UPqWZrGPZi!AD+h1hoOa;{mu9tLp+nB;3WkIeYT(J zeE6!=4>8PXQ~C#%fVAunm@fy2%c$d}leU9R*phdWD~^MWiwMhghmycOUvPSAXfOHc zCw*NTY!-(DdR-a1CvjkPvi@k_hk=uh8LsKfv9gR_5JuY1HGRhN!zM6EJ0O^qmm9Pn z9J#AexgAby0G}dMm-s_HaB5+UzB0@ zZKrpwi$eNPqk=VOh!+NFB>EiCrtJY0iWJGwAz@dRhkAN!C2SmENgUCKQyQt>}V>CkZxLj*mGYxG?iF$%YNks7xTi^#(LD zcY4gLHdMXRasiaEaTiEgXKZGd*`155$8*4N<4qwG%H z-!que`EVlw<(Wc~4>lmuH2nB9H4wYwLyVkL__uC&XvM==>|Jlhe}moES<|yMv37xrZ!PdlObVO)$wF(eRwKKFUhoQne zxP^(3cItjj*chdlJ{cE?{^1h5jbL1Yb~23lgJ-CjJxk0 zYNau%N@bql?-iNc$@?-Vt1ezcAemkxm>RlWjktgd_Env-RA@JF1!Z}P48|6%3b8@8 zci>%D3>ni4Y0W}vHv2OZ1LF%r;5C>qC@`L1P*fleV#wNJENqDM6}q5>;;X9+yM*^+ zZPMwo7BQTL>65Z;$U_}FHez5Jh9+9U;*$efwUbg6Y%TD7IM^Rmwbm%u=@XJv(}ep+ zS?(4M1Q5||H1)6B07bC<4t@}tx3H(oGuzD>??pHZ#?2*%O0l>f##zOaXjnbzII8+? zEW8Zx<3IO?Wk3%#QaMHesRZg`HOuLy8;Ue?9iN5N{ZqJ&Ho;Y1mm;PC8!eDbUS0yp z#I6339n`bMa?yiS^tM`E;mn$aroYB)sNpH?sOkt!UgQNzsx-fb{SUiq+TCtEmI5Gb z=w?W>*4m|o8d)gRJFb)EgUzM%~j!y7nijjiRV|3maa;^;)AphyyCh%KSjkj?h)7 zwY~-q-Fg6Agz~zcnHRYREsuKER~#lJGxJ^+#Y2;vxSV8tI)C%``%ykgp})@2xO*Zk zB3;H00i+srN(7QVT2?+OY~9mlfLpS46zQr@#Jo(Y;o#89c0+}*Daku@PA#pR#XBKR zV7*b8#LR|b#lTL)4-=eV#bOSl{JE@t>)L^GF04AqB*Ua7zI1{ZgaU4>oYy zHpjixSH`^;q!;eLLU|#`sj&BX(nvzF!uSuKtARTTS>7@?W%3#48ycl*Ah6Cw~JgUXa?!f!LfQD~!0ue3vksz(i@y@7BrK z*auBy?!%BewRu|cZoi~aqM|6_9qd8DsK~58gZPBxcqew6p+O)>V-ns$+xdz(hiR|@ zDs_bejG&G#jY3^MVT>9*mIgl8oQT<5^f?MOPJP34&0(DQ9pS*dVjd;CG!b`U>xWfmlHFhV`6Qov zF2LYbaEln2vT~vUJuad&_|tt@UzH%ntR*SVg@0YH&d(0q4|NzX4_h8d;_or+QDEPx%U#06YCs3JnTf+ z>kl+{1^XikntI%v%4x0q1;fRo!e8)yE0aN?BQsFnlLf13%}LeIk|d&zucM;MqrOfG z9t$cm{ze*K3pujyOZW67I-Ff-+lS26{N;R;acQZ9m{mZUhZvnU-R#pFIgl-Z;vPDQ zbix88@j_s@HOzPGO?w1kH?q%h(IJKLJb*%l?L|f)-ZmB>j@z3HCVPd0#Q^PUFGe%d zsEv>Nvi0UgaV zY)9|tZR5?x;FhSrCAou~7HJDS*GqTHc#&NffBzA*gGqU~N_fQkIIHMH^j`}^4PMoSoYa@tvRj~V29yYgx zQH9-#bz(&LoEdaW&O~-;MCn}Cic0AUtf#G9b`)?^z^XN2&fDMemj{1Jn^>46E^D>& zNgc6jW+hPmj9ErH zq|Kh+IiE6obrwyF9jkh0;a=?vv@>)!t`rPY6Co)qwg=*Li4e$g$uTLLI z62P1rg6qJ4Q}84is`yErJ=Eay?w*}P8$Ttneo&D|B8|2(Oca%(3s+fVql1 znZ%Dmo0JsmE~r?PtWi5uqsmRd$J@b|yN3)C9HcywphPIT#4*HVkk1R78#K`jeOUA* zTo4+tPj)ywA`ydV7#6AD8|5?7R#xbjFCDIL^J|K2IY475y_AF?_Lo%3tiW+e+ITJd z?iE_Jn4P_I^4XKqWYGUq2(0A6df8UBPH9L>S+UVP0eLO6{m3j^V^&q+a@L@e*!BOY zfTYrDPmKC*FdF}+BEc5(H3SZ+HLjL;SzzY> zqXVocPL$WjT%;aO&)DoTY7*U0H=!gvd|4}%AA~!p^vBH323*ah*sx3O2*jJG3Qg5y4r< z(i*ZjE1uPYqT$t-_c53daohub;%^^!;KXd-1lse?)FIZSV-$T5rWH%|uz~@;)V?HC zAl)}fKof7+&=8?POa<)W6cE-GbM8_&2npP@MtF)&v+ZYbydmgn^ZLR#IRxb!F^HcP zeq1VMpMGIM)7DdAY(IbCKC5zfE zLfI$K4ns)N`MuVdu=kn-M}IY#EyqE5cH9aoHfZOW(#^v)m(391+glH;1zk`2cNsU; z{uF&>z{(O>O2TEV^JG^AgmI>|xv2uQvqgSq4)}Q>7IW12GH^*DY6ry9ah0|J^w`~e zJ5Y)4q%DP=q@#u!3(Kr{^Aakh0s=-(9NET=H2#D#FoJmCx| z`s+?keJW58kGxRf#WaJ)fg1e!&R{Mp&i!+7n#>@K^Nk4a=TIm^Gd)L|BXGPFBNlQ! z2}uu_x$O*$tVvfvl&9|C{$Aj3m*GvDh`jG6B6%G@yB=!9KV=ngcPHewtv-qk_NkI2 zhj!M`ch929f6lYPGIny8$! zb(;|m4B4xtd=&h=-l2E(8wBg=cy$$cw3lp*xOd@;i8$yFAC#W(Il)S#7)b#pdFi?` zn%vQLS}olN)xA=rbxwN!dldakA7oYLpSh6TKlsnENl(wh*1}m&@87WL|1-N$QiUvI zxl@D`m!6iDmOGN99-mPOinx3Jgm52_#B^rO!Z!jqr z8BTxm(AK`YlP8lerIl7kkVu009~qbvEJ0_#CIIjp3Gw}HL zYyY1d>{46HZ9_ES&q?iR{!11R3iLmjgKOx?sX)BBSh#IrWOmgEgf*}XG8-fzfFdLD z;O$B)wFjsOm2{fyqb?_07CPett_eD%XnZb&5I_#wWn+oSzO@3m;<=4B?r zIGJ(|m3RtAWvmV9qDdpn@LhLU22L&*b!Uv5Qnys4F(I8J@s4;ZIZb6)adi8rL<801 z#b))QJ4J=6G`&KkF)muY!-tQYML;;4sp33?kT*prpEl32AF@1=o4g%>rIJk@BUpRGZT?b!%~tDuDG zyAL!IEDZ5!e6}+mLdd}dS~^YEij+^;a+!I-{f;-NnL>cpIWUiWSC90e9Xlep%49)# zCe)D13hWZ(jmR9}%5eBouSm^$dpb3*ZF<(LykvT)IlUJ*gV8p<{bWm_j5nbu(ykc@f861wDaO1u zN~1VQ;k}Zp`Zf(e3fdf&k_U>s0-6f5r0x@?65hyA`^j3iz$w07vCQi8v<))Ds#*7k zK;&Hj^~sZmWkEBmw&79>QLLnbUrV5@7eRDLyTDYCc7Y6#oeMVxryTw#^hc5qcZF)@9~2syYASVi<^8N|ePwnLjQ|4PSuPMMP~8;1K)Z?lKtG^<*&?^h zU2#(^u6+MCGQ$BRkSFvEeD_y&3vg6blXM&Nk#uS8i80|(%boxpP*{YPXqHf$U$My) zrUS6{##$itID~-BU#G^F#AX>5bg;p3Y~k4`U{+r`hp4XFz-j!lgx%Se&k&PUWtlr}mf~Pb&vIGH2rrHU{6T|I z!RCipQ-GVMxIgHgtF>=b;q=GLG8rV znIaK!at6vk1GA&Wy2CI-0W0LSveR}PfBm=pnn^kFWSn3j~zQVL03 zcas#rfpQlt5MSlZ()5>sMB!)e#2wf>y0J(%X+2V;PAJGV4P>;;i&*pUrT|s>V`1A0 z&@=)Fckvq0J8ac}rha}PEREA-|Gqe4{vn3B%)!LW{sWA1QV?y7Pf!|&7O>|1l@*NA zxu@Z|mxIiFufq~9)EV%cy1mw*bkX{jVDZ0gC&Drv%DXxN&nq#XE@u7FLAF7E%|`3qc*9N5|F*~rufq~+{q1g7o+C8hML zI>hEJB%APvp?*GmlRvV@klwOGnG?P$%x+53Msm9fkzKyMt(AWBX$^(~3{x7`7*qS< zQ*W^jLJF82f6yo2Pfl`!a&%tJr!^H#**Mvt_tf~235Y8}NW`f1*-<=y#XQ0Ko#JR5 z)d>SV9Tbcwah3bG-Cxf5ibze8Puf@^X!PV9M4{kO=PV{iY$QWRoPMvG+rAAI)@A~V%GJ@PeVhGB!y zUHL4G_WfMmY;?}*X6DtOf|rH}$u*kan~qAwVVcmEc`Fz#RDBl7`?YDw0Ws$58Un6c zu4k+GcYPafRSgV(VfTq1Ay+ZF1R-bM|JrvSLO*mR0bB}fTf6lx#STJ{`FX?_oKMu` z^r14YMC-=(Uv7QkkDsLyRL!+*`B>=zo5sJY!YBsoKvts%ZYh6q(}Z<=BRY&m*kNa@ z{`d!16T#6V)a!F!8G;3{&f@Kt(JuGHr+>-rmtiz04h_;tG>JOXj8s+t!);e8ml#*C z^Z5ENd4lXvu}bj)gCN9y3ny8Oo#ML^DISrBf!mcL4%QAe_6C2@hiIJL%>8rS&wuz3BLY?sA(5}SNuFMF)5yN7tngf#33)h4s< z7>X1^iMp`Bd_q#rgTHM|MSj+1>$%fugA-zIA3q32((==q@8`X4K&!?mkXUcNVozcM zg?#GYaMMad{vj=oU_g3iw87Lctid)eGTgjC?rVZ*s=hqGU{_Wdh7%zM+Bo{h%8j=P zHkB*4P{kqA)IP&DcTkM1j$M%yIPrHy+%Zi{5NlDUqPIWemO*|YQZK_Ko6tJLd9729 z;oQz2qD6WSm__#y!nO3!E)&9T2M;y@0I?1~r7d*lHBNa!;(33Qo!)TifF zE^x+>NJJ2Bra(k#pb>OnLdi4)16hRhptT-Sl2aFQtFkH%9tj+_>;YVEQK+L_W>cpi zUy$$uQ`UNz@GD5#7c={miQfQF60Eh82;1VSC<~)P(WSB zu`KnvDeCv~elD8B`3X;CgO63&?UAsvn=7(xl4;cN2dnDaIvogKKen-!Gb?1P4%Dnx zS`s!6&*?4$KEL8>WTAcqFvy<2fEx0mEmQ zhJMVMFpwoBfWxf!8At-OU7hA{90`TTocF9JOWT?5s3#%j$4U$+|{ z;tsW($2)G4UQUD+(0hl=s?y^2s7w%D75P~UhY_t(+2Ft;Ys!^7rzmG%=uM$N_C^>R z4CM$Q9h>K7UG!)s{SCNkh!9fXk5=_AFRb2yFh`!ZUH`kK*Pwo z-1{o0x}FCGcLh$W#>O{)6gcvpG>ysYzt-fclNE&(#I6pA!nNOunp;?=;&dt(a`Ar> z>RDa;jIcDSGsnSe{aqt+xb6H#87nZX3m$(Dqwi;$zhWn4fXI7F(!8vXzUn2!=)ufc zGiHRksCm9j7MlgfmU)SnBX@ER&0XX?&`f^9j&&W%AFq{VU*sw>Jl@@&RyTg&bLc zw!hS771;4C?s4ipn{=@d>qy~Ha`f-8L5OE}=lT6ev~m>IGK14G)aD|0f}%Ul@}OTM z4n2IGV|Y88YfH|x!i4QS17dBVGs{PODHa3$P|U^&-SY5H+mKnsk^~-7t`w?Z<#Y&( zy_=opxMdrRbptXR54L@KSud>6(qm;BTz!g)K<=~=6{E?y zCG7>cXg2#@L{*Ecat||DN4~0W264?ys$hK>ScfG__v~j!-o@a&BYlWSWrt21=VzbL z1e;soDj=k}>xq1p0^oGJ6!mLsxC)YT5ac9sPgtUkf*WylF)SM>H=Ip!OQBocUx#ei zGyNfzO#a9Hg_eSkgnxcoeb(jQZ~W*x1stde{RPQwJD>G;Z-8o`d?cJ+uu$wxSsA~; ze%kSlBk4cysvQZ8I68`~VS%uw)j9qNsR+^aK8>FMpl>!s%r)=c$&5n7<0AQ`w&)LpIIRzYpcsr`#SMT$@t=*&UshP7ZVn z8MG3F)FM*Q;aD|iO9nkdy9+IaNuR-b)RA957k`1jR+`J7NtGzTw}!9`j6)~I{Jm=f zo8pAr78{NydWvZR>V6Kd?X;vad`wX|LM zA0#;>xI7^>_R4JR3a3m6V!{cmcdE|HzXL6?uN@&vU=YuA%n845y~|0*fp7 zyeS{KctN{c)JNz7RShuM@1W02%g2$yJ)`bJ1Mo%MnGOE-^Gj&xoSye?HTCi87-eeE72ih3Jv zRs`B-egAC;T?WL~_^XS1eHFTl2(1V;<(vgTni-i|lN3Hgo?3f9PrZ)Ug}oIOKiA3KM2xeBy+Fjp;?u|7G3gN?nyYi$38-2OK^W&gb-GP+Z-(ZZB^l!hv#;o z90Q4TCP=(`U24>;ExQ#EB7in5Q<8kmD-R5Aio&fn7F51&cH^}K_{M@qpNbgNB#EU} zsMkD_0e}>^W>QOLAyI-GD5>no!%b=)!od6zs^Xw@EM5;jf$FYb@;#iK=+FYle#5wR znT6yFn$Gyl^3d5xeUp^pZFkquvd`5UJ|r(xPLa_>65~An$?WX-)^k>?9`p6ceR=nN zc{*vU)pJ(wjciRw_ z-ha>T-c9=N%{i3Y#=)7wO(4If0%A%wrF1NK4wQNWt;(xOw>y7%R6&u8F&QRWBR~5( zEBw()*Zj`!{PjR=Fex{lUuK!j-3pdPE@r{^y;<94^_&e?q_fgd*3(?~U5x*hZX1&e`5^p2ayK{1W%Te)Y%<>GHIH7^4Lj`X zwc2i#ii9BiPH{UP_vT8Offx&#yDa@wXmaO5Kmn;PBnr$9t3JTn=vh}lno!w#jhst( zPx1S>GM`M^?C#YHN!kKqFR6VCWzSUHGK_Z9%ssWxoeX#m(tN-EkRQ>nn}Dq8Lcyl~x|=*&v2<$uJ=v>G;!f0F?I47UIxulC*5N-Qob*U!zw%=M?V?s#e~ zi(r)mjST9tD#-aQy%OJM?^9o5Be+P8(&*z1HyhKQtnWh{^LC>(TB0zK5aYs`Ph4KC zJ5ZIj;_kDvh@$~+l}cma&p0U4d=~@scuF)Ot_7=Lk720u;5xm-w)@d03|q8FKn!Mo z&=@fN0A4-JpoR|8Lub?nGUT%(X}B7W4=nG$>7#M`PRzx_>C4(fyecX-@PVsK zbdIdH!tPuNMwVxwr7(Q9xK==kaC_aBd7#m?yhu@q*uiEUA{NJiLsNAqIULNxG1SB* zh29^tm>2qu7}m*Si3*F*-tIU|9Ww+#k9~j2<}T_@%A}l#7B(H&Y;s7q4DnQ7H!pE= zaz2xz6njPB_SSl8C>xDjgJo+rd*`0O#=mshe4ITRf}kh6&*0$y>x)P>R18_-E3?oG zKXod^ec4y#*(HAmabw!6^Fbux^0B^;C+B27P0c-is#`{&$hz zQ%7JR68z9gApJwsPDgPPrmrq+a8OlC=`633ZI*-Gm7sLH1bM78XnhWNnOSXg(UH#d z+&f(Ah3P!l=`_iR<1We>sL^ZS>%3(4#h0eJMf0`NmM-c#^yA}WM%p}d!>qb0qvXxZU<~eD@Lf7vCFodt4H4|Zg3=F{N{V#AER4oVP|)&>(0$#XC9lr-&bR4=pP9vSnWR`EhvHTR&H>UpK&#hX1J#FW&+#p^ z5$<p<$4Ki0#`!ANWKCS75%`o_v0h86kiTt`DAT_e5s6POM7H}D)}7z1XT;v1U2_k) zAUo`0`T7}g89}M~3cHpqu<`=TbDo-X!`RFLWTMN!7I5u*G;N7ulaZ@iuv1h+DI{l7 zdA%L(9Na>)e@-W1FC@e){PLe3E&@;J8yw0w5Y#RYBkkaywQTZl8A&i-zuc7W%5TH1 zyQS;8qrvceL`<}F{oB*X1s1$ zVWsJtEchLoS8D;h@oHI%_W4Msbg>;%w88MH=g{)_a*9qE?EF!$AzN$6^0__D%utW0 zKWtBKg-4c8wkj`$zOCcU<3GHcA%hFK{hoFC`LpcCdphyKI)v6>WGvckVnJ>ZQ0(06 z$$M5(Xs1#wfhi}#TB3P$$D3%LG|ttPiaa2o{DXYdb(sF%OY}){feq04X)>({dmz9b z=6q^2@9M_pjtBLgu)29Za!aC&_$~fRrdlei;E~UUu%Y9AKBu2mY%RxVX|W?#_HX3W z?^_MR?iCqv(Ft`C^9j*l{o@6geR!vOoHetK*^fY~JlcPDC#3=zuE(d$tl z=Ua%-Wx4`xxqutft1#seO{LRVOH4hX*Udrb<>qYi_5hUGnOF49Hg?y-Yj%E=aqxM#D)xz@n(6oEJs*Q=f>)#>gS zVXDNBVHp95*bPXPU+65#dgM>G>6$Q2HJ9(TX{a=WGotKADm?fJ#PiYks8{6}VlG{8 zvv@JDhOC+L!h(#rB~erIIh!Zg6iZjIq29d*?DAV**t?IkS9>c*|_$0Zqyr0bT#!hjwIHb-LGcwdLX*Ah2< z_4YFUG@6d_eaTm!M9(N8G1Q{-D?`oL3$%;rX{2O~z5Q)oCkcx90iL6kIEBuzICnzv z$QvA8USyNCT2>W%t9v2lsbQYhSBVVV)&fx{haND(s5ev{xt(op)!$jc?H<)n0V--QDf5Z($ zqs#trzcD7H7ip&mFNQk0hd*Hw#=`xvp<;)dV$)L}?uAy{A}*6Z84~%r`yN$;xopO7 z4qiU6*W;KzMz^%^sE zwWWcuG$OJ{_5224K36mk_3TQ7TH=_YzP8v`P*JRBoVBK99l4f4qUTE@*gzS!TtO(Y zRKILUR&bJOA$Py%(BuM=AvpsMR%x>b4>-}EK9C6{!`oOhq*LGzzBERszz=-KNOSG{iA9>9L{cmMG8Fz`hQ-s@+b}YLw6~1-~A&JR)-L|B%om3!x z+G>)$0OUy~85;Zc^VMCQwMHHPF)WLbo+?taB_%WdJ|Cto>s6%NC@gE{cdb(VzpT6)V8gy8s+2q5fia@MI`I{|a7R#akd~ z_rk`;lMj~O_*elzE>b(E_Dp87&let&FzWEiecCJ^B5k4W6J2_?*JID$Xy<6lDSunh zUX7(xV+@VC2fA4)eiY1j3uSEn2Sk4jTE!^eW&@9|L&%OwGgvAcvJef?{T|$QBxPje zNl0^Zn)ZBon5Ahuybn8g^AMW z8{4*R+qP}nwr!l)$%$<{C$??dPUfq*o2$9#>R-^itDbuHT3a zhLF@C1X8HsnjUb8c}a9Q`n6#q@7zYsq%yv!S}2OJDA**}3_?<@Ti9q&bN}rM?E)L7 zBJFLslwyM|fD%|BB$}UbF8X4h@|COqYZ$jo3C+_0KAG$)WpwSn`O~p~)vJ|>lPjb@ z16Re%3{r=4IY4UXLW|mV#dhi~bQt-NTCL$&oR|-UshXV{QIb_`{K5mgAY{AgD3ySI z%($JJLJcSyA{wyfp221?KY^bw*F?j`gRxx%+`tQUdh$`N?$=`A8(({#Rh4C*0|nh_ zLDoX^VsDZ(zH#!?{lrwg0~fft zlH*(gnbA?X6OSKGwq3&OU^5l~5v&Q95I;E&y<&-TKKu*k$lAk$f+jB}`8yZ4SM1=3 zx_0Ef^(TfIkr=9}-x#Oe`z0%=@+DoA=)kipI;<3$DSP92E!$uY&p~DC8~NEuSbuvoYwWoU;10+lA$*)fnR*t7t1LBRKK%<0aR8&aWg!H!{DkVannz5O1Q1>Ft5B1}*% zc0bze%o+#zwQ&@&H4Pp34yBOfd@&e&L^%Wp&ea%H!22vd6{Y9mir-u%>p6+vvhZV# zzhhMyLyHUtequOx`8Sd3I#D%}yYSTJwgt45t)GTIZo_a%sA;ejh+$Vd+TNc73+`V~ zh1r^%q-cwz@^qw~*3C@?_VHCOrq{Tc#1$o)HO5~N#XT_MJBOGaMwV*y0{pKV#lFsR zVRpzWyotFI!0JQi-j=a3 z&vYqg6=g7#l)e4;1Se#7?-Tao-Qf$6&Vx?z>7yA}t*SD*>=k2zLMjam)z}dhipvOe z1Eb}02DZlx%US!kKQLc(5NYi9JgB#iR))j%Y$ox?!Qx0nMa+%J#Eqkkk1savD*L2; z;k`j#B%WjT%(z+p-+fDX@Q`|D&h&=cT)7#YKnl?QPr6(c{i5*#s;dGqS~_=vAKtQMSen3^W^QNW`{PvYP1c72@MuFp5C&hQ(}zFhZ0b%)83)TNi+{9a6I z6ncJdA@HH9k5+7Bcd_d+G}zi{Gq$YAVT*|DXgu-CFDk9XIYej;L~Xo1{)Jfe*hu-b z7u>-kby~a$6xzeX(r(hnDtnkG0F(NxMFIsH@M!0XY6h7kPy*go3d>7M?q(^l8a7Pp z&`+y>DLN*E0H+2gF?-cPOb`mg*SlXrIzg%O2t;8+2SwadtB51`4olu!o{;@}Hjbnv zPlFZ>XS-@Kwj|2VO)O~>F<4HRnXbM4MlTG>875G0COkYDy?YcDO22SpVFz$vNYF0o8Cw4JZzmt<5F$mxT5O_(}0{5E`Cz zQEu)kb;@5a9y}PA{StTE6o*Rd!iU&MysidBh@eh4ftB{-{ieb^GDtSs=5H}6mS%=R zjZ7K%%h^z!`g6k=`KO{Ak;LMq;*Su#-ByQ5$jtQFNte{~nZn><{juhAS<{B+!AsI#bphADjx=VQYgkJ%JP_Wm;Q z#D?*RQNh&{ftIM7sJzBBPwC_qnLKtqw_S^`>Gu}|-n)H}AGs(DssU)t#vuo$ToP`P zP_x z*y#RW?@T%D}y(t4PbU5_88svjnpLBEY+xV=UAzYqNYkXw|Z zn*QA}k}mDAq7sHIV1=!EBl`Z6c5NYhqOB{CuDl^6eB{}pZVnhFmy$zD!ImoyNBd5FocFmD69 zfofW^67cWY2+!&aX(09}Ak&?ny^J6q7c>bhFmr;(4`4N|E7Gnb^U7+A-33?#jP7G? zxKdRU=a{k#|B=#PzB8g++BfTy#4?tskykiI& zdXWI@uSukolVk}dFwyo+p_Eq0zKAyZr5W>|cDFKQR+=v2WT~=VNiUi;3`;mkv6dF< z6uNJ!ETTE_Zqa7t$@lkGtE*kb0BNGg*T+P&Q`Yx`hMdl)Us*ga%lCT$tvxJ&^hz8$%Y!Qv<)H`7hm*RV>x2BwG6p{*2)~Lfa29hLnze0njwGycAWYWp2>! z974$xL^M_E#@BHv(TIH)U&r3PS*AC*NX80g2t{5=2ZTI9j=OWEt;zM&SuDgzSjR(f z#uW_B{C&C9N_}p0Vt4bk&NRL~;& zOK7IG&``tB{UbX)m7B)eF>ayWLsnQeO^5=Qf(ULYAl3FH!NgU0MsKmc< zOlMqC8t@)I0Tg8D-t3&&+o=bUw3Eay81)yuC(FEOW~HjEuIKRi;{5W(%G>6h%qN`N zXXfjHFP+&>hDIbnF(Kp;qKpeyw6mdkpMHQ`TN1S`hef;! zXLz>VdgKk&hu1>G&RaX;>OrP`DD2^G*#u@oIy0}8ql8rPhf25td_BO@n$!RKu|+*$gN?eH!6lGWRKc49vKuo_1?Tx zH$J00W?K+(J%^O-68d0IiN1fksk0jtf2A6r{5GN0ip%eB8l$FnGjImHi@dhJAgO9> zkHUrGGjfVByS&pA%Ze;T{SPf%6=aLv`B?mk5P62#KJmOtw|D0PTw!Nxv!`%uG{$50 zzi9li-hH)xEJae7(N?jDSsp4?*~+~LBfB}apewJisuBjRc@Q0Yy00p|s-@#FURn}Q z4{pPeP8ytsLqR|L_x+nGY2E!zeE&tw8RX84Pk?WY=V413`S6Jt6gdGFN&Y*Wo&C8i zMA3QM&ea(ca8jXLW`&YRYEmG<5MfxL!6V3R<#+sU$H#vo2xWFfCz+ml$M&e^5PC{| zllF}mVW&|O-nU1`=o9veyr9oMXtp@3_;!=+xlStC*dOY=k4e36UPPDoYHb?yDelNE zK3<&f4`Ysf(%Q8SlJ0HNWT;w6G}iB{rBG#W;e{03aDp3gpezyxoz*()5Mv&ViU|D2 z{e{YS2ZUgGN+mcvn2OUn48s@U>P|aw$ zR<5TIGVfO9fmn=XvW!1mZW(tlGx+~x*R#XWUE9@q?hR0h8w^SK*67+E7z)!Y z+F6crecT_u?-hNe*~;Memvj*ARZ>^w{gjKgbLA+iKVoQ1BMW;w_2=W@h)OZh{+pjU zJRv;%=R->;JDJbXlOJ^7_&ZX`WtI>{^ z@`EZLO80!^jH2w}t#q=sk?DxNa7)R#iH-sTee1gBuZW+}Z^=R?N7~DZmQ9_noi8(+ zzCXu3%swS;iRpZoEvHap?rkp^Ke12PG4oNSg~$FAMH~NUb1ln}l+iY)%H|`jjAzS) z^2s%QV~-?0x@I^8RqhtmN%Nqt=guQE6g>1HMDN*pSy^MN(Rkbd;+%P6B)N!9p^!aT z9oe9EP7bt_*%tk)0b$^T(VV8g;wN|ulj+~6PB*3-yr1$f-i#}{3^>jNC$wu55C)T9m$1yzL_ zL{}ALCDv*nmJ4QI)n*Z*U4xiOCdfFmch_amu!{PbjZrDZ1cD1*$%yzDK+unyG(2#t z$$ZW1#jRrP8j{7vrj1xgnaFDz;b_&K6f86R#`J9ae+Svc^Du>0=x7guwcPb0 ztctVaCSQIL-KDq|Yp1hk+$5#wouM z6?QaKO&RGef-xb#EpnWmrp}k7-85SAv1NBU2IxyXLBYN&2x9|5-UaKH| zBC!=T=%!WvS_d({Ce(!jSl-m1R%p9%9i8~Bs_B8G5MjHb+|5p>9&;3{((Jk>)z{RU{m?_ z2YRqGZ)rIIS&OHlu#Z2NmbP>`Pz(CWO_L4c%OC7v-`c=r zgxCyaeJE3G^|_7(r%~GcG$FB^=g=>9YGCgaz zE2M@aD4~Jl^SqF=mWS{4rrhk*`TCN%Lqi{R9UEqIz2Zy{6ulc7@omUU)AER`)&|Yz zSoT{eMXL^zhQM;ni(}&=F%35A;JYu-O|B7z$r_1&{XptXaU)I|7*_W_mSx7jEd^iZ zpfKjhkecFQh7wSMpC$vrODMe7-vi5*ez1BGGkat08uv8QHa5>gJL~M|U1WhP#X3+_ z)O!?zvDInb{Z*cHq50^pY5CZMXQS zV_2zzriLSXP)1A_T3=g~g*a5q;vC<4A$6^C~OX6Vt!M5Xh`! z5{RF-W}41PuMAh`4#jzP-$$FeY|-L3>yLZ(WOYY&1Ec85FR@mb4Qz^4RZIa|lSw~B zYUd!AJ4oRULX>hlVlIKRF>Xg2!(IuST)H$d)GVXgqWN*?-qp|QD|kmO`#|*h%=@hD zff1%6;tp32BR z`pQGRp`3ruBTo=F40XX6L}46aqTj49E9;}j7a7#SeW8YtMvF?cV@9GkUL5jRr_j1N za_!-mz3sD`>vLD@mw|^e&)3GKZ#1VDMW<~19;)X3dAYb79&zW+_u_GUUwf)4^d2ypzl8u9b63(%pbYE@;_NWu>)f+(eA~gSU!B*wo zel=k2AI_a%V5dcg{LG(>;Z7&#Xo3f(WyNan319upZT7Yikx$&UihnFR!XPoi+!(x% zu>rXP*m%mo1H%rF@IMk4zx@EI)4B`}XZVVI24v!KwhdK-r9JdSTe4&~;?oZjc?$z+F_4-N3mPETfEM_0d^)_I9+MGMjow*fs2o`C z&mtE;z_hW31%L)H(phMLY-}avyYd@>c7Iv5j$;^EPGo>M{Cw)CF@68^%_pMp0ANWt zKrz)P<_$4Fp+mpIMetD=XVaUcI)N+%*>IuGdM zfk=TInug@Bv|urpQ73-YfpoIP$s(w^B)anl<>LFke;NLM-5q(tcws^pbAC72N|k4T$+<<_4mEsj3L~I06lpJAWz;z6w0gH;mfDqR@J_{ z-hU$2Ex+jVsfmrBF!TQIi%qnppZTTmD|B|t5kaf7P*-kV1ZolAZ4eybwiB+q$QdQI0vbRWlVlm-@r#1nAKZSu&eur{N3- zzp>}#4h5*M&g`rA<|e_ckKeDqpHF^2rnt9|Rs{Z9*P0@z@a$}-(;Bb2VsJBECR_@? zG*+Y;;OO}}^km)65&`KTr$GJkLAd53lY9_vLPiUvW};^Y@2)R|X%0BZSjMwd4URnM z>1|7Sp4Pl)YX5#c&c<|TIlN5<_N;pss1_?tF1cvq}APr7?rUL64%vGf~V_8b{Q#oye5C9N-XD9LrpThJ6?A+Tf zbR%S`kbv z*-ONy+qBp_LfOWhu(^js!y{(#%_bHQaB5PfnhgHFKlURYUq)2iphC1~8@FRwrsYHXl2w*|NGq%p2dK03p^z#gz|so#i@Mvr zH_3}$(xWvqKmH-vd7ytZybIpwIcm3bp?`yXVO`3_^>I69%?2q@naaOn@dca9x0G7& z7#zaK`U_&dzS97)!#dw-bm1;&UuO(RRNlsHtF*@&88{!f{{g~opIUdXfdzg&)9=Td zJwe?K$MUanwaKsRddxhsM9gisKmme~LsIYALx&q&Wue(M`HA;-*Q2wEW{3KZ42I)v z#6x>iKhf;3!X>Iqs4;hV8Ytq-RioT-%D|*HzwB*Y!U-ym)W9@^;kECv`~e1~d{1u= zKw71pMO0Pw-q@t;6g8>Se`VL-M)0e2`D+M=@LxbYQ$aa-Omf_7XI+=q9bZK8Ln`h5 znT-#;R^Hn4yre)+oOqFgyp4_#XG# zMw(ShgP;ynOaN__wE{U_ zf1BzeaH`Oxgy13nK|D?LriDwT<>^cihPw=3?OW4@sSrji$CGuv+AS zeuo(hw;7IvAPCDinWx}tVr)_x`ZdGm^Q2uJD(fIX$5#jrTx1+h7>6iA)D!_PHrDh8 z?j&||!yVT!qVfBq2RW)B0+%1T!F-t5B)=s?Vi2(5mYE8ULKnGjVluq6V@)OsVY7mv z>vRXaV7k_B=gFxkSXi-CD($$$o42pKxf^H~+=6XP%ycsXm-Z1Y zv@KndJ^k|ENt2XKPf{XAWq8U1#|9e7aSXvsAG`5%%wkI4 zFQc!Mv-ge0aJC=iD|o)uP0+{DtHQgE8Rv~NWIuBx4EQ8jnBfJs1v2U9(3z0e^+Ibka5yaAUFer#4H|~v7?_nks z6V*fM1gaJzFcM1)iCBoK_yJH3Xx0PSZ{-!6Q&8XN+Y50!H;|509?*MA2H|^d(^qc^ zSS6V`-YhgG$XjJb*clv9e;~#bnF)f~S-xayaCqaj9&&=2l?OoY)pOp-JYjA}iO5)< zP^~*mQPgsJh_SZaZ08_7D-?19+qmf_(msW#$7pzS?3okvx6BnpDQ4tn1BNVL0Jos} ztUngL*d!6m)8k>s%&dzB-7=EpNpJ}QOYtHh9R&hNSEX-*hr8m}sYTFKfW$h;6z@t? zC02TjHDsEBsPD9GoHTdhoteE}6>gWLp~FxO(l$vpCjTO6g8i!eY|PS5U+t9i08m!H z4`uGiLGAF;;ZEssaKvAIwsZHTbLD99G6UB<_`H4rBrJ$oxh#HyIk;6%K?(cw*2{l? z0t6|>adWq$Vp>C@jGHUC0wW^zIg8|mm?Q?%S|O?Uz?UcKHRRbB~X&y zOVadvV`~uSn?ODVm8gY&H|@vn8dnJ|$2x9-v)!8|YHL4rPNJPEQj?3*Zo76m_^Z?} z#R40T+7QNRgEQL*xfOkYY+8Q62W&;C;gF)VAI6Sm$k?uWuV0E*T8maitHx87x93bH zNwqbDR^gs)a=Mr`U-qtviPcNpq&ZjRkXw`2S!i(96GGD4t_bP;%)%?qSXiU&NH$@c ztA(-8VK|W(P9Ck=R(C6-a?Ytrle#6d_0{nPI5n$XxYin{wNbARA64e=;S**fC=7gE z>I3+Bebcs)eX6fYN#SjziQrr$9BET#!8LVj`C@Qs7U49U{(XYMnrnsVNHia&_Gn%t zw%m+v;<#3t*b+S2Fig={|3uW*r06IX_d`MZb(*WNMDxiy1Sce&)w>IEVdE|NN(Rhf zhp)wA?R1@o z@im`bY|0v8k?U(Dvvwm<4mwHfa&|QEj5E5pJ6f;24e;#*t5kYgk=7>XungH>Nupa2 zRZ&D4|M%-6j~^SsQhuh@mKQKI$hm!kO>yQTAo70A+{F3&v+T8xJbsjNSC%byMm61< zDPPRtQE8PaQ$1Y)?w5I-5>EOLf*XanM@&_4rq$lRVhXk25StQ|w>aAeunmNp;xcUD zrDoC8S`fIj=1$$N%Q~PTP6)t8Ym8ZsRdi*fUCF`C_;n$3VTVu_VL#hJeH+M6&wQdZ8Vc0EFS97LDopWfip2Y z$wiUrD3BvixlcMZll_%?kzuys4l0#@L&{QqVbUr zPE5#@kJ&Yue*POt7(*JYGzfezW&*YDbW(rnz-Zq+It2{EQ}GFwNj^yjVuU4gdFmwh z^~Hy;DgYRgWWvG(9;e^9`XV!vpH)yl>!4r4UnK98*theooUhocI7C1nPP5AOdO1Zu z`1V#zKU91z-2N=L=PA@bHPm+E;Mx&l7{b136nBj8vkXF96(@RC*QRES0bVR;ci|mQ zOzAQV^S!sFqibYeSL&VFlVpp~z6c~y!{B|ca^S@`J}^Uk3ceoo#;@eRYEJn_eU;`|htEg%fmtrfwqKF5Ql3gAKnwka)_|@joC_ zg!Vkef$!zIT7#^fJn_1`m*4N_$R1rdFS!+y&~R)cF~@Wk1B!&Kyls5?ja}V2j6>?; z-FUjY_pwWz6=E6)A>OJDwmUKyVJ(t6x4lcwhj3&v@dEV*1o1d%!bS}#vI?TwK zLK>??qlnxH9crS7%!+x%o%ui9Zh1~~C-JM76l8u-V-R=~^DkG#p_0%uXTNw&F!)3IOQhRWZ+k?1bkcHF^8d96$; zOxP%%%P1eWA%BK6B|a(2oMgH($X>UhR>^XI1x2f}?{jU-G-CUdc4>oUHm zsPY=4f4Ro=+>gH@wZUo=TE;k)KZ4DpePEZBtoE@Cbz9K&+!o@X3z<6$(-Puc_WHoJEcsZfr`yzB znal}hJIaC+({IN14N0qUE6#5ORR$H6T?HGoehxCH#Rh?PhtSNzIi5K6IG*h?@~pC# zaY8}1HHUE6(!$;3wRe$YYv?po->64nNt>A{(ibiRY#{Kjj1#RHZ?6x00A@4V6CM^Z zE>S|F*&c-UJbt5H5f&5<@hd>&0Iq|W7rlS$O__}iY zl7bRlfP2+fIoMvBN*vT5y_`T2p;%Qs2yCVjWaq$qo3+BT2)ksB4;-&Az;5*Et12}> zm3o9EtY?+gn-Q-eDFI1gR=*B1#jp$$Xc;`~`&-p1`;V62#EhOnXz&MrWFuzJaN1)L z$-ROwIWQidCxr2XC~%mG+bLl(C=gTW%(F(D%22_j#o}Wfyrp zpglc&r~0&RZzsK>4Lwu&&_Z+`(5q(DT0TYv_}`uRZ!r(rHBQ2deI7^oze;|-i%6$3 zA|nh4?YpRg1`e1735T+s*?QSQXzHX15xn0sYpf;f^$n(_7Y&XD0J*aj$X@2vq@xJn z%VX<-qto!;&bGZ*SzN9MZqBHI!SqT&b}bc1&C)uj*M0<8h&brlj$UU7jY_iWfw&Ib z`-D)Ia4PDabU)qZEMth!?-GI482c5zPH3BCMo_k zGO-_Zqc#^yXS_gjsg|;8f35xzSXS({m=W{$=R|#DY9D;ZL%Djsg5m7eo>_&#sTc9L z*(;$7z}}TZE@m#XUdw>XECroAZ4VcEE8_h*4`hPPLLp=qmvl+34wdbQ`9&YdhFeIct+~2}ijH!=UdXC>eNx&^Bk@VkE z-xFl7Q5KR4MZ*!$X;e3Rxus)bmKYm3dJv^~fj>iS6^>_j>ciMM301mOI)u_7$axM~ z>}+q4gIpU$|7XtA>6g@J@QDP8k6-+c2JTr!(|)E!*%~~T~oAvpi_qaD!<(fIrrFL^oy^!)y9j450<$* zQG>T@w}&k`!rsjC=5k~6fC^0{q806g>O7o5mhLNUaga{ownF-luPIFY)}KQHu{mgQ z=JNRh3VHefLnueP|H&XD9c!h z^}k8)AVZJ89RW0`Kxt!{Qw|{qKAISwAwtrjn7Bo6YIxm$7p<&G)tsu$!Fj(!7$U76 zXIlvG-NH*QNHHQS^>+)Zp??h3!5?lltjvNrV9<#Y+Ah$CJV(el3HKorAT6;=kmnmG zu+GB<8d=5C(Yt0hm%Lph(*5e%E9z>Mgik0Z#9DH8QlNRENcG%R!W(UbU4}icIy!0{ zB=Z{g5}d8fm_&KP1gc1Z2oO~L@XxB_r~98WiwFZe z{CpA1t5KTd+4Reii58c!&Pausa5s+cKnDm!p!R-qJy;~Uxb9sGF_fpymP#9JN13ih&;*v2#iD4|%&Im=lxob(=GM*G&6~7L{aHfy~$VOTrq9n@-#ID2@ z!cg%Wa-M^S2`xy)g&;mOFz7n}7x`pf>b*)#dk8$8`m}odQVf=h^@)Gh52$#w-`WCt zyvo)Zz#!yOYyOVb6kT}JZ%4$_+d-{r3t=7`0a1+j@3kf;SORLUO zc@Dwys9!q?szr(Or3_$YR3iZk-YMT!GRsu2{}JqL)GsU+>ZX3}fQ2nXTsvY1%>o{R zqnE0ArVWqgX2_JRw1dPtc8v0_*ZFvsNX#735lg|yw|w(tD*o6oT<2f zY%c`fA(uati2hkG^`KKizHwlDKAp`MCq#euoX+;GZT_}$`Z(1GydCfN3POU(cN0Y; zYPJVUXs1qQg(X>7i}$eiCC~s92_|^@@;aj5O1Nu6+Vy)uU*q+!&3t46Ipmv#ZPBqV zZm1cGc9QK+St`&yb2k^f!tBS9`tl(bhIXfkkW7m>N=%HKLT!?cyyo%o@Nx3^7#@jL zN6eNL{daas0qF)=?nW}2>Zapp%bl1zCLv_Jkpl8z{aS+X1pwQLiHZ5?mr+5G@rXLn z1RVqzpwGB_Tggf3(|s0y{4qbvJ^49OX*!sQ zglt$K%*J&e*@`@Y!)YA_K?~GyksN#&kiqU;2bTT#g}P}jr)>PVu#?r&Elnh7W?Z%9 z_(c7L5qsKz588D@%pp_RD<-JIm|(v;>$TZKyB$2o2ZR6xD02m2Q ztI#2ae6+XNdQ8Ob2O|+L3BBbYt2m%@OU~fO*S+!g`tIuM$?#T^fu?;q_-e$@{V+}M zyg5Bl!_yG;NQ#3e4$<(XX|(D&_&Chl=`;WPVE4Q4ZH}p4)OqmBej0mF;f^pFHWN7g z1-f26{H9vO##>3{Z{}aa`LbHevp3KSbwZ0B)8LBq$^i?<7L{w{h@q>i8XjFBz~B=RjQ ze8Us@RpZiOh<-lk^V2?V>^fl`K+v0TkrTo>iS(=yID}<8StFa_l(8wfwjHpq(N47$ z2mYcB0Z6z6e4=a#6ANFvjOr2sv(O8)iZ#i{@AEY&fV^Qd2A7Hts~`V8l)A(*j@EWj znTOyNz~REXWiKRYY?cl7aw;ZiUfeWbgm(uo#eNi7oy-kppk2K5YMN*lF+X8wYtjSW z>(w&9Jqc_?$ck0cb9ybZq>zTCg}xPs{4$)w0K&9lQ$eF->c?fH2tw zYiZK;n`9bjg|RS$wOtfXK6Qg5zKJC&7IwZ9h;dv33`ON+^TDB9hXJ1zz0hh5DU1D3 zGAov(^1@3^8yG}%FQVP;3Hmc4-Ue@pxG!I_;FX3_-}M9&XkM0ohg?NI>^J}x`6I=1 zxRE4;4)hh!xF&9dlW>HHufJ!sue22DDb_Xv&Yudx-0}ykR3HVb%x2LW{y66;#z-5o z@E-g@f^#M9{h;yQ_ECy6WFF9ux+>)mUh^8bvsqTano$#Br7^XPw?#JzR<|8q7=c50 zsjpB;`1KFx*0#YtxIw2O`#}zDR&0Cbp(v%c@+J_`r{6z?z=c&GSE{p2E_PA=#CdY@ z5{^`nKuG$kbN5k7<(nCddf<%Vro-12-1^isC3 zowJU8MQ0d)JzCuZy&!5|rqk}9H+Pvpd2b8g54k#tZnm)*9rkc`5SeRod5scTTRu&) zutzjb1KLn=XO6rG_5B$FjD(D#iT_}2jZ5W2w8kU;C%8H)v8<^VMC`w0h)WOMY1)wu zR5@;2^4XFMX+~WzA`LDZ>p~7G7^930Cy*oJ-&*F%eWCBY9fSR(nd=y{eIYBJxgvVr|8W+>0SyC zwNgel+IBLN;g_(K$RFQ!9~;gbWM&|`f8Lv?RXVJ{I65O3u^l+Yp%deK&Vz?h>mm zEe@mzIoU5|1*0H&yYoPS{|P1Yue<$Vw9@fnT<~?igN*T>jztsqQMxD1BIG~e!^hwV zyc$J;pAgu(sD4+j(qr~3yS2FoK?Ir4Mo%a&xmj?ii$um@!auemN6&HZigAhFzk5LM zBG8|A+#j}`2ihP0$IMG==_0uHtAD#Zb68eaOrFu&ja9PV#d&Wl&@pu!|I>#>3s7lm zr@gcinHPHvJC&{z*w0cp@r|DMqNsy=%z8$XIk&-IU_h2a)HM?fT<~{$$Tawpv{K*o zqF|TU%SYCezv>l9l}dssy>wN1;E6C6Y482Z%V9RbG2$;ZdF?ahTo ze35rLmje+0vq`0_VC=HM4+RW#Ig8or3ClZO0_3N%nyFFfVl6``X8Z`*YfSBKrYb4%sSngYbTt6LnR|?I|K9;n}k7rl%xlDf857XIxf#eJs_4%fUYVl6EshR=2 z&Wia_)(`V`CA}v{++;TlLI?sOeAq-kUEc#tE`l!PojlUoJs}0hbH=P(5#mFr-Vp+h z$(W%V)br>WJ8bf-XH> zDf1Wz(}mB4TO0X68yVB)mX!f(Z$O(iJ0Gn>Psu;@ zTgrPqr}&?pD|uxwM#UA@$W`;H`EG@X?II=?X@H_LIC$J%i&+eZUSt4`Py#^_M7VQ4 zYZWpbR$xeqS;3@|rJ<})>vl{STA$b$9Pi!l(Xu2AuM*B4u|CllHWb&LnI-jSeZ)9{ zuy1@K5v4apI3hE!55Ing3rP3?1U}o#Oh|NqQhzDWy_W0%w#N-oKG%{wKk_p&g|{Mh zuazhH&k2TS#Qx-r;o=G|=E-|EVT6PVn1?s4b7jrm4DEt0Na=P3BGX}Ur$TS zHPGPrEkj#e6H*h%Ju@zKBJhQOYPKLi^Xm)f- z_vUr&SoZx^ycoiO@N1u;VhcFH8{<74`}uNf^Tf_JGExcRlq+NXPDIHRqFG>qZDQA* zCcpGj*{N`- z3|S00{0`ZFM_}O0UTp5}$Vdh2@GV9p5Rb2bRa;l)V_^j7yze(Tgj?+5_afq~?2q4P zj{UfdDS7BHAZ3A(A-tw*jAzwNS8f4S=dlL;ol!7oB${&MnW{HhX-OF_=0M*61^K^} zZlI$-tBY^|fRh9OfcbxL*#9ZvY|Ttu|1-J%e;V&=T?2_{rT zv5)>`i(C_+vxR6!J^N5TKi#8ww>Q`B+@HQs#ipNE9C^nqNHOox-!lwigCtTa8Rq9X zU<0n13@k{X|9JuyXe}m5gQ;Q&%q>{AxbIJcSyv-pz)&W;)pK=M)ra?WBn|E@PlVNd)7 zi{F&3AG2V7YvIPR`95FbXhXAsPx;NHNYw{MD$!*I?nwsI;4=)gX12$Yt`SEjk06gz ztqF-Bn|m*s-MD@tmZHLzd@+Tq<(cbq=@@tcgObsfo9gNhUN5pjC2S1AxbP zgAqSXCx1Z$QAf{PO3r9Vv5o%LBtuOCkO>vJpgvfq7Q>K%dtalVni0D4pwRw_3qx9_ zlmMj1|BgkQ&r28zQ3Wan(kUSy3jy|V6o804!^z_ekz(8s204+V^7#Fwv!6Il*rz1d z5avWeLTABv4E4U|*+6-0QyZN?S>lT;AT@{#5&JdcGi7tZosfjX@-9fHQCcf9*!r_BS{0C1II{BniEiCy=3nTAg zuhIOl#~HB5Xh{Lnp}0g@g8{NcXg69`nDi_sQ9ovVpvg_H?XVUgYl{8YAaa*O`~$eF z4M+WyMa57u9l^W2Lt>7WzXjYb7tIho~q(R9T{mu6=AuIs0Jh<|kw0Ur@K8fbzesk-6 zxJ1++SIp7G6jW`%M5*#If4`NetRU!NreL|6ymIv?0yua9FwY+!se#?Fv|0xm2)Opm zOr+jOHJ7hsS%fY-cP)!srVSxls$ux9P>bgcVdJFgu=w8C9}3&WOfUa#(ZuSkN_H-;vmC|L@4Md4Rv^md(r(1*d_GkQ;r zUl(jL{DyaX;y`daNX4mX?O>G7>he$nk4WjeiI)P{;DPh^Ge1V$SjSaR?b{)VJ{icK zWIst0Qpxc4?9y+pUs!gpaeYQ45PGDBUAp1GKts?*d3XsB5~$e53P!sDf;co^W~*Un z%Ug9TqL@7`61L~1Gc`nZ=5l(RIK%#;CL|$6rMhVR6v=R2okjDijAbj#8I zY3wY3>S(ugy>Ws=Ah-p03-0d0osGM@ySs(p?ydKwvpgGda7Ywvx54Rv1Hcb#~E+~1u zL^2-5oPZX$G6V1W0+DD>)0``jyc0oUJS}m~fc|?Av)gd0G7Suru$W>yG*_HPOnCAM zLxTm;{tQ+!P*K}7h|bw^_TAP2-t-KCWTw*?D0X*z2=FEZp<@ za-c*VKPI4vgM`15I{^PIgt@3Cq^e!A9gBRqM5x^M$pUs7qWPR8z}`a{9oml_F;cn{ z5U-JqL#lE!PlI{-dr{2NmF67Mc-KFN)M*_DnAYE-qLyCDb8sPJ=qFld08ulJ&91gs z;x6y{`iI2hp^&siLRYgY@P?(YMt z8S)1Sb_g|nQyTRy9_8~i4AyC48&*PfFoC5WLu>{FHTA_1iWkKVk&ul;uRy-FMq}6j z5AI8f;+p1@9h?+Za3dUC(GVPgdZ-uV$-EEBl5*+qROhqoHNv>vh;}ccA-z5nYCA5D za-9!1wpGFlSV)e(%d7sEGv-U^F&MM^f_;ZbpXh+gNk^*di=B_Vs#tM-B8nl+lCsig za35p3wV634MHz=o5Pp@?X(m)g9VxCKZkfcYD+)37;p+XIop>|BD90x&t8jRpH*K5! zRHlfx~IK0Whp>xr16baB7eD(xk*JH}viGtA(z z+wZWL_(r`xme}0}4LAue>xX9gG@a;Wp9%sRIBsfV({tk!W`;|P&zDVsIdekZ)mNE! zvA4aMv_E6VfG;+RSD#F*M>mXS7E`+(R;peGrvAtp?Dz$+;s{Q_mZZPXhvScJw-c}A z5}-=rCl895p`7~XEWPp7`lnq#ougx(5F zH^@bKk5?}kGQ#>JF^1`=Rcu<-Sm!Wq8U~9oviS&m4LZ9q9JwXipu zdRA(a5;3skc*Q-uEVqIi0=hiX8*z_sx$RWStlP}QoW2Ih@-m2a+m=(Pla_tP#T>B2 zl9R^|cJ*1HFm&wp&f>;9YPZKE85JtRemPa z64A~%O0~W%G5w|qrCphmF*M~2_v6IKuGNf4+0t51RDEn(|3GQ&jWd<)=&VVfeXc-? zEO4XTDK7NpCwH2niyBd@sS!-Aa8F8WH% z<%mHUtIJIFn@iqD>#(}f9B(Yp&7O)?AvjOoT4cbFF*Ixx;&B+LO z>fj+0S?+NXC|MG);$?CBV+}Z@UMp(L&(%5VI+X8NyW+LAJF~UpaC5gaS(y3DtNv@~ zzrQ5@3{G{0`vcIUTyjQ>$z6wJBWUP zHe=$unGiFL1qd6|SK6?Ue>u#YPWHX3ao}^0PTKe|fi3;e)HeHdvI}2AcSBZuu z3b_A6M3iT~^O&0xt;IeymKo{c{&@AMe>t*Y!0e$iu@s64wq10GZ%!y>&4{UU*!JleS9#&{vt?EcDNSP2^YUuG4LVRed7?t1vpc_cdn8>yc zHR465sv#4yWj#slA>wqU%xiTU$Y(B8XBkVfueuLe@k8U~UqasjcrnbBFsCcEW$a>6 zCFe(dulJnbZR6;2+Is=1RTdJT z+wCeKNs8A*ZnM%t3f+gfC%R3bsxmyGPyUv6 zOW=++Cdv7{Kq{DV$wjiuC{lt?+!vp&1HJlfQ3Ij5z9DM)C*Q7C!5Tr8;LXkX?Qxg^ zyaFodas6u-wfKo63qfPS?o_Fuo|&1)o1Fshi@Vd8r1lIdL132vul#0C?w@;wm$nmM z4=!KtbUmV9={cRjDO7Obb4%t_l5@_Ts#(%JY=x!rI@RhJl+3jN$(G|UI?c^w5W`iq zjri%b%B7WUPd{+osFyUu4sBs##TG9%KdFuLhJUu_6nwGOUc%bKf8&FRtUl(wEz$~7 zEOH(h_YE$pOp!$kcY$H@zNo-?S1 zrk_J{D4;UPnKG5wx{0uBH-SXior3cEM7cy+1aDLCzgqdbz21Yk0k}nB-m{o{QpWF( ziP%(gAe~;pAHEQEX5x!qc1-om#9x`Teuanjc8{O*jSw-IGh&lOd8Rj8W_)STjL}Tn z>Qp55;VB6Rq*6?DXx9P?IC~?!kcMDS>+mb|QolsOyuNZUnK)_j8t;@=bDlwYrLns+ z$J%|}Rr?u){^FK*-U8}661RjW6kH0(Y2S*a$HTSHO?a3}o)6L%M;SR3O{173`8{;Z zdgs~M_6m4Q6?oGGO9K^CM?<~P!`5B zkOW;JfsTy%{LBO8SOarU*8ZV)!;JgLa$3U3^rV7XhnDvx~`ApsZy#sBFMc^QUT zR-+0*D-;-8dm}`;%VTZAICaOXU>HCs+uyvCZhs+|SDxYf1oRhL~EqgtWa{#>VyY5FwPs_(QZGxQ*2>|+x79wz^h`pMY@S~f~9N>egq~fR?ARTG*SSu4drBG$4_&+SJJh_BwJM+v^PloNPjdSfXa? zusu{=Z_(3$i_sOZrfkNab;q=t!WIr``-QCNXl1ImZYBM+v4mnirQ~(jab0`iVIUE+ zWhWs=6g{?9J4@54sP+lc$UI)gvhkrQHl7ZYSnfOV=`7*Q1B1+bp*~LG8yVnp;8ZnO zv70PuuoT8P+#Z#%LlUw-gxy8-{iL^te;bvNT(U?<&?b)?)+YK<)EBLtmV(=_XFXqh zmW-B}ZHAnqD#tXb-f^s#VI%Z7&nSP&{I{uH3=?J?cU89=X4dypmru`bKT(q~2wEC0p?B*jCcdM9XE#xY_qskEwH0 zn3#KqwV`ea#t1W;b{*O=~D!@AS2qlGT{9&Zw~qddB`!K{@~|D`6o#I09jc zRt!~-;+#?)BW7u+iwSF~Q^4#Ju2CJ?=O!NnQXr3jYSntLe^-=D$jm4*_wk!u^APg`Vee(X`PG1}PdE zs7~DmdwL=e(Le_r2U5$gkM-DMp0d5&>k?&jkyg$k;a}Yt3A`()Pgy3}L_k^F_IOlh z8bkr%hzNJmVAsZG>PxK+YV(8@;srmYFL@7FDCLfQknyU?S9;Ht*&4?cIu!Z5Ud20J zv7cZ|2};x)$=9w&xLD?Qf89M))x55+Seq_jMpT#78cn9%|5hS`>tW!Wr4z2`F>?}U zh=HZSAwZu@XOe{04;T#Ro<-}VVS9Hx{L20+&ScqpREmVU)iwkS-JXtVULh2Vq%)z1 z0{luF8R)-xqgN;CDYyE_4@w}7;Pvhe*E2pQM`>^(t8sXSB~@yAi3)b%42DkBS_BW$ zLMKj6i;xJr;LT%coH-;jTw)@Y>;YE&#Ku9a+2eg0uj3A4OY4)PKEj`{5)~#&*N0yL zWS52$wk!it*6!cLOfM|B9n<3?sKSYC4s6CVcIvD(MecEK z)YFEN50&D)UD$7B9z&W(M8DjJG>6nm^g?#M;T>DOR7p(E85{wL?-eY7-A$QXAwqR`JExRh(iFwO^&rpdmgVC#76f=UT%ma^H9Gchv?I= zI`qAfwu`CzaSTQnLyR!m75zi(j6nR`D6vwrBWo^X;^vS1AuZpWOr1LEz73oBMw(4D zPG@S7YNdKbzz^O;ux|rh*qDdz5{nYCSy_h!lX+%t$yVnhrb6ng^W&ah;BtJtyM-n@ zsqC%;Q1e7D#=*ToW;7fXWdM~d50y(a7_QnKl@6(15JL?TM7noDlB*H^z z+;nzD_Uw<*dR^af+?#f*Zam2kW9VApbl{cwYBo!KO8c>Pf@0t!ZX$&!1gW z!uFzY^He3dV;Q%s$ysAqaOa{JkMjhMq(pOEvhbZs;Vf$2IJ%-<^5hX_*Z0zH5wl00 z1(}bwmK*lk{P66KOeW!% z*2BTRxK-*&N2zB2$~P7YuXsS8!7f0R)w>}0@o+28U$q#*7MS?)WEr}Bo@Dl|gXZzyX$u{nS@N4r7KR9<`e`z0#6WX(I39jZ7UN=I|z`>+rQO`N(EL)kP$$wZf?u zBk0Iv6%kvlR5i_*igSpTG*9o&v6NC{)Ux|W@;zgx+jgwSrY$`N_T2Qq{hxfL_*FIp z@29ww*>^t)3!0FCXfhTzcVB;MyB8yu%`m#J#)b>lok}U+UWvG+>(y|@NT>R7!<6T7 zaD^V(@6kr5PF@N-#WyG*Q7|Hh3tIlk5Z1lPUfe^Y!k zJw&ONZNybU%PVTvn7o0UqG#_cI={A~dr!Nu3cZNBZ0(lXM*UouH@_XiqfoPBtQV=H zHO`G&;`VhIZi&Wd&8m3xNgdpYiI<*dGmqGKT^oKp>{(nV?sv>>TVAQ#x{qGzID3V({1`bUuQno8N_;|4S%?4Ks%RrGSf%bsJOjbrI@!GeAbEnN z?sVA5c)&^SW?g2RQpnET17%m55#{n{1fTy;|5-J#)!CMgk&}*FZ2XSdnDqy;(lj%V z94o=CL&o{&EskSzq6)no91!7`QaHT4@~&1-#K%3}wQPk_&uGuGM!q=(x*F;NUHX~T z4V~Hc04c}LOOy7Engs+dt4iXt{Bh-}QG&Z?q?zM;^Q?JgkJ|>Vj`r!bAP%S@19go& zcuWlx79j+U~CW;FqZ(#)aZTcqoieFtmKmt1{FGYJb!1d$2TUf+xS=t~Ec^&FygLU?P?c~JZ)7Q& z4>y@%YN)9}NA|J^#9qS{I&L1YV}Gz(8Lj_$(MD|W#5pDTbP_`JT4+0&tBgtjj$y`0 zUb%V$F0;5vo08{#XxPM0M{~dUy^QbnM6prP3`3$bA-vUpyhDoU-H~c@&&l}tR{5K! zUpD&@UP1dj_yozdI7F#QJ*z-QF8lPC|HeJ%GLBOuh}tey`f z1hF$7rYpBy$i;z}=NU`T(uPT$RDCH_E-Py4Q?g^r%r2~o)MXf=;zA?y!Z;r5DK73j zEzD77#X|qd7%nf<&+%HR9!Y($ksyDDns^a9zq?L}_{I+m#BCpXZF|h9-pKt}jyV$a zXI)6_T{w#2=cY2r$owLgO)L}pY0ax)LPA?fg~9Wlb@Xt#m++^W@y&-VQ!gK$&UvDd zhn>@lbI`#4Qw-lea}2~xl5w!EnY&XU)jr(qEy?VNF>sA4?#rgaLUj-|?o@rHuFw=AJi2;O8BfoSwm{~~g|a5fsk&<6p+Zk`*DcD^O+F7?4Jug< z&Yg6Wn(x6Ay)1#>#YywqtK{Pe;)6MJ$+z$50>$18gK+1J`laFK_8SCQM8&0W_9cZa z%r^?%mgfbObeZ0kO@Ml3A^niVZm~`iH2v(vg1EwT2pN!8uOcK(%4G~XX*?G<=96O(ZMCKv7&6S!KBVQUquokytQO3^ zI{{WS`SzKzjTrX{I`&uy#2wenydOea+Mg(Jj}pXCSCydY3QW11r`xme4|)^~461qk zFrr`n3_@%}c|DV_CgXPSv}AP&K8z$@v4jvT5X5x10r}oeNX+?>BVi#PwvQc=Lso1)(;N{GNCn-0XmcA=IM8q?!=$04pZvSEM zioFzMo(|7{uU$@Hp?-qH?9GgtLH6{UQ{}nJY?jPgYH8Mo@I#tmM$z=k=Gbc#YJ$ng zR>FL=IAIFRl>Ume5zo2r#ZLPhjz8j?H_H(r_ZHRcw`|N4JWjC=1zhGfvT&?!K)j9B z>(lx#xb3fz4O4@)Cu9qKQ3e-GKkiXwo0b@GU+M$ncH;3g`lbR6<=+~&H0d@`%;g;il%1L95g=~(~ zSA-!yqNU&_yQT9FJUHN(NQ?N?nR5bnGphS0-Fvm<6tQupbQFxDuc9M^1ca8xUIfCo z6A4#3T-sAkDHm=qbc=27c zBVFAzoRHVEM(LR>tv$*sl#UKk_AA!Kw#q}bDCkDpG*i1s{6u+!s3ObRj7an;L^~So zm?QK#n(0`Om9YHUfQGaxLwy-72|W>Lb+yzl2jG1L(EArS@Cs<)i&tq*I*np&4;S5b z&Eedb`(Y_46ErZ;Qwgdwzv>w1WA^i7d8Dvd@E^;GbGlQzwaY5pD=xw*JfLX6#QZuWd}n(E*4l#Ggr7ZvvxB9$|^J-a}AU~~0!)u_5< zq4fKq(Oq-fepT}W>Ke>3@W%J?e4lgCSKQ*P#SSJWS>RqC?r~eopWrqoPH)Z-6X?=c zD${PQgzpG6{!FbZ{d6rR)>yoZHIZwK;{G$R`$~O16Y1QqIEaUI$OI)uwFRM`%KUa9 zrrLPEFy-FIi;Mr&(dzLl5G@n}{y_Ed>$F`>VCH&j(c*fe9eDh`*Jf>(h$s_fzM-zZF_|F z^zh1#zEV!L3ujAfRzKs&SR1rDH^=#;!dvqDkXt)$?z1kgp|L07>e-n>a(h~%KG}SX z<#uq@B8_=FiPjv26RaWSoRpc@QID=iX~%3(wadM`8(Dw5W6@CZZ*qTv;h$sL{_dw0 ze}lepbm&Np}*IExgP-*VM9^=x0F|dA{1+If9{8 zQ)d^CD|sL1^~tDxP~Q;O536^YaRIDK@;tANez0!^2Dee~$GMa~$0g53cn*I~AxHOb zO4oehcDZ>W&Xej=T7o7Zj%1I1;w-e=v*qUQ!d~U9{80gh{T}0`AsVW(Be~{@qs8#X z2%BO(!ePDIrcx%zHtC03{)qHVkfjbM4J@Tn;X?JJRdC60TlRjLANps1j&tyEQry^} z&sg&n2r7=WkcSvd@k@O0`YfQOJKM+m4Og(Oja+$S$z4$4OR|HpkwldSCuos4Q ztp0fzgx;(Rf#l33p+)DD?h*akP_XxvH2Jf0fUYBNC6qog4(OrvTZTHLMm*-11Ki^W zxNqKE{8#avOPE?uf!vY)Bu!)7*7 zt|e5Qc?Nu+PBR!yoH|-Q^zK?`c5-HXQs9ZmuRXkVwcS%@q>4=HeYmBhkbJnm#%|B~ zbtJwmrY}(~!`()_UDxz;O$Yw5sVDmkFJeCV5!z8G9FC-8QyO75mDJC;YBvCh@!?e- zwQUAMNavTWAi%=vnipclYi!vV~j8nE; zN~m^Y&lj_+QN1aUicY_2%nx*)9ZR^Ik!wywSKfyRk(K|WlB^&pbaN!<5ipy*z3~RA zC<_5e1ceCz0N??9d5Kz+PZ?p2FaW>@bN~SLJpuEt3F5sK1^%_<;Q7Uh$=TA}&d}yR zb1-25P-NPM=B^%y00`*2_YI-{Lj5Nc;jcXa)MiZDg|!GwxbMquZaQ&Cq4C?=}1IP`amgV1j)c(Ewor**9PpZDYrJbdVzW%?c|3er2 zrPUdUxV=Xk0JsZXdjCQG uKl6P5cYXZ#knq3LcO(Bv|Gxsn@8thhH_ZD0dcOqT=a27$fw%atwf_L2J_&aK literal 0 HcmV?d00001 diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index f595b26450e39..ae20fc1efdef6 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -343,7 +343,7 @@ def run(self): license="http://www.apache.org/licenses/LICENSE-2.0", # Don't forget to update python/docs/source/getting_started/install.rst # if you're updating the versions or dependencies. - install_requires=["py4j==0.10.9.8"], + install_requires=["py4j==0.10.9.9"], extras_require={ "ml": ["numpy>=%s" % _minimum_numpy_version], "mllib": ["numpy>=%s" % _minimum_numpy_version], diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 825ea0c63c6f6..814e17f147129 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" - export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.8-src.zip:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.9-src.zip:${PYTHONPATH}" export PYSPARK_PYTHONPATH_SET=1 fi From 0593ac6d345a695da325499e644212d03f321735 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 14 Jan 2025 20:22:29 -0800 Subject: [PATCH 433/438] [SPARK-50541][SQL][FOLLOWUP] Migrate DESC TABLE AS JSON to v2 command ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/49139 to use v2 command to simplify the code. Now we only need one logical plan and all the implementation is centralized to that logical plan, no need to touch other analyzer/planner rules. ### Why are the changes needed? code simplification ### Does this PR introduce _any_ user-facing change? no, this feature is not released yet. ### How was this patch tested? update tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49466 from cloud-fan/as-json. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/parser/AstBuilder.scala | 43 +- .../catalyst/plans/logical/v2Commands.scala | 13 - .../AnalysisExceptionPositionSuite.scala | 1 - .../analysis/ResolveSessionCatalog.scala | 4 - .../spark/sql/execution/SparkSqlParser.scala | 44 +- .../command/DescribeRelationJsonCommand.scala | 313 ++++++++++++++ .../spark/sql/execution/command/tables.scala | 259 ----------- .../spark/sql/execution/command/views.scala | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 3 - .../analyzer-results/describe.sql.out | 8 +- .../sql-tests/results/describe.sql.out | 5 +- .../command/DescribeTableParserSuite.scala | 19 +- .../command/PlanResolutionSuite.scala | 53 +-- .../command/v1/DescribeTableSuite.scala | 409 +++++++----------- 14 files changed, 544 insertions(+), 632 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 5858425f66462..b408fcefcfb26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3494,7 +3494,7 @@ class AstBuilder extends DataTypeAstBuilder /** * Create an [[UnresolvedTableOrView]] from a multi-part identifier. */ - private def createUnresolvedTableOrView( + protected def createUnresolvedTableOrView( ctx: IdentifierReferenceContext, commandName: String, allowTempView: Boolean = true): LogicalPlan = withOrigin(ctx) { @@ -5198,47 +5198,6 @@ class AstBuilder extends DataTypeAstBuilder visitLocationSpec(ctx.locationSpec)) } - /** - * Create a [[DescribeColumn]] or [[DescribeRelation]] commands. - */ - override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { - val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null - val asJson = ctx.JSON != null - if (asJson && !isExtended) { - val tableName = ctx.identifierReference.getText.split("\\.").lastOption.getOrElse("table") - throw QueryCompilationErrors.describeJsonNotExtendedError(tableName) - } - val relation = createUnresolvedTableOrView(ctx.identifierReference, "DESCRIBE TABLE") - if (ctx.describeColName != null) { - if (ctx.partitionSpec != null) { - throw QueryParsingErrors.descColumnForPartitionUnsupportedError(ctx) - } else if (asJson) { - throw QueryCompilationErrors.describeColJsonUnsupportedError() - } else { - DescribeColumn( - relation, - UnresolvedAttribute(ctx.describeColName.nameParts.asScala.map(_.getText).toSeq), - isExtended) - } - } else { - val partitionSpec = if (ctx.partitionSpec != null) { - // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. - visitPartitionSpec(ctx.partitionSpec).map { - case (key, Some(value)) => key -> value - case (key, _) => - throw QueryParsingErrors.emptyPartitionKeyError(key, ctx.partitionSpec) - } - } else { - Map.empty[String, String] - } - if (asJson) { - DescribeRelationJson(relation, partitionSpec, isExtended) - } else { - DescribeRelation(relation, partitionSpec, isExtended) - } - } - } - /** * Create an [[AnalyzeTable]], or an [[AnalyzeColumn]]. * Example SQL for analyzing a table or a set of partitions : diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index b486a1fd0a72a..58c62a90225aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -691,19 +691,6 @@ object DescribeRelation { def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes() } -/** - * The logical plan of the DESCRIBE relation_name AS JSON command. - */ -case class DescribeRelationJson( - relation: LogicalPlan, - partitionSpec: TablePartitionSpec, - isExtended: Boolean) extends UnaryCommand { - override val output: Seq[Attribute] = DescribeCommandSchema.describeJsonTableAttributes() - override def child: LogicalPlan = relation - override protected def withNewChildInternal(newChild: LogicalPlan): DescribeRelationJson = - copy(relation = newChild) -} - /** * The logical plan of the DESCRIBE relation_name col_name command. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala index 55f59f7a22574..325862127d366 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala @@ -40,7 +40,6 @@ class AnalysisExceptionPositionSuite extends AnalysisTest { } test("SPARK-34057: UnresolvedTableOrView should retain sql text position") { - verifyTableOrViewPosition("DESCRIBE TABLE unknown", "unknown") verifyTableOrPermanentViewPosition("ANALYZE TABLE unknown COMPUTE STATISTICS", "unknown") verifyTableOrViewPosition("ANALYZE TABLE unknown COMPUTE STATISTICS FOR COLUMNS col", "unknown") verifyTableOrViewPosition("ANALYZE TABLE unknown COMPUTE STATISTICS FOR ALL COLUMNS", "unknown") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 3b58518b98da9..b73ea2f80452b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -152,10 +152,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case RenameTable(ResolvedV1TableOrViewIdentifier(oldIdent), newName, isView) => AlterTableRenameCommand(oldIdent, newName.asTableIdentifier, isView) - case DescribeRelationJson( - ResolvedV1TableOrViewIdentifier(ident), partitionSpec, isExtended) => - DescribeTableJsonCommand(ident, partitionSpec, isExtended) - // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. case DescribeRelation( ResolvedV1TableOrViewIdentifier(ident), partitionSpec, isExtended, output) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 744ab03d5d037..2b7be9b34b9aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -27,7 +27,7 @@ import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, PlanWithUnresolvedIdentifier, SchemaEvolution, SchemaTypeEvolution, UnresolvedFunctionName, UnresolvedIdentifier, UnresolvedNamespace} +import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, PlanWithUnresolvedIdentifier, SchemaEvolution, SchemaTypeEvolution, UnresolvedAttribute, UnresolvedFunctionName, UnresolvedIdentifier, UnresolvedNamespace} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.catalyst.parser._ @@ -1153,4 +1153,46 @@ class SparkSqlAstBuilder extends AstBuilder { withIdentClause(ctx.identifierReference(), UnresolvedNamespace(_)), cleanedProperties) } + + /** + * Create a [[DescribeColumn]] or [[DescribeRelation]] or [[DescribeRelationAsJsonCommand]] + * command. + */ + override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { + val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null + val asJson = ctx.JSON != null + if (asJson && !isExtended) { + val tableName = ctx.identifierReference.getText.split("\\.").lastOption.getOrElse("table") + throw QueryCompilationErrors.describeJsonNotExtendedError(tableName) + } + val relation = createUnresolvedTableOrView(ctx.identifierReference, "DESCRIBE TABLE") + if (ctx.describeColName != null) { + if (ctx.partitionSpec != null) { + throw QueryParsingErrors.descColumnForPartitionUnsupportedError(ctx) + } else if (asJson) { + throw QueryCompilationErrors.describeColJsonUnsupportedError() + } else { + DescribeColumn( + relation, + UnresolvedAttribute(ctx.describeColName.nameParts.asScala.map(_.getText).toSeq), + isExtended) + } + } else { + val partitionSpec = if (ctx.partitionSpec != null) { + // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. + visitPartitionSpec(ctx.partitionSpec).map { + case (key, Some(value)) => key -> value + case (key, _) => + throw QueryParsingErrors.emptyPartitionKeyError(key, ctx.partitionSpec) + } + } else { + Map.empty[String, String] + } + if (asJson) { + DescribeRelationJsonCommand(relation, partitionSpec, isExtended) + } else { + DescribeRelation(relation, partitionSpec, isExtended) + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala new file mode 100644 index 0000000000000..6abe34f0ea156 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeRelationJsonCommand.scala @@ -0,0 +1,313 @@ +/* + * 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.command + +import scala.collection.mutable + +import org.json4s._ +import org.json4s.JsonAST.JObject +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.analysis.{ResolvedPersistentView, ResolvedTable, ResolvedTempView} +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.quoteIfNeeded +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.connector.catalog.V1Table +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.PartitioningUtils + +/** + * The command for `DESCRIBE ... AS JSON`. + */ +case class DescribeRelationJsonCommand( + child: LogicalPlan, + partitionSpec: TablePartitionSpec, + isExtended: Boolean, + override val output: Seq[Attribute] = Seq( + AttributeReference( + "json_metadata", + StringType, + nullable = false, + new MetadataBuilder().putString("comment", "JSON metadata of the table").build())() + )) extends UnaryRunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val jsonMap = mutable.LinkedHashMap[String, JValue]() + child match { + case v: ResolvedTempView => + if (partitionSpec.nonEmpty) { + throw QueryCompilationErrors.descPartitionNotAllowedOnTempView(v.identifier.name()) + } + describeIdentifier(Seq("system", "session", v.identifier.name()), jsonMap) + describeColsJson(v.metadata.schema, jsonMap) + describeFormattedTableInfoJson(v.metadata, jsonMap) + + case v: ResolvedPersistentView => + if (partitionSpec.nonEmpty) { + throw QueryCompilationErrors.descPartitionNotAllowedOnView(v.identifier.name()) + } + describeIdentifier(v.identifier.toQualifiedNameParts(v.catalog), jsonMap) + describeColsJson(v.metadata.schema, jsonMap) + describeFormattedTableInfoJson(v.metadata, jsonMap) + + case ResolvedTable(catalog, identifier, V1Table(metadata), _) => + describeIdentifier(identifier.toQualifiedNameParts(catalog), jsonMap) + val schema = if (metadata.schema.isEmpty) { + // In older versions of Spark, + // the table schema can be empty and should be inferred at runtime. + sparkSession.table(metadata.identifier).schema + } else { + metadata.schema + } + describeColsJson(schema, jsonMap) + describeClusteringInfoJson(metadata, jsonMap) + if (partitionSpec.nonEmpty) { + // Outputs the partition-specific info for the DDL command: + // "DESCRIBE [EXTENDED|FORMATTED] table_name PARTITION (partitionVal*)" + describePartitionInfoJson( + sparkSession, sparkSession.sessionState.catalog, metadata, jsonMap) + } else { + describeFormattedTableInfoJson(metadata, jsonMap) + } + + case _ => throw QueryCompilationErrors.describeAsJsonNotSupportedForV2TablesError() + } + + Seq(Row(compact(render(JObject(jsonMap.toList))))) + } + + private def addKeyValueToMap( + key: String, + value: JValue, + jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + // Rename some JSON keys that are pre-named in describe table implementation + val renames = Map( + "inputformat" -> "input_format", + "outputformat" -> "output_format" + ) + + val normalizedKey = key.toLowerCase().replace(" ", "_") + val renamedKey = renames.getOrElse(normalizedKey, normalizedKey) + + if (!jsonMap.contains(renamedKey) && !excludedKeys.contains(renamedKey)) { + jsonMap += renamedKey -> value + } + } + + private def describeIdentifier( + ident: Seq[String], + jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + addKeyValueToMap("table_name", JString(ident.last), jsonMap) + addKeyValueToMap("catalog_name", JString(ident.head), jsonMap) + val namespace = ident.init.tail + addKeyValueToMap("namespace", JArray(namespace.map(JString).toList), jsonMap) + if (namespace.nonEmpty) { + addKeyValueToMap("schema_name", JString(namespace.last), jsonMap) + } + } + + /** + * Util to recursively form JSON string representation of data type, used for DESCRIBE AS JSON. + * Differs from `json` in DataType.scala by providing additional fields for some types. + */ + private def jsonType(dataType: DataType): JValue = { + dataType match { + case arrayType: ArrayType => + JObject( + "name" -> JString("array"), + "element_type" -> jsonType(arrayType.elementType), + "element_nullable" -> JBool(arrayType.containsNull) + ) + + case mapType: MapType => + JObject( + "name" -> JString("map"), + "key_type" -> jsonType(mapType.keyType), + "value_type" -> jsonType(mapType.valueType), + "value_nullable" -> JBool(mapType.valueContainsNull) + ) + + case structType: StructType => + val fieldsJson = structType.fields.map { field => + val baseJson = List( + "name" -> JString(field.name), + "type" -> jsonType(field.dataType), + "nullable" -> JBool(field.nullable) + ) + val commentJson = field.getComment().map(comment => "comment" -> JString(comment)).toList + val defaultJson = + field.getCurrentDefaultValue().map(default => "default" -> JString(default)).toList + + JObject(baseJson ++ commentJson ++ defaultJson: _*) + }.toList + + JObject( + "name" -> JString("struct"), + "fields" -> JArray(fieldsJson) + ) + + case decimalType: DecimalType => + JObject( + "name" -> JString("decimal"), + "precision" -> JInt(decimalType.precision), + "scale" -> JInt(decimalType.scale) + ) + + case varcharType: VarcharType => + JObject( + "name" -> JString("varchar"), + "length" -> JInt(varcharType.length) + ) + + case charType: CharType => + JObject( + "name" -> JString("char"), + "length" -> JInt(charType.length) + ) + + // Only override TimestampType; TimestampType_NTZ type is already timestamp_ntz + case _: TimestampType => + JObject("name" -> JString("timestamp_ltz")) + + case yearMonthIntervalType: YearMonthIntervalType => + def getFieldName(field: Byte): String = YearMonthIntervalType.fieldToString(field) + + JObject( + "name" -> JString("interval"), + "start_unit" -> JString(getFieldName(yearMonthIntervalType.startField)), + "end_unit" -> JString(getFieldName(yearMonthIntervalType.endField)) + ) + + case dayTimeIntervalType: DayTimeIntervalType => + def getFieldName(field: Byte): String = DayTimeIntervalType.fieldToString(field) + + JObject( + "name" -> JString("interval"), + "start_unit" -> JString(getFieldName(dayTimeIntervalType.startField)), + "end_unit" -> JString(getFieldName(dayTimeIntervalType.endField)) + ) + + case _ => + JObject("name" -> JString(dataType.simpleString)) + } + } + + private def describeColsJson( + schema: StructType, + jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + val columnsJson = jsonType(StructType(schema.fields)) + .asInstanceOf[JObject].find(_.isInstanceOf[JArray]).get + addKeyValueToMap("columns", columnsJson, jsonMap) + } + + private def describeClusteringInfoJson( + table: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + table.clusterBySpec.foreach { clusterBySpec => + val clusteringColumnsJson: JValue = JArray( + clusterBySpec.columnNames.map { fieldNames => + val nestedFieldOpt = table.schema.findNestedField(fieldNames.fieldNames.toIndexedSeq) + assert(nestedFieldOpt.isDefined, + "The clustering column " + + s"${fieldNames.fieldNames.map(quoteIfNeeded).mkString(".")} " + + s"was not found in the table schema ${table.schema.catalogString}." + ) + val (path, field) = nestedFieldOpt.get + JObject( + "name" -> JString((path :+ field.name).map(quoteIfNeeded).mkString(".")), + "type" -> jsonType(field.dataType), + "comment" -> field.getComment().map(JString).getOrElse(JNull) + ) + }.toList + ) + addKeyValueToMap("clustering_information", clusteringColumnsJson, jsonMap) + } + } + + private def describeFormattedTableInfoJson( + table: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + table.bucketSpec match { + case Some(spec) => + spec.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + case _ => + } + table.storage.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + + val filteredTableInfo = table.toJsonLinkedHashMap + + filteredTableInfo.map { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + } + + private def describePartitionInfoJson( + spark: SparkSession, + catalog: SessionCatalog, + metadata: CatalogTable, + jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { + if (metadata.tableType == CatalogTableType.VIEW) { + throw QueryCompilationErrors.descPartitionNotAllowedOnView(metadata.identifier.identifier) + } + + DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION") + val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( + partitionSpec, + metadata.partitionSchema, + metadata.identifier.quotedString, + spark.sessionState.conf.resolver) + val partition = catalog.getPartition(metadata.identifier, normalizedPartSpec) + + // First add partition details to jsonMap. + // `addKeyValueToMap` only adds unique keys, so this ensures the + // more detailed partition information is added + // in the case of duplicated key names (e.g. storage_information). + partition.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + + metadata.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + + metadata.bucketSpec match { + case Some(spec) => + spec.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + case _ => + } + metadata.storage.toJsonLinkedHashMap.foreach { case (key, value) => + addKeyValueToMap(key, value, jsonMap) + } + } + + // Already added to jsonMap in DescribeTableJsonCommand + private val excludedKeys = Set("catalog", "schema", "database", "table") + + override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = { + copy(child = newChild) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 73aaed0627946..a58e8fac6e36d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -19,16 +19,12 @@ package org.apache.spark.sql.execution.command import java.net.{URI, URISyntaxException} -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileContext, FsConstants, Path} import org.apache.hadoop.fs.permission.{AclEntry, AclEntryScope, AclEntryType, FsAction, FsPermission} -import org.json4s._ -import org.json4s.JsonAST.JObject -import org.json4s.jackson.JsonMethods._ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{SQLConfHelper, TableIdentifier} @@ -750,261 +746,6 @@ case class DescribeTableCommand( } } -/** - * Command that looks like - * {{{ - * DESCRIBE [EXTENDED|FORMATTED] table_name partitionSpec? [AS JSON]; - * }}} - */ -case class DescribeTableJsonCommand( - table: TableIdentifier, - partitionSpec: TablePartitionSpec, - isExtended: Boolean) extends LeafRunnableCommand { - override val output = DescribeCommandSchema.describeJsonTableAttributes() - // Already added to jsonMap in DescribeTableJsonCommand - private val excludedKeys = Set("catalog", "schema", "database", "table") - - override def run(sparkSession: SparkSession): Seq[Row] = { - val jsonMap = mutable.LinkedHashMap[String, JValue]() - val catalog = sparkSession.sessionState.catalog - - if (catalog.isTempView(table)) { - if (partitionSpec.nonEmpty) { - throw QueryCompilationErrors.descPartitionNotAllowedOnTempView(table.identifier) - } - val schema = catalog.getTempViewOrPermanentTableMetadata(table).schema - describeColsJson(schema, jsonMap, header = false) - } else { - val metadata = catalog.getTableRawMetadata(table) - val schema = if (metadata.schema.isEmpty) { - // In older versions of Spark, - // the table schema can be empty and should be inferred at runtime. - sparkSession.table(metadata.identifier).schema - } else { - metadata.schema - } - - addKeyValueToMap("table_name", JString(metadata.identifier.table), jsonMap) - table.catalog.foreach(catalog => addKeyValueToMap("catalog_name", JString(catalog), jsonMap)) - table.database.foreach { db => - addKeyValueToMap("namespace", JArray(List(JString(db))), jsonMap) - addKeyValueToMap("schema_name", JString(db), jsonMap) - } - - describeColsJson(schema, jsonMap, header = false) - describeClusteringInfoJson(metadata, jsonMap) - - if (partitionSpec.nonEmpty) { - // Outputs the partition-specific info for the DDL command: - // "DESCRIBE [EXTENDED|FORMATTED] table_name PARTITION (partitionVal*)" - describePartitionInfoJson(sparkSession, catalog, metadata, jsonMap) - } else { - describeFormattedTableInfoJson(metadata, jsonMap) - } - } - - Seq(Row(compact(render(JObject(jsonMap.toList))))) - } - - private def addKeyValueToMap( - key: String, - value: JValue, - jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { - // Rename some JSON keys that are pre-named in describe table implementation - val renames = Map( - "inputformat" -> "input_format", - "outputformat" -> "output_format" - ) - - val normalizedKey = key.toLowerCase().replace(" ", "_") - val renamedKey = renames.getOrElse(normalizedKey, normalizedKey) - - if (!jsonMap.contains(renamedKey) && !excludedKeys.contains(renamedKey)) { - jsonMap += renamedKey -> value - } - } - - /** - * Util to recursively form JSON string representation of data type, used for DESCRIBE AS JSON. - * Differs from `json` in DataType.scala by providing additional fields for some types. - */ - private def jsonType( - dataType: DataType): JValue = { - dataType match { - case arrayType: ArrayType => - JObject( - "name" -> JString("array"), - "element_type" -> jsonType(arrayType.elementType), - "element_nullable" -> JBool(arrayType.containsNull) - ) - - case mapType: MapType => - JObject( - "name" -> JString("map"), - "key_type" -> jsonType(mapType.keyType), - "value_type" -> jsonType(mapType.valueType), - "value_nullable" -> JBool(mapType.valueContainsNull) - ) - - case structType: StructType => - val fieldsJson = structType.fields.map { field => - val baseJson = List( - "name" -> JString(field.name), - "type" -> jsonType(field.dataType), - "nullable" -> JBool(field.nullable) - ) - val commentJson = field.getComment().map(comment => "comment" -> JString(comment)).toList - val defaultJson = - field.getCurrentDefaultValue().map(default => "default" -> JString(default)).toList - - JObject(baseJson ++ commentJson ++ defaultJson: _*) - }.toList - - JObject( - "name" -> JString("struct"), - "fields" -> JArray(fieldsJson) - ) - - case decimalType: DecimalType => - JObject( - "name" -> JString("decimal"), - "precision" -> JInt(decimalType.precision), - "scale" -> JInt(decimalType.scale) - ) - - case varcharType: VarcharType => - JObject( - "name" -> JString("varchar"), - "length" -> JInt(varcharType.length) - ) - - case charType: CharType => - JObject( - "name" -> JString("char"), - "length" -> JInt(charType.length) - ) - - // Only override TimestampType; TimestampType_NTZ type is already timestamp_ntz - case _: TimestampType => - JObject("name" -> JString("timestamp_ltz")) - - case yearMonthIntervalType: YearMonthIntervalType => - def getFieldName(field: Byte): String = YearMonthIntervalType.fieldToString(field) - - JObject( - "name" -> JString("interval"), - "start_unit" -> JString(getFieldName(yearMonthIntervalType.startField)), - "end_unit" -> JString(getFieldName(yearMonthIntervalType.endField)) - ) - - case dayTimeIntervalType: DayTimeIntervalType => - def getFieldName(field: Byte): String = DayTimeIntervalType.fieldToString(field) - - JObject( - "name" -> JString("interval"), - "start_unit" -> JString(getFieldName(dayTimeIntervalType.startField)), - "end_unit" -> JString(getFieldName(dayTimeIntervalType.endField)) - ) - - case _ => - JObject("name" -> JString(dataType.simpleString)) - } - } - - private def describeColsJson( - schema: StructType, - jsonMap: mutable.LinkedHashMap[String, JValue], - header: Boolean): Unit = { - val columnsJson = jsonType(StructType(schema.fields)) - .asInstanceOf[JObject].find(_.isInstanceOf[JArray]).get - addKeyValueToMap("columns", columnsJson, jsonMap) - } - - private def describeClusteringInfoJson( - table: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { - table.clusterBySpec.foreach { clusterBySpec => - val clusteringColumnsJson: JValue = JArray( - clusterBySpec.columnNames.map { fieldNames => - val nestedFieldOpt = table.schema.findNestedField(fieldNames.fieldNames.toIndexedSeq) - assert(nestedFieldOpt.isDefined, - "The clustering column " + - s"${fieldNames.fieldNames.map(quoteIfNeeded).mkString(".")} " + - s"was not found in the table schema ${table.schema.catalogString}." - ) - val (path, field) = nestedFieldOpt.get - JObject( - "name" -> JString((path :+ field.name).map(quoteIfNeeded).mkString(".")), - "type" -> jsonType(field.dataType), - "comment" -> field.getComment().map(JString).getOrElse(JNull) - ) - }.toList - ) - addKeyValueToMap("clustering_information", clusteringColumnsJson, jsonMap) - } - } - - private def describeFormattedTableInfoJson( - table: CatalogTable, jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { - table.bucketSpec match { - case Some(spec) => - spec.toJsonLinkedHashMap.foreach { case (key, value) => - addKeyValueToMap(key, value, jsonMap) - } - case _ => - } - table.storage.toJsonLinkedHashMap.foreach { case (key, value) => - addKeyValueToMap(key, value, jsonMap) - } - - val filteredTableInfo = table.toJsonLinkedHashMap - - filteredTableInfo.map { case (key, value) => - addKeyValueToMap(key, value, jsonMap) - } - } - - private def describePartitionInfoJson( - spark: SparkSession, - catalog: SessionCatalog, - metadata: CatalogTable, - jsonMap: mutable.LinkedHashMap[String, JValue]): Unit = { - if (metadata.tableType == CatalogTableType.VIEW) { - throw QueryCompilationErrors.descPartitionNotAllowedOnView(table.identifier) - } - - DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION") - val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( - partitionSpec, - metadata.partitionSchema, - table.quotedString, - spark.sessionState.conf.resolver) - val partition = catalog.getPartition(table, normalizedPartSpec) - - // First add partition details to jsonMap. - // `addKeyValueToMap` only adds unique keys, so this ensures the - // more detailed partition information is added - // in the case of duplicated key names (e.g. storage_information). - partition.toJsonLinkedHashMap.map { case (key, value) => - addKeyValueToMap(key, value, jsonMap) - } - - metadata.toJsonLinkedHashMap.map { case (key, value) => - addKeyValueToMap(key, value, jsonMap) - } - - metadata.bucketSpec match { - case Some(spec) => - spec.toJsonLinkedHashMap.map { case (key, value) => - addKeyValueToMap(key, value, jsonMap) - } - case _ => - } - metadata.storage.toJsonLinkedHashMap.map { case (key, value) => - addKeyValueToMap(key, value, jsonMap) - } - } -} - /** * Command that looks like * {{{ 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 f654c846c8a57..6428583c9e1ea 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 @@ -793,7 +793,6 @@ object ViewHelper extends SQLConfHelper with Logging { originalText: String, tempFunctions: Seq[String]): CatalogTable = { - val catalog = session.sessionState.catalog val tempViews = collectTemporaryViews(analyzedPlan) val tempVariables = collectTemporaryVariables(analyzedPlan) // TBLPROPERTIES is not allowed for temporary view, so we don't use it for @@ -808,6 +807,7 @@ object ViewHelper extends SQLConfHelper with Logging { storage = CatalogStorageFormat.empty, schema = viewSchema, viewText = Some(originalText), + createVersion = org.apache.spark.SPARK_VERSION, properties = newProperties) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index f7a3be9254758..499721fbae4e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -344,9 +344,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DescribeNamespace(ResolvedNamespace(catalog, ns, _), extended, output) => DescribeNamespaceExec(output, catalog.asNamespaceCatalog, ns, extended) :: Nil - case DescribeRelationJson(_, _, _) => - throw QueryCompilationErrors.describeAsJsonNotSupportedForV2TablesError() - case DescribeRelation(r: ResolvedTable, partitionSpec, isExtended, output) => if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.describeDoesNotSupportPartitionForV2TablesError() diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out index f52f69a5ff808..307a0a3e25fbd 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -59,7 +59,8 @@ DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_typ -- !query DESCRIBE EXTENDED t AS JSON -- !query analysis -DescribeTableJsonCommand `spark_catalog`.`default`.`t`, true +DescribeRelationJsonCommand true, [json_metadata#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.t, V1Table(default.t), [a#x, b#x, c#x, d#x] -- !query @@ -142,7 +143,8 @@ DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=1], false, [col_nam -- !query DESC EXTENDED t PARTITION (c='Us', d=1) AS JSON -- !query analysis -DescribeTableJsonCommand `spark_catalog`.`default`.`t`, [c=Us, d=1], true +DescribeRelationJsonCommand [c=Us, d=1], true, [json_metadata#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.t, V1Table(default.t), [a#x, b#x, c#x, d#x] -- !query @@ -328,7 +330,7 @@ ExplainCommand 'DescribeRelation [c=Us, d=2], false, [col_name#x, data_type#x, c -- !query EXPLAIN DESCRIBE EXTENDED t PARTITION (c='Us', d=2) AS JSON -- !query analysis -ExplainCommand 'DescribeRelationJson [c=Us, d=2], true, SimpleMode +ExplainCommand 'DescribeRelationJsonCommand [c=Us, d=2], true, [json_metadata#x], SimpleMode -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 870ad02e71414..70870131163e5 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -693,8 +693,9 @@ EXPLAIN DESCRIBE EXTENDED t PARTITION (c='Us', d=2) AS JSON struct -- !query output == Physical Plan == -Execute DescribeTableJsonCommand - +- DescribeTableJsonCommand `spark_catalog`.`default`.`t`, [c=Us, d=2], true +Execute DescribeRelationJsonCommand + +- DescribeRelationJsonCommand [c=Us, d=2], true, [json_metadata#x] + +- ResolvedTable V2SessionCatalog(spark_catalog), default.t, V1Table(default.t), [a#x, b#x, c#x, d#x] -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala index d81f007e2a4d3..f8174d24c9499 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedTableOrView} -import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumn, DescribeRelation} +import org.apache.spark.sql.test.SharedSparkSession + +class DescribeTableParserSuite extends SharedSparkSession with AnalysisTest { + private def parsePlan(statement: String) = spark.sessionState.sqlParser.parsePlan(statement) -class DescribeTableParserSuite extends AnalysisTest { test("SPARK-17328: Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeRelation( @@ -92,4 +94,17 @@ class DescribeTableParserSuite extends AnalysisTest { start = 0, stop = 47)) } + + test("retain sql text position") { + val tbl = "unknown" + val sqlStatement = s"DESCRIBE TABLE $tbl" + val startPos = sqlStatement.indexOf(tbl) + assert(startPos != -1) + assertAnalysisErrorCondition( + parsePlan(sqlStatement), + "TABLE_OR_VIEW_NOT_FOUND", + Map("relationName" -> s"`$tbl`"), + Array(ExpectedContext(tbl, startPos, startPos + tbl.length - 1)) + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 541fec1cb3740..2cc203129817b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -31,8 +31,8 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AnalysisTest, An import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog, TempVariableManager} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, Literal, StringLiteral} import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke -import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} -import org.apache.spark.sql.catalyst.plans.logical.{AlterColumn, AnalysisOnlyCommand, AppendData, Assignment, CreateTable, CreateTableAsSelect, DeleteAction, DeleteFromTable, DescribeRelation, DescribeRelationJson, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, OverwriteByExpression, OverwritePartitionsDynamic, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.{AlterColumn, AnalysisOnlyCommand, AppendData, Assignment, CreateTable, CreateTableAsSelect, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, OverwriteByExpression, OverwritePartitionsDynamic, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.connector.FakeV2Provider @@ -45,11 +45,12 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} import org.apache.spark.sql.sources.SimpleScanSource +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BooleanType, CharType, DoubleType, IntegerType, LongType, StringType, StructField, StructType, VarcharType} import org.apache.spark.unsafe.types.UTF8String -class PlanResolutionSuite extends AnalysisTest { - import CatalystSqlParser._ +class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { + private def parsePlan(statement: String) = spark.sessionState.sqlParser.parsePlan(statement) private val v1Format = classOf[SimpleScanSource].getName private val v2Format = classOf[FakeV2Provider].getName @@ -240,7 +241,7 @@ class PlanResolutionSuite extends AnalysisTest { } // We don't check analysis here by default, as we expect the plan to be unresolved // such as `CreateTable`. - val analyzed = analyzer.execute(CatalystSqlParser.parsePlan(query)) + val analyzed = analyzer.execute(parsePlan(query)) if (checkAnalysis) { analyzer.checkAnalysis(analyzed) } @@ -961,43 +962,6 @@ class PlanResolutionSuite extends AnalysisTest { assert(parsed4.isInstanceOf[DescribeTableCommand]) } - test("DESCRIBE AS JSON relation") { - Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { - case (tblName, useV1Command) => - val sql = s"DESC TABLE EXTENDED $tblName AS JSON" - val parsed = parseAndResolve(sql) - if (useV1Command) { - val expected2 = DescribeTableJsonCommand( - TableIdentifier(tblName, Some("default"), Some(SESSION_CATALOG_NAME)), - Map.empty, true) - - comparePlans(parsed, expected2) - } else { - parsed match { - case DescribeRelationJson(_: ResolvedTable, _, isExtended) => - assert(isExtended) - case _ => fail("Expect DescribeTable, but got:\n" + parsed.treeString) - } - } - - val sql2 = s"DESC TABLE EXTENDED $tblName PARTITION(a=1) AS JSON" - val parsed2 = parseAndResolve(sql2) - if (useV1Command) { - val expected2 = DescribeTableJsonCommand( - TableIdentifier(tblName, Some("default"), Some(SESSION_CATALOG_NAME)), - Map("a" -> "1"), true) - comparePlans(parsed2, expected2) - } else { - parsed2 match { - case DescribeRelationJson(_: ResolvedTable, partitionSpec, isExtended) => - assert(isExtended) - assert(partitionSpec == Map("a" -> "1")) - case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) - } - } - } - } - test("DELETE FROM") { Seq("v2Table", "testcat.tab").foreach { tblName => val sql1 = s"DELETE FROM $tblName" @@ -2904,9 +2868,8 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[ParseException] { parsePlan(query) }, - condition = "_LEGACY_ERROR_TEMP_0035", - parameters = Map( - "message" -> "CREATE TEMPORARY TABLE ..., use CREATE TEMPORARY VIEW instead"), + condition = "_LEGACY_ERROR_TEMP_0046", + parameters = Map(), context = ExpectedContext(fragment = query, start = 0, stop = 48)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index cae56754ba465..3602853e53aa8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -210,108 +210,6 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase Row("histogram", "NULL"))) } } -} - -/** - * The class contains tests for the `DESCRIBE TABLE` command to check V1 In-Memory - * table catalog. - */ -class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { - override def commandVersion: String = super[DescribeTableSuiteBase].commandVersion - - test("DESCRIBE TABLE EXTENDED of a partitioned table") { - withNamespaceAndTable("ns", "table") { tbl => - spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing" + - " PARTITIONED BY (id)" + - " TBLPROPERTIES ('bar'='baz')" + - " COMMENT 'this is a test table'" + - " DEFAULT COLLATION unicode" + - " LOCATION 'file:/tmp/testcat/table_name'") - val descriptionDf = spark.sql(s"DESCRIBE TABLE EXTENDED $tbl") - assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( - ("col_name", StringType), - ("data_type", StringType), - ("comment", StringType))) - QueryTest.checkAnswer( - descriptionDf.filter("!(col_name in ('Created Time', 'Created By'))"), - Seq( - Row("data", "string", null), - Row("id", "bigint", null), - Row("# Partition Information", "", ""), - Row("# col_name", "data_type", "comment"), - Row("id", "bigint", null), - Row("", "", ""), - Row("# Detailed Table Information", "", ""), - Row("Catalog", SESSION_CATALOG_NAME, ""), - Row("Database", "ns", ""), - Row("Table", "table", ""), - Row("Last Access", "UNKNOWN", ""), - Row("Type", "EXTERNAL", ""), - Row("Provider", getProvider(), ""), - Row("Comment", "this is a test table", ""), - Row("Collation", "UNICODE", ""), - Row("Table Properties", "[bar=baz]", ""), - Row("Location", "file:/tmp/testcat/table_name", ""), - Row("Partition Provider", "Catalog", ""))) - } - } - - test("DESCRIBE TABLE EXTENDED of a table with a default column value") { - withTable("t") { - spark.sql(s"CREATE TABLE t (id bigint default 42) $defaultUsing") - val descriptionDf = spark.sql(s"DESCRIBE TABLE EXTENDED t") - assert(descriptionDf.schema.map { field => - (field.name, field.dataType) - } === Seq( - ("col_name", StringType), - ("data_type", StringType), - ("comment", StringType))) - QueryTest.checkAnswer( - descriptionDf.filter( - "!(col_name in ('Created Time', 'Created By', 'Database', 'Location', " + - "'Provider', 'Type'))"), - Seq( - Row("id", "bigint", null), - Row("", "", ""), - Row("# Detailed Table Information", "", ""), - Row("Catalog", SESSION_CATALOG_NAME, ""), - Row("Table", "t", ""), - Row("Last Access", "UNKNOWN", ""), - Row("", "", ""), - Row("# Column Default Values", "", ""), - Row("id", "bigint", "42") - )) - } - } - - test("DESCRIBE AS JSON throws when not EXTENDED") { - withNamespaceAndTable("ns", "table") { t => - val tableCreationStr = - s""" - |CREATE TABLE $t ( - | employee_id INT, - | employee_name STRING, - | department STRING, - | hire_date DATE - |) USING parquet - |OPTIONS ('compression' = 'snappy', 'max_records' = '1000') - |PARTITIONED BY (department, hire_date) - |CLUSTERED BY (employee_id) SORTED BY (employee_name ASC) INTO 4 BUCKETS - |COMMENT 'Employee data table for testing partitions and buckets' - |TBLPROPERTIES ('version' = '1.0') - |""".stripMargin - spark.sql(tableCreationStr) - - val error = intercept[AnalysisException] { - spark.sql(s"DESCRIBE $t AS JSON") - } - - checkError( - exception = error, - condition = "DESCRIBE_JSON_NOT_EXTENDED", - parameters = Map("tableName" -> "table")) - } - } test("DESCRIBE AS JSON partitions, clusters, buckets") { withNamespaceAndTable("ns", "table") { t => @@ -356,10 +254,11 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { table_properties = Some(Map( "version" -> "1.0" )), - location = Some(""), - serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), - inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + serde_library = if (getProvider() == "hive") { + Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe") + } else { + None + }, storage_properties = Some(Map( "compression" -> "snappy", "max_records" -> "1000" @@ -368,17 +267,9 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { partition_columns = Some(List("department", "hire_date")) ) - if (getProvider() == "hive") { - assert(expectedOutput == parsedOutput.copy(owner = None, - created_time = None, - location = Some(""))) - } else { - assert(expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) - == parsedOutput.copy(owner = None, created_time = None, location = Some(""))) - } - parsedOutput.created_time.foreach { createdTime => - assert(iso8601Regex.matches(createdTime)) - } + assert(parsedOutput.location.isDefined) + assert(iso8601Regex.matches(parsedOutput.created_time.get)) + assert(expectedOutput == parsedOutput.copy(location = None, created_time = None)) } } @@ -426,33 +317,20 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { table_properties = Some(Map( "t" -> "test" )), - serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), - inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), - storage_properties = Some(Map( - "serialization.format" -> "1" - )), + serde_library = if (getProvider() == "hive") { + Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe") + } else { + None + }, partition_provider = Some("Catalog"), partition_columns = Some(List("region", "category")), partition_values = Some(Map("region" -> "USA", "category" -> "tech")) ) - val filteredParsedStorageProperties = - parsedOutput.storage_properties.map(_.filterNot { case (key, _) => key == "path" }) - - if (getProvider() == "hive") { - assert(expectedOutput == - parsedOutput.copy(location = None, created_time = None, owner = None, - storage_properties = filteredParsedStorageProperties)) - } else { - assert(expectedOutput.copy( - inputformat = None, outputformat = None, serde_library = None, storage_properties = None) - == parsedOutput.copy(location = None, created_time = None, owner = None, - storage_properties = filteredParsedStorageProperties)) - } - parsedOutput.created_time.foreach { createdTime => - assert(iso8601Regex.matches(createdTime)) - } + assert(parsedOutput.location.isDefined) + assert(iso8601Regex.matches(parsedOutput.created_time.get)) + assert(expectedOutput == parsedOutput.copy( + location = None, created_time = None, storage_properties = None)) } } @@ -494,113 +372,67 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { bucket_columns = Some(Nil), sort_columns = Some(Nil), comment = Some("table_comment"), - serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), - inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + serde_library = if (getProvider() == "hive") { + Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe") + } else { + None + }, table_properties = None ) - if (getProvider() == "hive") { - assert( - expectedOutput == - parsedOutput.copy(location = None, created_time = None, owner = None) - ) - } else { - assert( - expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) == - parsedOutput.copy(location = None, created_time = None, owner = None) - ) - } - parsedOutput.created_time.foreach { createdTime => - assert(iso8601Regex.matches(createdTime)) - } - } - } - - test("DESCRIBE AS JSON temp view") { - withNamespaceAndTable("ns", "table") { t => - withTempView("temp_view") { - val tableCreationStr = - s""" - |CREATE TABLE $t (id INT, name STRING, created_at TIMESTAMP) - | USING parquet - | OPTIONS ('compression' 'snappy') - | CLUSTERED BY (id, name) SORTED BY (created_at) INTO 4 BUCKETS - | COMMENT 'test temp view' - | TBLPROPERTIES ('parquet.encryption' = 'true') - |""".stripMargin - spark.sql(tableCreationStr) - spark.sql(s"CREATE TEMPORARY VIEW temp_view AS SELECT * FROM $t") - val descriptionDf = spark.sql(s"DESCRIBE EXTENDED temp_view AS JSON") - val firstRow = descriptionDf.select("json_metadata").head() - val jsonValue = firstRow.getString(0) - val parsedOutput = parse(jsonValue).extract[DescribeTableJson] - - val expectedOutput = DescribeTableJson( - columns = Some(List( - TableColumn("id", Type("int")), - TableColumn("name", Type("string")), - TableColumn("created_at", Type("timestamp_ltz")) - )) - ) - - assert(expectedOutput == parsedOutput) - } + assert(parsedOutput.location.isDefined) + assert(iso8601Regex.matches(parsedOutput.created_time.get)) + assert(expectedOutput == parsedOutput.copy(location = None, created_time = None)) } } - test("DESCRIBE AS JSON persistent view") { - withNamespaceAndTable("ns", "table") { t => - withView("view") { - val tableCreationStr = - s""" - |CREATE TABLE $t (id INT, name STRING, created_at TIMESTAMP) - | USING parquet - | OPTIONS ('compression' 'snappy') - | CLUSTERED BY (id, name) SORTED BY (created_at) INTO 4 BUCKETS - | COMMENT 'test temp view' - | TBLPROPERTIES ('parquet.encryption' = 'true') - |""".stripMargin - spark.sql(tableCreationStr) - spark.sql(s"CREATE VIEW view AS SELECT * FROM $t") - val descriptionDf = spark.sql(s"DESCRIBE EXTENDED view AS JSON") - val firstRow = descriptionDf.select("json_metadata").head() - val jsonValue = firstRow.getString(0) - val parsedOutput = parse(jsonValue).extract[DescribeTableJson] - - val expectedOutput = DescribeTableJson( - table_name = Some("view"), - catalog_name = Some("spark_catalog"), - namespace = Some(List("default")), - schema_name = Some("default"), - columns = Some(List( - TableColumn("id", Type("int")), - TableColumn("name", Type("string")), - TableColumn("created_at", Type("timestamp_ltz")) - )), - serde_library = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"), - inputformat = Some("org.apache.hadoop.mapred.SequenceFileInputFormat"), - outputformat = Some("org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"), - storage_properties = Some(Map("serialization.format" -> "1")), - last_access = Some("UNKNOWN"), - created_by = Some(s"Spark $SPARK_VERSION"), - `type` = Some("VIEW"), - view_text = Some("SELECT * FROM spark_catalog.ns.table"), - view_original_text = Some("SELECT * FROM spark_catalog.ns.table"), - view_schema_mode = Some("COMPENSATION"), - view_catalog_and_namespace = Some("spark_catalog.default"), - view_query_output_columns = Some(List("id", "name", "created_at")) - ) + test("DESCRIBE AS JSON view") { + Seq(true, false).foreach { isTemp => + withNamespaceAndTable("ns", "table") { t => + withView("view") { + val tableCreationStr = + s""" + |CREATE TABLE $t (id INT, name STRING, created_at TIMESTAMP) + | USING parquet + | OPTIONS ('compression' 'snappy') + | CLUSTERED BY (id, name) SORTED BY (created_at) INTO 4 BUCKETS + | COMMENT 'test temp view' + | TBLPROPERTIES ('parquet.encryption' = 'true') + |""".stripMargin + spark.sql(tableCreationStr) + val viewType = if (isTemp) "TEMP VIEW" else "VIEW" + spark.sql(s"CREATE $viewType view AS SELECT * FROM $t") + val descriptionDf = spark.sql(s"DESCRIBE EXTENDED view AS JSON") + val firstRow = descriptionDf.select("json_metadata").head() + val jsonValue = firstRow.getString(0) + val parsedOutput = parse(jsonValue).extract[DescribeTableJson] + + val expectedOutput = DescribeTableJson( + table_name = Some("view"), + catalog_name = if (isTemp) Some("system") else Some("spark_catalog"), + namespace = if (isTemp) Some(List("session")) else Some(List("default")), + schema_name = if (isTemp) Some("session") else Some("default"), + columns = Some(List( + TableColumn("id", Type("int")), + TableColumn("name", Type("string")), + TableColumn("created_at", Type("timestamp_ltz")) + )), + last_access = Some("UNKNOWN"), + created_by = Some(s"Spark $SPARK_VERSION"), + `type` = Some("VIEW"), + view_text = Some("SELECT * FROM spark_catalog.ns.table"), + view_original_text = if (isTemp) None else Some("SELECT * FROM spark_catalog.ns.table"), + // TODO: this is unexpected and temp view should also use COMPENSATION mode. + view_schema_mode = if (isTemp) Some("BINDING") else Some("COMPENSATION"), + view_catalog_and_namespace = Some("spark_catalog.default"), + view_query_output_columns = Some(List("id", "name", "created_at")) + ) - if (getProvider() == "hive") { - assert(expectedOutput == - parsedOutput.copy(table_properties = None, created_time = None, owner = None)) - } else { - assert(expectedOutput.copy(inputformat = None, - outputformat = None, serde_library = None, storage_properties = None) - == parsedOutput.copy(table_properties = None, created_time = None, owner = None)) - } - parsedOutput.created_time.foreach { createdTime => - assert(iso8601Regex.matches(createdTime)) + assert(iso8601Regex.matches(parsedOutput.created_time.get)) + assert(expectedOutput == parsedOutput.copy( + created_time = None, + table_properties = None, + storage_properties = None, + serde_library = None)) } } } @@ -755,9 +587,11 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { default = None ) )), - serde_library = Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"), - inputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputformat = Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + serde_library = if (getProvider() == "hive") { + Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") + } else { + None + }, storage_properties = Some(Map( "option1" -> "value1", "option2" -> "value2" @@ -775,16 +609,82 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { partition_columns = Some(List("id")) ) - if (getProvider() == "hive") { - assert(expectedOutput == - parsedOutput.copy(location = None, created_time = None, owner = None)) - } else { - assert(expectedOutput.copy(inputformat = None, outputformat = None, serde_library = None) - == parsedOutput.copy(location = None, created_time = None, owner = None)) - } - parsedOutput.created_time.foreach { createdTime => - assert(iso8601Regex.matches(createdTime)) - } + assert(parsedOutput.location.isDefined) + assert(iso8601Regex.matches(parsedOutput.created_time.get)) + assert(expectedOutput == parsedOutput.copy(location = None, created_time = None)) + } + } +} + +/** + * The class contains tests for the `DESCRIBE TABLE` command to check V1 In-Memory + * table catalog. + */ +class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { + override def commandVersion: String = super[DescribeTableSuiteBase].commandVersion + + test("DESCRIBE TABLE EXTENDED of a partitioned table") { + withNamespaceAndTable("ns", "table") { tbl => + spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing" + + " PARTITIONED BY (id)" + + " TBLPROPERTIES ('bar'='baz')" + + " COMMENT 'this is a test table'" + + " DEFAULT COLLATION unicode" + + " LOCATION 'file:/tmp/testcat/table_name'") + val descriptionDf = spark.sql(s"DESCRIBE TABLE EXTENDED $tbl") + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( + ("col_name", StringType), + ("data_type", StringType), + ("comment", StringType))) + QueryTest.checkAnswer( + descriptionDf.filter("!(col_name in ('Created Time', 'Created By'))"), + Seq( + Row("data", "string", null), + Row("id", "bigint", null), + Row("# Partition Information", "", ""), + Row("# col_name", "data_type", "comment"), + Row("id", "bigint", null), + Row("", "", ""), + Row("# Detailed Table Information", "", ""), + Row("Catalog", SESSION_CATALOG_NAME, ""), + Row("Database", "ns", ""), + Row("Table", "table", ""), + Row("Last Access", "UNKNOWN", ""), + Row("Type", "EXTERNAL", ""), + Row("Provider", getProvider(), ""), + Row("Comment", "this is a test table", ""), + Row("Collation", "UNICODE", ""), + Row("Table Properties", "[bar=baz]", ""), + Row("Location", "file:/tmp/testcat/table_name", ""), + Row("Partition Provider", "Catalog", ""))) + } + } + + test("DESCRIBE TABLE EXTENDED of a table with a default column value") { + withTable("t") { + spark.sql(s"CREATE TABLE t (id bigint default 42) $defaultUsing") + val descriptionDf = spark.sql(s"DESCRIBE TABLE EXTENDED t") + assert(descriptionDf.schema.map { field => + (field.name, field.dataType) + } === Seq( + ("col_name", StringType), + ("data_type", StringType), + ("comment", StringType))) + QueryTest.checkAnswer( + descriptionDf.filter( + "!(col_name in ('Created Time', 'Created By', 'Database', 'Location', " + + "'Provider', 'Type'))"), + Seq( + Row("id", "bigint", null), + Row("", "", ""), + Row("# Detailed Table Information", "", ""), + Row("Catalog", SESSION_CATALOG_NAME, ""), + Row("Table", "t", ""), + Row("Last Access", "UNKNOWN", ""), + Row("", "", ""), + Row("# Column Default Values", "", ""), + Row("id", "bigint", "42") + )) } } } @@ -796,7 +696,6 @@ case class DescribeTableJson( namespace: Option[List[String]] = Some(Nil), schema_name: Option[String] = None, columns: Option[List[TableColumn]] = Some(Nil), - owner: Option[String] = None, created_time: Option[String] = None, last_access: Option[String] = None, created_by: Option[String] = None, @@ -808,8 +707,6 @@ case class DescribeTableJson( table_properties: Option[Map[String, String]] = None, location: Option[String] = None, serde_library: Option[String] = None, - inputformat: Option[String] = None, - outputformat: Option[String] = None, storage_properties: Option[Map[String, String]] = None, partition_provider: Option[String] = None, partition_columns: Option[List[String]] = Some(Nil), From 7e1248ab770b8cf8b4b9929f551c8599d467a462 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 15 Jan 2025 15:52:56 +0900 Subject: [PATCH 434/438] [SPARK-50824][PYTHON] Avoid importing optional Python packages for checking ### What changes were proposed in this pull request? This PR proposes to avoid importing optional Python packages for checking, by using `importlib.util.find_spec` instead of actually loading/importing the package. ### Why are the changes needed? https://github.com/apache/spark/commit/a40919912f5ce7f63fff2907b30e473dd4155227 changed to import optional dependencies in main code. After that, technically https://github.com/apache/spark/commit/f223b8da9e23e4e028e145e0d4dd74eeae5d2d52 broke the Python Spark Core tests, (because now we will import `pyspark.testing`, and it will import optional dependencies) but it did not run the tests. By importing `deepspeed`, via logger, it can show stdout (https://github.com/microsoft/DeepSpeed/blob/master/accelerator/real_accelerator.py#L182). This broke the test in `pyspark.conf`. After that, the real test failure was found when core change was triggered at https://github.com/apache/spark/commit/6f3b778e1a12901726c2a35072904f36f46f7888. In the PR, build passed because it was before https://github.com/apache/spark/commit/f223b8da9e23e4e028e145e0d4dd74eeae5d2d52 was merged. ### Does this PR introduce _any_ user-facing change? Technically yes. There might be some side effects by importing optional dependencies, and this PR avoid them. ### How was this patch tested? CI in this PR should verify it. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49500 from HyukjinKwon/SPARK-50824. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/testing/utils.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index 233b432766b75..76f5b48ff9bb0 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -52,13 +52,9 @@ def have_package(name: str) -> bool: - try: - import importlib + import importlib - importlib.import_module(name) - return True - except Exception: - return False + return importlib.util.find_spec(name) is not None have_numpy = have_package("numpy") From 15173157c33e54ff2c390710c252aedcb36d4c38 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 14 Jan 2025 23:06:44 -0800 Subject: [PATCH 435/438] [SPARK-49907][ML][CONNECT][FOLLOWUP] Remove unused `expressions.proto` import ### What changes were proposed in this pull request? This is a follow-up to remove unused `expressions.proto` import statement from `ml.proto`. - #48791 ### Why are the changes needed? This is a clean up for 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 #49497 from dongjoon-hyun/SPARK-49907. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/connect/proto/ml_pb2.py | 35 +++++++++---------- .../src/main/protobuf/spark/connect/ml.proto | 1 - 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py index 46692010e3ffc..5005f82d5d533 100644 --- a/python/pyspark/sql/connect/proto/ml_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_pb2.py @@ -34,13 +34,12 @@ _sym_db = _symbol_database.Default() -from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 from pyspark.sql.connect.proto import ml_common_pb2 as spark_dot_connect_dot_ml__common__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x1dspark/connect/ml_common.proto"\xc6\x07\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x1a\xa2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12/\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61taset\x1a;\n\x06\x44\x65lete\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x1a\xf0\x02\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12/\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12)\n\x10should_overwrite\x18\x05 \x01(\x08R\x0fshouldOverwrite\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x06\n\x04type\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04pathB\t\n\x07\x63ommand"\xe9\x02\n\x0fMlCommandResult\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\xa6\x01\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x10\n\x03uid\x18\x03 \x01(\tR\x03uid\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\x06\n\x04typeB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1dspark/connect/relations.proto\x1a\x1dspark/connect/ml_common.proto"\xc6\x07\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x1a\xa2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12/\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61taset\x1a;\n\x06\x44\x65lete\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x1a\xf0\x02\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12/\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12)\n\x10should_overwrite\x18\x05 \x01(\x08R\x0fshouldOverwrite\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x06\n\x04type\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04pathB\t\n\x07\x63ommand"\xe9\x02\n\x0fMlCommandResult\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\xa6\x01\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x10\n\x03uid\x18\x03 \x01(\tR\x03uid\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\x06\n\x04typeB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -53,20 +52,20 @@ ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._loaded_options = None _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_options = b"8\001" - _globals["_MLCOMMAND"]._serialized_start = 137 - _globals["_MLCOMMAND"]._serialized_end = 1103 - _globals["_MLCOMMAND_FIT"]._serialized_start = 415 - _globals["_MLCOMMAND_FIT"]._serialized_end = 577 - _globals["_MLCOMMAND_DELETE"]._serialized_start = 579 - _globals["_MLCOMMAND_DELETE"]._serialized_end = 638 - _globals["_MLCOMMAND_WRITE"]._serialized_start = 641 - _globals["_MLCOMMAND_WRITE"]._serialized_end = 1009 - _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 943 - _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 1001 - _globals["_MLCOMMAND_READ"]._serialized_start = 1011 - _globals["_MLCOMMAND_READ"]._serialized_end = 1092 - _globals["_MLCOMMANDRESULT"]._serialized_start = 1106 - _globals["_MLCOMMANDRESULT"]._serialized_end = 1467 - _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 1286 - _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 1452 + _globals["_MLCOMMAND"]._serialized_start = 104 + _globals["_MLCOMMAND"]._serialized_end = 1070 + _globals["_MLCOMMAND_FIT"]._serialized_start = 382 + _globals["_MLCOMMAND_FIT"]._serialized_end = 544 + _globals["_MLCOMMAND_DELETE"]._serialized_start = 546 + _globals["_MLCOMMAND_DELETE"]._serialized_end = 605 + _globals["_MLCOMMAND_WRITE"]._serialized_start = 608 + _globals["_MLCOMMAND_WRITE"]._serialized_end = 976 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 910 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 968 + _globals["_MLCOMMAND_READ"]._serialized_start = 978 + _globals["_MLCOMMAND_READ"]._serialized_end = 1059 + _globals["_MLCOMMANDRESULT"]._serialized_start = 1073 + _globals["_MLCOMMANDRESULT"]._serialized_end = 1434 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 1253 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 1419 # @@protoc_insertion_point(module_scope) diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto index 591004b000daa..48b04a6e14cd0 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto @@ -19,7 +19,6 @@ syntax = 'proto3'; package spark.connect; -import "spark/connect/expressions.proto"; import "spark/connect/relations.proto"; import "spark/connect/ml_common.proto"; From 47d831ea58dce53f9e3a6547ccb2e5f30beaf582 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 14 Jan 2025 23:54:11 -0800 Subject: [PATCH 436/438] [SPARK-50810][BUILD] Enable SBT CI for profiler module ### What changes were proposed in this pull request? Update GitHub workflow files and related scripts to enable SBT CI for the `profiler` module. ### Why are the changes needed? Currently, the `profiler` module is not covered in regular PR's SBT CI. ### Does this PR introduce _any_ user-facing change? No, dev only. ### How was this patch tested? https://github.com/pan3793/spark/actions/runs/12770426351/job/35595345586 ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49480 from pan3793/SPARK-50810. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 2 +- connector/profiler/README.md | 9 ++++++++- dev/sparktestsupport/modules.py | 9 +++++++++ project/SparkBuild.scala | 10 +++++----- 4 files changed, 23 insertions(+), 7 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 8040169fcb2c8..ef11c8416b0ae 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -242,7 +242,7 @@ jobs: - >- api, catalyst, hive-thriftserver - >- - mllib-local, mllib, graphx + mllib-local, mllib, graphx, profiler - >- streaming, sql-kafka-0-10, streaming-kafka-0-10, streaming-kinesis-asl, kubernetes, hadoop-cloud, spark-ganglia-lgpl, protobuf, connect diff --git a/connector/profiler/README.md b/connector/profiler/README.md index 4d97b15eb96ab..30d897f21b065 100644 --- a/connector/profiler/README.md +++ b/connector/profiler/README.md @@ -3,8 +3,15 @@ ## Build To build + +``` +./build/mvn clean package -DskipTests -Pjvm-profiler -pl :spark-profiler_2.13 -am +``` + +or + ``` - ./build/mvn clean package -DskipTests -Pjvm-profiler +./build/sbt -Pjvm-profiler clean "profiler/package" ``` ## Executor Code Profiling diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index d43feceb76648..f785a72e6a1fe 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -309,6 +309,15 @@ def __hash__(self): ], ) +profiler = Module( + name="profiler", + dependencies=[], + build_profile_flags=["-Pjvm-profiler"], + source_file_regexes=[ + "connector/profiler", + ], +) + protobuf = Module( name="protobuf", dependencies=[sql], diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4cf512c0f3cd3..d84c0f17d2b2b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -65,10 +65,10 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects ++ connectProjects val optionallyEnabledProjects@Seq(kubernetes, yarn, - sparkGangliaLgpl, streamingKinesisAsl, + sparkGangliaLgpl, streamingKinesisAsl, profiler, dockerIntegrationTests, hadoopCloud, kubernetesIntegrationTests) = Seq("kubernetes", "yarn", - "ganglia-lgpl", "streaming-kinesis-asl", + "ganglia-lgpl", "streaming-kinesis-asl", "profiler", "docker-integration-tests", "hadoop-cloud", "kubernetes-integration-tests").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(networkYarn, streamingKafka010Assembly, streamingKinesisAslAssembly) = @@ -371,7 +371,7 @@ object SparkBuild extends PomBuild { Seq( spark, hive, hiveThriftServer, repl, networkCommon, networkShuffle, networkYarn, unsafe, tags, tokenProviderKafka010, sqlKafka010, connectCommon, connect, connectClient, - variant, connectShims + variant, connectShims, profiler ).contains(x) } @@ -1469,11 +1469,11 @@ object SparkUnidoc extends SharedUnidocSettings { (ScalaUnidoc / unidoc / unidocProjectFilter) := inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, yarn, tags, streamingKafka010, sqlKafka010, connectCommon, connect, connectClient, - connectShims, protobuf), + connectShims, protobuf, profiler), (JavaUnidoc / unidoc / unidocProjectFilter) := inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, yarn, tags, streamingKafka010, sqlKafka010, connectCommon, connect, connectClient, - connectShims, protobuf), + connectShims, protobuf, profiler), ) } From 21a37a79df972b4140cd8c7e8c27abb421e32a0d Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 15 Jan 2025 10:54:48 +0300 Subject: [PATCH 437/438] [SPARK-50814][DOCS] Remove unused SQL error pages ### What changes were proposed in this pull request? Remove standalone SQL error pages that were made obsolete by the work completed in #44971. Also fix the formatting of the error message for `QUERY_ONLY_CORRUPT_RECORD_COLUMN`, since it was incorrect and overflowing the table cell it belongs to. ### Why are the changes needed? These error pages are either already captured completely in `common/utils/src/main/resources/error/error-conditions.json`, or are obsolete and not needed (and are not being rendered in the documentation output anyway). The formatting of `QUERY_ONLY_CORRUPT_RECORD_COLUMN` before and after: ### Does this PR introduce _any_ user-facing change? Yes, documentation formatting. ### How was this patch tested? Built the docs locally and reviewed them in my browser. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49486 from nchammas/SPARK-50814-unused-error-docs. Authored-by: Nicholas Chammas Signed-off-by: Max Gekk --- .../resources/error/error-conditions.json | 10 +-- ...ditions-codec-not-available-error-class.md | 41 ---------- ...nditions-collation-mismatch-error-class.md | 41 ---------- ...conditions-failed-read-file-error-class.md | 52 ------------ ...s-illegal-state-store-value-error-class.md | 41 ---------- ...ns-invalid-aggregate-filter-error-class.md | 49 ----------- ...nditions-invalid-conf-value-error-class.md | 41 ---------- ...ns-invalid-datetime-pattern-error-class.md | 41 ---------- ...ons-invalid-delimiter-value-error-class.md | 49 ----------- ...ons-invalid-interval-format-error-class.md | 81 ------------------- ...-numeric-value-out-of-range-error-class.md | 41 ---------- ...ditions-syntax-discontinued-error-class.md | 39 --------- ...conditions-unsupported-call-error-class.md | 41 ---------- ...tions-unsupported-collation-error-class.md | 37 --------- 14 files changed, 5 insertions(+), 599 deletions(-) delete mode 100644 docs/sql-error-conditions-codec-not-available-error-class.md delete mode 100644 docs/sql-error-conditions-collation-mismatch-error-class.md delete mode 100644 docs/sql-error-conditions-failed-read-file-error-class.md delete mode 100644 docs/sql-error-conditions-illegal-state-store-value-error-class.md delete mode 100644 docs/sql-error-conditions-invalid-aggregate-filter-error-class.md delete mode 100644 docs/sql-error-conditions-invalid-conf-value-error-class.md delete mode 100644 docs/sql-error-conditions-invalid-datetime-pattern-error-class.md delete mode 100644 docs/sql-error-conditions-invalid-delimiter-value-error-class.md delete mode 100644 docs/sql-error-conditions-invalid-interval-format-error-class.md delete mode 100644 docs/sql-error-conditions-numeric-value-out-of-range-error-class.md delete mode 100644 docs/sql-error-conditions-syntax-discontinued-error-class.md delete mode 100644 docs/sql-error-conditions-unsupported-call-error-class.md delete mode 100644 docs/sql-error-conditions-unsupported-collation-error-class.md diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 71eae4de89c8a..44d69b6675937 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5485,12 +5485,12 @@ "message" : [ "Queries from raw JSON/CSV/XML files are disallowed when the", "referenced columns only include the internal corrupt record column", - "(named _corrupt_record by default). For example:", - "spark.read.schema(schema).json(file).filter($\"_corrupt_record\".isNotNull).count()", - "and spark.read.schema(schema).json(file).select(\"_corrupt_record\").show().", + "(named `_corrupt_record` by default). For example:", + "`spark.read.schema(schema).json(file).filter($\"_corrupt_record\".isNotNull).count()`", + "and `spark.read.schema(schema).json(file).select(\"_corrupt_record\").show()`.", "Instead, you can cache or save the parsed results and then send the same query.", - "For example, val df = spark.read.schema(schema).json(file).cache() and then", - "df.filter($\"_corrupt_record\".isNotNull).count()." + "For example, `val df = spark.read.schema(schema).json(file).cache()` and then", + "`df.filter($\"_corrupt_record\".isNotNull).count()`." ] }, "REMOVE_NAMESPACE_COMMENT" : { diff --git a/docs/sql-error-conditions-codec-not-available-error-class.md b/docs/sql-error-conditions-codec-not-available-error-class.md deleted file mode 100644 index bb93f56206ba1..0000000000000 --- a/docs/sql-error-conditions-codec-not-available-error-class.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: global -title: CODEC_NOT_AVAILABLE error class -displayTitle: CODEC_NOT_AVAILABLE error class -license: | - 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. ---- - - - -SQLSTATE: 56038 - -The codec `` is not available. - -This error class has the following derived error classes: - -## WITH_AVAILABLE_CODECS_SUGGESTION - -Available codecs are ``. - -## WITH_CONF_SUGGESTION - -Consider to set the config `` to ``. - - diff --git a/docs/sql-error-conditions-collation-mismatch-error-class.md b/docs/sql-error-conditions-collation-mismatch-error-class.md deleted file mode 100644 index 79aaaf00ee47c..0000000000000 --- a/docs/sql-error-conditions-collation-mismatch-error-class.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: global -title: COLLATION_MISMATCH error class -displayTitle: COLLATION_MISMATCH error class -license: | - 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. ---- - - - -[SQLSTATE: 42P21](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) - -Could not determine which collation to use for string functions and operators. - -This error class has the following derived error classes: - -## EXPLICIT - -Error occurred due to the mismatch between explicit collations: ``. Decide on a single explicit collation and remove others. - -## IMPLICIT - -Error occurred due to the mismatch between implicit collations: ``. Use COLLATE function to set the collation explicitly. - - diff --git a/docs/sql-error-conditions-failed-read-file-error-class.md b/docs/sql-error-conditions-failed-read-file-error-class.md deleted file mode 100644 index a4344666c59c6..0000000000000 --- a/docs/sql-error-conditions-failed-read-file-error-class.md +++ /dev/null @@ -1,52 +0,0 @@ ---- -layout: global -title: FAILED_READ_FILE error class -displayTitle: FAILED_READ_FILE error class -license: | - 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. ---- - - - -SQLSTATE: KD001 - -Encountered error while reading file ``. - -This error class has the following derived error classes: - -## CANNOT_READ_FILE_FOOTER - -Could not read footer. Please ensure that the file is in either ORC or Parquet format. -If not, please convert it to a valid format. If the file is in the valid format, please check if it is corrupt. -If it is, you can choose to either ignore it or fix the corruption. - -## FILE_NOT_EXIST - -File does not exist. It is possible the underlying files have been updated. -You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved. - -## NO_HINT - - - -## PARQUET_COLUMN_DATA_TYPE_MISMATCH - -Data type mismatches when reading Parquet column ``. Expected Spark type ``, actual Parquet type ``. - - diff --git a/docs/sql-error-conditions-illegal-state-store-value-error-class.md b/docs/sql-error-conditions-illegal-state-store-value-error-class.md deleted file mode 100644 index e6457e58b7b4d..0000000000000 --- a/docs/sql-error-conditions-illegal-state-store-value-error-class.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: global -title: ILLEGAL_STATE_STORE_VALUE error class -displayTitle: ILLEGAL_STATE_STORE_VALUE error class -license: | - 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. ---- - - - -[SQLSTATE: 42601](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) - -Illegal value provided to the State Store - -This error class has the following derived error classes: - -## EMPTY_LIST_VALUE - -Cannot write empty list values to State Store for StateName ``. - -## NULL_VALUE - -Cannot write null values to State Store for StateName ``. - - diff --git a/docs/sql-error-conditions-invalid-aggregate-filter-error-class.md b/docs/sql-error-conditions-invalid-aggregate-filter-error-class.md deleted file mode 100644 index 8a3441ca133d4..0000000000000 --- a/docs/sql-error-conditions-invalid-aggregate-filter-error-class.md +++ /dev/null @@ -1,49 +0,0 @@ ---- -layout: global -title: INVALID_AGGREGATE_FILTER error class -displayTitle: INVALID_AGGREGATE_FILTER error class -license: | - 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. ---- - - - -[SQLSTATE: 42903](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) - -The FILTER expression `` in an aggregate function is invalid. - -This error class has the following derived error classes: - -## CONTAINS_AGGREGATE - -Expected a FILTER expression without an aggregation, but found ``. - -## CONTAINS_WINDOW_FUNCTION - -Expected a FILTER expression without a window function, but found ``. - -## NON_DETERMINISTIC - -Expected a deterministic FILTER expression. - -## NOT_BOOLEAN - -Expected a FILTER expression of the BOOLEAN type. - - diff --git a/docs/sql-error-conditions-invalid-conf-value-error-class.md b/docs/sql-error-conditions-invalid-conf-value-error-class.md deleted file mode 100644 index ac430956340f8..0000000000000 --- a/docs/sql-error-conditions-invalid-conf-value-error-class.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: global -title: INVALID_CONF_VALUE error class -displayTitle: INVALID_CONF_VALUE error class -license: | - 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. ---- - - - -[SQLSTATE: 22022](sql-error-conditions-sqlstates.html#class-22-data-exception) - -The value '``' in the config "``" is invalid. - -This error class has the following derived error classes: - -## DEFAULT_COLLATION - -Cannot resolve the given default collation. Did you mean '``'? - -## TIME_ZONE - -Cannot resolve the given timezone. - - diff --git a/docs/sql-error-conditions-invalid-datetime-pattern-error-class.md b/docs/sql-error-conditions-invalid-datetime-pattern-error-class.md deleted file mode 100644 index 10e9fc97027c0..0000000000000 --- a/docs/sql-error-conditions-invalid-datetime-pattern-error-class.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: global -title: INVALID_DATETIME_PATTERN error class -displayTitle: INVALID_DATETIME_PATTERN error class -license: | - 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. ---- - - - -[SQLSTATE: 22007](sql-error-conditions-sqlstates.html#class-22-data-exception) - -Unrecognized datetime pattern: ``. - -This error class has the following derived error classes: - -## ILLEGAL_CHARACTER - -Illegal pattern character found in datetime pattern: ``. Please provide legal character. - -## LENGTH - -Too many letters in datetime pattern: ``. Please reduce pattern length. - - diff --git a/docs/sql-error-conditions-invalid-delimiter-value-error-class.md b/docs/sql-error-conditions-invalid-delimiter-value-error-class.md deleted file mode 100644 index 815fe78bce945..0000000000000 --- a/docs/sql-error-conditions-invalid-delimiter-value-error-class.md +++ /dev/null @@ -1,49 +0,0 @@ ---- -layout: global -title: INVALID_DELIMITER_VALUE error class -displayTitle: INVALID_DELIMITER_VALUE error class -license: | - 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. ---- - - - -[SQLSTATE: 42602](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) - -Invalid value for delimiter. - -This error class has the following derived error classes: - -## DELIMITER_LONGER_THAN_EXPECTED - -Delimiter cannot be more than one character: ``. - -## EMPTY_STRING - -Delimiter cannot be empty string. - -## SINGLE_BACKSLASH - -Single backslash is prohibited. It has special meaning as beginning of an escape sequence. To get the backslash character, pass a string with two backslashes as the delimiter. - -## UNSUPPORTED_SPECIAL_CHARACTER - -Unsupported special character for delimiter: ``. - - diff --git a/docs/sql-error-conditions-invalid-interval-format-error-class.md b/docs/sql-error-conditions-invalid-interval-format-error-class.md deleted file mode 100644 index 28cccd5e12887..0000000000000 --- a/docs/sql-error-conditions-invalid-interval-format-error-class.md +++ /dev/null @@ -1,81 +0,0 @@ ---- -layout: global -title: INVALID_INTERVAL_FORMAT error class -displayTitle: INVALID_INTERVAL_FORMAT error class -license: | - 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. ---- - - - -[SQLSTATE: 22006](sql-error-conditions-sqlstates.html#class-22-data-exception) - -Error parsing '``' to interval. Please ensure that the value provided is in a valid format for defining an interval. You can reference the documentation for the correct format. - -This error class has the following derived error classes: - -## ARITHMETIC_EXCEPTION - -Uncaught arithmetic exception while parsing '``'. - -## INPUT_IS_EMPTY - -Interval string cannot be empty. - -## INPUT_IS_NULL - -Interval string cannot be null. - -## INVALID_FRACTION - -`` cannot have fractional part. - -## INVALID_PRECISION - -Interval can only support nanosecond precision, `` is out of range. - -## INVALID_PREFIX - -Invalid interval prefix ``. - -## INVALID_UNIT - -Invalid unit ``. - -## INVALID_VALUE - -Invalid value ``. - -## MISSING_NUMBER - -Expect a number after `` but hit EOL. - -## MISSING_UNIT - -Expect a unit name after `` but hit EOL. - -## UNKNOWN_PARSING_ERROR - -Unknown error when parsing ``. - -## UNRECOGNIZED_NUMBER - -Unrecognized number ``. - - diff --git a/docs/sql-error-conditions-numeric-value-out-of-range-error-class.md b/docs/sql-error-conditions-numeric-value-out-of-range-error-class.md deleted file mode 100644 index 690bbeec07473..0000000000000 --- a/docs/sql-error-conditions-numeric-value-out-of-range-error-class.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: global -title: NUMERIC_VALUE_OUT_OF_RANGE error class -displayTitle: NUMERIC_VALUE_OUT_OF_RANGE error class -license: | - 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. ---- - - - -[SQLSTATE: 22003](sql-error-conditions-sqlstates.html#class-22-data-exception) - - - -This error class has the following derived error classes: - -## WITHOUT_SUGGESTION - -The `` rounded half up from `` cannot be represented as Decimal(``, ``). - -## WITH_SUGGESTION - -`` cannot be represented as Decimal(``, ``). If necessary set `` to "false" to bypass this error, and return NULL instead. - - diff --git a/docs/sql-error-conditions-syntax-discontinued-error-class.md b/docs/sql-error-conditions-syntax-discontinued-error-class.md deleted file mode 100644 index 966e11004364e..0000000000000 --- a/docs/sql-error-conditions-syntax-discontinued-error-class.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -layout: global -title: SYNTAX_DISCONTINUED error class -displayTitle: SYNTAX_DISCONTINUED error class -license: | - 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. ---- - - - -[SQLSTATE: 42601](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) - -Support of the clause or keyword: `` has been discontinued in this context. - -This error class has the following derived error classes: - -## BANG_EQUALS_NOT - -The '!' keyword is supported as a prefix operator in a logical operation only. -Use the 'NOT' keyword instead for clauses such as `NOT LIKE`, `NOT IN`, `NOT BETWEEN`, etc. -To re-enable the '!' keyword, set "spark.sql.legacy.bangEqualsNot" to "true". - - diff --git a/docs/sql-error-conditions-unsupported-call-error-class.md b/docs/sql-error-conditions-unsupported-call-error-class.md deleted file mode 100644 index 38c7859e88fe6..0000000000000 --- a/docs/sql-error-conditions-unsupported-call-error-class.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: global -title: UNSUPPORTED_CALL error class -displayTitle: UNSUPPORTED_CALL error class -license: | - 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. ---- - - - -[SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) - -Cannot call the method "``" of the class "``". - -This error class has the following derived error classes: - -## FIELD_INDEX - -The row shall have a schema to get an index of the field ``. - -## WITHOUT_SUGGESTION - - - - diff --git a/docs/sql-error-conditions-unsupported-collation-error-class.md b/docs/sql-error-conditions-unsupported-collation-error-class.md deleted file mode 100644 index ae410a30317a1..0000000000000 --- a/docs/sql-error-conditions-unsupported-collation-error-class.md +++ /dev/null @@ -1,37 +0,0 @@ ---- -layout: global -title: UNSUPPORTED_COLLATION error class -displayTitle: UNSUPPORTED_COLLATION error class -license: | - 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. ---- - - - -[SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) - -Collation `` is not supported for: - -This error class has the following derived error classes: - -## FOR_FUNCTION - -function ``. Please try to use a different collation. - - From 39bb2d84ce2203ce11189e50e44109c7f7323618 Mon Sep 17 00:00:00 2001 From: Harsh Motwani Date: Wed, 15 Jan 2025 16:14:45 +0800 Subject: [PATCH 438/438] [SPARK-50815][PYTHON][SQL] Fix bug where passing null Variants in createDataFrame causes it to fail and add Variant support in createDataFrame in Spark Connect ### What changes were proposed in this pull request? In this PR, we add a case to handle None in `VariantType.toInternal`. Also, variants can be used with `createDataFrame` when using Spark Connect. ### Why are the changes needed? Previously, `spark.createDataFrame([(VariantVal(bytearray([12, 1]), bytearray([1, 0, 0])),), (None,)], "v variant").show()` failed because there was no way of handling nulls. Also, `createDataFrame` did not work with Variants prior to this PR - now it does. ### Does this PR introduce _any_ user-facing change? Yes, it fixes a bug where `None` values couldn't be handled with Variant schemas, and allows users to use createDataFrame with Variants in the Python client. ### How was this patch tested? Unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #49487 from harshmotw-db/harsh-motwani_data/variant_bug_fix. Authored-by: Harsh Motwani Signed-off-by: Wenchen Fan --- python/pyspark/sql/connect/conversion.py | 16 +++++++------- python/pyspark/sql/tests/test_types.py | 27 ++++++++++++++++++++++++ python/pyspark/sql/types.py | 2 ++ 3 files changed, 37 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/connect/conversion.py b/python/pyspark/sql/connect/conversion.py index d803f37c5b9f1..b6b0bd65adcb8 100644 --- a/python/pyspark/sql/connect/conversion.py +++ b/python/pyspark/sql/connect/conversion.py @@ -104,6 +104,7 @@ def _need_converter( def _create_converter( dataType: DataType, nullable: bool = True, + variants_as_dicts: bool = False, # some code paths may require python internal types ) -> Callable: assert dataType is not None and isinstance(dataType, DataType) assert isinstance(nullable, bool) @@ -126,8 +127,7 @@ def convert_null(value: Any) -> Any: field_convs = [ LocalDataToArrowConversion._create_converter( - field.dataType, - field.nullable, + field.dataType, field.nullable, variants_as_dicts ) for field in dataType.fields ] @@ -170,8 +170,7 @@ def convert_struct(value: Any) -> Any: elif isinstance(dataType, ArrayType): element_conv = LocalDataToArrowConversion._create_converter( - dataType.elementType, - dataType.containsNull, + dataType.elementType, dataType.containsNull, variants_as_dicts ) def convert_array(value: Any) -> Any: @@ -188,8 +187,7 @@ def convert_array(value: Any) -> Any: elif isinstance(dataType, MapType): key_conv = LocalDataToArrowConversion._create_converter(dataType.keyType) value_conv = LocalDataToArrowConversion._create_converter( - dataType.valueType, - dataType.valueContainsNull, + dataType.valueType, dataType.valueContainsNull, variants_as_dicts ) def convert_map(value: Any) -> Any: @@ -303,8 +301,11 @@ def convert_variant(value: Any) -> Any: isinstance(value, dict) and all(key in value for key in ["value", "metadata"]) and all(isinstance(value[key], bytes) for key in ["value", "metadata"]) + and not variants_as_dicts ): return VariantVal(value["value"], value["metadata"]) + elif isinstance(value, VariantVal) and variants_as_dicts: + return VariantType().toInternal(value) else: raise PySparkValueError(errorClass="MALFORMED_VARIANT") @@ -331,8 +332,7 @@ def convert(data: Sequence[Any], schema: StructType) -> "pa.Table": column_convs = [ LocalDataToArrowConversion._create_converter( - field.dataType, - field.nullable, + field.dataType, field.nullable, variants_as_dicts=True ) for field in schema.fields ] diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 75c28ac0dec1d..ab05502ad229d 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -2251,6 +2251,33 @@ def test_variant_type(self): self.assertEqual(parse_json_spark_output.value, parse_json_python_output.value) self.assertEqual(parse_json_spark_output.metadata, parse_json_python_output.metadata) + # Test createDataFrame + create_df_variants = self.spark.createDataFrame( + [ + ( + VariantVal.parseJson("2"), + [VariantVal.parseJson("3")], + {"v": VariantVal.parseJson("4")}, + {"v": VariantVal.parseJson("5")}, + ), + (None, [None], {"v": None}, {"v": None}), + (None, None, None, None), + ], + "v variant, a array, s struct, m map", + ).collect() + self.assertEqual(create_df_variants[0][0].toJson(), "2") + self.assertEqual(create_df_variants[0][1][0].toJson(), "3") + self.assertEqual(create_df_variants[0][2][0].toJson(), "4") + self.assertEqual(create_df_variants[0][3]["v"].toJson(), "5") + self.assertEqual(create_df_variants[1][0], None) + self.assertEqual(create_df_variants[1][1][0], None) + self.assertEqual(create_df_variants[1][2][0], None) + self.assertEqual(create_df_variants[1][3]["v"], None) + self.assertEqual(create_df_variants[2][0], None) + self.assertEqual(create_df_variants[2][1], None) + self.assertEqual(create_df_variants[2][2], None) + self.assertEqual(create_df_variants[2][3], None) + def test_to_ddl(self): schema = StructType().add("a", NullType()).add("b", BooleanType()).add("c", BinaryType()) self.assertEqual(schema.toDDL(), "a VOID,b BOOLEAN,c BINARY") diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index cc0285233dcc4..7c3b97f951d67 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1580,6 +1580,8 @@ def fromInternal(self, obj: Dict) -> Optional["VariantVal"]: return VariantVal(obj["value"], obj["metadata"]) def toInternal(self, variant: Any) -> Any: + if variant is None: + return None assert isinstance(variant, VariantVal) return {"value": variant.value, "metadata": variant.metadata}
    Sets a separator for each field and value. This separator can be one or more characters. read/write
    extensioncsvSets the file extension for the output files. Limited to letters. Length must equal 3.write
    encoding
    charset
    UTF-8spark.executor.profiling.dfsDir (none) - An HDFS compatible path to which the profiler's output files are copied. The output files will be written as dfsDir/application_id/profile-appname-exec-executor_id.jfr
    + An HDFS compatible path to which the profiler's output files are copied. The output files will be written as dfsDir/{{APP_ID}}/profile-exec-{{EXECUTOR_ID}}.jfr
    If no dfsDir is specified then the files are not copied over. Users should ensure there is sufficient disk space available otherwise it may lead to corrupt jfr files.
    4.0.0event=wall,interval=10ms,alloc=2m,lock=10ms,chunktime=300s Options to pass to the profiler. Detailed options are documented in the comments here: - Profiler arguments. + Profiler arguments. Note that the options to start, stop, specify output format, and output file do not have to be specified. 4.0.0